You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2017/11/20 17:02:10 UTC

hbase git commit: HBASE-19299 Assert only one Connection is constructed when calculating splits in a MultiTableInputFormat

Repository: hbase
Updated Branches:
  refs/heads/branch-2 b554bbc65 -> eb17a2f28


HBASE-19299 Assert only one Connection is constructed when calculating splits in a MultiTableInputFormat

Adds a test suite that has one test in it. Does a bunch of mocking so
getSplits can run. Has counter in mocked Connection constructor so can
count how many Connections made during call to getSplits. Verified that
assert fails if more than one Connection made.


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/eb17a2f2
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/eb17a2f2
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/eb17a2f2

Branch: refs/heads/branch-2
Commit: eb17a2f285fab68cc722065336af1e7f65a02eb2
Parents: b554bbc
Author: Michael Stack <st...@apache.org>
Authored: Sat Nov 18 16:03:16 2017 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Mon Nov 20 09:01:56 2017 -0800

----------------------------------------------------------------------
 .../TestMultiTableInputFormatBase.java          | 244 +++++++++++++++++++
 1 file changed, 244 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/eb17a2f2/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormatBase.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormatBase.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormatBase.java
new file mode 100644
index 0000000..47d7c0c
--- /dev/null
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormatBase.java
@@ -0,0 +1,244 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.mapreduce;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CategoryBasedTimeout;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.BufferedMutator;
+import org.apache.hadoop.hbase.client.BufferedMutatorParams;
+import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableBuilder;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+import org.junit.rules.TestRule;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.apache.hadoop.hbase.client.Scan.SCAN_ATTRIBUTES_TABLE_NAME;
+
+/**
+ * Tests of MultiTableInputFormatBase.
+ */
+@Category({SmallTests.class})
+public class TestMultiTableInputFormatBase {
+  @Rule public final TestName name = new TestName();
+  @Rule public final TestRule timeout = CategoryBasedTimeout.builder()
+      .withTimeout(this.getClass())
+      .withLookingForStuckThread(true)
+      .build();
+
+  /**
+   * Test getSplits only puts up one Connection.
+   * In past it has put up many Connections. Each Connection setup comes with a fresh new cache
+   * so we have to do fresh hit on hbase:meta. Should only do one Connection when doing getSplits
+   * even if a MultiTableInputFormat.
+   * @throws IOException
+   */
+  @Test
+  public void testMRSplitsConnectionCount() throws IOException {
+    // Make instance of MTIFB.
+    MultiTableInputFormatBase mtif = new MultiTableInputFormatBase() {
+      @Override
+      public RecordReader<ImmutableBytesWritable, Result> createRecordReader(InputSplit split,
+          TaskAttemptContext context)
+      throws IOException, InterruptedException {
+        return super.createRecordReader(split, context);
+      }
+    };
+    // Pass it a mocked JobContext. Make the JC return our Configuration.
+    // Load the Configuration so it returns our special Connection so we can interpolate
+    // canned responses.
+    JobContext mockedJobContext = Mockito.mock(JobContext.class);
+    Configuration c = HBaseConfiguration.create();
+    c.set(ClusterConnection.HBASE_CLIENT_CONNECTION_IMPL, MRSplitsConnection.class.getName());
+    Mockito.when(mockedJobContext.getConfiguration()).thenReturn(c);
+    // Invent a bunch of scans. Have each Scan go against a different table so a good spread.
+    List<Scan> scans = new ArrayList<>();
+    for (int i = 0; i < 10; i++) {
+      Scan scan = new Scan();
+      String tableName = this.name.getMethodName() + i;
+      scan.setAttribute(SCAN_ATTRIBUTES_TABLE_NAME, Bytes.toBytes(tableName));
+      scans.add(scan);
+    }
+    mtif.setScans(scans);
+    // Get splits. Assert that that more than one.
+    List<InputSplit> splits = mtif.getSplits(mockedJobContext);
+    Assert.assertTrue(splits.size() > 0);
+    // Assert only one Connection was made (see the static counter we have in the mocked
+    // Connection MRSplitsConnection Constructor.
+    Assert.assertEquals(1, MRSplitsConnection.creations.get());
+  }
+
+  /**
+   * Connection to use above in Test.
+   */
+  public static class MRSplitsConnection implements Connection {
+    private final Configuration configuration;
+    static final AtomicInteger creations = new AtomicInteger(0);
+
+    MRSplitsConnection (Configuration conf, ExecutorService pool, User user) throws IOException {
+      this.configuration = conf;
+      creations.incrementAndGet();
+    }
+
+    @Override
+    public void abort(String why, Throwable e) {
+
+    }
+
+    @Override
+    public boolean isAborted() {
+      return false;
+    }
+
+    @Override
+    public Configuration getConfiguration() {
+      return this.configuration;
+    }
+
+    @Override
+    public BufferedMutator getBufferedMutator(TableName tableName) throws IOException {
+      return null;
+    }
+
+    @Override
+    public BufferedMutator getBufferedMutator(BufferedMutatorParams params) throws IOException {
+      return null;
+    }
+
+    @Override
+    public RegionLocator getRegionLocator(final TableName tableName) throws IOException {
+      // Make up array of start keys. We start off w/ empty byte array.
+      final byte [][] startKeys = new byte [][] {HConstants.EMPTY_BYTE_ARRAY,
+          Bytes.toBytes("aaaa"), Bytes.toBytes("bbb"),
+          Bytes.toBytes("ccc"), Bytes.toBytes("ddd"), Bytes.toBytes("eee"),
+          Bytes.toBytes("fff"), Bytes.toBytes("ggg"), Bytes.toBytes("hhh"),
+          Bytes.toBytes("iii"), Bytes.toBytes("lll"), Bytes.toBytes("mmm"),
+          Bytes.toBytes("nnn"), Bytes.toBytes("ooo"), Bytes.toBytes("ppp"),
+          Bytes.toBytes("qqq"), Bytes.toBytes("rrr"), Bytes.toBytes("sss"),
+          Bytes.toBytes("ttt"), Bytes.toBytes("uuu"), Bytes.toBytes("vvv"),
+          Bytes.toBytes("zzz")};
+      // Make an array of end keys. We end with the empty byte array.
+      final byte [][] endKeys = new byte[][] {
+          Bytes.toBytes("aaaa"), Bytes.toBytes("bbb"),
+          Bytes.toBytes("ccc"), Bytes.toBytes("ddd"), Bytes.toBytes("eee"),
+          Bytes.toBytes("fff"), Bytes.toBytes("ggg"), Bytes.toBytes("hhh"),
+          Bytes.toBytes("iii"), Bytes.toBytes("lll"), Bytes.toBytes("mmm"),
+          Bytes.toBytes("nnn"), Bytes.toBytes("ooo"), Bytes.toBytes("ppp"),
+          Bytes.toBytes("qqq"), Bytes.toBytes("rrr"), Bytes.toBytes("sss"),
+          Bytes.toBytes("ttt"), Bytes.toBytes("uuu"), Bytes.toBytes("vvv"),
+          Bytes.toBytes("zzz"),
+          HConstants.EMPTY_BYTE_ARRAY};
+      // Now make a map of start keys to HRegionLocations. Let the server namber derive from
+      // the start key.
+      final Map<byte [], HRegionLocation> map =
+          new TreeMap<byte [], HRegionLocation>(Bytes.BYTES_COMPARATOR);
+      for (byte [] startKey: startKeys) {
+        HRegionLocation hrl = new HRegionLocation(
+            RegionInfoBuilder.newBuilder(tableName).setStartKey(startKey).build(),
+            ServerName.valueOf(Bytes.toString(startKey), 0, 0));
+        map.put(startKey, hrl);
+      }
+      // Get a list of the locations.
+      final List<HRegionLocation> locations = new ArrayList<HRegionLocation>(map.values());
+      // Now make a RegionLocator mock backed by the abpve map and list of locations.
+      RegionLocator mockedRegionLocator = Mockito.mock(RegionLocator.class);
+      Mockito.when(mockedRegionLocator.getRegionLocation(Mockito.any(byte [].class),
+            Mockito.anyBoolean())).
+          thenAnswer(new Answer<HRegionLocation>() {
+            @Override
+            public HRegionLocation answer(InvocationOnMock invocationOnMock) throws Throwable {
+              Object [] args = invocationOnMock.getArguments();
+              byte [] key = (byte [])args[0];
+              return map.get(key);
+            }
+          });
+      Mockito.when(mockedRegionLocator.getAllRegionLocations()).thenReturn(locations);
+      Mockito.when(mockedRegionLocator.getStartEndKeys()).
+          thenReturn(new Pair<byte [][], byte[][]>(startKeys, endKeys));
+      Mockito.when(mockedRegionLocator.getName()).thenReturn(tableName);
+      return mockedRegionLocator;
+    }
+
+    @Override
+    public Admin getAdmin() throws IOException {
+      Admin admin = Mockito.mock(Admin.class);
+      Mockito.when(admin.getConfiguration()).thenReturn(getConfiguration());
+      return admin;
+    }
+
+    @Override
+    public Table getTable(TableName tableName) throws IOException {
+      Table table = Mockito.mock(Table.class);
+      Mockito.when(table.getName()).thenReturn(tableName);
+      return table;
+    }
+
+    @Override
+    public void close() throws IOException {
+
+    }
+
+    @Override
+    public boolean isClosed() {
+      return false;
+    }
+
+    @Override
+    public TableBuilder getTableBuilder(TableName tableName, ExecutorService pool) {
+      return Mockito.mock(TableBuilder.class);
+    }
+  }
+}