001/* 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase; 019 020import java.security.InvalidParameterException; 021import java.util.Map; 022import java.util.Set; 023import java.util.TreeMap; 024import java.util.concurrent.atomic.AtomicLong; 025import org.apache.hadoop.conf.Configuration; 026import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; 027import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; 028import org.apache.hadoop.hbase.client.Connection; 029import org.apache.hadoop.hbase.client.ConnectionFactory; 030import org.apache.hadoop.hbase.client.Result; 031import org.apache.hadoop.hbase.client.ResultScanner; 032import org.apache.hadoop.hbase.client.Scan; 033import org.apache.hadoop.hbase.client.Table; 034import org.apache.hadoop.hbase.client.TableDescriptorBuilder; 035import org.apache.hadoop.hbase.filter.Filter; 036import org.apache.hadoop.hbase.filter.SingleColumnValueFilter; 037import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; 038import org.apache.hadoop.hbase.testclassification.IntegrationTests; 039import org.apache.hadoop.hbase.util.Bytes; 040import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; 041import org.apache.hadoop.hbase.util.LoadTestKVGenerator; 042import org.apache.hadoop.hbase.util.MultiThreadedWriter; 043import org.apache.hadoop.hbase.util.RegionSplitter; 044import org.apache.hadoop.hbase.util.test.LoadTestDataGenerator; 045import org.junit.After; 046import org.junit.Assert; 047import org.junit.Before; 048import org.junit.Test; 049import org.junit.experimental.categories.Category; 050import org.slf4j.Logger; 051import org.slf4j.LoggerFactory; 052 053/** 054 * Integration test that verifies lazy CF loading during scans by doing repeated scans with this 055 * feature while multiple threads are continuously writing values; and verifying the result. 056 */ 057@Category(IntegrationTests.class) 058public class IntegrationTestLazyCfLoading { 059 private static final TableName TABLE_NAME = 060 TableName.valueOf(IntegrationTestLazyCfLoading.class.getSimpleName()); 061 @SuppressWarnings("InlineFormatString") 062 private static final String TIMEOUT_KEY = "hbase.%s.timeout"; 063 @SuppressWarnings("InlineFormatString") 064 private static final String ENCODING_KEY = "hbase.%s.datablock.encoding"; 065 066 /** A soft test timeout; duration of the test, as such, depends on number of keys to put. */ 067 private static final int DEFAULT_TIMEOUT_MINUTES = 10; 068 069 private static final int NUM_SERVERS = 1; 070 /** Set regions per server low to ensure splits happen during test */ 071 private static final int REGIONS_PER_SERVER = 3; 072 private static final int KEYS_TO_WRITE_PER_SERVER = 20000; 073 private static final int WRITER_THREADS = 10; 074 private static final int WAIT_BETWEEN_SCANS_MS = 1000; 075 076 private static final Logger LOG = LoggerFactory.getLogger(IntegrationTestLazyCfLoading.class); 077 private IntegrationTestingUtility util = new IntegrationTestingUtility(); 078 private final DataGenerator dataGen = new DataGenerator(); 079 080 /** 081 * Custom LoadTestDataGenerator. Uses key generation and verification from LoadTestKVGenerator. 082 * Creates 3 column families; one with an integer column to filter on, the 2nd one with an integer 083 * column that matches the first integer column (for test-specific verification), and byte[] value 084 * that is used for general verification; and the third one with just the value. 085 */ 086 private static class DataGenerator extends LoadTestDataGenerator { 087 private static final int MIN_DATA_SIZE = 4096; 088 private static final int MAX_DATA_SIZE = 65536; 089 public static final byte[] ESSENTIAL_CF = Bytes.toBytes("essential"); 090 public static final byte[] JOINED_CF1 = Bytes.toBytes("joined"); 091 public static final byte[] JOINED_CF2 = Bytes.toBytes("joined2"); 092 public static final byte[] FILTER_COLUMN = Bytes.toBytes("filter"); 093 public static final byte[] VALUE_COLUMN = Bytes.toBytes("val"); 094 public static final long ACCEPTED_VALUE = 1L; 095 096 private static final Map<byte[], byte[][]> columnMap = new TreeMap<>(Bytes.BYTES_COMPARATOR); 097 098 private final AtomicLong expectedNumberOfKeys = new AtomicLong(0); 099 private final AtomicLong totalNumberOfKeys = new AtomicLong(0); 100 101 public DataGenerator() { 102 super(MIN_DATA_SIZE, MAX_DATA_SIZE); 103 columnMap.put(ESSENTIAL_CF, new byte[][] { FILTER_COLUMN }); 104 columnMap.put(JOINED_CF1, new byte[][] { FILTER_COLUMN, VALUE_COLUMN }); 105 columnMap.put(JOINED_CF2, new byte[][] { VALUE_COLUMN }); 106 } 107 108 public long getExpectedNumberOfKeys() { 109 return expectedNumberOfKeys.get(); 110 } 111 112 public long getTotalNumberOfKeys() { 113 return totalNumberOfKeys.get(); 114 } 115 116 @Override 117 public byte[] getDeterministicUniqueKey(long keyBase) { 118 return Bytes.toBytes(LoadTestKVGenerator.md5PrefixedKey(keyBase)); 119 } 120 121 @Override 122 public byte[][] getColumnFamilies() { 123 return columnMap.keySet().toArray(new byte[columnMap.size()][]); 124 } 125 126 @Override 127 public byte[][] generateColumnsForCf(byte[] rowKey, byte[] cf) { 128 return columnMap.get(cf); 129 } 130 131 @Override 132 public byte[] generateValue(byte[] rowKey, byte[] cf, byte[] column) { 133 if (Bytes.BYTES_COMPARATOR.compare(column, FILTER_COLUMN) == 0) { 134 // Random deterministic way to make some values "on" and others "off" for filters. 135 long value = Long.parseLong(Bytes.toString(rowKey, 0, 4), 16) & ACCEPTED_VALUE; 136 if (Bytes.BYTES_COMPARATOR.compare(cf, ESSENTIAL_CF) == 0) { 137 totalNumberOfKeys.incrementAndGet(); 138 if (value == ACCEPTED_VALUE) { 139 expectedNumberOfKeys.incrementAndGet(); 140 } 141 } 142 return Bytes.toBytes(value); 143 } else if (Bytes.BYTES_COMPARATOR.compare(column, VALUE_COLUMN) == 0) { 144 return kvGenerator.generateRandomSizeValue(rowKey, cf, column); 145 } 146 String error = "Unknown column " + Bytes.toString(column); 147 assert false : error; 148 throw new InvalidParameterException(error); 149 } 150 151 @Override 152 public boolean verify(byte[] rowKey, byte[] cf, byte[] column, byte[] value) { 153 if (Bytes.BYTES_COMPARATOR.compare(column, FILTER_COLUMN) == 0) { 154 // Relies on the filter from getScanFilter being used. 155 return Bytes.toLong(value) == ACCEPTED_VALUE; 156 } else if (Bytes.BYTES_COMPARATOR.compare(column, VALUE_COLUMN) == 0) { 157 return LoadTestKVGenerator.verify(value, rowKey, cf, column); 158 } 159 return false; // some bogus value from read, we don't expect any such thing. 160 } 161 162 @Override 163 public boolean verify(byte[] rowKey, byte[] cf, Set<byte[]> columnSet) { 164 return columnMap.get(cf).length == columnSet.size(); 165 } 166 167 public Filter getScanFilter() { 168 SingleColumnValueFilter scf = new SingleColumnValueFilter(ESSENTIAL_CF, FILTER_COLUMN, 169 CompareOperator.EQUAL, Bytes.toBytes(ACCEPTED_VALUE)); 170 scf.setFilterIfMissing(true); 171 return scf; 172 } 173 } 174 175 @Before 176 public void setUp() throws Exception { 177 LOG.info("Initializing cluster with " + NUM_SERVERS + " servers"); 178 util.initializeCluster(NUM_SERVERS); 179 LOG.info("Done initializing cluster"); 180 createTable(); 181 // after table creation, ACLs need time to be propagated to RSs in a secure deployment 182 // so we sleep a little bit because we don't have a good way to know when permissions 183 // are received by RSs 184 Thread.sleep(3000); 185 } 186 187 private void createTable() throws Exception { 188 deleteTable(); 189 LOG.info("Creating table"); 190 Configuration conf = util.getConfiguration(); 191 String encodingKey = String.format(ENCODING_KEY, this.getClass().getSimpleName()); 192 DataBlockEncoding blockEncoding = DataBlockEncoding.valueOf(conf.get(encodingKey, "FAST_DIFF")); 193 TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(TABLE_NAME); 194 for (byte[] cf : dataGen.getColumnFamilies()) { 195 ColumnFamilyDescriptor familyDescriptor = 196 ColumnFamilyDescriptorBuilder.newBuilder(cf).setDataBlockEncoding(blockEncoding).build(); 197 builder.setColumnFamily(familyDescriptor); 198 } 199 int serverCount = 200 util.getHBaseClusterInterface().getClusterMetrics().getLiveServerMetrics().size(); 201 byte[][] splits = new RegionSplitter.HexStringSplit().split(serverCount * REGIONS_PER_SERVER); 202 util.getAdmin().createTable(builder.build(), splits); 203 LOG.info("Created table"); 204 } 205 206 private void deleteTable() throws Exception { 207 if (util.getAdmin().tableExists(TABLE_NAME)) { 208 LOG.info("Deleting table"); 209 util.deleteTable(TABLE_NAME); 210 LOG.info("Deleted table"); 211 } 212 } 213 214 @After 215 public void tearDown() throws Exception { 216 deleteTable(); 217 LOG.info("Restoring the cluster"); 218 util.restoreCluster(); 219 LOG.info("Done restoring the cluster"); 220 } 221 222 @Test 223 public void testReadersAndWriters() throws Exception { 224 Configuration conf = util.getConfiguration(); 225 String timeoutKey = String.format(TIMEOUT_KEY, this.getClass().getSimpleName()); 226 long maxRuntime = conf.getLong(timeoutKey, DEFAULT_TIMEOUT_MINUTES); 227 long serverCount = 228 util.getHBaseClusterInterface().getClusterMetrics().getLiveServerMetrics().size(); 229 long keysToWrite = serverCount * KEYS_TO_WRITE_PER_SERVER; 230 Connection connection = ConnectionFactory.createConnection(conf); 231 Table table = connection.getTable(TABLE_NAME); 232 233 // Create multi-threaded writer and start it. We write multiple columns/CFs and verify 234 // their integrity, therefore multi-put is necessary. 235 MultiThreadedWriter writer = new MultiThreadedWriter(dataGen, conf, TABLE_NAME); 236 writer.setMultiPut(true); 237 238 LOG.info("Starting writer; the number of keys to write is " + keysToWrite); 239 // TODO : Need to see if tag support has to be given here in the integration test suite 240 writer.start(1, keysToWrite, WRITER_THREADS); 241 242 // Now, do scans. 243 long now = EnvironmentEdgeManager.currentTime(); 244 long timeLimit = now + (maxRuntime * 60000); 245 boolean isWriterDone = false; 246 while (now < timeLimit && !isWriterDone) { 247 LOG 248 .info("Starting the scan; wrote approximately " + dataGen.getTotalNumberOfKeys() + " keys"); 249 isWriterDone = writer.isDone(); 250 if (isWriterDone) { 251 LOG.info("Scanning full result, writer is done"); 252 } 253 Scan scan = new Scan(); 254 for (byte[] cf : dataGen.getColumnFamilies()) { 255 scan.addFamily(cf); 256 } 257 scan.setFilter(dataGen.getScanFilter()); 258 scan.setLoadColumnFamiliesOnDemand(true); 259 // The number of keys we can expect from scan - lower bound (before scan). 260 // Not a strict lower bound - writer knows nothing about filters, so we report 261 // this from generator. Writer might have generated the value but not put it yet. 262 long onesGennedBeforeScan = dataGen.getExpectedNumberOfKeys(); 263 long startTs = EnvironmentEdgeManager.currentTime(); 264 ResultScanner results = table.getScanner(scan); 265 long resultCount = 0; 266 Result result = null; 267 // Verify and count the results. 268 while ((result = results.next()) != null) { 269 boolean isOk = writer.verifyResultAgainstDataGenerator(result, true, true); 270 Assert.assertTrue("Failed to verify [" + Bytes.toString(result.getRow()) + "]", isOk); 271 ++resultCount; 272 } 273 long timeTaken = EnvironmentEdgeManager.currentTime() - startTs; 274 // Verify the result count. 275 long onesGennedAfterScan = dataGen.getExpectedNumberOfKeys(); 276 Assert.assertTrue( 277 "Read " + resultCount + " keys when at most " + onesGennedAfterScan + " were generated ", 278 onesGennedAfterScan >= resultCount); 279 if (isWriterDone) { 280 Assert.assertTrue("Read " + resultCount + " keys; the writer is done and " 281 + onesGennedAfterScan + " keys were generated", onesGennedAfterScan == resultCount); 282 } else if (onesGennedBeforeScan * 0.9 > resultCount) { 283 LOG.warn("Read way too few keys (" + resultCount + "/" + onesGennedBeforeScan 284 + ") - there might be a problem, or the writer might just be slow"); 285 } 286 LOG.info("Scan took " + timeTaken + "ms"); 287 if (!isWriterDone) { 288 Thread.sleep(WAIT_BETWEEN_SCANS_MS); 289 now = EnvironmentEdgeManager.currentTime(); 290 } 291 } 292 Assert.assertEquals("There are write failures", 0, writer.getNumWriteFailures()); 293 Assert.assertTrue("Writer is not done", isWriterDone); 294 // Assert.fail("Boom!"); 295 connection.close(); 296 } 297}