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}