View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  
19  package org.apache.hadoop.hbase;
20  
21  import java.security.InvalidParameterException;
22  import java.util.Map;
23  import java.util.Set;
24  import java.util.TreeMap;
25  import java.util.concurrent.atomic.AtomicLong;
26  
27  import org.apache.commons.logging.Log;
28  import org.apache.commons.logging.LogFactory;
29  import org.apache.hadoop.conf.Configuration;
30  import org.apache.hadoop.hbase.client.HTable;
31  import org.apache.hadoop.hbase.client.Result;
32  import org.apache.hadoop.hbase.client.ResultScanner;
33  import org.apache.hadoop.hbase.client.Scan;
34  import org.apache.hadoop.hbase.filter.CompareFilter;
35  import org.apache.hadoop.hbase.filter.Filter;
36  import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
37  import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
38  import org.apache.hadoop.hbase.testclassification.IntegrationTests;
39  import org.apache.hadoop.hbase.util.Bytes;
40  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
41  import org.apache.hadoop.hbase.util.MultiThreadedWriter;
42  import org.apache.hadoop.hbase.util.RegionSplitter;
43  import org.apache.hadoop.hbase.util.test.LoadTestDataGenerator;
44  import org.apache.hadoop.hbase.util.test.LoadTestKVGenerator;
45  import org.junit.After;
46  import org.junit.Assert;
47  import org.junit.Before;
48  import org.junit.Test;
49  import org.junit.experimental.categories.Category;
50  
51  /**
52   * Integration test that verifies lazy CF loading during scans by doing repeated scans
53   * with this feature while multiple threads are continuously writing values; and
54   * verifying the result.
55   */
56  @Category(IntegrationTests.class)
57  public class IntegrationTestLazyCfLoading {
58    private static final TableName TABLE_NAME =
59        TableName.valueOf(IntegrationTestLazyCfLoading.class.getSimpleName());
60    private static final String TIMEOUT_KEY = "hbase.%s.timeout";
61    private static final String ENCODING_KEY = "hbase.%s.datablock.encoding";
62  
63    /** A soft test timeout; duration of the test, as such, depends on number of keys to put. */
64    private static final int DEFAULT_TIMEOUT_MINUTES = 10;
65  
66    private static final int NUM_SERVERS = 1;
67    /** Set regions per server low to ensure splits happen during test */
68    private static final int REGIONS_PER_SERVER = 3;
69    private static final int KEYS_TO_WRITE_PER_SERVER = 20000;
70    private static final int WRITER_THREADS = 10;
71    private static final int WAIT_BETWEEN_SCANS_MS = 1000;
72  
73    private static final Log LOG = LogFactory.getLog(IntegrationTestLazyCfLoading.class);
74    private IntegrationTestingUtility util = new IntegrationTestingUtility();
75    private final DataGenerator dataGen = new DataGenerator();
76  
77    /** Custom LoadTestDataGenerator. Uses key generation and verification from
78     * LoadTestKVGenerator. Creates 3 column families; one with an integer column to
79     * filter on, the 2nd one with an integer column that matches the first integer column (for
80     * test-specific verification), and byte[] value that is used for general verification; and
81     * the third one with just the value.
82     */
83    private static class DataGenerator extends LoadTestDataGenerator {
84      private static final int MIN_DATA_SIZE = 4096;
85      private static final int MAX_DATA_SIZE = 65536;
86      public static final byte[] ESSENTIAL_CF = Bytes.toBytes("essential");
87      public static final byte[] JOINED_CF1 = Bytes.toBytes("joined");
88      public static final byte[] JOINED_CF2 = Bytes.toBytes("joined2");
89      public static final byte[] FILTER_COLUMN = Bytes.toBytes("filter");
90      public static final byte[] VALUE_COLUMN = Bytes.toBytes("val");
91      public static final long ACCEPTED_VALUE = 1L;
92  
93      private static final Map<byte[], byte[][]> columnMap = new TreeMap<byte[], byte[][]>(
94          Bytes.BYTES_COMPARATOR);
95  
96      private final AtomicLong expectedNumberOfKeys = new AtomicLong(0);
97      private final AtomicLong totalNumberOfKeys = new AtomicLong(0);
98  
99      public DataGenerator() {
100       super(MIN_DATA_SIZE, MAX_DATA_SIZE);
101       columnMap.put(ESSENTIAL_CF, new byte[][] { FILTER_COLUMN });
102       columnMap.put(JOINED_CF1, new byte[][] { FILTER_COLUMN, VALUE_COLUMN });
103       columnMap.put(JOINED_CF2, new byte[][] { VALUE_COLUMN });
104     }
105 
106     public long getExpectedNumberOfKeys() {
107       return expectedNumberOfKeys.get();
108     }
109 
110     public long getTotalNumberOfKeys() {
111       return totalNumberOfKeys.get();
112     }
113 
114     @Override
115     public byte[] getDeterministicUniqueKey(long keyBase) {
116       return LoadTestKVGenerator.md5PrefixedKey(keyBase).getBytes();
117     }
118 
119     @Override
120     public byte[][] getColumnFamilies() {
121       return columnMap.keySet().toArray(new byte[columnMap.size()][]);
122     }
123 
124     @Override
125     public byte[][] generateColumnsForCf(byte[] rowKey, byte[] cf) {
126       return columnMap.get(cf);
127     }
128 
129     @Override
130     public byte[] generateValue(byte[] rowKey, byte[] cf, byte[] column) {
131       if (Bytes.BYTES_COMPARATOR.compare(column, FILTER_COLUMN) == 0) {
132         // Random deterministic way to make some values "on" and others "off" for filters.
133         long value = Long.parseLong(Bytes.toString(rowKey, 0, 4), 16) & ACCEPTED_VALUE;
134         if (Bytes.BYTES_COMPARATOR.compare(cf, ESSENTIAL_CF) == 0) {
135           totalNumberOfKeys.incrementAndGet();
136           if (value == ACCEPTED_VALUE) {
137             expectedNumberOfKeys.incrementAndGet();
138           }
139         }
140         return Bytes.toBytes(value);
141       } else if (Bytes.BYTES_COMPARATOR.compare(column, VALUE_COLUMN) == 0) {
142         return kvGenerator.generateRandomSizeValue(rowKey, cf, column);
143       }
144       String error = "Unknown column " + Bytes.toString(column);
145       assert false : error;
146       throw new InvalidParameterException(error);
147     }
148 
149     @Override
150     public boolean verify(byte[] rowKey, byte[] cf, byte[] column, byte[] value) {
151       if (Bytes.BYTES_COMPARATOR.compare(column, FILTER_COLUMN) == 0) {
152         // Relies on the filter from getScanFilter being used.
153         return Bytes.toLong(value) == ACCEPTED_VALUE;
154       } else if (Bytes.BYTES_COMPARATOR.compare(column, VALUE_COLUMN) == 0) {
155         return LoadTestKVGenerator.verify(value, rowKey, cf, column);
156       }
157       return false; // some bogus value from read, we don't expect any such thing.
158     }
159 
160     @Override
161     public boolean verify(byte[] rowKey, byte[] cf, Set<byte[]> columnSet) {
162       return columnMap.get(cf).length == columnSet.size();
163     }
164 
165     public Filter getScanFilter() {
166       SingleColumnValueFilter scf = new SingleColumnValueFilter(ESSENTIAL_CF, FILTER_COLUMN,
167           CompareFilter.CompareOp.EQUAL, Bytes.toBytes(ACCEPTED_VALUE));
168       scf.setFilterIfMissing(true);
169       return scf;
170     }
171   }
172 
173   @Before
174   public void setUp() throws Exception {
175     LOG.info("Initializing cluster with " + NUM_SERVERS + " servers");
176     util.initializeCluster(NUM_SERVERS);
177     LOG.info("Done initializing cluster");
178     createTable();
179     // after table creation, ACLs need time to be propagated to RSs in a secure deployment
180     // so we sleep a little bit because we don't have a good way to know when permissions
181     // are received by RSs
182     Thread.sleep(3000);
183   }
184 
185   private void createTable() throws Exception {
186     deleteTable();
187     LOG.info("Creating table");
188     Configuration conf = util.getConfiguration();
189     String encodingKey = String.format(ENCODING_KEY, this.getClass().getSimpleName());
190     DataBlockEncoding blockEncoding = DataBlockEncoding.valueOf(conf.get(encodingKey, "FAST_DIFF"));
191     HTableDescriptor htd = new HTableDescriptor(TABLE_NAME);
192     for (byte[] cf : dataGen.getColumnFamilies()) {
193       HColumnDescriptor hcd = new HColumnDescriptor(cf);
194       hcd.setDataBlockEncoding(blockEncoding);
195       htd.addFamily(hcd);
196     }
197     int serverCount = util.getHBaseClusterInterface().getClusterStatus().getServersSize();
198     byte[][] splits = new RegionSplitter.HexStringSplit().split(serverCount * REGIONS_PER_SERVER);
199     util.getHBaseAdmin().createTable(htd, splits);
200     LOG.info("Created table");
201   }
202 
203   private void deleteTable() throws Exception {
204     if (util.getHBaseAdmin().tableExists(TABLE_NAME)) {
205       LOG.info("Deleting table");
206       util.deleteTable(TABLE_NAME);
207       LOG.info("Deleted table");
208     }
209   }
210 
211   @After
212   public void tearDown() throws Exception {
213     deleteTable();
214     LOG.info("Restoring the cluster");
215     util.restoreCluster();
216     LOG.info("Done restoring the cluster");
217   }
218 
219   @Test
220   public void testReadersAndWriters() throws Exception {
221     Configuration conf = util.getConfiguration();
222     String timeoutKey = String.format(TIMEOUT_KEY, this.getClass().getSimpleName());
223     long maxRuntime = conf.getLong(timeoutKey, DEFAULT_TIMEOUT_MINUTES);
224     long serverCount = util.getHBaseClusterInterface().getClusterStatus().getServersSize();
225     long keysToWrite = serverCount * KEYS_TO_WRITE_PER_SERVER;
226     HTable table = new HTable(conf, TABLE_NAME);
227 
228     // Create multi-threaded writer and start it. We write multiple columns/CFs and verify
229     // their integrity, therefore multi-put is necessary.
230     MultiThreadedWriter writer =
231       new MultiThreadedWriter(dataGen, conf, TABLE_NAME);
232     writer.setMultiPut(true);
233 
234     LOG.info("Starting writer; the number of keys to write is " + keysToWrite);
235     // TODO : Need to see if tag support has to be given here in the integration test suite
236     writer.start(1, keysToWrite, WRITER_THREADS);
237 
238     // Now, do scans.
239     long now = EnvironmentEdgeManager.currentTimeMillis();
240     long timeLimit = now + (maxRuntime * 60000);
241     boolean isWriterDone = false;
242     while (now < timeLimit && !isWriterDone) {
243       LOG.info("Starting the scan; wrote approximately "
244         + dataGen.getTotalNumberOfKeys() + " keys");
245       isWriterDone = writer.isDone();
246       if (isWriterDone) {
247         LOG.info("Scanning full result, writer is done");
248       }
249       Scan scan = new Scan();
250       for (byte[] cf : dataGen.getColumnFamilies()) {
251         scan.addFamily(cf);
252       }
253       scan.setFilter(dataGen.getScanFilter());
254       scan.setLoadColumnFamiliesOnDemand(true);
255       // The number of keys we can expect from scan - lower bound (before scan).
256       // Not a strict lower bound - writer knows nothing about filters, so we report
257       // this from generator. Writer might have generated the value but not put it yet.
258       long onesGennedBeforeScan = dataGen.getExpectedNumberOfKeys();
259       long startTs = EnvironmentEdgeManager.currentTimeMillis();
260       ResultScanner results = table.getScanner(scan);
261       long resultCount = 0;
262       Result result = null;
263       // Verify and count the results.
264       while ((result = results.next()) != null) {
265         boolean isOk = writer.verifyResultAgainstDataGenerator(result, true, true);
266         Assert.assertTrue("Failed to verify [" + Bytes.toString(result.getRow())+ "]", isOk);
267         ++resultCount;
268       }
269       long timeTaken = EnvironmentEdgeManager.currentTimeMillis() - startTs;
270       // Verify the result count.
271       long onesGennedAfterScan = dataGen.getExpectedNumberOfKeys();
272       Assert.assertTrue("Read " + resultCount + " keys when at most " + onesGennedAfterScan
273         + " were generated ", onesGennedAfterScan >= resultCount);
274       if (isWriterDone) {
275         Assert.assertTrue("Read " + resultCount + " keys; the writer is done and "
276           + onesGennedAfterScan + " keys were generated", onesGennedAfterScan == resultCount);
277       } else if (onesGennedBeforeScan * 0.9 > resultCount) {
278         LOG.warn("Read way too few keys (" + resultCount + "/" + onesGennedBeforeScan
279           + ") - there might be a problem, or the writer might just be slow");
280       }
281       LOG.info("Scan took " + timeTaken + "ms");
282       if (!isWriterDone) {
283         Thread.sleep(WAIT_BETWEEN_SCANS_MS);
284         now = EnvironmentEdgeManager.currentTimeMillis();
285       }
286     }
287     Assert.assertEquals("There are write failures", 0, writer.getNumWriteFailures());
288     Assert.assertTrue("Writer is not done", isWriterDone);
289     // Assert.fail("Boom!");
290   }
291 }