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