You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by la...@apache.org on 2016/11/01 19:49:25 UTC

[01/50] [abbrv] hbase git commit: HBASE-16771 VerifyReplication should increase GOODROWS counter if re-comparison passes

Repository: hbase
Updated Branches:
  refs/heads/branch-1 9f343b587 -> 5fdddae55


HBASE-16771 VerifyReplication should increase GOODROWS counter if re-comparison passes


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

Branch: refs/heads/branch-1
Commit: 364a57a9500b942163932e2c644c05bff3e55090
Parents: 4b75614
Author: tedyu <yu...@gmail.com>
Authored: Sun Oct 9 20:51:23 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Sun Oct 9 20:51:23 2016 -0700

----------------------------------------------------------------------
 .../hbase/mapreduce/replication/VerifyReplication.java  | 12 ++++++++----
 1 file changed, 8 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/364a57a9/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
index 8bb1592..bf320ee 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
@@ -102,6 +102,7 @@ public class VerifyReplication extends Configured implements Tool {
     private Result currentCompareRowInPeerTable;
     private Table replicatedTable;
     private int sleepMsBeforeReCompare;
+    private String delimiter = "";
     private boolean verbose = false;
 
     /**
@@ -119,6 +120,7 @@ public class VerifyReplication extends Configured implements Tool {
       if (replicatedScanner == null) {
         Configuration conf = context.getConfiguration();
         sleepMsBeforeReCompare = conf.getInt(NAME +".sleepMsBeforeReCompare", 0);
+        delimiter = conf.get(NAME + ".delimiter", "");
         verbose = conf.getBoolean(NAME +".verbose", false);
         final Scan scan = new Scan();
         scan.setBatch(batch);
@@ -179,7 +181,6 @@ public class VerifyReplication extends Configured implements Tool {
             }
           } catch (Exception e) {
             logFailRowAndIncreaseCounter(context, Counters.CONTENT_DIFFERENT_ROWS, value);
-            LOG.error("Exception while comparing row : " + e);
           }
           currentCompareRowInPeerTable = replicatedScanner.next();
           break;
@@ -203,9 +204,11 @@ public class VerifyReplication extends Configured implements Tool {
           Result sourceResult = sourceTable.get(new Get(row.getRow()));
           Result replicatedResult = replicatedTable.get(new Get(row.getRow()));
           Result.compareResults(sourceResult, replicatedResult);
-          context.getCounter(Counters.GOODROWS).increment(1);
-          if (verbose) {
-            LOG.info("Good row key: " + delimiter + Bytes.toString(row.getRow()) + delimiter);
+          if (!sourceResult.isEmpty()) {
+            context.getCounter(Counters.GOODROWS).increment(1);
+            if (verbose) {
+              LOG.info("Good row key: " + delimiter + Bytes.toString(row.getRow()) + delimiter);
+            }
           }
           return;
         } catch (Exception e) {
@@ -309,6 +312,7 @@ public class VerifyReplication extends Configured implements Tool {
     conf.setLong(NAME+".startTime", startTime);
     conf.setLong(NAME+".endTime", endTime);
     conf.setInt(NAME +".sleepMsBeforeReCompare", sleepMsBeforeReCompare);
+    conf.set(NAME + ".delimiter", delimiter);
     conf.setBoolean(NAME +".verbose", verbose);
     if (families != null) {
       conf.set(NAME+".families", families);


[40/50] [abbrv] hbase git commit: HBASE-16910 Avoid NPE when starting StochasticLoadBalancer

Posted by la...@apache.org.
HBASE-16910 Avoid NPE when starting StochasticLoadBalancer

Signed-off-by: Ashish Singhi <as...@apache.org>


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

Branch: refs/heads/branch-1
Commit: ae502a9d5ce3dc5c4a485c3ff364d433bdf29a10
Parents: 16823ff
Author: Guanghao Zhang <zg...@gmail.com>
Authored: Tue Oct 25 11:58:41 2016 +0530
Committer: Ashish Singhi <as...@apache.org>
Committed: Tue Oct 25 11:58:41 2016 +0530

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hbase/master/HMaster.java      | 2 +-
 .../hadoop/hbase/master/balancer/StochasticLoadBalancer.java       | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ae502a9d/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index e079b3b..ba067e7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -774,8 +774,8 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
     }
 
     //initialize load balancer
-    this.balancer.setClusterStatus(getClusterStatus());
     this.balancer.setMasterServices(this);
+    this.balancer.setClusterStatus(getClusterStatus());
     this.balancer.initialize();
 
     // Check if master is shutting down because of some issue

http://git-wip-us.apache.org/repos/asf/hbase/blob/ae502a9d/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
index d497d42..7d7dc8e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
@@ -232,7 +232,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
 
       updateMetricsSize(tablesCount * (functionsCount + 1)); // +1 for overall
     } catch (Exception e) {
-      LOG.error("failed to get the size of all tables, exception = " + e.getMessage());
+      LOG.error("failed to get the size of all tables", e);
     }
   }
 


[28/50] [abbrv] hbase git commit: HBASE-16754 All WALSplitter OutputSinks should process compaction events

Posted by la...@apache.org.
HBASE-16754 All WALSplitter OutputSinks should process compaction events


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

Branch: refs/heads/branch-1
Commit: ae151334ba5c20cb2ad4017a25d52aff8bd3994b
Parents: 019c7f9
Author: Gary Helmling <ga...@apache.org>
Authored: Thu Oct 13 15:31:42 2016 -0700
Committer: Gary Helmling <ga...@apache.org>
Committed: Wed Oct 19 10:37:35 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/wal/WALEdit.java  |   13 +-
 .../RegionReplicaReplicationEndpoint.java       |    2 +-
 .../apache/hadoop/hbase/wal/WALSplitter.java    |   19 +-
 .../apache/hadoop/hbase/wal/TestWALSplit.java   | 1423 ++++++++++++++++++
 .../hbase/wal/TestWALSplitCompressed.java       |   36 +
 5 files changed, 1483 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ae151334/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java
index f7a3967..4fd1c41 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java
@@ -363,13 +363,22 @@ public class WALEdit implements Writable, HeapSize {
    * @return deserialized CompactionDescriptor or null.
    */
   public static CompactionDescriptor getCompaction(Cell kv) throws IOException {
-    if (CellUtil.matchingColumn(kv, METAFAMILY, COMPACTION)) {
-      return CompactionDescriptor.parseFrom(kv.getValue());
+    if (isCompactionMarker(kv)) {
+      return CompactionDescriptor.parseFrom(CellUtil.cloneValue(kv));
     }
     return null;
   }
 
   /**
+   * Returns true if the given cell is a serialized {@link CompactionDescriptor}
+   *
+   * @see #getCompaction(Cell)
+   */
+  public static boolean isCompactionMarker(Cell cell) {
+    return CellUtil.matchingColumn(cell, METAFAMILY, COMPACTION);
+  }
+
+  /**
    * Create a bulk loader WALEdit
    *
    * @param hri                The HRegionInfo for the region in which we are bulk loading

http://git-wip-us.apache.org/repos/asf/hbase/blob/ae151334/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
index f64b367..0697013 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
@@ -368,7 +368,7 @@ public class RegionReplicaReplicationEndpoint extends HBaseReplicationEndpoint {
     }
 
     @Override
-    public boolean keepRegionEvents() {
+    public boolean keepRegionEvent(Entry entry) {
       return true;
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/ae151334/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
index 8bf9fb5..cb60e68 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
@@ -369,7 +369,7 @@ public class WALSplitter {
           continue;
         }
         // Don't send Compaction/Close/Open region events to recovered edit type sinks.
-        if (entry.getEdit().isMetaEdit() && !outputSink.keepRegionEvents()) {
+        if (entry.getEdit().isMetaEdit() && !outputSink.keepRegionEvent(entry)) {
           editsSkipped++;
           continue;
         }
@@ -1282,12 +1282,11 @@ public class WALSplitter {
 
     /**
      * Some WALEdit's contain only KV's for account on what happened to a region.
-     * Not all sinks will want to get those edits.
+     * Not all sinks will want to get all of those edits.
      *
-     * @return Return true if this sink wants to get all WALEdit's regardless of if it's a region
-     * event.
+     * @return Return true if this sink wants to accept this region-level WALEdit.
      */
-    public abstract boolean keepRegionEvents();
+    public abstract boolean keepRegionEvent(Entry entry);
   }
 
   /**
@@ -1632,7 +1631,13 @@ public class WALSplitter {
     }
 
     @Override
-    public boolean keepRegionEvents() {
+    public boolean keepRegionEvent(Entry entry) {
+      ArrayList<Cell> cells = entry.getEdit().getCells();
+      for (int i = 0; i < cells.size(); i++) {
+        if (WALEdit.isCompactionMarker(cells.get(i))) {
+          return true;
+        }
+      }
       return false;
     }
 
@@ -2085,7 +2090,7 @@ public class WALSplitter {
     }
 
     @Override
-    public boolean keepRegionEvents() {
+    public boolean keepRegionEvent(Entry entry) {
       return true;
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/ae151334/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
new file mode 100644
index 0000000..4c77d25
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
@@ -0,0 +1,1423 @@
+/**
+ *
+ * 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.wal;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.protobuf.ByteString;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.lang.reflect.Method;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableSet;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
+import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.wal.FaultySequenceFileLogReader;
+import org.apache.hadoop.hbase.regionserver.wal.InstrumentedLogWriter;
+import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CancelableProgressable;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.Threads;
+import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.hadoop.hbase.wal.WAL.Reader;
+import org.apache.hadoop.hbase.wal.WALProvider.Writer;
+import org.apache.hadoop.hbase.wal.WALSplitter.CorruptedLogFileException;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
+import org.apache.hadoop.ipc.RemoteException;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+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.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+/**
+ * Testing {@link WAL} splitting code.
+ */
+@Category({RegionServerTests.class, LargeTests.class})
+public class TestWALSplit {
+  {
+    // Uncomment the following lines if more verbosity is needed for
+    // debugging (see HBASE-12285 for details).
+    //((Log4JLogger)DataNode.LOG).getLogger().setLevel(Level.ALL);
+    //((Log4JLogger)LeaseManager.LOG).getLogger().setLevel(Level.ALL);
+    //((Log4JLogger)FSNamesystem.LOG).getLogger().setLevel(Level.ALL);
+  }
+  private final static Log LOG = LogFactory.getLog(TestWALSplit.class);
+
+  private static Configuration conf;
+  private FileSystem fs;
+
+  protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+  private Path HBASEDIR;
+  private Path WALDIR;
+  private Path OLDLOGDIR;
+  private Path CORRUPTDIR;
+  private Path TABLEDIR;
+
+  private static final int NUM_WRITERS = 10;
+  private static final int ENTRIES = 10; // entries per writer per region
+
+  private static final String FILENAME_BEING_SPLIT = "testfile";
+  private static final TableName TABLE_NAME =
+      TableName.valueOf("t1");
+  private static final byte[] FAMILY = "f1".getBytes();
+  private static final byte[] QUALIFIER = "q1".getBytes();
+  private static final byte[] VALUE = "v1".getBytes();
+  private static final String WAL_FILE_PREFIX = "wal.dat.";
+  private static List<String> REGIONS = new ArrayList<String>();
+  private static final String HBASE_SKIP_ERRORS = "hbase.hlog.split.skip.errors";
+  private static String ROBBER;
+  private static String ZOMBIE;
+  private static String [] GROUP = new String [] {"supergroup"};
+  private RecoveryMode mode;
+
+  static enum Corruptions {
+    INSERT_GARBAGE_ON_FIRST_LINE,
+    INSERT_GARBAGE_IN_THE_MIDDLE,
+    APPEND_GARBAGE,
+    TRUNCATE,
+    TRUNCATE_TRAILER
+  }
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    conf = TEST_UTIL.getConfiguration();
+    conf.setClass("hbase.regionserver.hlog.writer.impl",
+        InstrumentedLogWriter.class, Writer.class);
+    // This is how you turn off shortcircuit read currently.  TODO: Fix.  Should read config.
+    System.setProperty("hbase.tests.use.shortcircuit.reads", "false");
+    // Create fake maping user to group and set it to the conf.
+    Map<String, String []> u2g_map = new HashMap<String, String []>(2);
+    ROBBER = User.getCurrent().getName() + "-robber";
+    ZOMBIE = User.getCurrent().getName() + "-zombie";
+    u2g_map.put(ROBBER, GROUP);
+    u2g_map.put(ZOMBIE, GROUP);
+    DFSTestUtil.updateConfWithFakeGroupMapping(conf, u2g_map);
+    conf.setInt("dfs.heartbeat.interval", 1);
+    TEST_UTIL.startMiniDFSCluster(2);
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    TEST_UTIL.shutdownMiniDFSCluster();
+  }
+
+  @Rule
+  public TestName name = new TestName();
+  private WALFactory wals = null;
+
+  @Before
+  public void setUp() throws Exception {
+    LOG.info("Cleaning up cluster for new test.");
+    fs = TEST_UTIL.getDFSCluster().getFileSystem();
+    HBASEDIR = TEST_UTIL.createRootDir();
+    OLDLOGDIR = new Path(HBASEDIR, HConstants.HREGION_OLDLOGDIR_NAME);
+    CORRUPTDIR = new Path(HBASEDIR, HConstants.CORRUPT_DIR_NAME);
+    TABLEDIR = FSUtils.getTableDir(HBASEDIR, TABLE_NAME);
+    REGIONS.clear();
+    Collections.addAll(REGIONS, "bbb", "ccc");
+    InstrumentedLogWriter.activateFailure = false;
+    this.mode = (conf.getBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false) ?
+        RecoveryMode.LOG_REPLAY : RecoveryMode.LOG_SPLITTING);
+    wals = new WALFactory(conf, null, name.getMethodName());
+    WALDIR = new Path(HBASEDIR, DefaultWALProvider.getWALDirectoryName(name.getMethodName()));
+    //fs.mkdirs(WALDIR);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    try {
+      wals.close();
+    } catch(IOException exception) {
+      // Some tests will move WALs out from under us. In those cases, we'll get an error on close.
+      LOG.info("Ignoring an error while closing down our WALFactory. Fine for some tests, but if" +
+          " you see a failure look here.");
+      LOG.debug("exception details", exception);
+    } finally {
+      wals = null;
+      fs.delete(HBASEDIR, true);
+    }
+  }
+
+  /**
+   * Simulates splitting a WAL out from under a regionserver that is still trying to write it.
+   * Ensures we do not lose edits.
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  @Test (timeout=300000)
+  public void testLogCannotBeWrittenOnceParsed() throws IOException, InterruptedException {
+    final AtomicLong counter = new AtomicLong(0);
+    AtomicBoolean stop = new AtomicBoolean(false);
+    // Region we'll write edits too and then later examine to make sure they all made it in.
+    final String region = REGIONS.get(0);
+    final int numWriters = 3;
+    Thread zombie = new ZombieLastLogWriterRegionServer(counter, stop, region, numWriters);
+    try {
+      long startCount = counter.get();
+      zombie.start();
+      // Wait till writer starts going.
+      while (startCount == counter.get()) Threads.sleep(1);
+      // Give it a second to write a few appends.
+      Threads.sleep(1000);
+      final Configuration conf2 = HBaseConfiguration.create(this.conf);
+      final User robber = User.createUserForTesting(conf2, ROBBER, GROUP);
+      int count = robber.runAs(new PrivilegedExceptionAction<Integer>() {
+        @Override
+        public Integer run() throws Exception {
+          StringBuilder ls = new StringBuilder("Contents of WALDIR (").append(WALDIR)
+              .append("):\n");
+          for (FileStatus status : fs.listStatus(WALDIR)) {
+            ls.append("\t").append(status.toString()).append("\n");
+          }
+          LOG.debug(ls);
+          LOG.info("Splitting WALs out from under zombie. Expecting " + numWriters + " files.");
+          WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, fs, conf2, wals);
+          LOG.info("Finished splitting out from under zombie.");
+          Path[] logfiles = getLogForRegion(HBASEDIR, TABLE_NAME, region);
+          assertEquals("wrong number of split files for region", numWriters, logfiles.length);
+          int count = 0;
+          for (Path logfile: logfiles) {
+            count += countWAL(logfile);
+          }
+          return count;
+        }
+      });
+      LOG.info("zombie=" + counter.get() + ", robber=" + count);
+      assertTrue("The log file could have at most 1 extra log entry, but can't have less. " +
+              "Zombie could write " + counter.get() + " and logfile had only " + count,
+          counter.get() == count || counter.get() + 1 == count);
+    } finally {
+      stop.set(true);
+      zombie.interrupt();
+      Threads.threadDumpingIsAlive(zombie);
+    }
+  }
+
+  /**
+   * This thread will keep writing to a 'wal' file even after the split process has started.
+   * It simulates a region server that was considered dead but woke up and wrote some more to the
+   * last log entry. Does its writing as an alternate user in another filesystem instance to
+   * simulate better it being a regionserver.
+   */
+  class ZombieLastLogWriterRegionServer extends Thread {
+    final AtomicLong editsCount;
+    final AtomicBoolean stop;
+    final int numOfWriters;
+    /**
+     * Region to write edits for.
+     */
+    final String region;
+    final User user;
+
+    public ZombieLastLogWriterRegionServer(AtomicLong counter, AtomicBoolean stop,
+        final String region, final int writers)
+        throws IOException, InterruptedException {
+      super("ZombieLastLogWriterRegionServer");
+      setDaemon(true);
+      this.stop = stop;
+      this.editsCount = counter;
+      this.region = region;
+      this.user = User.createUserForTesting(conf, ZOMBIE, GROUP);
+      numOfWriters = writers;
+    }
+
+    @Override
+    public void run() {
+      try {
+        doWriting();
+      } catch (IOException e) {
+        LOG.warn(getName() + " Writer exiting " + e);
+      } catch (InterruptedException e) {
+        LOG.warn(getName() + " Writer exiting " + e);
+      }
+    }
+
+    private void doWriting() throws IOException, InterruptedException {
+      this.user.runAs(new PrivilegedExceptionAction<Object>() {
+        @Override
+        public Object run() throws Exception {
+          // Index of the WAL we want to keep open.  generateWALs will leave open the WAL whose
+          // index we supply here.
+          int walToKeepOpen = numOfWriters - 1;
+          // The below method writes numOfWriters files each with ENTRIES entries for a total of
+          // numOfWriters * ENTRIES added per column family in the region.
+          Writer writer = null;
+          try {
+            writer = generateWALs(numOfWriters, ENTRIES, walToKeepOpen);
+          } catch (IOException e1) {
+            throw new RuntimeException("Failed", e1);
+          }
+          // Update counter so has all edits written so far.
+          editsCount.addAndGet(numOfWriters * ENTRIES);
+          loop(writer);
+          // If we've been interruped, then things should have shifted out from under us.
+          // closing should error
+          try {
+            writer.close();
+            fail("Writing closing after parsing should give an error.");
+          } catch (IOException exception) {
+            LOG.debug("ignoring error when closing final writer.", exception);
+          }
+          return null;
+        }
+      });
+    }
+
+    private void loop(final Writer writer) {
+      byte [] regionBytes = Bytes.toBytes(this.region);
+      while (!stop.get()) {
+        try {
+          long seq = appendEntry(writer, TABLE_NAME, regionBytes,
+              ("r" + editsCount.get()).getBytes(), regionBytes, QUALIFIER, VALUE, 0);
+          long count = editsCount.incrementAndGet();
+          LOG.info(getName() + " sync count=" + count + ", seq=" + seq);
+          try {
+            Thread.sleep(1);
+          } catch (InterruptedException e) {
+            //
+          }
+        } catch (IOException ex) {
+          LOG.error(getName() + " ex " + ex.toString());
+          if (ex instanceof RemoteException) {
+            LOG.error("Juliet: got RemoteException " + ex.getMessage() +
+                " while writing " + (editsCount.get() + 1));
+          } else {
+            LOG.error(getName() + " failed to write....at " + editsCount.get());
+            fail("Failed to write " + editsCount.get());
+          }
+          break;
+        } catch (Throwable t) {
+          LOG.error(getName() + " HOW? " + t);
+          LOG.debug("exception details", t);
+          break;
+        }
+      }
+      LOG.info(getName() + " Writer exiting");
+    }
+  }
+
+  /**
+   * @throws IOException
+   * @see https://issues.apache.org/jira/browse/HBASE-3020
+   */
+  @Test (timeout=300000)
+  public void testRecoveredEditsPathForMeta() throws IOException {
+    byte [] encoded = HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes();
+    Path tdir = FSUtils.getTableDir(HBASEDIR, TableName.META_TABLE_NAME);
+    Path regiondir = new Path(tdir,
+        HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
+    fs.mkdirs(regiondir);
+    long now = System.currentTimeMillis();
+    Entry entry =
+        new Entry(new WALKey(encoded,
+            TableName.META_TABLE_NAME, 1, now, HConstants.DEFAULT_CLUSTER_ID),
+            new WALEdit());
+    Path p = WALSplitter.getRegionSplitEditsPath(fs, entry, HBASEDIR,
+        FILENAME_BEING_SPLIT);
+    String parentOfParent = p.getParent().getParent().getName();
+    assertEquals(parentOfParent, HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
+  }
+
+  /**
+   * Test old recovered edits file doesn't break WALSplitter.
+   * This is useful in upgrading old instances.
+   */
+  @Test (timeout=300000)
+  public void testOldRecoveredEditsFileSidelined() throws IOException {
+    byte [] encoded = HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes();
+    Path tdir = FSUtils.getTableDir(HBASEDIR, TableName.META_TABLE_NAME);
+    Path regiondir = new Path(tdir,
+        HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
+    fs.mkdirs(regiondir);
+    long now = System.currentTimeMillis();
+    Entry entry =
+        new Entry(new WALKey(encoded,
+            TableName.META_TABLE_NAME, 1, now, HConstants.DEFAULT_CLUSTER_ID),
+            new WALEdit());
+    Path parent = WALSplitter.getRegionDirRecoveredEditsDir(regiondir);
+    assertEquals(parent.getName(), HConstants.RECOVERED_EDITS_DIR);
+    fs.createNewFile(parent); // create a recovered.edits file
+
+    Path p = WALSplitter.getRegionSplitEditsPath(fs, entry, HBASEDIR,
+        FILENAME_BEING_SPLIT);
+    String parentOfParent = p.getParent().getParent().getName();
+    assertEquals(parentOfParent, HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
+    WALFactory.createRecoveredEditsWriter(fs, p, conf).close();
+  }
+
+  private void useDifferentDFSClient() throws IOException {
+    // make fs act as a different client now
+    // initialize will create a new DFSClient with a new client ID
+    fs.initialize(fs.getUri(), conf);
+  }
+
+  @Test (timeout=300000)
+  public void testSplitPreservesEdits() throws IOException{
+    final String REGION = "region__1";
+    REGIONS.clear();
+    REGIONS.add(REGION);
+
+    generateWALs(1, 10, -1, 0);
+    useDifferentDFSClient();
+    WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
+    Path originalLog = (fs.listStatus(OLDLOGDIR))[0].getPath();
+    Path[] splitLog = getLogForRegion(HBASEDIR, TABLE_NAME, REGION);
+    assertEquals(1, splitLog.length);
+
+    assertTrue("edits differ after split", logsAreEqual(originalLog, splitLog[0]));
+  }
+
+  @Test (timeout=300000)
+  public void testSplitRemovesRegionEventsEdits() throws IOException{
+    final String REGION = "region__1";
+    REGIONS.clear();
+    REGIONS.add(REGION);
+
+    generateWALs(1, 10, -1, 100);
+    useDifferentDFSClient();
+    WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
+    Path originalLog = (fs.listStatus(OLDLOGDIR))[0].getPath();
+    Path[] splitLog = getLogForRegion(HBASEDIR, TABLE_NAME, REGION);
+    assertEquals(1, splitLog.length);
+
+    assertFalse("edits differ after split", logsAreEqual(originalLog, splitLog[0]));
+    // split log should only have the test edits
+    assertEquals(10, countWAL(splitLog[0]));
+  }
+
+
+  @Test (timeout=300000)
+  public void testSplitLeavesCompactionEventsEdits() throws IOException{
+    HRegionInfo hri = new HRegionInfo(TABLE_NAME);
+    REGIONS.clear();
+    REGIONS.add(hri.getEncodedName());
+    Path regionDir = new Path(FSUtils.getTableDir(HBASEDIR, TABLE_NAME), hri.getEncodedName());
+    LOG.info("Creating region directory: " + regionDir);
+    assertTrue(fs.mkdirs(regionDir));
+
+    Writer writer = generateWALs(1, 10, 0, 10);
+    String[] compactInputs = new String[]{"file1", "file2", "file3"};
+    String compactOutput = "file4";
+    appendCompactionEvent(writer, hri, compactInputs, compactOutput);
+    writer.close();
+
+    useDifferentDFSClient();
+    WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
+
+    Path originalLog = (fs.listStatus(OLDLOGDIR))[0].getPath();
+    // original log should have 10 test edits, 10 region markers, 1 compaction marker
+    assertEquals(21, countWAL(originalLog));
+
+    Path[] splitLog = getLogForRegion(HBASEDIR, TABLE_NAME, hri.getEncodedName());
+    assertEquals(1, splitLog.length);
+
+    assertFalse("edits differ after split", logsAreEqual(originalLog, splitLog[0]));
+    // split log should have 10 test edits plus 1 compaction marker
+    assertEquals(11, countWAL(splitLog[0]));
+  }
+
+  /**
+   * @param expectedEntries -1 to not assert
+   * @return the count across all regions
+   */
+  private int splitAndCount(final int expectedFiles, final int expectedEntries)
+      throws IOException {
+    useDifferentDFSClient();
+    WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
+    int result = 0;
+    for (String region : REGIONS) {
+      Path[] logfiles = getLogForRegion(HBASEDIR, TABLE_NAME, region);
+      assertEquals(expectedFiles, logfiles.length);
+      int count = 0;
+      for (Path logfile: logfiles) {
+        count += countWAL(logfile);
+      }
+      if (-1 != expectedEntries) {
+        assertEquals(expectedEntries, count);
+      }
+      result += count;
+    }
+    return result;
+  }
+
+  @Test (timeout=300000)
+  public void testEmptyLogFiles() throws IOException {
+    testEmptyLogFiles(true);
+  }
+
+  @Test (timeout=300000)
+  public void testEmptyOpenLogFiles() throws IOException {
+    testEmptyLogFiles(false);
+  }
+
+  private void testEmptyLogFiles(final boolean close) throws IOException {
+    // we won't create the hlog dir until getWAL got called, so
+    // make dir here when testing empty log file
+    fs.mkdirs(WALDIR);
+    injectEmptyFile(".empty", close);
+    generateWALs(Integer.MAX_VALUE);
+    injectEmptyFile("empty", close);
+    splitAndCount(NUM_WRITERS, NUM_WRITERS * ENTRIES); // skip 2 empty
+  }
+
+  @Test (timeout=300000)
+  public void testOpenZeroLengthReportedFileButWithDataGetsSplit() throws IOException {
+    // generate logs but leave wal.dat.5 open.
+    generateWALs(5);
+    splitAndCount(NUM_WRITERS, NUM_WRITERS * ENTRIES);
+  }
+
+  @Test (timeout=300000)
+  public void testTralingGarbageCorruptionFileSkipErrorsPasses() throws IOException {
+    conf.setBoolean(HBASE_SKIP_ERRORS, true);
+    generateWALs(Integer.MAX_VALUE);
+    corruptWAL(new Path(WALDIR, WAL_FILE_PREFIX + "5"),
+        Corruptions.APPEND_GARBAGE, true);
+    splitAndCount(NUM_WRITERS, NUM_WRITERS * ENTRIES);
+  }
+
+  @Test (timeout=300000)
+  public void testFirstLineCorruptionLogFileSkipErrorsPasses() throws IOException {
+    conf.setBoolean(HBASE_SKIP_ERRORS, true);
+    generateWALs(Integer.MAX_VALUE);
+    corruptWAL(new Path(WALDIR, WAL_FILE_PREFIX + "5"),
+        Corruptions.INSERT_GARBAGE_ON_FIRST_LINE, true);
+    splitAndCount(NUM_WRITERS - 1, (NUM_WRITERS - 1) * ENTRIES); //1 corrupt
+  }
+
+  @Test (timeout=300000)
+  public void testMiddleGarbageCorruptionSkipErrorsReadsHalfOfFile() throws IOException {
+    conf.setBoolean(HBASE_SKIP_ERRORS, true);
+    generateWALs(Integer.MAX_VALUE);
+    corruptWAL(new Path(WALDIR, WAL_FILE_PREFIX + "5"),
+        Corruptions.INSERT_GARBAGE_IN_THE_MIDDLE, false);
+    // the entries in the original logs are alternating regions
+    // considering the sequence file header, the middle corruption should
+    // affect at least half of the entries
+    int goodEntries = (NUM_WRITERS - 1) * ENTRIES;
+    int firstHalfEntries = (int) Math.ceil(ENTRIES / 2) - 1;
+    int allRegionsCount = splitAndCount(NUM_WRITERS, -1);
+    assertTrue("The file up to the corrupted area hasn't been parsed",
+        REGIONS.size() * (goodEntries + firstHalfEntries) <= allRegionsCount);
+  }
+
+  @Test (timeout=300000)
+  public void testCorruptedFileGetsArchivedIfSkipErrors() throws IOException {
+    conf.setBoolean(HBASE_SKIP_ERRORS, true);
+    for (FaultySequenceFileLogReader.FailureType  failureType :
+        FaultySequenceFileLogReader.FailureType.values()) {
+      final Set<String> walDirContents = splitCorruptWALs(failureType);
+      final Set<String> archivedLogs = new HashSet<String>();
+      final StringBuilder archived = new StringBuilder("Archived logs in CORRUPTDIR:");
+      for (FileStatus log : fs.listStatus(CORRUPTDIR)) {
+        archived.append("\n\t").append(log.toString());
+        archivedLogs.add(log.getPath().getName());
+      }
+      LOG.debug(archived.toString());
+      assertEquals(failureType.name() + ": expected to find all of our wals corrupt.",
+          walDirContents, archivedLogs);
+    }
+  }
+
+  /**
+   * @return set of wal names present prior to split attempt.
+   * @throws IOException if the split process fails
+   */
+  private Set<String> splitCorruptWALs(final FaultySequenceFileLogReader.FailureType failureType)
+      throws IOException {
+    Class<?> backupClass = conf.getClass("hbase.regionserver.hlog.reader.impl",
+        Reader.class);
+    InstrumentedLogWriter.activateFailure = false;
+
+    try {
+      conf.setClass("hbase.regionserver.hlog.reader.impl",
+          FaultySequenceFileLogReader.class, Reader.class);
+      conf.set("faultysequencefilelogreader.failuretype", failureType.name());
+      // Clean up from previous tests or previous loop
+      try {
+        wals.shutdown();
+      } catch (IOException exception) {
+        // since we're splitting out from under the factory, we should expect some closing failures.
+        LOG.debug("Ignoring problem closing WALFactory.", exception);
+      }
+      wals.close();
+      try {
+        for (FileStatus log : fs.listStatus(CORRUPTDIR)) {
+          fs.delete(log.getPath(), true);
+        }
+      } catch (FileNotFoundException exception) {
+        LOG.debug("no previous CORRUPTDIR to clean.");
+      }
+      // change to the faulty reader
+      wals = new WALFactory(conf, null, name.getMethodName());
+      generateWALs(-1);
+      // Our reader will render all of these files corrupt.
+      final Set<String> walDirContents = new HashSet<String>();
+      for (FileStatus status : fs.listStatus(WALDIR)) {
+        walDirContents.add(status.getPath().getName());
+      }
+      useDifferentDFSClient();
+      WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
+      return walDirContents;
+    } finally {
+      conf.setClass("hbase.regionserver.hlog.reader.impl", backupClass,
+          Reader.class);
+    }
+  }
+
+  @Test (timeout=300000, expected = IOException.class)
+  public void testTrailingGarbageCorruptionLogFileSkipErrorsFalseThrows()
+      throws IOException {
+    conf.setBoolean(HBASE_SKIP_ERRORS, false);
+    splitCorruptWALs(FaultySequenceFileLogReader.FailureType.BEGINNING);
+  }
+
+  @Test (timeout=300000)
+  public void testCorruptedLogFilesSkipErrorsFalseDoesNotTouchLogs()
+      throws IOException {
+    conf.setBoolean(HBASE_SKIP_ERRORS, false);
+    try {
+      splitCorruptWALs(FaultySequenceFileLogReader.FailureType.BEGINNING);
+    } catch (IOException e) {
+      LOG.debug("split with 'skip errors' set to 'false' correctly threw");
+    }
+    assertEquals("if skip.errors is false all files should remain in place",
+        NUM_WRITERS, fs.listStatus(WALDIR).length);
+  }
+
+  private void ignoreCorruption(final Corruptions corruption, final int entryCount,
+      final int expectedCount) throws IOException {
+    conf.setBoolean(HBASE_SKIP_ERRORS, false);
+
+    final String REGION = "region__1";
+    REGIONS.clear();
+    REGIONS.add(REGION);
+
+    Path c1 = new Path(WALDIR, WAL_FILE_PREFIX + "0");
+    generateWALs(1, entryCount, -1, 0);
+    corruptWAL(c1, corruption, true);
+
+    useDifferentDFSClient();
+    WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
+
+    Path[] splitLog = getLogForRegion(HBASEDIR, TABLE_NAME, REGION);
+    assertEquals(1, splitLog.length);
+
+    int actualCount = 0;
+    Reader in = wals.createReader(fs, splitLog[0]);
+    @SuppressWarnings("unused")
+    Entry entry;
+    while ((entry = in.next()) != null) ++actualCount;
+    assertEquals(expectedCount, actualCount);
+    in.close();
+
+    // should not have stored the EOF files as corrupt
+    FileStatus[] archivedLogs = fs.listStatus(CORRUPTDIR);
+    assertEquals(archivedLogs.length, 0);
+
+  }
+
+  @Test (timeout=300000)
+  public void testEOFisIgnored() throws IOException {
+    int entryCount = 10;
+    ignoreCorruption(Corruptions.TRUNCATE, entryCount, entryCount-1);
+  }
+
+  @Test (timeout=300000)
+  public void testCorruptWALTrailer() throws IOException {
+    int entryCount = 10;
+    ignoreCorruption(Corruptions.TRUNCATE_TRAILER, entryCount, entryCount);
+  }
+
+  @Test (timeout=300000)
+  public void testLogsGetArchivedAfterSplit() throws IOException {
+    conf.setBoolean(HBASE_SKIP_ERRORS, false);
+    generateWALs(-1);
+    useDifferentDFSClient();
+    WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
+    FileStatus[] archivedLogs = fs.listStatus(OLDLOGDIR);
+    assertEquals("wrong number of files in the archive log", NUM_WRITERS, archivedLogs.length);
+  }
+
+  @Test (timeout=300000)
+  public void testSplit() throws IOException {
+    generateWALs(-1);
+    splitAndCount(NUM_WRITERS, NUM_WRITERS * ENTRIES);
+  }
+
+  @Test (timeout=300000)
+  public void testLogDirectoryShouldBeDeletedAfterSuccessfulSplit()
+      throws IOException {
+    generateWALs(-1);
+    useDifferentDFSClient();
+    WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
+    FileStatus [] statuses = null;
+    try {
+      statuses = fs.listStatus(WALDIR);
+      if (statuses != null) {
+        fail("Files left in log dir: " +
+            Joiner.on(",").join(FileUtil.stat2Paths(statuses)));
+      }
+    } catch (FileNotFoundException e) {
+      // hadoop 0.21 throws FNFE whereas hadoop 0.20 returns null
+    }
+  }
+
+  @Test(timeout=300000, expected = IOException.class)
+  public void testSplitWillFailIfWritingToRegionFails() throws Exception {
+    //leave 5th log open so we could append the "trap"
+    Writer writer = generateWALs(4);
+    useDifferentDFSClient();
+
+    String region = "break";
+    Path regiondir = new Path(TABLEDIR, region);
+    fs.mkdirs(regiondir);
+
+    InstrumentedLogWriter.activateFailure = false;
+    appendEntry(writer, TABLE_NAME, Bytes.toBytes(region),
+        ("r" + 999).getBytes(), FAMILY, QUALIFIER, VALUE, 0);
+    writer.close();
+
+    try {
+      InstrumentedLogWriter.activateFailure = true;
+      WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
+    } catch (IOException e) {
+      assertTrue(e.getMessage().
+          contains("This exception is instrumented and should only be thrown for testing"));
+      throw e;
+    } finally {
+      InstrumentedLogWriter.activateFailure = false;
+    }
+  }
+
+  @Test (timeout=300000)
+  public void testSplitDeletedRegion() throws IOException {
+    REGIONS.clear();
+    String region = "region_that_splits";
+    REGIONS.add(region);
+
+    generateWALs(1);
+    useDifferentDFSClient();
+
+    Path regiondir = new Path(TABLEDIR, region);
+    fs.delete(regiondir, true);
+    WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
+    assertFalse(fs.exists(regiondir));
+  }
+
+  @Test (timeout=300000)
+  public void testIOEOnOutputThread() throws Exception {
+    conf.setBoolean(HBASE_SKIP_ERRORS, false);
+
+    generateWALs(-1);
+    useDifferentDFSClient();
+    FileStatus[] logfiles = fs.listStatus(WALDIR);
+    assertTrue("There should be some log file",
+        logfiles != null && logfiles.length > 0);
+    // wals with no entries (like the one we don't use in the factory)
+    // won't cause a failure since nothing will ever be written.
+    // pick the largest one since it's most likely to have entries.
+    int largestLogFile = 0;
+    long largestSize = 0;
+    for (int i = 0; i < logfiles.length; i++) {
+      if (logfiles[i].getLen() > largestSize) {
+        largestLogFile = i;
+        largestSize = logfiles[i].getLen();
+      }
+    }
+    assertTrue("There should be some log greater than size 0.", 0 < largestSize);
+    // Set up a splitter that will throw an IOE on the output side
+    WALSplitter logSplitter = new WALSplitter(wals,
+        conf, HBASEDIR, fs, null, null, this.mode) {
+      @Override
+      protected Writer createWriter(Path logfile) throws IOException {
+        Writer mockWriter = Mockito.mock(Writer.class);
+        Mockito.doThrow(new IOException("Injected")).when(
+            mockWriter).append(Mockito.<Entry>any());
+        return mockWriter;
+      }
+    };
+    // Set up a background thread dumper.  Needs a thread to depend on and then we need to run
+    // the thread dumping in a background thread so it does not hold up the test.
+    final AtomicBoolean stop = new AtomicBoolean(false);
+    final Thread someOldThread = new Thread("Some-old-thread") {
+      @Override
+      public void run() {
+        while(!stop.get()) Threads.sleep(10);
+      }
+    };
+    someOldThread.setDaemon(true);
+    someOldThread.start();
+    final Thread t = new Thread("Background-thread-dumper") {
+      public void run() {
+        try {
+          Threads.threadDumpingIsAlive(someOldThread);
+        } catch (InterruptedException e) {
+          e.printStackTrace();
+        }
+      }
+    };
+    t.setDaemon(true);
+    t.start();
+    try {
+      logSplitter.splitLogFile(logfiles[largestLogFile], null);
+      fail("Didn't throw!");
+    } catch (IOException ioe) {
+      assertTrue(ioe.toString().contains("Injected"));
+    } finally {
+      // Setting this to true will turn off the background thread dumper.
+      stop.set(true);
+    }
+  }
+
+  /**
+   * @param spiedFs should be instrumented for failure.
+   */
+  private void retryOverHdfsProblem(final FileSystem spiedFs) throws Exception {
+    generateWALs(-1);
+    useDifferentDFSClient();
+
+    try {
+      WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, spiedFs, conf, wals);
+      assertEquals(NUM_WRITERS, fs.listStatus(OLDLOGDIR).length);
+      assertFalse(fs.exists(WALDIR));
+    } catch (IOException e) {
+      fail("There shouldn't be any exception but: " + e.toString());
+    }
+  }
+
+  // Test for HBASE-3412
+  @Test (timeout=300000)
+  public void testMovedWALDuringRecovery() throws Exception {
+    // This partial mock will throw LEE for every file simulating
+    // files that were moved
+    FileSystem spiedFs = Mockito.spy(fs);
+    // The "File does not exist" part is very important,
+    // that's how it comes out of HDFS
+    Mockito.doThrow(new LeaseExpiredException("Injected: File does not exist")).
+        when(spiedFs).append(Mockito.<Path>any());
+    retryOverHdfsProblem(spiedFs);
+  }
+
+  @Test (timeout=300000)
+  public void testRetryOpenDuringRecovery() throws Exception {
+    FileSystem spiedFs = Mockito.spy(fs);
+    // The "Cannot obtain block length", "Could not obtain the last block",
+    // and "Blocklist for [^ ]* has changed.*" part is very important,
+    // that's how it comes out of HDFS. If HDFS changes the exception
+    // message, this test needs to be adjusted accordingly.
+    //
+    // When DFSClient tries to open a file, HDFS needs to locate
+    // the last block of the file and get its length. However, if the
+    // last block is under recovery, HDFS may have problem to obtain
+    // the block length, in which case, retry may help.
+    Mockito.doAnswer(new Answer<FSDataInputStream>() {
+      private final String[] errors = new String[] {
+          "Cannot obtain block length", "Could not obtain the last block",
+          "Blocklist for " + OLDLOGDIR + " has changed"};
+      private int count = 0;
+
+      public FSDataInputStream answer(InvocationOnMock invocation) throws Throwable {
+        if (count < 3) {
+          throw new IOException(errors[count++]);
+        }
+        return (FSDataInputStream)invocation.callRealMethod();
+      }
+    }).when(spiedFs).open(Mockito.<Path>any(), Mockito.anyInt());
+    retryOverHdfsProblem(spiedFs);
+  }
+
+  @Test (timeout=300000)
+  public void testTerminationAskedByReporter() throws IOException, CorruptedLogFileException {
+    generateWALs(1, 10, -1);
+    FileStatus logfile = fs.listStatus(WALDIR)[0];
+    useDifferentDFSClient();
+
+    final AtomicInteger count = new AtomicInteger();
+
+    CancelableProgressable localReporter
+        = new CancelableProgressable() {
+      @Override
+      public boolean progress() {
+        count.getAndIncrement();
+        return false;
+      }
+    };
+
+    FileSystem spiedFs = Mockito.spy(fs);
+    Mockito.doAnswer(new Answer<FSDataInputStream>() {
+      public FSDataInputStream answer(InvocationOnMock invocation) throws Throwable {
+        Thread.sleep(1500); // Sleep a while and wait report status invoked
+        return (FSDataInputStream)invocation.callRealMethod();
+      }
+    }).when(spiedFs).open(Mockito.<Path>any(), Mockito.anyInt());
+
+    try {
+      conf.setInt("hbase.splitlog.report.period", 1000);
+      boolean ret = WALSplitter.splitLogFile(
+          HBASEDIR, logfile, spiedFs, conf, localReporter, null, null, this.mode, wals);
+      assertFalse("Log splitting should failed", ret);
+      assertTrue(count.get() > 0);
+    } catch (IOException e) {
+      fail("There shouldn't be any exception but: " + e.toString());
+    } finally {
+      // reset it back to its default value
+      conf.setInt("hbase.splitlog.report.period", 59000);
+    }
+  }
+
+  /**
+   * Test log split process with fake data and lots of edits to trigger threading
+   * issues.
+   */
+  @Test (timeout=300000)
+  public void testThreading() throws Exception {
+    doTestThreading(20000, 128*1024*1024, 0);
+  }
+
+  /**
+   * Test blocking behavior of the log split process if writers are writing slower
+   * than the reader is reading.
+   */
+  @Test (timeout=300000)
+  public void testThreadingSlowWriterSmallBuffer() throws Exception {
+    doTestThreading(200, 1024, 50);
+  }
+
+  /**
+   * Sets up a log splitter with a mock reader and writer. The mock reader generates
+   * a specified number of edits spread across 5 regions. The mock writer optionally
+   * sleeps for each edit it is fed.
+   * *
+   * After the split is complete, verifies that the statistics show the correct number
+   * of edits output into each region.
+   *
+   * @param numFakeEdits number of fake edits to push through pipeline
+   * @param bufferSize size of in-memory buffer
+   * @param writerSlowness writer threads will sleep this many ms per edit
+   */
+  private void doTestThreading(final int numFakeEdits,
+      final int bufferSize,
+      final int writerSlowness) throws Exception {
+
+    Configuration localConf = new Configuration(conf);
+    localConf.setInt("hbase.regionserver.hlog.splitlog.buffersize", bufferSize);
+
+    // Create a fake log file (we'll override the reader to produce a stream of edits)
+    Path logPath = new Path(WALDIR, WAL_FILE_PREFIX + ".fake");
+    FSDataOutputStream out = fs.create(logPath);
+    out.close();
+
+    // Make region dirs for our destination regions so the output doesn't get skipped
+    final List<String> regions = ImmutableList.of("r0", "r1", "r2", "r3", "r4");
+    makeRegionDirs(regions);
+
+    // Create a splitter that reads and writes the data without touching disk
+    WALSplitter logSplitter = new WALSplitter(wals,
+        localConf, HBASEDIR, fs, null, null, this.mode) {
+
+      /* Produce a mock writer that doesn't write anywhere */
+      @Override
+      protected Writer createWriter(Path logfile) throws IOException {
+        Writer mockWriter = Mockito.mock(Writer.class);
+        Mockito.doAnswer(new Answer<Void>() {
+          int expectedIndex = 0;
+
+          @Override
+          public Void answer(InvocationOnMock invocation) {
+            if (writerSlowness > 0) {
+              try {
+                Thread.sleep(writerSlowness);
+              } catch (InterruptedException ie) {
+                Thread.currentThread().interrupt();
+              }
+            }
+            Entry entry = (Entry) invocation.getArguments()[0];
+            WALEdit edit = entry.getEdit();
+            List<Cell> cells = edit.getCells();
+            assertEquals(1, cells.size());
+            Cell cell = cells.get(0);
+
+            // Check that the edits come in the right order.
+            assertEquals(expectedIndex, Bytes.toInt(cell.getRowArray(), cell.getRowOffset(),
+                cell.getRowLength()));
+            expectedIndex++;
+            return null;
+          }
+        }).when(mockWriter).append(Mockito.<Entry>any());
+        return mockWriter;
+      }
+
+      /* Produce a mock reader that generates fake entries */
+      @Override
+      protected Reader getReader(Path curLogFile, CancelableProgressable reporter)
+          throws IOException {
+        Reader mockReader = Mockito.mock(Reader.class);
+        Mockito.doAnswer(new Answer<Entry>() {
+          int index = 0;
+
+          @Override
+          public Entry answer(InvocationOnMock invocation) throws Throwable {
+            if (index >= numFakeEdits) return null;
+
+            // Generate r0 through r4 in round robin fashion
+            int regionIdx = index % regions.size();
+            byte region[] = new byte[] {(byte)'r', (byte) (0x30 + regionIdx)};
+
+            Entry ret = createTestEntry(TABLE_NAME, region,
+                Bytes.toBytes((int)(index / regions.size())),
+                FAMILY, QUALIFIER, VALUE, index);
+            index++;
+            return ret;
+          }
+        }).when(mockReader).next();
+        return mockReader;
+      }
+    };
+
+    logSplitter.splitLogFile(fs.getFileStatus(logPath), null);
+
+    // Verify number of written edits per region
+    Map<byte[], Long> outputCounts = logSplitter.outputSink.getOutputCounts();
+    for (Map.Entry<byte[], Long> entry : outputCounts.entrySet()) {
+      LOG.info("Got " + entry.getValue() + " output edits for region " +
+          Bytes.toString(entry.getKey()));
+      assertEquals((long)entry.getValue(), numFakeEdits / regions.size());
+    }
+    assertEquals("Should have as many outputs as regions", regions.size(), outputCounts.size());
+  }
+
+  // Does leaving the writer open in testSplitDeletedRegion matter enough for two tests?
+  @Test (timeout=300000)
+  public void testSplitLogFileDeletedRegionDir() throws IOException {
+    LOG.info("testSplitLogFileDeletedRegionDir");
+    final String REGION = "region__1";
+    REGIONS.clear();
+    REGIONS.add(REGION);
+
+    generateWALs(1, 10, -1);
+    useDifferentDFSClient();
+
+    Path regiondir = new Path(TABLEDIR, REGION);
+    LOG.info("Region directory is" + regiondir);
+    fs.delete(regiondir, true);
+    WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
+    assertFalse(fs.exists(regiondir));
+  }
+
+  @Test (timeout=300000)
+  public void testSplitLogFileEmpty() throws IOException {
+    LOG.info("testSplitLogFileEmpty");
+    // we won't create the hlog dir until getWAL got called, so
+    // make dir here when testing empty log file
+    fs.mkdirs(WALDIR);
+    injectEmptyFile(".empty", true);
+    useDifferentDFSClient();
+
+    WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
+    Path tdir = FSUtils.getTableDir(HBASEDIR, TABLE_NAME);
+    assertFalse(fs.exists(tdir));
+
+    assertEquals(0, countWAL(fs.listStatus(OLDLOGDIR)[0].getPath()));
+  }
+
+  @Test (timeout=300000)
+  public void testSplitLogFileMultipleRegions() throws IOException {
+    LOG.info("testSplitLogFileMultipleRegions");
+    generateWALs(1, 10, -1);
+    splitAndCount(1, 10);
+  }
+
+  @Test (timeout=300000)
+  public void testSplitLogFileFirstLineCorruptionLog()
+      throws IOException {
+    conf.setBoolean(HBASE_SKIP_ERRORS, true);
+    generateWALs(1, 10, -1);
+    FileStatus logfile = fs.listStatus(WALDIR)[0];
+
+    corruptWAL(logfile.getPath(),
+        Corruptions.INSERT_GARBAGE_ON_FIRST_LINE, true);
+
+    useDifferentDFSClient();
+    WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
+
+    final Path corruptDir = new Path(FSUtils.getRootDir(conf), HConstants.CORRUPT_DIR_NAME);
+    assertEquals(1, fs.listStatus(corruptDir).length);
+  }
+
+  /**
+   * @throws IOException
+   * @see https://issues.apache.org/jira/browse/HBASE-4862
+   */
+  @Test (timeout=300000)
+  public void testConcurrentSplitLogAndReplayRecoverEdit() throws IOException {
+    LOG.info("testConcurrentSplitLogAndReplayRecoverEdit");
+    // Generate wals for our destination region
+    String regionName = "r0";
+    final Path regiondir = new Path(TABLEDIR, regionName);
+    REGIONS.clear();
+    REGIONS.add(regionName);
+    generateWALs(-1);
+
+    wals.getWAL(Bytes.toBytes(regionName), null);
+    FileStatus[] logfiles = fs.listStatus(WALDIR);
+    assertTrue("There should be some log file",
+        logfiles != null && logfiles.length > 0);
+
+    WALSplitter logSplitter = new WALSplitter(wals,
+        conf, HBASEDIR, fs, null, null, this.mode) {
+      @Override
+      protected Writer createWriter(Path logfile)
+          throws IOException {
+        Writer writer = wals.createRecoveredEditsWriter(this.fs, logfile);
+        // After creating writer, simulate region's
+        // replayRecoveredEditsIfAny() which gets SplitEditFiles of this
+        // region and delete them, excluding files with '.temp' suffix.
+        NavigableSet<Path> files = WALSplitter.getSplitEditFilesSorted(fs, regiondir);
+        if (files != null && !files.isEmpty()) {
+          for (Path file : files) {
+            if (!this.fs.delete(file, false)) {
+              LOG.error("Failed delete of " + file);
+            } else {
+              LOG.debug("Deleted recovered.edits file=" + file);
+            }
+          }
+        }
+        return writer;
+      }
+    };
+    try{
+      logSplitter.splitLogFile(logfiles[0], null);
+    } catch (IOException e) {
+      LOG.info(e);
+      fail("Throws IOException when spliting "
+          + "log, it is most likely because writing file does not "
+          + "exist which is caused by concurrent replayRecoveredEditsIfAny()");
+    }
+    if (fs.exists(CORRUPTDIR)) {
+      if (fs.listStatus(CORRUPTDIR).length > 0) {
+        fail("There are some corrupt logs, "
+            + "it is most likely caused by concurrent replayRecoveredEditsIfAny()");
+      }
+    }
+  }
+
+  private Writer generateWALs(int leaveOpen) throws IOException {
+    return generateWALs(NUM_WRITERS, ENTRIES, leaveOpen, 0);
+  }
+
+  private Writer generateWALs(int writers, int entries, int leaveOpen) throws IOException {
+    return generateWALs(writers, entries, leaveOpen, 7);
+  }
+
+  private void makeRegionDirs(List<String> regions) throws IOException {
+    for (String region : regions) {
+      LOG.debug("Creating dir for region " + region);
+      fs.mkdirs(new Path(TABLEDIR, region));
+    }
+  }
+
+  /**
+   * @param leaveOpen index to leave un-closed. -1 to close all.
+   * @return the writer that's still open, or null if all were closed.
+   */
+  private Writer generateWALs(int writers, int entries, int leaveOpen, int regionEvents) throws IOException {
+    makeRegionDirs(REGIONS);
+    fs.mkdirs(WALDIR);
+    Writer [] ws = new Writer[writers];
+    int seq = 0;
+    int numRegionEventsAdded = 0;
+    for (int i = 0; i < writers; i++) {
+      ws[i] = wals.createWALWriter(fs, new Path(WALDIR, WAL_FILE_PREFIX + i));
+      for (int j = 0; j < entries; j++) {
+        int prefix = 0;
+        for (String region : REGIONS) {
+          String row_key = region + prefix++ + i + j;
+          appendEntry(ws[i], TABLE_NAME, region.getBytes(), row_key.getBytes(), FAMILY, QUALIFIER,
+              VALUE, seq++);
+
+          if (numRegionEventsAdded < regionEvents) {
+            numRegionEventsAdded ++;
+            appendRegionEvent(ws[i], region);
+          }
+        }
+      }
+      if (i != leaveOpen) {
+        ws[i].close();
+        LOG.info("Closing writer " + i);
+      }
+    }
+    if (leaveOpen < 0 || leaveOpen >= writers) {
+      return null;
+    }
+    return ws[leaveOpen];
+  }
+
+
+
+  private Path[] getLogForRegion(Path rootdir, TableName table, String region)
+      throws IOException {
+    Path tdir = FSUtils.getTableDir(rootdir, table);
+    @SuppressWarnings("deprecation")
+    Path editsdir = WALSplitter.getRegionDirRecoveredEditsDir(HRegion.getRegionDir(tdir,
+        Bytes.toString(region.getBytes())));
+    FileStatus[] files = fs.listStatus(editsdir, new PathFilter() {
+      @Override
+      public boolean accept(Path p) {
+        if (WALSplitter.isSequenceIdFile(p)) {
+          return false;
+        }
+        return true;
+      }
+    });
+    Path[] paths = new Path[files.length];
+    for (int i = 0; i < files.length; i++) {
+      paths[i] = files[i].getPath();
+    }
+    return paths;
+  }
+
+  private void corruptWAL(Path path, Corruptions corruption, boolean close) throws IOException {
+    FSDataOutputStream out;
+    int fileSize = (int) fs.listStatus(path)[0].getLen();
+
+    FSDataInputStream in = fs.open(path);
+    byte[] corrupted_bytes = new byte[fileSize];
+    in.readFully(0, corrupted_bytes, 0, fileSize);
+    in.close();
+
+    switch (corruption) {
+    case APPEND_GARBAGE:
+      fs.delete(path, false);
+      out = fs.create(path);
+      out.write(corrupted_bytes);
+      out.write("-----".getBytes());
+      closeOrFlush(close, out);
+      break;
+
+    case INSERT_GARBAGE_ON_FIRST_LINE:
+      fs.delete(path, false);
+      out = fs.create(path);
+      out.write(0);
+      out.write(corrupted_bytes);
+      closeOrFlush(close, out);
+      break;
+
+    case INSERT_GARBAGE_IN_THE_MIDDLE:
+      fs.delete(path, false);
+      out = fs.create(path);
+      int middle = (int) Math.floor(corrupted_bytes.length / 2);
+      out.write(corrupted_bytes, 0, middle);
+      out.write(0);
+      out.write(corrupted_bytes, middle, corrupted_bytes.length - middle);
+      closeOrFlush(close, out);
+      break;
+
+    case TRUNCATE:
+      fs.delete(path, false);
+      out = fs.create(path);
+      out.write(corrupted_bytes, 0, fileSize
+          - (32 + ProtobufLogReader.PB_WAL_COMPLETE_MAGIC.length + Bytes.SIZEOF_INT));
+      closeOrFlush(close, out);
+      break;
+
+    case TRUNCATE_TRAILER:
+      fs.delete(path, false);
+      out = fs.create(path);
+      out.write(corrupted_bytes, 0, fileSize - Bytes.SIZEOF_INT);// trailer is truncated.
+      closeOrFlush(close, out);
+      break;
+    }
+  }
+
+  private void closeOrFlush(boolean close, FSDataOutputStream out)
+      throws IOException {
+    if (close) {
+      out.close();
+    } else {
+      Method syncMethod = null;
+      try {
+        syncMethod = out.getClass().getMethod("hflush", new Class<?> []{});
+      } catch (NoSuchMethodException e) {
+        try {
+          syncMethod = out.getClass().getMethod("sync", new Class<?> []{});
+        } catch (NoSuchMethodException ex) {
+          throw new IOException("This version of Hadoop supports " +
+              "neither Syncable.sync() nor Syncable.hflush().");
+        }
+      }
+      try {
+        syncMethod.invoke(out, new Object[]{});
+      } catch (Exception e) {
+        throw new IOException(e);
+      }
+      // Not in 0out.hflush();
+    }
+  }
+
+  private int countWAL(Path log) throws IOException {
+    int count = 0;
+    Reader in = wals.createReader(fs, log);
+    while (in.next() != null) {
+      count++;
+    }
+    in.close();
+    return count;
+  }
+
+  private static void appendCompactionEvent(Writer w, HRegionInfo hri, String[] inputs,
+      String output) throws IOException {
+    WALProtos.CompactionDescriptor.Builder desc = WALProtos.CompactionDescriptor.newBuilder();
+    desc.setTableName(ByteString.copyFrom(hri.getTable().toBytes()))
+        .setEncodedRegionName(ByteString.copyFrom(hri.getEncodedNameAsBytes()))
+        .setRegionName(ByteString.copyFrom(hri.getRegionName()))
+        .setFamilyName(ByteString.copyFrom(FAMILY))
+        .setStoreHomeDir(hri.getEncodedName() + "/" + Bytes.toString(FAMILY))
+        .addAllCompactionInput(Arrays.asList(inputs))
+        .addCompactionOutput(output);
+
+    WALEdit edit = WALEdit.createCompaction(hri, desc.build());
+    WALKey key = new WALKey(hri.getEncodedNameAsBytes(), TABLE_NAME, 1,
+        EnvironmentEdgeManager.currentTime(), HConstants.DEFAULT_CLUSTER_ID);
+    w.append(new Entry(key, edit));
+    w.sync();
+  }
+
+  private static void appendRegionEvent(Writer w, String region) throws IOException {
+    WALProtos.RegionEventDescriptor regionOpenDesc = ProtobufUtil.toRegionEventDescriptor(
+        WALProtos.RegionEventDescriptor.EventType.REGION_OPEN,
+        TABLE_NAME.toBytes(),
+        region.getBytes(),
+        String.valueOf(region.hashCode()).getBytes(),
+        1,
+        ServerName.parseServerName("ServerName:9099"), ImmutableMap.<byte[], List<Path>>of());
+    final long time = EnvironmentEdgeManager.currentTime();
+    KeyValue kv = new KeyValue(region.getBytes(), WALEdit.METAFAMILY, WALEdit.REGION_EVENT,
+        time, regionOpenDesc.toByteArray());
+    final WALKey walKey = new WALKey(region.getBytes(), TABLE_NAME, 1, time,
+        HConstants.DEFAULT_CLUSTER_ID);
+    w.append(
+        new Entry(walKey, new WALEdit().add(kv)));
+    w.sync();
+  }
+
+  public static long appendEntry(Writer writer, TableName table, byte[] region,
+      byte[] row, byte[] family, byte[] qualifier,
+      byte[] value, long seq)
+      throws IOException {
+    LOG.info(Thread.currentThread().getName() + " append");
+    writer.append(createTestEntry(table, region, row, family, qualifier, value, seq));
+    LOG.info(Thread.currentThread().getName() + " sync");
+    writer.sync();
+    return seq;
+  }
+
+  private static Entry createTestEntry(
+      TableName table, byte[] region,
+      byte[] row, byte[] family, byte[] qualifier,
+      byte[] value, long seq) {
+    long time = System.nanoTime();
+
+    seq++;
+    final KeyValue cell = new KeyValue(row, family, qualifier, time, KeyValue.Type.Put, value);
+    WALEdit edit = new WALEdit();
+    edit.add(cell);
+    return new Entry(new WALKey(region, table, seq, time,
+        HConstants.DEFAULT_CLUSTER_ID), edit);
+  }
+
+  private void injectEmptyFile(String suffix, boolean closeFile)
+      throws IOException {
+    Writer writer = wals.createWALWriter(fs, new Path(WALDIR, WAL_FILE_PREFIX + suffix),
+        conf);
+    if (closeFile) writer.close();
+  }
+
+  private boolean logsAreEqual(Path p1, Path p2) throws IOException {
+    Reader in1, in2;
+    in1 = wals.createReader(fs, p1);
+    in2 = wals.createReader(fs, p2);
+    Entry entry1;
+    Entry entry2;
+    while ((entry1 = in1.next()) != null) {
+      entry2 = in2.next();
+      if ((entry1.getKey().compareTo(entry2.getKey()) != 0) ||
+          (!entry1.getEdit().toString().equals(entry2.getEdit().toString()))) {
+        return false;
+      }
+    }
+    in1.close();
+    in2.close();
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ae151334/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplitCompressed.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplitCompressed.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplitCompressed.java
new file mode 100644
index 0000000..f47951a
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplitCompressed.java
@@ -0,0 +1,36 @@
+/**
+ * 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.wal;
+
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.junit.BeforeClass;
+import org.junit.experimental.categories.Category;
+
+@Category({RegionServerTests.class, LargeTests.class})
+public class TestWALSplitCompressed extends TestWALSplit {
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    TestWALSplit.setUpBeforeClass();
+    TEST_UTIL.getConfiguration().setBoolean(HConstants.ENABLE_WAL_COMPRESSION, true);
+  }
+}
\ No newline at end of file


[39/50] [abbrv] hbase git commit: HBASE-16931 Setting cell's seqId to zero in compaction flow might cause RS down.

Posted by la...@apache.org.
HBASE-16931 Setting cell's seqId to zero in compaction flow might cause RS down.

Signed-off-by: Yu Li <li...@apache.org>


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

Branch: refs/heads/branch-1
Commit: 16823ff55e7da4822d82fb2a2108b4a253fd42f9
Parents: a1536c2
Author: binlijin <bi...@gmail.com>
Authored: Mon Oct 24 23:31:14 2016 +0800
Committer: Yu Li <li...@apache.org>
Committed: Mon Oct 24 23:33:34 2016 +0800

----------------------------------------------------------------------
 .../regionserver/compactions/Compactor.java     | 17 +++-
 .../hbase/regionserver/TestCompaction.java      | 83 +++++++++++++++++++-
 2 files changed, 94 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/16823ff5/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
index c6fc0c6..b6d145d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
@@ -26,8 +26,6 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 
-import com.google.common.io.Closeables;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -58,6 +56,8 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
 
+import com.google.common.io.Closeables;
+
 /**
  * A compactor is a compaction algorithm associated a given policy. Base class also contains
  * reusable parts for implementing compactors (what is common and what isn't is evolving).
@@ -75,7 +75,7 @@ public abstract class Compactor<T extends CellSink> {
   protected final Compression.Algorithm compactionCompression;
 
   /** specify how many days to keep MVCC values during major compaction **/ 
-  protected final int keepSeqIdPeriod;
+  protected int keepSeqIdPeriod;
 
   //TODO: depending on Store is not good but, realistically, all compactors currently do.
   Compactor(final Configuration conf, final Store store) {
@@ -435,9 +435,16 @@ public abstract class Compactor<T extends CellSink> {
           now = EnvironmentEdgeManager.currentTime();
         }
         // output to writer:
+        Cell lastCleanCell = null;
+        long lastCleanCellSeqId = 0;
         for (Cell c : cells) {
           if (cleanSeqId && c.getSequenceId() <= smallestReadPoint) {
+            lastCleanCell = c;
+            lastCleanCellSeqId = c.getSequenceId();
             CellUtil.setSequenceId(c, 0);
+          } else {
+            lastCleanCell = null;
+            lastCleanCellSeqId = 0;
           }
           writer.append(c);
           int len = KeyValueUtil.length(c);
@@ -459,6 +466,10 @@ public abstract class Compactor<T extends CellSink> {
             }
           }
         }
+        if (lastCleanCell != null) {
+          // HBASE-16931, set back sequence id to avoid affecting scan order unexpectedly
+          CellUtil.setSequenceId(lastCleanCell, lastCleanCellSeqId);
+        }
         // Log the progress of long running compactions every minute if
         // logging at DEBUG level
         if (LOG.isDebugEnabled()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/16823ff5/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
index da0bf42..6b1382f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
@@ -50,10 +50,9 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestCase;
 import org.apache.hadoop.hbase.HBaseTestCase.HRegionIncommon;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Put;
@@ -64,10 +63,12 @@ import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor;
 import org.apache.hadoop.hbase.regionserver.throttle.CompactionThroughputControllerFactory;
 import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
-import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Threads;
+import org.apache.hadoop.hbase.wal.WAL;
 import org.junit.After;
 import org.junit.Assume;
 import org.junit.Before;
@@ -98,6 +99,7 @@ public class TestCompaction {
   private int compactionThreshold;
   private byte[] secondRowBytes, thirdRowBytes;
   private static final long MAX_FILES_TO_COMPACT = 10;
+  private final byte[] FAMILY = Bytes.toBytes("cf");
 
   /** constructor */
   public TestCompaction() {
@@ -120,6 +122,15 @@ public class TestCompaction {
   @Before
   public void setUp() throws Exception {
     this.htd = UTIL.createTableDescriptor(name.getMethodName());
+    if (name.getMethodName().equals("testCompactionSeqId")) {
+      UTIL.getConfiguration().set("hbase.hstore.compaction.kv.max", "10");
+      UTIL.getConfiguration().set(
+          DefaultStoreEngine.DEFAULT_COMPACTOR_CLASS_KEY,
+          DummyCompactor.class.getName());
+      HColumnDescriptor hcd = new HColumnDescriptor(FAMILY);
+      hcd.setMaxVersions(65536);
+      this.htd.addFamily(hcd);
+    }
     this.r = UTIL.createLocalHRegion(htd, null, null);
   }
 
@@ -604,6 +615,72 @@ public class TestCompaction {
     cst.interruptIfNecessary();
   }
 
+  /**
+   * Firstly write 10 cells (with different time stamp) to a qualifier and flush
+   * to hfile1, then write 10 cells (with different time stamp) to the same
+   * qualifier and flush to hfile2. The latest cell (cell-A) in hfile1 and the
+   * oldest cell (cell-B) in hfile2 are with the same time stamp but different
+   * sequence id, and will get scanned successively during compaction.
+   * <p/>
+   * We set compaction.kv.max to 10 so compaction will scan 10 versions each
+   * round, meanwhile we set keepSeqIdPeriod=0 in {@link DummyCompactor} so all
+   * 10 versions of hfile2 will be written out with seqId cleaned (set to 0)
+   * including cell-B, then when scanner goes to cell-A it will cause a scan
+   * out-of-order assertion error before HBASE-16931
+   *
+   * @throws Exception
+   *           if error occurs during the test
+   */
+  @Test
+  public void testCompactionSeqId() throws Exception {
+    final byte[] ROW = Bytes.toBytes("row");
+    final byte[] QUALIFIER = Bytes.toBytes("qualifier");
+
+    long timestamp = 10000;
+
+    // row1/cf:a/10009/Put/vlen=2/seqid=11 V: v9
+    // row1/cf:a/10008/Put/vlen=2/seqid=10 V: v8
+    // row1/cf:a/10007/Put/vlen=2/seqid=9 V: v7
+    // row1/cf:a/10006/Put/vlen=2/seqid=8 V: v6
+    // row1/cf:a/10005/Put/vlen=2/seqid=7 V: v5
+    // row1/cf:a/10004/Put/vlen=2/seqid=6 V: v4
+    // row1/cf:a/10003/Put/vlen=2/seqid=5 V: v3
+    // row1/cf:a/10002/Put/vlen=2/seqid=4 V: v2
+    // row1/cf:a/10001/Put/vlen=2/seqid=3 V: v1
+    // row1/cf:a/10000/Put/vlen=2/seqid=2 V: v0
+    for (int i = 0; i < 10; i++) {
+      Put put = new Put(ROW);
+      put.addColumn(FAMILY, QUALIFIER, timestamp + i, Bytes.toBytes("v" + i));
+      r.put(put);
+    }
+    r.flush(true);
+
+    // row1/cf:a/10018/Put/vlen=3/seqid=16 V: v18
+    // row1/cf:a/10017/Put/vlen=3/seqid=17 V: v17
+    // row1/cf:a/10016/Put/vlen=3/seqid=18 V: v16
+    // row1/cf:a/10015/Put/vlen=3/seqid=19 V: v15
+    // row1/cf:a/10014/Put/vlen=3/seqid=20 V: v14
+    // row1/cf:a/10013/Put/vlen=3/seqid=21 V: v13
+    // row1/cf:a/10012/Put/vlen=3/seqid=22 V: v12
+    // row1/cf:a/10011/Put/vlen=3/seqid=23 V: v11
+    // row1/cf:a/10010/Put/vlen=3/seqid=24 V: v10
+    // row1/cf:a/10009/Put/vlen=2/seqid=25 V: v9
+    for (int i = 18; i > 8; i--) {
+      Put put = new Put(ROW);
+      put.addColumn(FAMILY, QUALIFIER, timestamp + i, Bytes.toBytes("v" + i));
+      r.put(put);
+    }
+    r.flush(true);
+    r.compact(true);
+  }
+
+  public static class DummyCompactor extends DefaultCompactor {
+    public DummyCompactor(Configuration conf, Store store) {
+      super(conf, store);
+      this.keepSeqIdPeriod = 0;
+    }
+  }
+
   private static StoreFile createFile() throws Exception {
     StoreFile sf = mock(StoreFile.class);
     when(sf.getPath()).thenReturn(new Path("file"));


[10/50] [abbrv] hbase git commit: HBASE-16663 JMX ConnectorServer stopped when unauthorized user try to stop HM/RS/cluster

Posted by la...@apache.org.
HBASE-16663 JMX ConnectorServer stopped when unauthorized user try to stop HM/RS/cluster

Signed-off-by: Andrew Purtell <ap...@apache.org>

Conflicts:

	hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java


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

Branch: refs/heads/branch-1
Commit: 6db4ef84796225dcf780021cbf567eaeed793f1e
Parents: 4f29c23
Author: Pankaj Kumar <pa...@huawei.com>
Authored: Sat Oct 8 12:29:06 2016 +0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Tue Oct 11 16:17:07 2016 -0700

----------------------------------------------------------------------
 .../hbase/master/MasterCoprocessorHost.java     |  56 ++++-
 .../RegionServerCoprocessorHost.java            |  51 ++++-
 .../hadoop/hbase/TestJMXConnectorServer.java    | 206 +++++++++++++++++++
 3 files changed, 310 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/6db4ef84/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
index 9e7021a..086d72a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
@@ -799,7 +799,9 @@ public class MasterCoprocessorHost
   }
 
   public void preShutdown() throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    // While stopping the cluster all coprocessors method should be executed first then the
+    // coprocessor should be cleaned up.
+    execShutdown(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
           throws IOException {
@@ -814,7 +816,9 @@ public class MasterCoprocessorHost
   }
 
   public void preStopMaster() throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    // While stopping master all coprocessors method should be executed first then the coprocessor
+    // environment should be cleaned up.
+    execShutdown(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
           throws IOException {
@@ -1200,4 +1204,52 @@ public class MasterCoprocessorHost
     }
     return bypass;
   }
+
+  /**
+   * Master coprocessor classes can be configured in any order, based on that priority is set and
+   * chained in a sorted order. For preStopMaster()/preShutdown(), coprocessor methods are invoked
+   * in call() and environment is shutdown in postEnvCall(). <br>
+   * Need to execute all coprocessor methods first then postEnvCall(), otherwise some coprocessors
+   * may remain shutdown if any exception occurs during next coprocessor execution which prevent
+   * Master stop or cluster shutdown. (Refer:
+   * <a href="https://issues.apache.org/jira/browse/HBASE-16663">HBASE-16663</a>
+   * @param ctx CoprocessorOperation
+   * @return true if bypaas coprocessor execution, false if not.
+   * @throws IOException
+   */
+  private boolean execShutdown(final CoprocessorOperation ctx) throws IOException {
+    if (ctx == null) return false;
+    boolean bypass = false;
+    List<MasterEnvironment> envs = coprocessors.get();
+    int envsSize = envs.size();
+    // Iterate the coprocessors and execute CoprocessorOperation's call()
+    for (int i = 0; i < envsSize; i++) {
+      MasterEnvironment env = envs.get(i);
+      if (env.getInstance() instanceof MasterObserver) {
+        ctx.prepare(env);
+        Thread currentThread = Thread.currentThread();
+        ClassLoader cl = currentThread.getContextClassLoader();
+        try {
+          currentThread.setContextClassLoader(env.getClassLoader());
+          ctx.call((MasterObserver) env.getInstance(), ctx);
+        } catch (Throwable e) {
+          handleCoprocessorThrowable(env, e);
+        } finally {
+          currentThread.setContextClassLoader(cl);
+        }
+        bypass |= ctx.shouldBypass();
+        if (ctx.shouldComplete()) {
+          break;
+        }
+      }
+    }
+
+    // Iterate the coprocessors and execute CoprocessorOperation's postEnvCall()
+    for (int i = 0; i < envsSize; i++) {
+      MasterEnvironment env = envs.get(i);
+      ctx.postEnvCall(env);
+    }
+    return bypass;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6db4ef84/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerCoprocessorHost.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerCoprocessorHost.java
index 50072c3..bb27dd2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerCoprocessorHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerCoprocessorHost.java
@@ -77,7 +77,9 @@ public class RegionServerCoprocessorHost extends
   }
 
   public void preStop(String message) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+    // While stopping the region server all coprocessors method should be executed first then the
+    // coprocessor should be cleaned up.
+    execShutdown(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override
       public void call(RegionServerObserver oserver,
           ObserverContext<RegionServerCoprocessorEnvironment> ctx) throws IOException {
@@ -270,6 +272,53 @@ public class RegionServerCoprocessorHost extends
   }
 
   /**
+   * RegionServer coprocessor classes can be configured in any order, based on that priority is set
+   * and chained in a sorted order. For preStop(), coprocessor methods are invoked in call() and
+   * environment is shutdown in postEnvCall(). <br>
+   * Need to execute all coprocessor methods first then postEnvCall(), otherwise some coprocessors
+   * may remain shutdown if any exception occurs during next coprocessor execution which prevent
+   * RegionServer stop. (Refer:
+   * <a href="https://issues.apache.org/jira/browse/HBASE-16663">HBASE-16663</a>
+   * @param ctx CoprocessorOperation
+   * @return true if bypaas coprocessor execution, false if not.
+   * @throws IOException
+   */
+  private boolean execShutdown(final CoprocessorOperation ctx) throws IOException {
+    if (ctx == null) return false;
+    boolean bypass = false;
+    List<RegionServerEnvironment> envs = coprocessors.get();
+    int envsSize = envs.size();
+    // Iterate the coprocessors and execute CoprocessorOperation's call()
+    for (int i = 0; i < envsSize; i++) {
+      RegionServerEnvironment env = envs.get(i);
+      if (env.getInstance() instanceof RegionServerObserver) {
+        ctx.prepare(env);
+        Thread currentThread = Thread.currentThread();
+        ClassLoader cl = currentThread.getContextClassLoader();
+        try {
+          currentThread.setContextClassLoader(env.getClassLoader());
+          ctx.call((RegionServerObserver) env.getInstance(), ctx);
+        } catch (Throwable e) {
+          handleCoprocessorThrowable(env, e);
+        } finally {
+          currentThread.setContextClassLoader(cl);
+        }
+        bypass |= ctx.shouldBypass();
+        if (ctx.shouldComplete()) {
+          break;
+        }
+      }
+    }
+
+    // Iterate the coprocessors and execute CoprocessorOperation's postEnvCall()
+    for (int i = 0; i < envsSize; i++) {
+      RegionServerEnvironment env = envs.get(i);
+      ctx.postEnvCall(env);
+    }
+    return bypass;
+  }
+
+  /**
    * Coprocessor environment extension providing access to region server
    * related services.
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/6db4ef84/hbase-server/src/test/java/org/apache/hadoop/hbase/TestJMXConnectorServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestJMXConnectorServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestJMXConnectorServer.java
new file mode 100644
index 0000000..44220f5
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestJMXConnectorServer.java
@@ -0,0 +1,206 @@
+/**
+ *
+ * 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;
+
+import java.io.IOException;
+
+import javax.management.remote.JMXConnector;
+import javax.management.remote.JMXConnectorFactory;
+import javax.naming.ServiceUnavailableException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessorEnvironment;
+import org.apache.hadoop.hbase.security.AccessDeniedException;
+import org.apache.hadoop.hbase.security.access.AccessController;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test case for JMX Connector Server.
+ */
+@Category({ MiscTests.class, MediumTests.class })
+public class TestJMXConnectorServer {
+  private static final Log LOG = LogFactory.getLog(TestJMXConnectorServer.class);
+  private static HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  private static Configuration conf = null;
+  private static Admin admin;
+  // RMI registry port
+  private static int rmiRegistryPort = 61120;
+  // Switch for customized Accesscontroller to throw ACD exception while executing test case
+  static boolean hasAccess;
+
+  @Before
+  public void setUp() throws Exception {
+    UTIL = new HBaseTestingUtility();
+    conf = UTIL.getConfiguration();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    // Set to true while stopping cluster
+    hasAccess = true;
+    admin.close();
+    UTIL.shutdownMiniCluster();
+  }
+
+  /**
+   * This tests to validate the HMaster's ConnectorServer after unauthorised stopMaster call.
+   */
+  @Test(timeout = 180000)
+  public void testHMConnectorServerWhenStopMaster() throws Exception {
+    conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY,
+      JMXListener.class.getName() + "," + MyAccessController.class.getName());
+    conf.setInt("master.rmi.registry.port", rmiRegistryPort);
+    UTIL.startMiniCluster();
+    admin = UTIL.getConnection().getAdmin();
+
+    // try to stop master
+    boolean accessDenied = false;
+    try {
+      hasAccess = false;
+      LOG.info("Stopping HMaster...");
+      admin.stopMaster();
+    } catch (AccessDeniedException e) {
+      LOG.info("Exception occured while stopping HMaster. ", e);
+      accessDenied = true;
+    }
+    Assert.assertTrue(accessDenied);
+
+    // Check whether HMaster JMX Connector server can be connected
+    JMXConnector connector = null;
+    try {
+      connector = JMXConnectorFactory
+          .connect(JMXListener.buildJMXServiceURL(rmiRegistryPort, rmiRegistryPort));
+    } catch (IOException e) {
+      if (e.getCause() instanceof ServiceUnavailableException) {
+        Assert.fail("Can't connect to HMaster ConnectorServer.");
+      }
+    }
+    Assert.assertNotNull("JMXConnector should not be null.", connector);
+    connector.close();
+  }
+
+  /**
+   * This tests to validate the RegionServer's ConnectorServer after unauthorised stopRegionServer
+   * call.
+   */
+  @Test(timeout = 180000)
+  public void testRSConnectorServerWhenStopRegionServer() throws Exception {
+    conf.set(CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY,
+      JMXListener.class.getName() + "," + MyAccessController.class.getName());
+    conf.setInt("regionserver.rmi.registry.port", rmiRegistryPort);
+    UTIL.startMiniCluster();
+    admin = UTIL.getConnection().getAdmin();
+
+    hasAccess = false;
+    ServerName serverName = UTIL.getHBaseCluster().getRegionServer(0).getServerName();
+    LOG.info("Stopping Region Server...");
+    admin.stopRegionServer(serverName.getHostname() + ":" + serverName.getPort());
+
+    // Check whether Region Sever JMX Connector server can be connected
+    JMXConnector connector = null;
+    try {
+      connector = JMXConnectorFactory
+          .connect(JMXListener.buildJMXServiceURL(rmiRegistryPort, rmiRegistryPort));
+    } catch (IOException e) {
+      if (e.getCause() instanceof ServiceUnavailableException) {
+        Assert.fail("Can't connect to Region Server ConnectorServer.");
+      }
+    }
+    Assert.assertNotNull("JMXConnector should not be null.", connector);
+    connector.close();
+  }
+
+  /**
+   * This tests to validate the HMaster's ConnectorServer after unauthorised shutdown call.
+   */
+  @Test(timeout = 180000)
+  public void testHMConnectorServerWhenShutdownCluster() throws Exception {
+    conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY,
+      JMXListener.class.getName() + "," + MyAccessController.class.getName());
+    conf.setInt("master.rmi.registry.port", rmiRegistryPort);
+
+    UTIL.startMiniCluster();
+    admin = UTIL.getConnection().getAdmin();
+
+    boolean accessDenied = false;
+    try {
+      hasAccess = false;
+      LOG.info("Stopping HMaster...");
+      admin.shutdown();
+    } catch (AccessDeniedException e) {
+      LOG.error("Exception occured while stopping HMaster. ", e);
+      accessDenied = true;
+    }
+    Assert.assertTrue(accessDenied);
+
+    // Check whether HMaster JMX Connector server can be connected
+    JMXConnector connector = null;
+    try {
+      connector = JMXConnectorFactory
+          .connect(JMXListener.buildJMXServiceURL(rmiRegistryPort, rmiRegistryPort));
+    } catch (IOException e) {
+      if (e.getCause() instanceof ServiceUnavailableException) {
+        Assert.fail("Can't connect to HMaster ConnectorServer.");
+      }
+    }
+    Assert.assertNotNull("JMXConnector should not be null.", connector);
+    connector.close();
+  }
+
+  /*
+   * Customized class for test case execution which will throw ACD exception while executing
+   * stopMaster/preStopRegionServer/preShutdown explicitly.
+   */
+  public static class MyAccessController extends AccessController {
+    @Override
+    public void preStopMaster(ObserverContext<MasterCoprocessorEnvironment> c) throws IOException {
+      if (!hasAccess) {
+        throw new AccessDeniedException("Insufficient permissions to stop master");
+      }
+    }
+
+    @Override
+    public void preStopRegionServer(ObserverContext<RegionServerCoprocessorEnvironment> ctx)
+        throws IOException {
+      if (!hasAccess) {
+        throw new AccessDeniedException("Insufficient permissions to stop region server.");
+      }
+    }
+
+    @Override
+    public void preShutdown(ObserverContext<MasterCoprocessorEnvironment> c) throws IOException {
+      if (!hasAccess) {
+        throw new AccessDeniedException("Insufficient permissions to shut down cluster.");
+      }
+    }
+  }
+}
\ No newline at end of file


[17/50] [abbrv] hbase git commit: HBASE-16810 HBase Balancer throws ArrayIndexOutOfBoundsException when regionservers are in /hbase/draining znode and unloaded (David Pope)

Posted by la...@apache.org.
HBASE-16810 HBase Balancer throws ArrayIndexOutOfBoundsException when regionservers are in /hbase/draining znode and unloaded (David Pope)


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

Branch: refs/heads/branch-1
Commit: ca581874b9d39cef5d8f142f25ab77fbdef98237
Parents: 27398ea
Author: tedyu <yu...@gmail.com>
Authored: Thu Oct 13 08:42:49 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Thu Oct 13 08:42:49 2016 -0700

----------------------------------------------------------------------
 .../master/balancer/StochasticLoadBalancer.java |   3 +-
 .../hbase/master/MockNoopMasterServices.java    | 292 +++++++++++++++++++
 .../hbase/master/balancer/BalancerTestBase.java |  80 +++++
 .../balancer/TestStochasticLoadBalancer.java    |  16 +
 4 files changed, 389 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ca581874/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
index 181990b..d497d42 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
@@ -42,7 +42,6 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.master.RegionPlan;
-import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster;
 import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster.Action;
 import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster.Action.Type;
 import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster.AssignRegionAction;
@@ -1238,7 +1237,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
         if (index < 0) {
           cost += 1;
         } else {
-          cost += (1 - cluster.getLocalityOfRegion(i, index));
+          cost += (1 - cluster.getLocalityOfRegion(i, serverIndex));
         }
       }
       return scale(0, max, cost);

http://git-wip-us.apache.org/repos/asf/hbase/blob/ca581874/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
new file mode 100644
index 0000000..657d8e2
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
@@ -0,0 +1,292 @@
+/**
+ * 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.master;
+
+import com.google.protobuf.Service;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.*;
+import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.executor.ExecutorService;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
+import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+
+import java.io.IOException;
+import java.util.List;
+
+public class MockNoopMasterServices implements MasterServices, Server {
+  @Override
+  public SnapshotManager getSnapshotManager() {
+    return null;
+  }
+
+  @Override
+  public MasterProcedureManagerHost getMasterProcedureManagerHost() {
+    return null;
+  }
+
+  @Override
+  public AssignmentManager getAssignmentManager() {
+    return null;
+  }
+
+  @Override
+  public MasterFileSystem getMasterFileSystem() {
+    return null;
+  }
+
+  @Override
+  public ServerManager getServerManager() {
+    return null;
+  }
+
+  @Override
+  public ExecutorService getExecutorService() {
+    return null;
+  }
+
+  @Override
+  public TableLockManager getTableLockManager() {
+    return null;
+  }
+
+  @Override
+  public MasterCoprocessorHost getMasterCoprocessorHost() {
+    return null;
+  }
+
+  @Override
+  public TableNamespaceManager getTableNamespaceManager() {
+    return null;
+  }
+
+  @Override
+  public MasterQuotaManager getMasterQuotaManager() {
+    return null;
+  }
+
+  @Override
+  public ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
+    return null;
+  }
+
+  @Override
+  public void checkTableModifiable(TableName tableName) throws IOException, TableNotFoundException, TableNotDisabledException {
+
+  }
+
+  @Override
+  public long createTable(HTableDescriptor desc, byte[][] splitKeys, long nonceGroup, long nonce) throws IOException {
+    return 0;
+  }
+
+  @Override
+  public long createSystemTable(HTableDescriptor hTableDescriptor) throws IOException {
+    return 0;
+  }
+
+  @Override
+  public long deleteTable(TableName tableName, long nonceGroup, long nonce) throws IOException {
+    return 0;
+  }
+
+  @Override
+  public void truncateTable(TableName tableName, boolean preserveSplits, long nonceGroup, long nonce) throws IOException {
+
+  }
+
+  @Override
+  public void modifyTable(TableName tableName, HTableDescriptor descriptor, long nonceGroup, long nonce) throws IOException {
+
+  }
+
+  @Override
+  public long enableTable(TableName tableName, long nonceGroup, long nonce) throws IOException {
+    return 0;
+  }
+
+  @Override
+  public long disableTable(TableName tableName, long nonceGroup, long nonce) throws IOException {
+    return 0;
+  }
+
+  @Override
+  public void addColumn(TableName tableName, HColumnDescriptor column, long nonceGroup, long nonce) throws IOException {
+
+  }
+
+  @Override
+  public void modifyColumn(TableName tableName, HColumnDescriptor descriptor, long nonceGroup, long nonce) throws IOException {
+
+  }
+
+  @Override
+  public void deleteColumn(TableName tableName, byte[] columnName, long nonceGroup, long nonce) throws IOException {
+
+  }
+
+  @Override
+  public TableDescriptors getTableDescriptors() {
+    return null;
+  }
+
+  @Override
+  public boolean isServerCrashProcessingEnabled() {
+    return false;
+  }
+
+  @Override
+  public boolean registerService(Service instance) {
+    return false;
+  }
+
+  @Override
+  public void dispatchMergingRegions(HRegionInfo region_a, HRegionInfo region_b, boolean forcible, User user) throws IOException {
+
+  }
+
+  @Override
+  public boolean isInitialized() {
+    return false;
+  }
+
+  @Override
+  public void createNamespace(NamespaceDescriptor descriptor, long nonceGroup, long nonce) throws IOException {
+
+  }
+
+  @Override
+  public void createNamespaceSync(NamespaceDescriptor descriptor, long nonceGroup, long nonce) throws IOException {
+
+  }
+
+  @Override
+  public void modifyNamespace(NamespaceDescriptor descriptor, long nonceGroup, long nonce) throws IOException {
+
+  }
+
+  @Override
+  public void deleteNamespace(String name, long nonceGroup, long nonce) throws IOException {
+
+  }
+
+  @Override
+  public boolean isInMaintenanceMode() {
+    return false;
+  }
+
+  @Override
+  public boolean abortProcedure(long procId, boolean mayInterruptIfRunning) throws IOException {
+    return false;
+  }
+
+  @Override
+  public List<ProcedureInfo> listProcedures() throws IOException {
+    return null;
+  }
+
+  @Override
+  public NamespaceDescriptor getNamespaceDescriptor(String name) throws IOException {
+    return null;
+  }
+
+  @Override
+  public List<NamespaceDescriptor> listNamespaceDescriptors() throws IOException {
+    return null;
+  }
+
+  @Override
+  public List<HTableDescriptor> listTableDescriptorsByNamespace(String name) throws IOException {
+    return null;
+  }
+
+  @Override
+  public List<TableName> listTableNamesByNamespace(String name) throws IOException {
+    return null;
+  }
+
+  @Override
+  public long getLastMajorCompactionTimestamp(TableName table) throws IOException {
+    return 0;
+  }
+
+  @Override
+  public long getLastMajorCompactionTimestampForRegion(byte[] regionName) throws IOException {
+    return 0;
+  }
+
+  @Override
+  public Configuration getConfiguration() {
+    return null;
+  }
+
+  @Override
+  public ZooKeeperWatcher getZooKeeper() {
+    return null;
+  }
+
+  @Override
+  public ClusterConnection getConnection() {
+    return null;
+  }
+
+  @Override
+  public MetaTableLocator getMetaTableLocator() {
+    return null;
+  }
+
+  @Override
+  public ServerName getServerName() {
+    return null;
+  }
+
+  @Override
+  public CoordinatedStateManager getCoordinatedStateManager() {
+    return null;
+  }
+
+  @Override
+  public ChoreService getChoreService() {
+    return null;
+  }
+
+  @Override
+  public void abort(String why, Throwable e) {
+
+  }
+
+  @Override
+  public boolean isAborted() {
+    return false;
+  }
+
+  @Override
+  public void stop(String why) {
+
+  }
+
+  @Override
+  public boolean isStopped() {
+    return false;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ca581874/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase.java
index 7ae0133..5746e3f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedList;
@@ -150,6 +151,85 @@ public class BalancerTestBase {
 
   };
 
+  /**
+   * Data set for testLocalityCost:
+   *
+   * [test][regions][0] = [serverIndex] -> number of regions
+   * [test][regions][regionIndex+1] = {server hosting region, locality percentage, datanodes}
+   *
+   * For each [test], there is a list of cluster config information grouped by [regions].
+   * - [0] - the first element of the [regions] list is a list of servers with the value
+   *         indicating the number of regions it hosts.
+   * - [regionIndex+1] - the remaining elements of the array are regions, where the index value
+   *         is 1 greater than the regionIndex.  This element holds an array that identifies:
+   *     [0] - the serverIndex of the server hosting this region
+   *     [1] - the locality percentage returned by getLocalityOfRegion(region, server) when the
+   *           server is hosting both region and the hdfs blocks.
+   *     [.] - the serverIndex of servers hosting the hdfs blocks, where a value of -1 indicates
+   *         a dfs server not in the list of region servers.
+   */
+  protected int[][][] clusterRegionLocationMocks = new int[][][]{
+      // Test 1: Basic region placement with 1 region server not hosting dfs block
+      //     Locality Calculation:
+      //        region[0] = 1 - 80/100 = (.2)  - server[2] hosts both the region and dfs blocks
+      //        region[1] = 1.0                - server[0] only hosts the region, not dfs blocks
+      //        region[2] = 1 - 70/100 = (.3)  - server[1] hosts both the region and dfs blocks
+      //
+      //      RESULT = 0.2 + 1.0 + 0.3 / 3.0 (3.0 is max value)
+      //             = 1.5 / 3.0
+      //             = 0.5
+      new int[][]{
+          new int[]{1, 1, 1},         // 3 region servers with 1 region each
+          new int[]{2, 80, 1, 2, 0},  // region[0] on server[2] w/ 80% locality
+          new int[]{0, 50, 1, 2},     // region[1] on server[0] w/ 50% , but no local dfs blocks
+          new int[]{1, 70, 2, 0, 1},  // region[2] on server[1] w/ 70% locality
+      },
+
+      // Test 2: Sames as Test 1, but the last region has a datanode that isn't a region server
+      new int[][]{
+          new int[]{1, 1, 1},
+          new int[]{2, 80, 1, 2, 0},
+          new int[]{0, 50, 1, 2},
+          new int[]{1, 70, -1, 2, 0, 1},  // the first region location is not on a region server
+      },
+  };
+
+  // This mock allows us to test the LocalityCostFunction
+  protected class MockCluster extends BaseLoadBalancer.Cluster {
+
+    protected int[][] localityValue = null;   // [region][server] = percent of blocks
+
+    protected MockCluster(int[][] regions) {
+
+      // regions[0] is an array where index = serverIndex an value = number of regions
+      super(mockClusterServers(regions[0], 1), null, null, null);
+
+      localityValue = new int[regions.length-1][];
+      // the remaining elements in the regions array contain values for:
+      //   [0] - the serverIndex of the server hosting this region
+      //   [1] - the locality percentage (in whole numbers) for the hosting region server
+      //   [.] - a list of servers hosting dfs blocks for the region (-1 means its not one
+      //         of our region servers.
+      for (int i = 1; i < regions.length; i++){
+        int regionIndex = i - 1;
+        int serverIndex = regions[i][0];
+        int locality = regions[i][1];
+        int[] locations = Arrays.copyOfRange(regions[i], 2, regions[i].length);
+
+        regionIndexToServerIndex[regionIndex] = serverIndex;
+        localityValue[regionIndex] = new int[servers.length];
+        localityValue[regionIndex][serverIndex] = (locality > 100)? locality % 100 : locality;
+        regionLocations[regionIndex] = locations;
+      }
+    }
+
+    @Override
+    float getLocalityOfRegion(int region, int server) {
+      // convert the locality percentage to a fraction
+      return localityValue[region][server] / 100.0f;
+    }
+  }
+
   // This class is introduced because IP to rack resolution can be lengthy.
   public static class MockMapping implements DNSToSwitchMapping {
     public MockMapping(Configuration conf) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/ca581874/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer.java
index 9caf264..094687b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.RegionLoad;
 import org.apache.hadoop.hbase.ServerLoad;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.master.MockNoopMasterServices;
 import org.apache.hadoop.hbase.master.RackManager;
 import org.apache.hadoop.hbase.master.RegionPlan;
 import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster;
@@ -133,7 +134,22 @@ public class TestStochasticLoadBalancer extends BalancerTestBase {
         returnServer(entry.getKey());
       }
     }
+  }
+
+  @Test
+  public void testLocalityCost() throws Exception {
+    Configuration conf = HBaseConfiguration.create();
+    MockNoopMasterServices master = new MockNoopMasterServices();
+    StochasticLoadBalancer.CostFunction
+        costFunction = new StochasticLoadBalancer.LocalityCostFunction(conf, master);
 
+    for (int[][] clusterRegionLocations : clusterRegionLocationMocks) {
+      MockCluster cluster = new MockCluster(clusterRegionLocations);
+      costFunction.init(cluster);
+      double cost = costFunction.cost();
+
+      assertEquals(0.5f, cost, 0.001);
+    }
   }
 
   @Test


[46/50] [abbrv] hbase git commit: HBASE-16743 TestSimpleRpcScheduler#testCoDelScheduling is broke

Posted by la...@apache.org.
HBASE-16743 TestSimpleRpcScheduler#testCoDelScheduling is broke


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

Branch: refs/heads/branch-1
Commit: ea5b0a05d692cfaccc004d030df9d81fb4f7f47e
Parents: 0f158ed
Author: Mikhail Antonov <an...@apache.org>
Authored: Fri Oct 28 16:26:48 2016 -0700
Committer: Mikhail Antonov <an...@apache.org>
Committed: Fri Oct 28 16:32:58 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java  | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ea5b0a05/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java
index 514e42d..f93f250 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java
@@ -410,7 +410,8 @@ public class TestSimpleRpcScheduler {
   @Test
   public void testCoDelScheduling() throws Exception {
     CoDelEnvironmentEdge envEdge = new CoDelEnvironmentEdge();
-    envEdge.threadNamePrefixs.add("RpcServer.CodelBQ.default.handler");
+    envEdge.threadNamePrefixs.add("RpcServer.CodelFPBQ.default.handler");
+    envEdge.threadNamePrefixs.add("RpcServer.CodelRWQ.default.handler");
     Configuration schedConf = HBaseConfiguration.create();
     schedConf.setInt(RpcScheduler.IPC_SERVER_MAX_CALLQUEUE_LENGTH, 250);
 


[27/50] [abbrv] hbase git commit: HBASE-16824 Writer.flush() can be called on already closed streams in WAL roll

Posted by la...@apache.org.
HBASE-16824 Writer.flush() can be called on already closed streams in WAL roll


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

Branch: refs/heads/branch-1
Commit: 019c7f9303a7242b7c5d6713bed414b180b5c84a
Parents: 6694191
Author: Enis Soztutar <en...@apache.org>
Authored: Tue Oct 18 18:46:02 2016 -0700
Committer: Enis Soztutar <en...@apache.org>
Committed: Tue Oct 18 19:14:20 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/wal/FSHLog.java   | 31 ++++++++++++--
 .../wal/TestLogRollingNoCluster.java            | 43 ++++++++++++++------
 2 files changed, 57 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/019c7f93/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
index 11ebfef..a8b0372 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
@@ -1138,6 +1138,7 @@ public class FSHLog implements WAL {
     private volatile long sequence;
     // Keep around last exception thrown. Clear on successful sync.
     private final BlockingQueue<SyncFuture> syncFutures;
+    private volatile SyncFuture takeSyncFuture = null;
 
     /**
      * UPDATE!
@@ -1227,13 +1228,21 @@ public class FSHLog implements WAL {
       return sequence;
     }
 
+    boolean areSyncFuturesReleased() {
+      // check whether there is no sync futures offered, and no in-flight sync futures that is being
+      // processed.
+      return syncFutures.size() <= 0
+          && takeSyncFuture == null;
+    }
+
     public void run() {
       long currentSequence;
       while (!isInterrupted()) {
         int syncCount = 0;
-        SyncFuture takeSyncFuture;
+
         try {
           while (true) {
+            takeSyncFuture = null;
             // We have to process what we 'take' from the queue
             takeSyncFuture = this.syncFutures.take();
             currentSequence = this.sequence;
@@ -1744,9 +1753,21 @@ public class FSHLog implements WAL {
      * @return True if outstanding sync futures still
      */
     private boolean isOutstandingSyncs() {
+      // Look at SyncFutures in the EventHandler
       for (int i = 0; i < this.syncFuturesCount; i++) {
         if (!this.syncFutures[i].isDone()) return true;
       }
+
+      return false;
+    }
+
+    private boolean isOutstandingSyncsFromRunners() {
+      // Look at SyncFutures in the SyncRunners
+      for (SyncRunner syncRunner: syncRunners) {
+        if(syncRunner.isAlive() && !syncRunner.areSyncFuturesReleased()) {
+          return true;
+        }
+      }
       return false;
     }
 
@@ -1857,11 +1878,13 @@ public class FSHLog implements WAL {
         // Wait on outstanding syncers; wait for them to finish syncing (unless we've been
         // shutdown or unless our latch has been thrown because we have been aborted or unless
         // this WAL is broken and we can't get a sync/append to complete).
-        while (!this.shutdown && this.zigzagLatch.isCocked() &&
-            highestSyncedSequence.get() < currentSequence &&
+        while ((!this.shutdown && this.zigzagLatch.isCocked()
+            && highestSyncedSequence.get() < currentSequence &&
             // We could be in here and all syncs are failing or failed. Check for this. Otherwise
             // we'll just be stuck here for ever. In other words, ensure there syncs running.
-            isOutstandingSyncs()) {
+            isOutstandingSyncs())
+            // Wait for all SyncRunners to finish their work so that we can replace the writer
+            || isOutstandingSyncsFromRunners()) {
           synchronized (this.safePointWaiter) {
             this.safePointWaiter.wait(0, 1);
           }

http://git-wip-us.apache.org/repos/asf/hbase/blob/019c7f93/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java
index 7ce3615..bca4a7e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java
@@ -18,9 +18,9 @@
 package org.apache.hadoop.hbase.regionserver.wal;
 
 import static org.junit.Assert.assertFalse;
-
 import java.io.IOException;
-
+import java.util.TreeMap;
+import java.util.concurrent.ThreadLocalRandom;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALFactory;
 import org.apache.hadoop.hbase.wal.WALKey;
@@ -49,7 +50,18 @@ import org.junit.experimental.categories.Category;
 public class TestLogRollingNoCluster {
   private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private final static byte [] EMPTY_1K_ARRAY = new byte[1024];
-  private static final int THREAD_COUNT = 100; // Spin up this many threads
+  private static final int NUM_THREADS = 100; // Spin up this many threads
+  private static final int NUM_ENTRIES = 100; // How many entries to write
+
+  /** ProtobufLogWriter that simulates higher latencies in sync() call */
+  public static class HighLatencySyncWriter extends  ProtobufLogWriter {
+    @Override
+    public void sync() throws IOException {
+      Threads.sleep(ThreadLocalRandom.current().nextInt(10));
+      super.sync();
+      Threads.sleep(ThreadLocalRandom.current().nextInt(10));
+    }
+  }
 
   /**
    * Spin up a bunch of threads and have them all append to a WAL.  Roll the
@@ -58,37 +70,41 @@ public class TestLogRollingNoCluster {
    * @throws InterruptedException
    */
   @Test
-  public void testContendedLogRolling() throws IOException, InterruptedException {
-    Path dir = TEST_UTIL.getDataTestDir();
+  public void testContendedLogRolling() throws Exception {
+    TEST_UTIL.startMiniDFSCluster(3);
+    Path dir = TEST_UTIL.getDataTestDirOnTestFS();
+
     // The implementation needs to know the 'handler' count.
-    TEST_UTIL.getConfiguration().setInt(HConstants.REGION_SERVER_HANDLER_COUNT, THREAD_COUNT);
+    TEST_UTIL.getConfiguration().setInt(HConstants.REGION_SERVER_HANDLER_COUNT, NUM_THREADS);
     final Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
     FSUtils.setRootDir(conf, dir);
+    conf.set("hbase.regionserver.hlog.writer.impl", HighLatencySyncWriter.class.getName());
     final WALFactory wals = new WALFactory(conf, null, TestLogRollingNoCluster.class.getName());
     final WAL wal = wals.getWAL(new byte[]{}, null);
     
     Appender [] appenders = null;
 
-    final int count = THREAD_COUNT;
-    appenders = new Appender[count];
+    final int numThreads = NUM_THREADS;
+    appenders = new Appender[numThreads];
     try {
-      for (int i = 0; i < count; i++) {
+      for (int i = 0; i < numThreads; i++) {
         // Have each appending thread write 'count' entries
-        appenders[i] = new Appender(wal, i, count);
+        appenders[i] = new Appender(wal, i, NUM_ENTRIES);
       }
-      for (int i = 0; i < count; i++) {
+      for (int i = 0; i < numThreads; i++) {
         appenders[i].start();
       }
-      for (int i = 0; i < count; i++) {
+      for (int i = 0; i < numThreads; i++) {
         //ensure that all threads are joined before closing the wal
         appenders[i].join();
       }
     } finally {
       wals.close();
     }
-    for (int i = 0; i < count; i++) {
+    for (int i = 0; i < numThreads; i++) {
       assertFalse(appenders[i].isException());
     }
+    TEST_UTIL.shutdownMiniDFSCluster();
   }
 
   /**
@@ -137,6 +153,7 @@ public class TestLogRollingNoCluster {
           final HTableDescriptor htd = fts.get(TableName.META_TABLE_NAME);
           final long txid = wal.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(),
               TableName.META_TABLE_NAME, now, mvcc), edit, true);
+          Threads.sleep(ThreadLocalRandom.current().nextInt(5));
           wal.sync(txid);
         }
         String msg = getName() + " finished";


[04/50] [abbrv] hbase git commit: HBASE-16661 Add last major compaction age to per-region metrics

Posted by la...@apache.org.
HBASE-16661 Add last major compaction age to per-region metrics

Signed-off-by: Gary Helmling <ga...@apache.org>


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

Branch: refs/heads/branch-1
Commit: 59ca4dad70cee46314c992766fd9303d1e41ee2c
Parents: 66038b8
Author: Dustin Pho <ph...@fb.com>
Authored: Sat Sep 24 14:58:37 2016 -0700
Committer: Gary Helmling <ga...@apache.org>
Committed: Mon Oct 10 15:21:53 2016 -0700

----------------------------------------------------------------------
 .../hbase/regionserver/MetricsRegionSource.java    |  2 ++
 .../hbase/regionserver/MetricsRegionWrapper.java   |  5 +++++
 .../regionserver/MetricsRegionSourceImpl.java      |  4 ++++
 .../regionserver/TestMetricsRegionSourceImpl.java  |  5 +++++
 .../apache/hadoop/hbase/regionserver/HRegion.java  |  9 +++------
 .../regionserver/MetricsRegionWrapperImpl.java     | 17 +++++++++++++++++
 .../apache/hadoop/hbase/regionserver/Region.java   |  4 ++--
 .../regionserver/MetricsRegionWrapperStub.java     |  5 +++++
 8 files changed, 43 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/59ca4dad/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSource.java
index 911c757..12ef07c 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSource.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSource.java
@@ -29,10 +29,12 @@ public interface MetricsRegionSource extends Comparable<MetricsRegionSource> {
   String SIZE_VALUE_NAME = "size";
   String COMPACTIONS_COMPLETED_COUNT = "compactionsCompletedCount";
   String COMPACTIONS_FAILED_COUNT = "compactionsFailedCount";
+  String LAST_MAJOR_COMPACTION_AGE = "lastMajorCompactionAge";
   String NUM_BYTES_COMPACTED_COUNT = "numBytesCompactedCount";
   String NUM_FILES_COMPACTED_COUNT = "numFilesCompactedCount";
   String COMPACTIONS_COMPLETED_DESC = "Number of compactions that have completed.";
   String COMPACTIONS_FAILED_DESC = "Number of compactions that have failed.";
+  String LAST_MAJOR_COMPACTION_DESC = "Age of the last major compaction in milliseconds.";
   String  NUM_BYTES_COMPACTED_DESC =
       "Sum of filesize on all files entering a finished, successful or aborted, compaction";
   String NUM_FILES_COMPACTED_DESC =

http://git-wip-us.apache.org/repos/asf/hbase/blob/59ca4dad/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapper.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapper.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapper.java
index 0482d2a..9b7acd3 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapper.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapper.java
@@ -101,6 +101,11 @@ public interface MetricsRegionWrapper {
   long getNumCompactionsCompleted();
 
   /**
+   *  @return Age of the last major compaction
+   */
+  long getLastMajorCompactionAge();
+
+  /**
    * Returns the total number of compactions that have been reported as failed on this region.
    * Note that a given compaction can be reported as both completed and failed if an exception
    * is thrown in the processing after {@code HRegion.compact()}.

http://git-wip-us.apache.org/repos/asf/hbase/blob/59ca4dad/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java
index ae579cf..16f36bb 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java
@@ -269,6 +269,10 @@ public class MetricsRegionSourceImpl implements MetricsRegionSource {
           MetricsRegionSource.COMPACTIONS_FAILED_DESC),
           this.regionWrapper.getNumCompactionsFailed());
       mrb.addCounter(Interns.info(
+              regionNamePrefix + MetricsRegionSource.LAST_MAJOR_COMPACTION_AGE,
+              MetricsRegionSource.LAST_MAJOR_COMPACTION_DESC),
+          this.regionWrapper.getLastMajorCompactionAge());
+      mrb.addCounter(Interns.info(
               regionNamePrefix + MetricsRegionSource.NUM_BYTES_COMPACTED_COUNT,
               MetricsRegionSource.NUM_BYTES_COMPACTED_DESC),
           this.regionWrapper.getNumBytesCompacted());

http://git-wip-us.apache.org/repos/asf/hbase/blob/59ca4dad/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java
index 7498a74..e1cb6b8 100644
--- a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java
+++ b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java
@@ -144,6 +144,11 @@ public class TestMetricsRegionSourceImpl {
     }
 
     @Override
+    public long getLastMajorCompactionAge() {
+      return 0;
+    }
+
+    @Override
     public long getNumCompactionsCompleted() {
       return 0;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/59ca4dad/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index c399f4a..c18564d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -1663,7 +1663,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   }
 
   @Override
-  public long getOldestHfileTs(boolean majorCompactioOnly) throws IOException {
+  public long getOldestHfileTs(boolean majorCompactionOnly) throws IOException {
     long result = Long.MAX_VALUE;
     for (Store store : getStores()) {
       Collection<StoreFile> storeFiles = store.getStorefiles();
@@ -1673,12 +1673,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         if (sfReader == null) continue;
         HFile.Reader reader = sfReader.getHFileReader();
         if (reader == null) continue;
-        if (majorCompactioOnly) {
+        if (majorCompactionOnly) {
           byte[] val = reader.loadFileInfo().get(StoreFile.MAJOR_COMPACTION_KEY);
-          if (val == null) continue;
-          if (val == null || !Bytes.toBoolean(val)) {
-            continue;
-          }
+          if (val == null || !Bytes.toBoolean(val)) continue;
         }
         result = Math.min(result, reader.getFileContext().getFileCreateTime());
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/59ca4dad/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java
index 46e13f7..493ee18 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java
@@ -25,15 +25,20 @@ import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.metrics2.MetricsExecutor;
 
 @InterfaceAudience.Private
 public class MetricsRegionWrapperImpl implements MetricsRegionWrapper, Closeable {
 
+  private static final Log LOG = LogFactory.getLog(MetricsRegionWrapperImpl.class);
+
   public static final int PERIOD = 45;
   public static final String UNKNOWN = "unknown";
 
@@ -136,6 +141,18 @@ public class MetricsRegionWrapperImpl implements MetricsRegionWrapper, Closeable
   }
 
   @Override
+  public long getLastMajorCompactionAge() {
+    long lastMajorCompactionTs = 0L;
+    try {
+      lastMajorCompactionTs = this.region.getOldestHfileTs(true);
+    } catch (IOException ioe) {
+      LOG.error("Could not load HFile info ", ioe);
+    }
+    long now = EnvironmentEdgeManager.currentTime();
+    return now - lastMajorCompactionTs;
+  }
+
+  @Override
   public long getNumCompactionsFailed() {
     return this.region.compactionsFailed.get();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/59ca4dad/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
index 8827967..c763ac0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
@@ -141,10 +141,10 @@ public interface Region extends ConfigurationObserver {
 
   /**
    * This can be used to determine the last time all files of this region were major compacted.
-   * @param majorCompactioOnly Only consider HFile that are the result of major compaction
+   * @param majorCompactionOnly Only consider HFile that are the result of major compaction
    * @return the timestamp of the oldest HFile for all stores of this region
    */
-  long getOldestHfileTs(boolean majorCompactioOnly) throws IOException;
+  long getOldestHfileTs(boolean majorCompactionOnly) throws IOException;
 
   /**
    * @return map of column family names to max sequence id that was read from storage when this

http://git-wip-us.apache.org/repos/asf/hbase/blob/59ca4dad/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperStub.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperStub.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperStub.java
index ab340d9..8ae1180 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperStub.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperStub.java
@@ -116,6 +116,11 @@ public class MetricsRegionWrapperStub implements MetricsRegionWrapper {
   }
 
   @Override
+  public long getLastMajorCompactionAge() {
+    return 0;
+  }
+
+  @Override
   public long getNumCompactionsFailed() {
     return 0;
   }


[14/50] [abbrv] hbase git commit: HBASE-16716 OfflineMetaRepair leaves empty directory inside /hbase/WALs which remains forever (Pankaj Kumar)

Posted by la...@apache.org.
HBASE-16716 OfflineMetaRepair leaves empty directory inside /hbase/WALs which remains forever (Pankaj Kumar)


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

Branch: refs/heads/branch-1
Commit: e2278f9544380ec7abc92c3592bbe2068e62cb45
Parents: a3c29da
Author: tedyu <yu...@gmail.com>
Authored: Wed Oct 12 19:42:18 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Wed Oct 12 19:42:18 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/util/HBaseFsck.java | 21 ++++++++++++++++
 .../util/hbck/TestOfflineMetaRebuildBase.java   | 26 ++++++++++++++++++++
 2 files changed, 47 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e2278f95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
index 745adbf..458230f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
@@ -115,6 +115,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
+import org.apache.hadoop.hbase.regionserver.wal.FSHLog;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.util.Bytes.ByteArrayComparator;
@@ -1524,12 +1525,32 @@ public class HBaseFsck extends Configured implements Closeable {
     }
     meta.batchMutate(puts.toArray(new Put[puts.size()]));
     HRegion.closeHRegion(meta);
+    // clean up the temporary hbck meta recovery WAL directory
+    removeHBCKMetaRecoveryWALDir(meta);
     LOG.info("Success! hbase:meta table rebuilt.");
     LOG.info("Old hbase:meta is moved into " + backupDir);
     return true;
   }
 
   /**
+   * Removes the empty Meta recovery WAL directory.
+   * @param meta Meta region
+   */
+  private void removeHBCKMetaRecoveryWALDir(HRegion meta) throws IOException {
+    // TODO Since HBASE-11983 not available in this branch, so we don't know the walFactoryId.
+    // Retrieving WAL directory
+    Path walLogDir = ((FSHLog) meta.getWAL()).getCurrentFileName().getParent();
+    FileSystem fs = FSUtils.getCurrentFileSystem(getConf());
+    FileStatus[] walFiles = FSUtils.listStatus(fs, walLogDir, null);
+    if (walFiles == null || walFiles.length == 0) {
+      LOG.info("HBCK meta recovery WAL directory is empty, removing it now.");
+      if (!FSUtils.deleteDirectory(fs, walLogDir)) {
+        LOG.warn("Couldn't clear the HBCK Meta recovery WAL directory " + walLogDir);
+      }
+    }
+  }
+
+  /**
    * Log an appropriate message about whether or not overlapping merges are computed in parallel.
    */
   private void logParallelMerge() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/e2278f95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildBase.java
index 43b1444..b31e20e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildBase.java
@@ -20,17 +20,24 @@ package org.apache.hadoop.hbase.util.hbck;
 import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.assertErrors;
 import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.doFsck;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
+import java.io.IOException;
 import java.util.Arrays;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HBaseFsck;
 import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE;
 import org.junit.Test;
@@ -65,6 +72,7 @@ public class TestOfflineMetaRebuildBase extends OfflineMetaRebuildTestCore {
     // rebuild meta table from scratch
     HBaseFsck fsck = new HBaseFsck(conf);
     assertTrue(fsck.rebuildMeta(false));
+    assertTrue("HBCK meta recovery WAL directory exist.", validateHBCKMetaRecoveryWALDir());
 
     // bring up the minicluster
     TEST_UTIL.startMiniZKCluster();
@@ -87,4 +95,22 @@ public class TestOfflineMetaRebuildBase extends OfflineMetaRebuildTestCore {
     LOG.info("Table " + table + " has " + tableRowCount(conf, table) + " entries.");
     assertEquals(16, tableRowCount(conf, table));
   }
+
+  /**
+   * Validate whether Meta recovery empty WAL directory is removed.
+   * @return True if directory is removed otherwise false.
+   */
+  private boolean validateHBCKMetaRecoveryWALDir() throws IOException {
+    Path rootdir = FSUtils.getRootDir(TEST_UTIL.getConfiguration());
+    Path walLogDir = new Path(rootdir, HConstants.HREGION_LOGDIR_NAME);
+    FileSystem fs = TEST_UTIL.getTestFileSystem();
+    FileStatus[] walFiles = FSUtils.listStatus(fs, walLogDir, null);
+    assertNotNull(walFiles);
+    for (FileStatus fsStat : walFiles) {
+      if (fsStat.isDirectory() && fsStat.getPath().getName().startsWith("hregion-")) {
+        return false;
+      }
+    }
+    return true;
+  }
 }


[42/50] [abbrv] hbase git commit: HBASE-16948 Fix inconsistency between HRegion and Region javadoc on getRowLock

Posted by la...@apache.org.
HBASE-16948 Fix inconsistency between HRegion and Region javadoc on getRowLock


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

Branch: refs/heads/branch-1
Commit: 24a92ed63a2e483d43cf66f220c666c581b33484
Parents: 4edd8a6
Author: Michael Stack <st...@apache.org>
Authored: Wed Oct 26 16:09:43 2016 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Wed Oct 26 16:11:26 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/HRegion.java      | 10 --------
 .../hadoop/hbase/regionserver/Region.java       | 26 +++++++++++++-------
 2 files changed, 17 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/24a92ed6/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index ca37eb1..2757eae 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -5310,16 +5310,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     return getRowLock(row, false);
   }
 
-  /**
-   *
-   * Get a row lock for the specified row. All locks are reentrant.
-   *
-   * Before calling this function make sure that a region operation has already been
-   * started (the calling thread has already acquired the region-close-guard lock).
-   * @param row The row actions will be performed against
-   * @param readLock is the lock reader or writer. True indicates that a non-exlcusive
-   *                 lock is requested
-   */
   @Override
   public RowLock getRowLock(byte[] row, boolean readLock) throws IOException {
     // Make sure the row is inside of this region before getting the lock for it.

http://git-wip-us.apache.org/repos/asf/hbase/blob/24a92ed6/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
index c763ac0..35e2b35 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
@@ -269,15 +269,23 @@ public interface Region extends ConfigurationObserver {
   }
 
   /**
-   * Tries to acquire a lock on the given row.
-   * @param waitForLock if true, will block until the lock is available.
-   *        Otherwise, just tries to obtain the lock and returns
-   *        false if unavailable.
-   * @return the row lock if acquired,
-   *   null if waitForLock was false and the lock was not acquired
-   * @throws IOException if waitForLock was true and the lock could not be acquired after waiting
-   */
-  RowLock getRowLock(byte[] row, boolean waitForLock) throws IOException;
+   *
+   * Get a row lock for the specified row. All locks are reentrant.
+   *
+   * Before calling this function make sure that a region operation has already been
+   * started (the calling thread has already acquired the region-close-guard lock).
+   * 
+   * NOTE: the boolean passed here has changed. It used to be a boolean that
+   * stated whether or not to wait on the lock. Now it is whether it an exclusive
+   * lock is requested.
+   * 
+   * @param row The row actions will be performed against
+   * @param readLock is the lock reader or writer. True indicates that a non-exclusive
+   * lock is requested
+   * @see #startRegionOperation()
+   * @see #startRegionOperation(Operation)
+   */
+  RowLock getRowLock(byte[] row, boolean readLock) throws IOException;
 
   /**
    * If the given list of row locks is not null, releases all locks.


[34/50] [abbrv] hbase git commit: HBASE-16829 DemoClient should detect secure mode

Posted by la...@apache.org.
HBASE-16829 DemoClient should detect secure mode


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

Branch: refs/heads/branch-1
Commit: 9fda0dd09820de895313b9b06fc020e7f758c78f
Parents: 42e7a4a
Author: tedyu <yu...@gmail.com>
Authored: Fri Oct 21 16:36:50 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Fri Oct 21 16:36:50 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/thrift2/DemoClient.java   | 14 +++++++++++++-
 1 file changed, 13 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/9fda0dd0/hbase-examples/src/main/java/org/apache/hadoop/hbase/thrift2/DemoClient.java
----------------------------------------------------------------------
diff --git a/hbase-examples/src/main/java/org/apache/hadoop/hbase/thrift2/DemoClient.java b/hbase-examples/src/main/java/org/apache/hadoop/hbase/thrift2/DemoClient.java
index 72c6929..e96688e 100644
--- a/hbase-examples/src/main/java/org/apache/hadoop/hbase/thrift2/DemoClient.java
+++ b/hbase-examples/src/main/java/org/apache/hadoop/hbase/thrift2/DemoClient.java
@@ -31,6 +31,8 @@ import javax.security.auth.login.Configuration;
 import javax.security.auth.login.LoginContext;
 import javax.security.sasl.Sasl;
 
+import org.apache.hadoop.hbase.HBaseConfiguration;
+
 import org.apache.hadoop.hbase.thrift2.generated.TColumnValue;
 import org.apache.hadoop.hbase.thrift2.generated.TGet;
 import org.apache.hadoop.hbase.thrift2.generated.THBaseService;
@@ -48,6 +50,7 @@ public class DemoClient {
   private static String host = "localhost";
   private static int port = 9090;
   private static boolean secure = false;
+  private static String user = null;
 
   public static void main(String[] args) throws Exception {
     System.out.println("Thrift2 Demo");
@@ -61,6 +64,15 @@ public class DemoClient {
     if (args.length >= 2) {
       port = Integer.parseInt(args[1]);
     }
+    org.apache.hadoop.conf.Configuration conf = HBaseConfiguration.create();
+    String principal = conf.get("hbase.thrift.kerberos.principal");
+    if (principal != null) {
+      secure = true;
+      int slashIdx = principal.indexOf("/");
+      int atIdx = principal.indexOf("@");
+      int idx = slashIdx != -1 ? slashIdx : atIdx != -1 ? atIdx : principal.length();
+      user = principal.substring(0, idx);
+    }
     if (args.length >= 3) {
       secure = Boolean.parseBoolean(args[2]);
     }
@@ -93,7 +105,7 @@ public class DemoClient {
       Map<String, String> saslProperties = new HashMap<String, String>();
       saslProperties.put(Sasl.QOP, "auth-conf,auth-int,auth");
       transport = new TSaslClientTransport("GSSAPI", null,
-        "hbase", // Thrift server user name, should be an authorized proxy user.
+        user != null ? user : "hbase",// Thrift server user name, should be an authorized proxy user
         host, // Thrift server domain
         saslProperties, null, transport);
     }


[30/50] [abbrv] hbase git commit: HBASE-16752 Return error back to rpc client on exceeding rpc request size limit.

Posted by la...@apache.org.
HBASE-16752 Return error back to rpc client on exceeding rpc request size limit.

Signed-off-by: Gary Helmling <ga...@apache.org>


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

Branch: refs/heads/branch-1
Commit: 33e89fa9cfd4036fe972ff892df957603ae25bbf
Parents: a97aef5
Author: Ashu Pachauri <as...@gmail.com>
Authored: Thu Oct 6 17:09:19 2016 -0700
Committer: Gary Helmling <ga...@apache.org>
Committed: Wed Oct 19 15:50:42 2016 -0700

----------------------------------------------------------------------
 .../hbase/exceptions/ClientExceptionsUtil.java  |  2 +-
 .../exceptions/RequestTooBigException.java      | 43 +++++++++++++++++
 .../org/apache/hadoop/hbase/ipc/RpcServer.java  | 49 ++++++++++++++++++--
 .../hadoop/hbase/ipc/AbstractTestIPC.java       | 17 ++++---
 4 files changed, 101 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/33e89fa9/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ClientExceptionsUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ClientExceptionsUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ClientExceptionsUtil.java
index f367ed9..8acfe49 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ClientExceptionsUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ClientExceptionsUtil.java
@@ -63,7 +63,7 @@ public final class ClientExceptionsUtil {
         || cur instanceof RegionTooBusyException || cur instanceof ThrottlingException
         || cur instanceof MultiActionResultTooLarge || cur instanceof RetryImmediatelyException
         || cur instanceof CallQueueTooBigException || cur instanceof CallDroppedException
-        || cur instanceof NotServingRegionException);
+        || cur instanceof NotServingRegionException || cur instanceof RequestTooBigException);
   }
 
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/33e89fa9/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/RequestTooBigException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/RequestTooBigException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/RequestTooBigException.java
new file mode 100644
index 0000000..31baebb
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/RequestTooBigException.java
@@ -0,0 +1,43 @@
+/**
+ * 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.exceptions;
+
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * Thrown when the size of the rpc request received by the server is too large.
+ *
+ * On receiving such an exception, the client does not retry the offending rpc.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class RequestTooBigException extends DoNotRetryIOException {
+
+  private static final long serialVersionUID = -1593339239809586516L;
+
+  public RequestTooBigException() {
+    super();
+  }
+
+  public RequestTooBigException(String message) {
+    super(message);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/33e89fa9/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
index 90a53bc..9c21bbe 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
@@ -24,6 +24,7 @@ import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
+import java.io.InputStream;
 import java.net.BindException;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
@@ -85,6 +86,7 @@ import org.apache.hadoop.hbase.client.VersionInfoUtil;
 import org.apache.hadoop.hbase.codec.Codec;
 import org.apache.hadoop.hbase.conf.ConfigurationObserver;
 import org.apache.hadoop.hbase.exceptions.RegionMovedException;
+import org.apache.hadoop.hbase.exceptions.RequestTooBigException;
 import org.apache.hadoop.hbase.io.BoundedByteBufferPool;
 import org.apache.hadoop.hbase.io.ByteBufferInputStream;
 import org.apache.hadoop.hbase.io.ByteBufferOutputStream;
@@ -263,6 +265,9 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
   protected HBaseRPCErrorHandler errorHandler = null;
 
   static final String MAX_REQUEST_SIZE = "hbase.ipc.max.request.size";
+  private static final RequestTooBigException REQUEST_TOO_BIG_EXCEPTION =
+      new RequestTooBigException();
+
   private static final String WARN_RESPONSE_TIME = "hbase.ipc.warn.response.time";
   private static final String WARN_RESPONSE_SIZE = "hbase.ipc.warn.response.size";
 
@@ -1653,9 +1658,47 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
         }
 
         if (dataLength > maxRequestSize) {
-          throw new DoNotRetryIOException("RPC data length of " + dataLength + " received from "
-              + getHostAddress() + " is greater than max allowed " + maxRequestSize + ". Set \""
-              + MAX_REQUEST_SIZE + "\" on server to override this limit (not recommended)");
+          String msg = "RPC data length of " + dataLength + " received from "
+              + getHostAddress() + " is greater than max allowed "
+              + maxRequestSize + ". Set \"" + MAX_REQUEST_SIZE
+              + "\" on server to override this limit (not recommended)";
+          LOG.warn(msg);
+
+          if (connectionHeaderRead && connectionPreambleRead) {
+            incRpcCount();
+            // Construct InputStream for the non-blocking SocketChannel
+            // We need the InputStream because we want to read only the request header
+            // instead of the whole rpc.
+            final ByteBuffer buf = ByteBuffer.allocate(1);
+            InputStream is = new InputStream() {
+              @Override
+              public int read() throws IOException {
+                channelRead(channel, buf);
+                buf.flip();
+                int x = buf.get();
+                buf.flip();
+                return x;
+              }
+            };
+            CodedInputStream cis = CodedInputStream.newInstance(is);
+            int headerSize = cis.readRawVarint32();
+            Message.Builder builder = RequestHeader.newBuilder();
+            ProtobufUtil.mergeFrom(builder, cis, headerSize);
+            RequestHeader header = (RequestHeader) builder.build();
+
+            // Notify the client about the offending request
+            Call reqTooBig = new Call(header.getCallId(), this.service, null, null, null,
+                null, this, responder, 0, null, this.addr,0);
+            metrics.exception(REQUEST_TOO_BIG_EXCEPTION);
+            setupResponse(null, reqTooBig, REQUEST_TOO_BIG_EXCEPTION, msg);
+            // We are going to close the connection, make sure we process the response
+            // before that. In rare case when this fails, we still close the connection.
+            responseWriteLock.lock();
+            responder.processResponse(reqTooBig);
+            responseWriteLock.unlock();
+          }
+          // Close the connection
+          return -1;
         }
 
         data = ByteBuffer.allocate(dataLength);

http://git-wip-us.apache.org/repos/asf/hbase/blob/33e89fa9/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
index 69c8fe2..bbf8720 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
@@ -300,15 +300,18 @@ public abstract class AbstractTestIPC {
   @Test
   public void testRpcMaxRequestSize() throws IOException, InterruptedException {
     Configuration conf = new Configuration(CONF);
-    conf.setInt(RpcServer.MAX_REQUEST_SIZE, 100);
+    conf.setInt(RpcServer.MAX_REQUEST_SIZE, 1000);
     RpcServer rpcServer = new TestRpcServer(conf);
     AbstractRpcClient client = createRpcClient(conf);
     try {
       rpcServer.start();
       MethodDescriptor md = SERVICE.getDescriptorForType().findMethodByName("echo");
-      // set total RPC size bigger than 100 bytes
-      EchoRequestProto param = EchoRequestProto.newBuilder().setMessage("hello.hello.hello.hello."
-          + "hello.hello.hello.hello.hello.hello.hello.hello.hello.hello.hello.hello").build();
+      StringBuilder message = new StringBuilder(1200);
+      for (int i = 0; i < 200; i++) {
+        message.append("hello.");
+      }
+      // set total RPC size bigger than 1000 bytes
+      EchoRequestProto param = EchoRequestProto.newBuilder().setMessage(message.toString()).build();
       InetSocketAddress address = rpcServer.getListenerAddress();
       if (address == null) {
         throw new IOException("Listener channel is closed");
@@ -319,8 +322,10 @@ public abstract class AbstractTestIPC {
           md.getOutputType().toProto(), User.getCurrent(), address,
           new MetricsConnection.CallStats());
         fail("RPC should have failed because it exceeds max request size");
-      } catch(IOException ex) {
-        // pass
+      } catch(IOException e) {
+        LOG.info("Caught expected exception: " + e);
+        assertTrue(e.toString(),
+            StringUtils.stringifyException(e).contains("RequestTooBigException"));
       }
     } finally {
       rpcServer.stop();


[05/50] [abbrv] hbase git commit: HBASE-16788 Guard HFile archiving under a separate lock

Posted by la...@apache.org.
HBASE-16788 Guard HFile archiving under a separate lock


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

Branch: refs/heads/branch-1
Commit: 89bef67d0c020662599f682309c47a5ed25c9b32
Parents: 59ca4da
Author: Gary Helmling <ga...@apache.org>
Authored: Fri Oct 7 10:42:20 2016 -0700
Committer: Gary Helmling <ga...@apache.org>
Committed: Mon Oct 10 16:06:55 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/HStore.java       |  54 +++--
 .../TestCompactionArchiveConcurrentClose.java   | 198 +++++++++++++++++++
 2 files changed, 236 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/89bef67d/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
index 6ee6bb5..74f5a1c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
@@ -41,6 +41,7 @@ import java.util.concurrent.ExecutorCompletionService;
 import java.util.concurrent.Future;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import org.apache.commons.logging.Log;
@@ -149,6 +150,19 @@ public class HStore implements Store {
    *   - completing a compaction
    */
   final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+  /**
+   * Lock specific to archiving compacted store files.  This avoids races around
+   * the combination of retrieving the list of compacted files and moving them to
+   * the archive directory.  Since this is usually a background process (other than
+   * on close), we don't want to handle this with the store write lock, which would
+   * block readers and degrade performance.
+   *
+   * Locked by:
+   *   - CompactedHFilesDispatchHandler via closeAndArchiveCompactedFiles()
+   *   - close()
+   */
+  final ReentrantLock archiveLock = new ReentrantLock();
+
   private final boolean verifyBulkLoads;
 
   private ScanInfo scanInfo;
@@ -835,6 +849,7 @@ public class HStore implements Store {
 
   @Override
   public ImmutableCollection<StoreFile> close() throws IOException {
+    this.archiveLock.lock();
     this.lock.writeLock().lock();
     try {
       // Clear so metrics doesn't find them.
@@ -890,6 +905,7 @@ public class HStore implements Store {
       return result;
     } finally {
       this.lock.writeLock().unlock();
+      this.archiveLock.unlock();
     }
   }
 
@@ -2641,26 +2657,32 @@ public class HStore implements Store {
   }
 
   @Override
-  public void closeAndArchiveCompactedFiles() throws IOException {
-    lock.readLock().lock();
-    Collection<StoreFile> copyCompactedfiles = null;
+  public synchronized void closeAndArchiveCompactedFiles() throws IOException {
+    // ensure other threads do not attempt to archive the same files on close()
+    archiveLock.lock();
     try {
-      Collection<StoreFile> compactedfiles =
-          this.getStoreEngine().getStoreFileManager().getCompactedfiles();
-      if (compactedfiles != null && compactedfiles.size() != 0) {
-        // Do a copy under read lock
-        copyCompactedfiles = new ArrayList<StoreFile>(compactedfiles);
-      } else {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("No compacted files to archive");
-          return;
+      lock.readLock().lock();
+      Collection<StoreFile> copyCompactedfiles = null;
+      try {
+        Collection<StoreFile> compactedfiles =
+            this.getStoreEngine().getStoreFileManager().getCompactedfiles();
+        if (compactedfiles != null && compactedfiles.size() != 0) {
+          // Do a copy under read lock
+          copyCompactedfiles = new ArrayList<StoreFile>(compactedfiles);
+        } else {
+          if (LOG.isTraceEnabled()) {
+            LOG.trace("No compacted files to archive");
+            return;
+          }
         }
+      } finally {
+        lock.readLock().unlock();
+      }
+      if (copyCompactedfiles != null && !copyCompactedfiles.isEmpty()) {
+        removeCompactedfiles(copyCompactedfiles);
       }
     } finally {
-      lock.readLock().unlock();
-    }
-    if (copyCompactedfiles != null && !copyCompactedfiles.isEmpty()) {
-      removeCompactedfiles(copyCompactedfiles);
+      archiveLock.unlock();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/89bef67d/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java
new file mode 100644
index 0000000..03072e2
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java
@@ -0,0 +1,198 @@
+/*
+ * 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.regionserver;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * Tests a race condition between archiving of compacted files in CompactedHFilesDischarger chore
+ * and HRegion.close();
+ */
+@Category({RegionServerTests.class, MediumTests.class})
+public class TestCompactionArchiveConcurrentClose {
+  public HBaseTestingUtility testUtil;
+
+  private Path testDir;
+  private AtomicBoolean archived = new AtomicBoolean();
+
+  @Before
+  public void setup() throws Exception {
+    testUtil = HBaseTestingUtility.createLocalHTU();
+    testDir = testUtil.getDataTestDir("TestStoreFileRefresherChore");
+    FSUtils.setRootDir(testUtil.getConfiguration(), testDir);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    testUtil.cleanupTestDir();
+  }
+
+  @Test
+  public void testStoreCloseAndDischargeRunningInParallel() throws Exception {
+    byte[] fam = Bytes.toBytes("f");
+    byte[] col = Bytes.toBytes("c");
+    byte[] val = Bytes.toBytes("val");
+
+    TableName tableName = TableName.valueOf(getClass().getSimpleName());
+    HTableDescriptor htd = new HTableDescriptor(tableName);
+    htd.addFamily(new HColumnDescriptor(fam));
+    HRegionInfo info = new HRegionInfo(tableName, null, null, false);
+    final Region region = initHRegion(htd, info);
+    RegionServerServices rss = mock(RegionServerServices.class);
+    List<Region> regions = new ArrayList<Region>();
+    regions.add(region);
+    when(rss.getOnlineRegions()).thenReturn(regions);
+
+    // Create the cleaner object
+    final CompactedHFilesDischarger cleaner =
+        new CompactedHFilesDischarger(1000, (Stoppable) null, rss, false);
+    // Add some data to the region and do some flushes
+    int batchSize = 10;
+    int fileCount = 10;
+    for (int f = 0; f < fileCount; f++) {
+      int start = f * batchSize;
+      for (int i = start; i < start + batchSize; i++) {
+        Put p = new Put(Bytes.toBytes("row" + i));
+        p.addColumn(fam, col, val);
+        region.put(p);
+      }
+      // flush them
+      region.flush(true);
+    }
+
+    Store store = region.getStore(fam);
+    assertEquals(fileCount, store.getStorefilesCount());
+
+    Collection<StoreFile> storefiles = store.getStorefiles();
+    // None of the files should be in compacted state.
+    for (StoreFile file : storefiles) {
+      assertFalse(file.isCompactedAway());
+    }
+    // Do compaction
+    region.compact(true);
+
+    // now run the cleaner with a concurrent close
+    Thread cleanerThread = new Thread() {
+      public void run() {
+        cleaner.chore();
+      }
+    };
+    cleanerThread.start();
+    // wait for cleaner to pause
+    synchronized (archived) {
+      if (!archived.get()) {
+        archived.wait();
+      }
+    }
+    final AtomicReference<Exception> closeException = new AtomicReference<>();
+    Thread closeThread = new Thread() {
+      public void run() {
+        // wait for the chore to complete and call close
+        try {
+          ((HRegion) region).close();
+        } catch (IOException e) {
+          closeException.set(e);
+        }
+      }
+    };
+    closeThread.start();
+    // no error should occur after the execution of the test
+    closeThread.join();
+    cleanerThread.join();
+
+    if (closeException.get() != null) {
+      throw closeException.get();
+    }
+  }
+
+  private Region initHRegion(HTableDescriptor htd, HRegionInfo info)
+      throws IOException {
+    Configuration conf = testUtil.getConfiguration();
+    Path tableDir = FSUtils.getTableDir(testDir, htd.getTableName());
+
+    HRegionFileSystem fs = new WaitingHRegionFileSystem(conf, tableDir.getFileSystem(conf),
+        tableDir, info);
+    final Configuration walConf = new Configuration(conf);
+    FSUtils.setRootDir(walConf, tableDir);
+    final WALFactory wals = new WALFactory(walConf, null, "log_" + info.getEncodedName());
+    HRegion region =
+        new HRegion(fs, wals.getWAL(info.getEncodedNameAsBytes(), info.getTable().getNamespace()),
+            conf, htd, null);
+
+    region.initialize();
+
+    return region;
+  }
+
+  private class WaitingHRegionFileSystem extends HRegionFileSystem {
+
+    public WaitingHRegionFileSystem(final Configuration conf, final FileSystem fs,
+        final Path tableDir, final HRegionInfo regionInfo) {
+      super(conf, fs, tableDir, regionInfo);
+    }
+
+    @Override
+    public void removeStoreFiles(String familyName, Collection<StoreFile> storeFiles)
+        throws IOException {
+      super.removeStoreFiles(familyName, storeFiles);
+      archived.set(true);
+      synchronized (archived) {
+        archived.notifyAll();
+      }
+      try {
+        // unfortunately we can't use a stronger barrier here as the fix synchronizing
+        // the race condition will then block
+        Thread.sleep(100);
+      } catch (InterruptedException ie) {
+        throw new InterruptedIOException("Interrupted waiting for latch");
+      }
+    }
+  }
+}


[13/50] [abbrv] hbase git commit: HBASE-16505 Pass deadline to HRegion operations

Posted by la...@apache.org.
HBASE-16505 Pass deadline to HRegion operations

Signed-off-by: Michael Stack <st...@apache.org>


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

Branch: refs/heads/branch-1
Commit: a3c29da0b6c829a8c52012d50bdb81e9c07ceee3
Parents: 3830890
Author: Phil Yang <ud...@gmail.com>
Authored: Mon Oct 10 18:38:50 2016 +0800
Committer: Michael Stack <st...@apache.org>
Committed: Wed Oct 12 12:17:38 2016 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/hbase/ipc/CallRunner.java    | 8 ++++++--
 .../java/org/apache/hadoop/hbase/ipc/RpcCallContext.java     | 7 +++++++
 .../src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java | 8 +++++++-
 3 files changed, 20 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c29da0/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
index 5e91beb..1a9d8b2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
 import org.apache.hadoop.hbase.ipc.RpcServer.Call;
 import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
 import org.apache.hadoop.hbase.util.Pair;
@@ -95,7 +96,7 @@ public class CallRunner {
       }
       call.startTime = System.currentTimeMillis();
       if (call.startTime > call.deadline) {
-        RpcServer.LOG.info("Drop timeout call: " + call);
+        RpcServer.LOG.warn("Drop timeout call: " + call);
         return;
       }
       this.status.setStatus("Setting up call");
@@ -121,7 +122,10 @@ public class CallRunner {
         }
         // make the call
         resultPair = this.rpcServer.call(call.service, call.md, call.param, call.cellScanner,
-          call.timestamp, this.status, call.startTime, call.timeout);
+            call.timestamp, this.status, call.startTime, call.timeout);
+      } catch (TimeoutIOException e){
+        RpcServer.LOG.warn("Can not complete this request in time, drop it: " + call);
+        return;
       } catch (Throwable e) {
         RpcServer.LOG.debug(Thread.currentThread().getName() + ": " + call.toShortString(), e);
         errorThrowable = e;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c29da0/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCallContext.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCallContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCallContext.java
index b08afc2..508b1e8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCallContext.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCallContext.java
@@ -85,4 +85,11 @@ public interface RpcCallContext {
 
   long getResponseBlockSize();
   void incrementResponseBlockSize(long blockSize);
+
+  /**
+   * Return the deadline of this call. If we can not complete this call in time, we can throw a
+   * TimeoutIOException and RPCServer will drop it.
+   * @return The system timestamp of deadline.
+   */
+  long getDeadline();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c29da0/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
index 564639f..90a53bc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
@@ -407,7 +407,8 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
       return "callId: " + this.id + " service: " + serviceName +
           " methodName: " + ((this.md != null) ? this.md.getName() : "n/a") +
           " size: " + StringUtils.TraditionalBinaryPrefix.long2String(this.size, "", 1) +
-          " connection: " + connection.toString();
+          " connection: " + connection.toString() +
+          " deadline: " + deadline;
     }
 
     String toTraceString() {
@@ -573,6 +574,11 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
       responseBlockSize += blockSize;
     }
 
+    @Override
+    public long getDeadline() {
+      return deadline;
+    }
+
     public synchronized void sendResponseIfReady() throws IOException {
       // set param null to reduce memory pressure
       this.param = null;


[19/50] [abbrv] hbase git commit: HBASE-16832 Reduce the default number of versions in Meta table for branch-1 (binlijin)

Posted by la...@apache.org.
HBASE-16832 Reduce the default number of versions in Meta table for branch-1 (binlijin)


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

Branch: refs/heads/branch-1
Commit: 05b010cacb83a6449c205393e3c15b048b84b0a2
Parents: 57d3e9e
Author: tedyu <yu...@gmail.com>
Authored: Fri Oct 14 13:53:39 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Fri Oct 14 13:53:39 2016 -0700

----------------------------------------------------------------------
 hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/05b010ca/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
index ad9241f..f752cd0 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
@@ -775,7 +775,7 @@ public final class HConstants {
   /**
    * Default value of {@link #HBASE_META_VERSIONS}.
    */
-  public static int DEFAULT_HBASE_META_VERSIONS = 10;
+  public static int DEFAULT_HBASE_META_VERSIONS = 3;
 
   /**
    * Parameter name for number of versions, kept by meta table.


[20/50] [abbrv] hbase git commit: HBASE-16724 Snapshot owner can't clone

Posted by la...@apache.org.
HBASE-16724 Snapshot owner can't clone

Signed-off-by: Ashish Singhi <as...@apache.org>


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

Branch: refs/heads/branch-1
Commit: b7f283c6f6728238bb553c80aa6eafce0df0d650
Parents: 05b010c
Author: Pankaj Kumar <pa...@huawei.com>
Authored: Sat Oct 15 11:57:00 2016 +0530
Committer: Ashish Singhi <as...@apache.org>
Committed: Sat Oct 15 11:57:00 2016 +0530

----------------------------------------------------------------------
 .../hadoop/hbase/security/access/AccessController.java   | 11 ++++++++++-
 .../hbase/security/access/TestAccessController.java      | 10 ++++------
 src/main/asciidoc/_chapters/appendix_acl_matrix.adoc     |  2 +-
 3 files changed, 15 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b7f283c6/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
index 2152440..7be4540 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
@@ -1342,7 +1342,16 @@ public class AccessController extends BaseMasterAndRegionObserver
   public void preCloneSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
       final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
       throws IOException {
-    requirePermission("cloneSnapshot " + snapshot.getName(), Action.ADMIN);
+    User user = getActiveUser();
+    if (SnapshotDescriptionUtils.isSnapshotOwner(snapshot, user)
+        && hTableDescriptor.getNameAsString().equals(snapshot.getTable())) {
+      // Snapshot owner is allowed to create a table with the same name as the snapshot he took
+      AuthResult result = AuthResult.allow("cloneSnapshot " + snapshot.getName(),
+        "Snapshot owner check allowed", user, null, hTableDescriptor.getTableName(), null);
+      logResult(result);
+    } else {
+      requirePermission("cloneSnapshot " + snapshot.getName(), Action.ADMIN);
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/b7f283c6/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
index 221241e..79d65cd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
@@ -2122,15 +2122,13 @@ public class TestAccessController extends SecureTestUtil {
       @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER.preCloneSnapshot(ObserverContext.createAndPrepare(CP_ENV, null),
-          snapshot, null);
+          snapshot, htd);
         return null;
       }
     };
-    // Clone by snapshot owner is not allowed , because clone operation creates a new table,
-    // which needs global admin permission.
-    verifyAllowed(cloneAction, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
-    verifyDenied(cloneAction, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER,
-      USER_GROUP_READ, USER_GROUP_WRITE, USER_GROUP_CREATE);
+    verifyAllowed(cloneAction, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN, USER_OWNER);
+    verifyDenied(cloneAction, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ,
+      USER_GROUP_WRITE, USER_GROUP_CREATE);
   }
 
   @Test (timeout=180000)

http://git-wip-us.apache.org/repos/asf/hbase/blob/b7f283c6/src/main/asciidoc/_chapters/appendix_acl_matrix.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/appendix_acl_matrix.adoc b/src/main/asciidoc/_chapters/appendix_acl_matrix.adoc
index cb285f3..adc2b1f 100644
--- a/src/main/asciidoc/_chapters/appendix_acl_matrix.adoc
+++ b/src/main/asciidoc/_chapters/appendix_acl_matrix.adoc
@@ -100,7 +100,7 @@ In case the table goes out of date, the unit tests which check for accuracy of p
 |        | stopMaster | superuser\|global(A)
 |        | snapshot | superuser\|global(A)\|NS(A)\|TableOwner\|table(A)
 |        | listSnapshot | superuser\|global(A)\|SnapshotOwner
-|        | cloneSnapshot | superuser\|global(A)
+|        | cloneSnapshot | superuser\|global(A)\|(SnapshotOwner & TableName matches)
 |        | restoreSnapshot | superuser\|global(A)\|SnapshotOwner & (NS(A)\|TableOwner\|table(A))
 |        | deleteSnapshot | superuser\|global(A)\|SnapshotOwner
 |        | createNamespace | superuser\|global(A)


[36/50] [abbrv] hbase git commit: HBASE-16870 Add the metrics of replication sources which were transformed from other dead rs to ReplicationLoad

Posted by la...@apache.org.
HBASE-16870 Add the metrics of replication sources which were transformed from other dead rs to ReplicationLoad

Signed-off-by: zhangduo <zh...@apache.org>


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

Branch: refs/heads/branch-1
Commit: d76cc4c1f08d405ea152934a3871758aacc9382f
Parents: e395bf9
Author: Guanghao Zhang <zg...@gmail.com>
Authored: Thu Oct 20 09:33:03 2016 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sat Oct 22 14:27:01 2016 +0800

----------------------------------------------------------------------
 .../replication/regionserver/MetricsSource.java |  2 +-
 .../replication/regionserver/Replication.java   | 13 ++-
 .../regionserver/ReplicationLoad.java           | 26 ++++-
 .../regionserver/ReplicationSourceManager.java  |  6 ++
 .../hbase/replication/TestReplicationBase.java  |  3 +-
 .../replication/TestReplicationSmallTests.java  | 42 ---------
 .../replication/TestReplicationStatus.java      | 99 ++++++++++++++++++++
 7 files changed, 141 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d76cc4c1/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java
index a647d03..68f32f7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java
@@ -82,7 +82,7 @@ public class MetricsSource implements BaseSource {
   public void setAgeOfLastShippedOp(long timestamp, String walGroup) {
     long age = EnvironmentEdgeManager.currentTime() - timestamp;
     singleSourceSource.setLastShippedAge(age);
-    globalSourceSource.setLastShippedAge(age);
+    globalSourceSource.setLastShippedAge(Math.max(age, globalSourceSource.getLastShippedAge()));
     this.lastTimeStamps.put(walGroup, timestamp);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/d76cc4c1/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
index d9a20ac..4f5393a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
@@ -447,15 +447,24 @@ public class Replication extends WALActionsListener.Base implements
   }
 
   private void buildReplicationLoad() {
-    // get source
-    List<ReplicationSourceInterface> sources = this.replicationManager.getSources();
     List<MetricsSource> sourceMetricsList = new ArrayList<MetricsSource>();
 
+    // get source
+    List<ReplicationSourceInterface> sources = this.replicationManager.getSources();
     for (ReplicationSourceInterface source : sources) {
       if (source instanceof ReplicationSource) {
         sourceMetricsList.add(((ReplicationSource) source).getSourceMetrics());
       }
     }
+
+    // get old source
+    List<ReplicationSourceInterface> oldSources = this.replicationManager.getOldSources();
+    for (ReplicationSourceInterface source : oldSources) {
+      if (source instanceof ReplicationSource) {
+        sourceMetricsList.add(((ReplicationSource) source).getSourceMetrics());
+      }
+    }
+
     // get sink
     MetricsSink sinkMetrics = this.replicationSink.getSinkMetrics();
     this.replicationLoad.buildReplicationLoad(sourceMetricsList, sinkMetrics);

http://git-wip-us.apache.org/repos/asf/hbase/blob/d76cc4c1/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationLoad.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationLoad.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationLoad.java
index 8dd42bc..2ead3df 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationLoad.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationLoad.java
@@ -19,8 +19,10 @@
 package org.apache.hadoop.hbase.replication.regionserver;
 
 import java.util.Date;
+import java.util.HashMap;
 import java.util.List;
 import java.util.ArrayList;
+import java.util.Map;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
@@ -66,8 +68,14 @@ public class ReplicationLoad {
     this.replicationLoadSink = rLoadSinkBuild.build();
 
     // build the SourceLoad List
-    this.replicationLoadSourceList = new ArrayList<ClusterStatusProtos.ReplicationLoadSource>();
+    Map<String, ClusterStatusProtos.ReplicationLoadSource> replicationLoadSourceMap =
+        new HashMap<String, ClusterStatusProtos.ReplicationLoadSource>();
     for (MetricsSource sm : this.sourceMetricsList) {
+      // Get the actual peer id
+      String peerId = sm.getPeerID();
+      String[] parts = peerId.split("-", 2);
+      peerId = parts.length != 1 ? parts[0] : peerId;
+
       long ageOfLastShippedOp = sm.getAgeOfLastShippedOp();
       int sizeOfLogQueue = sm.getSizeOfLogQueue();
       long timeStampOfLastShippedOp = sm.getTimeStampOfLastShippedOp();
@@ -85,17 +93,27 @@ public class ReplicationLoad {
         replicationLag = 0;
       }
 
+      ClusterStatusProtos.ReplicationLoadSource rLoadSource = replicationLoadSourceMap.get(peerId);
+      if (rLoadSource != null) {
+        ageOfLastShippedOp = Math.max(rLoadSource.getAgeOfLastShippedOp(), ageOfLastShippedOp);
+        sizeOfLogQueue += rLoadSource.getSizeOfLogQueue();
+        timeStampOfLastShippedOp = Math.min(rLoadSource.getTimeStampOfLastShippedOp(),
+          timeStampOfLastShippedOp);
+        replicationLag = Math.max(rLoadSource.getReplicationLag(), replicationLag);
+      }
+
       ClusterStatusProtos.ReplicationLoadSource.Builder rLoadSourceBuild =
           ClusterStatusProtos.ReplicationLoadSource.newBuilder();
-      rLoadSourceBuild.setPeerID(sm.getPeerID());
+      rLoadSourceBuild.setPeerID(peerId);
       rLoadSourceBuild.setAgeOfLastShippedOp(ageOfLastShippedOp);
       rLoadSourceBuild.setSizeOfLogQueue(sizeOfLogQueue);
       rLoadSourceBuild.setTimeStampOfLastShippedOp(timeStampOfLastShippedOp);
       rLoadSourceBuild.setReplicationLag(replicationLag);
 
-      this.replicationLoadSourceList.add(rLoadSourceBuild.build());
+      replicationLoadSourceMap.put(peerId, rLoadSourceBuild.build());
     }
-
+    this.replicationLoadSourceList = new ArrayList<ClusterStatusProtos.ReplicationLoadSource>(
+        replicationLoadSourceMap.values());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/d76cc4c1/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
index d4f6d9d..b31cc54 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
@@ -541,6 +541,9 @@ public class ReplicationSourceManager implements ReplicationListener {
    */
   public void closeRecoveredQueue(ReplicationSourceInterface src) {
     LOG.info("Done with the recovered queue " + src.getPeerClusterZnode());
+    if (src instanceof ReplicationSource) {
+      ((ReplicationSource) src).getSourceMetrics().clear();
+    }
     this.oldsources.remove(src);
     deleteSource(src.getPeerClusterZnode(), false);
     this.walsByIdRecoveredQueues.remove(src.getPeerClusterZnode());
@@ -590,6 +593,9 @@ public class ReplicationSourceManager implements ReplicationListener {
       }
       for (ReplicationSourceInterface toRemove : srcToRemove) {
         toRemove.terminate(terminateMessage);
+        if (toRemove instanceof ReplicationSource) {
+          ((ReplicationSource) toRemove).getSourceMetrics().clear();
+        }
         this.sources.remove(toRemove);
       }
       deleteSource(id, true);

http://git-wip-us.apache.org/repos/asf/hbase/blob/d76cc4c1/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java
index 5283433..498d26a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java
@@ -66,6 +66,7 @@ public class TestReplicationBase {
 
   protected static HBaseTestingUtility utility1;
   protected static HBaseTestingUtility utility2;
+  protected static final String PEER_ID = "2";
   protected static final int NB_ROWS_IN_BATCH = 100;
   protected static final int NB_ROWS_IN_BIG_BATCH =
       NB_ROWS_IN_BATCH * 10;
@@ -126,7 +127,7 @@ public class TestReplicationBase {
 
     ReplicationPeerConfig rpc = new ReplicationPeerConfig();
     rpc.setClusterKey(utility2.getClusterKey());
-    admin.addPeer("2", rpc);
+    admin.addPeer(PEER_ID, rpc);
 
     LOG.info("Setup second Zk");
     CONF_WITH_LOCALFS = HBaseConfiguration.create(conf1);

http://git-wip-us.apache.org/repos/asf/hbase/blob/d76cc4c1/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
index ba634dd..5e3c9b1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
@@ -704,48 +704,6 @@ public class TestReplicationSmallTests extends TestReplicationBase {
     hadmin.close();
   }
 
-  /**
-   * Test for HBASE-9531
-   * put a few rows into htable1, which should be replicated to htable2
-   * create a ClusterStatus instance 'status' from HBaseAdmin
-   * test : status.getLoad(server).getReplicationLoadSourceList()
-   * test : status.getLoad(server).getReplicationLoadSink()
-   * * @throws Exception
-   */
-  @Test(timeout = 300000)
-  public void testReplicationStatus() throws Exception {
-    LOG.info("testReplicationStatus");
-
-    try (Admin admin = utility1.getConnection().getAdmin()) {
-
-      final byte[] qualName = Bytes.toBytes("q");
-      Put p;
-
-      for (int i = 0; i < NB_ROWS_IN_BATCH; i++) {
-        p = new Put(Bytes.toBytes("row" + i));
-        p.add(famName, qualName, Bytes.toBytes("val" + i));
-        htable1.put(p);
-      }
-
-      ClusterStatus status = admin.getClusterStatus();
-
-      for (ServerName server : status.getServers()) {
-        ServerLoad sl = status.getLoad(server);
-        List<ReplicationLoadSource> rLoadSourceList = sl.getReplicationLoadSourceList();
-        ReplicationLoadSink rLoadSink = sl.getReplicationLoadSink();
-
-        // check SourceList has at least one entry
-        assertTrue("failed to get ReplicationLoadSourceList", (rLoadSourceList.size() > 0));
-
-        // check Sink exist only as it is difficult to verify the value on the fly
-        assertTrue("failed to get ReplicationLoadSink.AgeOfLastShippedOp ",
-          (rLoadSink.getAgeOfLastAppliedOp() >= 0));
-        assertTrue("failed to get ReplicationLoadSink.TimeStampsOfLastAppliedOp ",
-          (rLoadSink.getTimeStampsOfLastAppliedOp() >= 0));
-      }
-    }
-  }
-  
   @Test(timeout=300000)
   public void testVerifyReplicationPrefixFiltering() throws Exception {
     final byte[] prefixRow = Bytes.toBytes("prefixrow");

http://git-wip-us.apache.org/repos/asf/hbase/blob/d76cc4c1/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatus.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatus.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatus.java
new file mode 100644
index 0000000..1ac4924
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatus.java
@@ -0,0 +1,99 @@
+/**
+ * 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.replication;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.ServerLoad;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(MediumTests.class)
+public class TestReplicationStatus extends TestReplicationBase {
+  private static final Log LOG = LogFactory.getLog(TestReplicationStatus.class);
+
+  /**
+   * Test for HBASE-9531
+   * put a few rows into htable1, which should be replicated to htable2
+   * create a ClusterStatus instance 'status' from HBaseAdmin
+   * test : status.getLoad(server).getReplicationLoadSourceList()
+   * test : status.getLoad(server).getReplicationLoadSink()
+   * * @throws Exception
+   */
+  @Test(timeout = 300000)
+  public void testReplicationStatus() throws Exception {
+    LOG.info("testReplicationStatus");
+
+    try (Admin hbaseAdmin = utility1.getConnection().getAdmin()) {
+      // disable peer
+      admin.disablePeer(PEER_ID);
+
+      final byte[] qualName = Bytes.toBytes("q");
+      Put p;
+
+      for (int i = 0; i < NB_ROWS_IN_BATCH; i++) {
+        p = new Put(Bytes.toBytes("row" + i));
+        p.add(famName, qualName, Bytes.toBytes("val" + i));
+        htable1.put(p);
+      }
+
+      ClusterStatus status = hbaseAdmin.getClusterStatus();
+
+      for (ServerName server : status.getServers()) {
+        ServerLoad sl = status.getLoad(server);
+        List<ReplicationLoadSource> rLoadSourceList = sl.getReplicationLoadSourceList();
+        ReplicationLoadSink rLoadSink = sl.getReplicationLoadSink();
+
+        // check SourceList only has one entry, beacuse only has one peer
+        assertTrue("failed to get ReplicationLoadSourceList", (rLoadSourceList.size() == 1));
+        assertEquals(PEER_ID, rLoadSourceList.get(0).getPeerID());
+
+        // check Sink exist only as it is difficult to verify the value on the fly
+        assertTrue("failed to get ReplicationLoadSink.AgeOfLastShippedOp ",
+          (rLoadSink.getAgeOfLastAppliedOp() >= 0));
+        assertTrue("failed to get ReplicationLoadSink.TimeStampsOfLastAppliedOp ",
+          (rLoadSink.getTimeStampsOfLastAppliedOp() >= 0));
+      }
+
+      // Stop rs1, then the queue of rs1 will be transfered to rs0
+      utility1.getHBaseCluster().getRegionServer(1).stop("Stop RegionServer");
+      Thread.sleep(10000);
+      status = hbaseAdmin.getClusterStatus();
+      ServerName server = utility1.getHBaseCluster().getRegionServer(0).getServerName();
+      ServerLoad sl = status.getLoad(server);
+      List<ReplicationLoadSource> rLoadSourceList = sl.getReplicationLoadSourceList();
+      // check SourceList still only has one entry
+      assertTrue("failed to get ReplicationLoadSourceList", (rLoadSourceList.size() == 1));
+      assertEquals(PEER_ID, rLoadSourceList.get(0).getPeerID());
+    } finally {
+      admin.enablePeer(PEER_ID);
+      utility1.getHBaseCluster().getRegionServer(1).start();
+    }
+  }
+}


[45/50] [abbrv] hbase git commit: HBASE-16951 1.3 assembly scripts dont package hbase-archetypes in the tarball

Posted by la...@apache.org.
HBASE-16951 1.3 assembly scripts dont package hbase-archetypes in the tarball


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

Branch: refs/heads/branch-1
Commit: 0f158edb3aba82101275ae5f6750827e09150d59
Parents: 0a41493
Author: Mikhail Antonov <an...@apache.org>
Authored: Wed Oct 26 23:20:42 2016 -0700
Committer: Mikhail Antonov <an...@apache.org>
Committed: Fri Oct 28 15:55:18 2016 -0700

----------------------------------------------------------------------
 hbase-assembly/src/main/assembly/src.xml | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/0f158edb/hbase-assembly/src/main/assembly/src.xml
----------------------------------------------------------------------
diff --git a/hbase-assembly/src/main/assembly/src.xml b/hbase-assembly/src/main/assembly/src.xml
index 16b22ee..b972bfd 100644
--- a/hbase-assembly/src/main/assembly/src.xml
+++ b/hbase-assembly/src/main/assembly/src.xml
@@ -32,6 +32,7 @@
       <useAllReactorProjects>true</useAllReactorProjects>
       <includes>
         <include>org.apache.hbase:hbase-annotations</include>
+        <include>org.apache.hbase:hbase-archetypes</include>
         <include>org.apache.hbase:hbase-assembly</include>
         <include>org.apache.hbase:hbase-checkstyle</include>
         <include>org.apache.hbase:hbase-client</include>


[16/50] [abbrv] hbase git commit: HBASE-16807, RegionServer will fail to report new active Hmaster until HMaster/RegionServer failover.

Posted by la...@apache.org.
HBASE-16807, RegionServer will fail to report new active Hmaster until HMaster/RegionServer failover.

Signed-off-by: chenheng <ch...@apache.org>


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

Branch: refs/heads/branch-1
Commit: 27398ea525be88f9d8089071e0d7f212f33c2ca8
Parents: 8f9fadf
Author: Pankaj Kumar <pa...@huawei.com>
Authored: Thu Oct 13 20:19:07 2016 +0800
Committer: chenheng <ch...@apache.org>
Committed: Thu Oct 13 22:17:28 2016 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/HRegionServer.java  | 18 +++++++++++++++---
 .../TestRegionServerReportForDuty.java            |  4 ++--
 2 files changed, 17 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/27398ea5/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 03dcd4e..f3eb27f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -1196,7 +1196,7 @@ public class HRegionServer extends HasThread implements
       }
       // Couldn't connect to the master, get location from zk and reconnect
       // Method blocks until new master is found or we are stopped
-      createRegionServerStatusStub();
+      createRegionServerStatusStub(true);
     }
   }
 
@@ -2235,12 +2235,24 @@ public class HRegionServer extends HasThread implements
    */
   @VisibleForTesting
   protected synchronized ServerName createRegionServerStatusStub() {
+    // Create RS stub without refreshing the master node from ZK, use cached data
+    return createRegionServerStatusStub(false);
+  }
+
+  /**
+   * Get the current master from ZooKeeper and open the RPC connection to it. To get a fresh
+   * connection, the current rssStub must be null. Method will block until a master is available.
+   * You can break from this block by requesting the server stop.
+   * @param refresh If true then master address will be read from ZK, otherwise use cached data
+   * @return master + port, or null if server has been stopped
+   */
+  @VisibleForTesting
+  protected synchronized ServerName createRegionServerStatusStub(boolean refresh) {
     if (rssStub != null) {
       return masterAddressTracker.getMasterAddress();
     }
     ServerName sn = null;
     long previousLogTime = 0;
-    boolean refresh = false; // for the first time, use cached data
     RegionServerStatusService.BlockingInterface intf = null;
     boolean interrupted = false;
     try {
@@ -2315,7 +2327,7 @@ public class HRegionServer extends HasThread implements
    * @throws IOException
    */
   private RegionServerStartupResponse reportForDuty() throws IOException {
-    ServerName masterServerName = createRegionServerStatusStub();
+    ServerName masterServerName = createRegionServerStatusStub(true);
     if (masterServerName == null) return null;
     RegionServerStartupResponse result = null;
     try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/27398ea5/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReportForDuty.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReportForDuty.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReportForDuty.java
index 5778c6c..77cb5b7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReportForDuty.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReportForDuty.java
@@ -166,8 +166,8 @@ public class TestRegionServerReportForDuty {
     }
 
     @Override
-    protected synchronized ServerName createRegionServerStatusStub() {
-      sn = super.createRegionServerStatusStub();
+    protected synchronized ServerName createRegionServerStatusStub(boolean refresh) {
+      sn = super.createRegionServerStatusStub(refresh);
       rpcStubCreatedFlag = true;
 
       // Wait for master switch over. Only do this for the second region server.


[48/50] [abbrv] hbase git commit: HBASE-16947 Some improvements for DumpReplicationQueues tool

Posted by la...@apache.org.
HBASE-16947 Some improvements for DumpReplicationQueues tool

Signed-off-by: Michael Stack <st...@apache.org>


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

Branch: refs/heads/branch-1
Commit: 722be3a3d8e8bde9b557c62fb1cbb83d21c0a38b
Parents: a969f8d
Author: Guanghao Zhang <zg...@gmail.com>
Authored: Fri Oct 28 13:20:50 2016 +0800
Committer: Michael Stack <st...@apache.org>
Committed: Mon Oct 31 21:00:45 2016 -0700

----------------------------------------------------------------------
 .../ReplicationQueuesClientZKImpl.java          |   4 +-
 .../replication/ReplicationQueuesZKImpl.java    |   8 +-
 .../regionserver/DumpReplicationQueues.java     | 123 ++++++++++++++-----
 3 files changed, 102 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/722be3a3/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
index cc407e3..9078e40 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
@@ -40,7 +40,9 @@ public class ReplicationQueuesClientZKImpl extends ReplicationStateZKBase implem
   @Override
   public void init() throws ReplicationException {
     try {
-      ZKUtil.createWithParents(this.zookeeper, this.queuesZNode);
+      if (ZKUtil.checkExists(this.zookeeper, this.queuesZNode) < 0) {
+        ZKUtil.createWithParents(this.zookeeper, this.queuesZNode);
+      }
     } catch (KeeperException e) {
       throw new ReplicationException("Internal error while initializing a queues client", e);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/722be3a3/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
index c7af78e..9beaba7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
@@ -82,14 +82,18 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
   public void init(String serverName) throws ReplicationException {
     this.myQueuesZnode = ZKUtil.joinZNode(this.queuesZNode, serverName);
     try {
-      ZKUtil.createWithParents(this.zookeeper, this.myQueuesZnode);
+      if (ZKUtil.checkExists(this.zookeeper, this.myQueuesZnode) < 0) {
+        ZKUtil.createWithParents(this.zookeeper, this.myQueuesZnode);
+      }
     } catch (KeeperException e) {
       throw new ReplicationException("Could not initialize replication queues.", e);
     }
     if (conf.getBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY,
       HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT)) {
       try {
-        ZKUtil.createWithParents(this.zookeeper, this.hfileRefsZNode);
+        if (ZKUtil.checkExists(this.zookeeper, this.hfileRefsZNode) < 0) {
+          ZKUtil.createWithParents(this.zookeeper, this.hfileRefsZNode);
+        }
       } catch (KeeperException e) {
         throw new ReplicationException("Could not initialize hfile references replication queue.",
             e);

http://git-wip-us.apache.org/repos/asf/hbase/blob/722be3a3/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java
index bf38d6f..0772f89 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java
@@ -40,6 +40,9 @@ import org.apache.hadoop.util.ToolRunner;
 import org.apache.zookeeper.KeeperException;
 import org.mortbay.util.IO;
 
+import com.google.common.util.concurrent.AtomicLongMap;
+
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.*;
 
@@ -55,6 +58,20 @@ public class DumpReplicationQueues extends Configured implements Tool {
 
   private static final Log LOG = LogFactory.getLog(DumpReplicationQueues.class.getName());
 
+  private List<String> deadRegionServers;
+  private List<String> deletedQueues;
+  private AtomicLongMap<String> peersQueueSize;
+  private long totalSizeOfWALs;
+  private long numWalsNotFound;
+
+  public DumpReplicationQueues() {
+    deadRegionServers = new ArrayList<String>();
+    deletedQueues = new ArrayList<String>();
+    peersQueueSize = AtomicLongMap.create();
+    totalSizeOfWALs = 0;
+    numWalsNotFound = 0;
+  }
+
   static class DumpOptions {
     boolean hdfs = false;
     boolean distributed = false;
@@ -155,13 +172,14 @@ public class DumpReplicationQueues extends Configured implements Tool {
     if (message != null && message.length() > 0) {
       System.err.println(message);
     }
-    System.err.println("Usage: java " + className + " \\");
+    System.err.println("Usage: bin/hbase " + className + " \\");
     System.err.println("  <OPTIONS> [-D<property=value>]*");
     System.err.println();
     System.err.println("General Options:");
-    System.err.println(" distributed  Poll each RS and print its own replication queue. "
+    System.err.println(" -h|--h|--help  Show this help and exit.");
+    System.err.println(" --distributed  Poll each RS and print its own replication queue. "
         + "Default only polls ZooKeeper");
-    System.err.println(" hdfs         Use HDFS to calculate usage of WALs by replication. It could be overestimated"
+    System.err.println(" --hdfs         Use HDFS to calculate usage of WALs by replication. It could be overestimated"
         + " if replicating to multiple peers. --distributed flag is also needed.");
   }
 
@@ -176,7 +194,6 @@ public class DumpReplicationQueues extends Configured implements Tool {
     HBaseAdmin.checkHBaseAvailable(conf);
     ReplicationAdmin replicationAdmin = new ReplicationAdmin(conf);
     ClusterConnection connection = (ClusterConnection) ConnectionFactory.createConnection(conf);
-    long deleted = 0;
 
     ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "DumpReplicationQueues" + System.currentTimeMillis(),
         new WarnOnlyAbortable(), true);
@@ -203,11 +220,8 @@ public class DumpReplicationQueues extends Configured implements Tool {
 
       if (opts.isDistributed()) {
         LOG.info("Found [--distributed], will poll each RegionServer.");
-        System.out.println(dumpQueues(connection, zkw, opts.isHdfs(), deleted));
-        if (deleted > 0) {
-          LOG.warn("Found " + deleted +" deleted queues"
-              + ", run hbck -fixReplication in order to remove the deleted replication queues");
-        }
+        System.out.println(dumpQueues(connection, peerConfigs.keySet(), zkw, opts.isHdfs()));
+        System.out.println(dumpReplicationSummary());
       } else {
         // use ZK instead
         System.out.print("Dumping replication znodes via ZooKeeper:");
@@ -221,21 +235,52 @@ public class DumpReplicationQueues extends Configured implements Tool {
     }
   }
 
+  public String dumpReplicationSummary() {
+    StringBuilder sb = new StringBuilder();
+    if (!deletedQueues.isEmpty()) {
+      sb.append("Found " + deletedQueues.size() + " deleted queues"
+          + ", run hbck -fixReplication in order to remove the deleted replication queues\n");
+      for (String deletedQueue : deletedQueues) {
+        sb.append("    " + deletedQueue + "\n");
+      }
+    }
+    if (!deadRegionServers.isEmpty()) {
+      sb.append("Found " + deadRegionServers.size() + " dead regionservers"
+          + ", restart one regionserver to transfer the queues of dead regionservers\n");
+      for (String deadRs : deadRegionServers) {
+        sb.append("    " + deadRs + "\n");
+      }
+    }
+    if (!peersQueueSize.isEmpty()) {
+      sb.append("Dumping all peers's number of WALs in replication queue\n");
+      for (Map.Entry<String, Long> entry : peersQueueSize.asMap().entrySet()) {
+        sb.append("    PeerId: " + entry.getKey() + " , sizeOfLogQueue: " + entry.getValue() + "\n");
+      }
+    }
+    sb.append("    Total size of WALs on HDFS: " + StringUtils.humanSize(totalSizeOfWALs) + "\n");
+    if (numWalsNotFound > 0) {
+      sb.append("    ERROR: There are " + numWalsNotFound + " WALs not found!!!\n");
+    }
+    return sb.toString();
+  }
+
   public String dumpPeersState(ReplicationAdmin replicationAdmin,
-                              Map<String, ReplicationPeerConfig> peerConfigs) throws Exception {
+      Map<String, ReplicationPeerConfig> peerConfigs) throws Exception {
     Map<String, String> currentConf;
     StringBuilder sb = new StringBuilder();
     for (Map.Entry<String, ReplicationPeerConfig> peer : peerConfigs.entrySet()) {
       try {
+        ReplicationPeerConfig peerConfig = peer.getValue();
         sb.append("Peer: " + peer.getKey() + "\n");
-        sb.append("    " + "State: " + (replicationAdmin.getPeerState(peer.getKey()) ? "ENABLED" : "DISABLED") + "\n");
-        sb.append("    " + "Cluster Name: " + peer.getValue() + "\n");
-        currentConf = peer.getValue().getConfiguration();
+        sb.append("    " + "State: "
+            + (replicationAdmin.getPeerState(peer.getKey()) ? "ENABLED" : "DISABLED") + "\n");
+        sb.append("    " + "Cluster Name: " + peerConfig.getClusterKey() + "\n");
+        currentConf = peerConfig.getConfiguration();
         // Only show when we have a custom configuration for the peer
         if (currentConf.size() > 1) {
           sb.append("    " + "Peer Configuration: " + currentConf + "\n");
         }
-        sb.append("    " + "Peer Table CFs: " + replicationAdmin.getPeerTableCFs(peer.getKey()) + "\n");
+        sb.append("    " + "Peer Table CFs: " + peerConfig.getTableCFsMap() + "\n");
       } catch (ReplicationException re) {
         sb.append("Got an exception while invoking ReplicationAdmin: " + re + "\n");
       }
@@ -243,33 +288,36 @@ public class DumpReplicationQueues extends Configured implements Tool {
     return sb.toString();
   }
 
-  public String dumpQueues(ClusterConnection connection, ZooKeeperWatcher zkw, boolean hdfs, long deleted)
-      throws Exception {
+  public String dumpQueues(ClusterConnection connection, Set<String> peerIds, ZooKeeperWatcher zkw,
+      boolean hdfs) throws Exception {
     ReplicationQueuesClient queuesClient;
     ReplicationPeers replicationPeers;
     ReplicationQueues replicationQueues;
-
+    ReplicationTracker replicationTracker;
     StringBuilder sb = new StringBuilder();
 
     queuesClient = ReplicationFactory.getReplicationQueuesClient(zkw, getConf(), connection);
     queuesClient.init();
     replicationQueues = ReplicationFactory.getReplicationQueues(zkw, getConf(), connection);
     replicationPeers = ReplicationFactory.getReplicationPeers(zkw, getConf(), queuesClient, connection);
-    replicationPeers.init();
+    replicationTracker = ReplicationFactory.getReplicationTracker(zkw, replicationPeers, getConf(),
+      new WarnOnlyAbortable(), new WarnOnlyStoppable());
+    List<String> liveRegionServers = replicationTracker.getListOfRegionServers();
 
     // Loops each peer on each RS and dumps the queues
-
-    Set<String> peerIds = new HashSet<String>(replicationPeers.getAllPeerIds());
     try {
       List<String> regionservers = queuesClient.getListOfReplicators();
       for (String regionserver : regionservers) {
         List<String> queueIds = queuesClient.getAllQueues(regionserver);
         replicationQueues.init(regionserver);
+        if (!liveRegionServers.contains(regionserver)) {
+          deadRegionServers.add(regionserver);
+        }
         for (String queueId : queueIds) {
           ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId);
           List<String> wals = queuesClient.getLogsInQueue(regionserver, queueId);
           if (!peerIds.contains(queueInfo.getPeerId())) {
-            deleted++;
+            deletedQueues.add(regionserver + "/" + queueId);
             sb.append(formatQueue(regionserver, replicationQueues, queueInfo, queueId, wals, true, hdfs));
           } else {
             sb.append(formatQueue(regionserver, replicationQueues, queueInfo, queueId, wals, false, hdfs));
@@ -282,14 +330,14 @@ public class DumpReplicationQueues extends Configured implements Tool {
     return sb.toString();
   }
 
-  private String formatQueue(String regionserver, ReplicationQueues replicationQueues, ReplicationQueueInfo queueInfo,
-                           String queueId, List<String> wals, boolean isDeleted, boolean hdfs) throws Exception {
-
+  private String formatQueue(String regionserver, ReplicationQueues replicationQueues,
+      ReplicationQueueInfo queueInfo, String queueId, List<String> wals, boolean isDeleted,
+      boolean hdfs) throws Exception {
     StringBuilder sb = new StringBuilder();
-
     List<String> deadServers ;
 
     sb.append("Dumping replication queue info for RegionServer: [" + regionserver + "]" + "\n");
+    sb.append("    Queue znode: " + queueId + "\n");
     sb.append("    PeerID: " + queueInfo.getPeerId() + "\n");
     sb.append("    Recovered: " + queueInfo.isQueueRecovered() + "\n");
     deadServers = queueInfo.getDeadRegionServers();
@@ -300,6 +348,8 @@ public class DumpReplicationQueues extends Configured implements Tool {
     }
     sb.append("    Was deleted: " + isDeleted + "\n");
     sb.append("    Number of WALs in replication queue: " + wals.size() + "\n");
+    peersQueueSize.addAndGet(queueInfo.getPeerId(), wals.size());
+
     for (String wal : wals) {
       long position = replicationQueues.getLogPosition(queueInfo.getPeerId(), wal);
       sb.append("    Replication position for " + wal + ": " + (position > 0 ? position : "0"
@@ -324,11 +374,18 @@ public class DumpReplicationQueues extends Configured implements Tool {
       try {
         fileStatus = (new WALLink(getConf(), server, wal)).getFileStatus(fs);
       } catch (IOException e) {
-        LOG.warn("WAL " + wal + " couldn't be found, skipping");
-        break;
+        if (e instanceof FileNotFoundException) {
+          numWalsNotFound++;
+          LOG.warn("WAL " + wal + " couldn't be found, skipping", e);
+        } else {
+          LOG.warn("Can't get file status of WAL " + wal + ", skipping", e);
+        }
+        continue;
       }
       size += fileStatus.getLen();
     }
+
+    totalSizeOfWALs += size;
     return size;
   }
 
@@ -347,9 +404,15 @@ public class DumpReplicationQueues extends Configured implements Tool {
     }
   }
 
-  private static void usage(final String errorMsg) {
-    if (errorMsg != null && errorMsg.length() > 0) {
-      LOG.error(errorMsg);
+  private static class WarnOnlyStoppable implements Stoppable {
+    @Override
+    public void stop(String why) {
+      LOG.warn("DumpReplicationQueue received stop, ignoring.  Reason: " + why);
+    }
+
+    @Override
+    public boolean isStopped() {
+      return false;
     }
   }
 }


[38/50] [abbrv] hbase git commit: Revert "HBASE-16562 ITBLL should fail to start if misconfigured"

Posted by la...@apache.org.
Revert "HBASE-16562 ITBLL should fail to start if misconfigured"

This reverts commit 38b946c2762d10f547de3e6c2b0676d3f7dfbd54.

See discussion on JIRA.


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

Branch: refs/heads/branch-1
Commit: a1536c28763014153510f3f14a82587d32c14572
Parents: 65c2dd4
Author: Sean Busbey <bu...@apache.org>
Authored: Mon Oct 24 09:16:53 2016 -0500
Committer: Sean Busbey <bu...@apache.org>
Committed: Mon Oct 24 09:16:53 2016 -0500

----------------------------------------------------------------------
 .../hadoop/hbase/test/IntegrationTestBigLinkedList.java  | 11 -----------
 1 file changed, 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a1536c28/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
index 366fc02..2d828f4 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
@@ -678,17 +678,6 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase {
         Integer width = (args.length < 4) ? null : Integer.parseInt(args[3]);
         Integer wrapMultiplier = (args.length < 5) ? null : Integer.parseInt(args[4]);
         Integer numWalkers = (args.length < 6) ? null : Integer.parseInt(args[5]);
-
-        long wrap = (long)width*wrapMultiplier;
-        if (wrap < numNodes && numNodes % wrap != 0) {
-          /**
-           *  numNodes should be a multiple of width*wrapMultiplier.
-           *  If numNodes less than wrap, wrap will be set to be equal with numNodes,
-           *  See {@link GeneratorMapper#setup(Mapper.Context)}
-           * */
-          System.err.println(USAGE);
-          return 1;
-        }
         return run(numMappers, numNodes, tmpOutput, width, wrapMultiplier, numWalkers);
       } catch (NumberFormatException e) {
         System.err.println("Parsing generator arguments failed: " + e.getMessage());


[22/50] [abbrv] hbase git commit: HBASE-16856 Exception message in SyncRunner.run() should print currentSequence (Allan Yang)

Posted by la...@apache.org.
HBASE-16856 Exception message in SyncRunner.run() should print currentSequence (Allan Yang)


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

Branch: refs/heads/branch-1
Commit: 08498c684835ac37700bd6fca22f9c2d47d70797
Parents: a4cbded
Author: tedyu <yu...@gmail.com>
Authored: Mon Oct 17 07:23:02 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Mon Oct 17 07:23:02 2016 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/08498c68/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
index ceced2f..11ebfef 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
@@ -1239,7 +1239,7 @@ public class FSHLog implements WAL {
             currentSequence = this.sequence;
             long syncFutureSequence = takeSyncFuture.getRingBufferSequence();
             if (syncFutureSequence > currentSequence) {
-              throw new IllegalStateException("currentSequence=" + syncFutureSequence +
+              throw new IllegalStateException("currentSequence=" + currentSequence +
                 ", syncFutureSequence=" + syncFutureSequence);
             }
             // See if we can process any syncfutures BEFORE we go sync.


[35/50] [abbrv] hbase git commit: HBASE-16815 Low scan ratio in RPC queue tuning triggers divide by zero exception

Posted by la...@apache.org.
HBASE-16815 Low scan ratio in RPC queue tuning triggers divide by zero exception

Signed-off-by: Matteo Bertozzi <ma...@cloudera.com>


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

Branch: refs/heads/branch-1
Commit: e395bf91dbafc7d892966122972e70c5446f3a09
Parents: 9fda0dd
Author: Guanghao Zhang <zg...@gmail.com>
Authored: Fri Oct 14 18:29:25 2016 +0800
Committer: Matteo Bertozzi <ma...@cloudera.com>
Committed: Fri Oct 21 17:00:24 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hbase/ipc/RWQueueRpcExecutor.java  | 15 ++++++++-------
 1 file changed, 8 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e395bf91/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RWQueueRpcExecutor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RWQueueRpcExecutor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RWQueueRpcExecutor.java
index 68e7b65..7ca9196 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RWQueueRpcExecutor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RWQueueRpcExecutor.java
@@ -145,17 +145,16 @@ public class RWQueueRpcExecutor extends RpcExecutor {
     this.scanBalancer = numScanQueues > 0 ? getBalancer(numScanQueues) : null;
 
     queues = new ArrayList<BlockingQueue<CallRunner>>(numWriteQueues + numReadQueues + numScanQueues);
-    LOG.debug(name + " writeQueues=" + numWriteQueues + " writeHandlers=" + writeHandlersCount +
-              " readQueues=" + numReadQueues + " readHandlers=" + readHandlersCount +
-              ((numScanQueues == 0) ? "" : " scanQueues=" + numScanQueues +
-                " scanHandlers=" + scanHandlersCount));
+    LOG.info(name + " writeQueues=" + numWriteQueues + " writeHandlers=" + writeHandlersCount
+        + " readQueues=" + numReadQueues + " readHandlers=" + readHandlersCount + " scanQueues="
+        + numScanQueues + " scanHandlers=" + scanHandlersCount);
+
     if (writeQueueInitArgs.length > 0) {
       currentQueueLimit = (int) writeQueueInitArgs[0];
       writeQueueInitArgs[0] = Math.max((int) writeQueueInitArgs[0],
         DEFAULT_CALL_QUEUE_SIZE_HARD_LIMIT);
     }
     for (int i = 0; i < numWriteQueues; ++i) {
-
       queues.add((BlockingQueue<CallRunner>)
         ReflectionUtils.newInstance(writeQueueClass, writeQueueInitArgs));
     }
@@ -175,8 +174,10 @@ public class RWQueueRpcExecutor extends RpcExecutor {
   protected void startHandlers(final int port) {
     startHandlers(".write", writeHandlersCount, queues, 0, numWriteQueues, port);
     startHandlers(".read", readHandlersCount, queues, numWriteQueues, numReadQueues, port);
-    startHandlers(".scan", scanHandlersCount, queues,
-                  numWriteQueues + numReadQueues, numScanQueues, port);
+    if (numScanQueues > 0) {
+      startHandlers(".scan", scanHandlersCount, queues, numWriteQueues + numReadQueues,
+        numScanQueues, port);
+    }
   }
 
   @Override


[41/50] [abbrv] hbase git commit: HBASE-16939 ExportSnapshot: set owner and permission on right directory

Posted by la...@apache.org.
HBASE-16939 ExportSnapshot: set owner and permission on right directory

Signed-off-by: Matteo Bertozzi <ma...@cloudera.com>


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

Branch: refs/heads/branch-1
Commit: 4edd8a63d2afa57a1c7029b2d6badeede8855097
Parents: ae502a9
Author: Guanghao Zhang <zg...@gmail.com>
Authored: Tue Oct 25 10:36:55 2016 +0800
Committer: Matteo Bertozzi <ma...@cloudera.com>
Committed: Tue Oct 25 14:11:59 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/snapshot/ExportSnapshot.java   | 29 ++++++++++++++++----
 1 file changed, 24 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/4edd8a63/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
index 35ce2d7..89dcd37 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
@@ -930,6 +930,19 @@ public class ExportSnapshot extends Configured implements Tool {
     Path outputSnapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(targetName, outputRoot);
     Path initialOutputSnapshotDir = skipTmp ? outputSnapshotDir : snapshotTmpDir;
 
+    // Find the necessary directory which need to change owner and group
+    Path needSetOwnerDir = SnapshotDescriptionUtils.getSnapshotRootDir(outputRoot);
+    if (outputFs.exists(needSetOwnerDir)) {
+      if (skipTmp) {
+        needSetOwnerDir = outputSnapshotDir;
+      } else {
+        needSetOwnerDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(outputRoot);
+        if (outputFs.exists(needSetOwnerDir)) {
+          needSetOwnerDir = snapshotTmpDir;
+        }
+      }
+    }
+
     // Check if the snapshot already exists
     if (outputFs.exists(outputSnapshotDir)) {
       if (overwrite) {
@@ -967,15 +980,21 @@ public class ExportSnapshot extends Configured implements Tool {
     try {
       LOG.info("Copy Snapshot Manifest");
       FileUtil.copy(inputFs, snapshotDir, outputFs, initialOutputSnapshotDir, false, false, conf);
+    } catch (IOException e) {
+      throw new ExportSnapshotException("Failed to copy the snapshot directory: from=" +
+        snapshotDir + " to=" + initialOutputSnapshotDir, e);
+    } finally {
       if (filesUser != null || filesGroup != null) {
-        setOwner(outputFs, snapshotTmpDir, filesUser, filesGroup, true);
+        LOG.warn((filesUser == null ? "" : "Change the owner of " + needSetOwnerDir + " to "
+            + filesUser)
+            + (filesGroup == null ? "" : ", Change the group of " + needSetOwnerDir + " to "
+            + filesGroup));
+        setOwner(outputFs, needSetOwnerDir, filesUser, filesGroup, true);
       }
       if (filesMode > 0) {
-        setPermission(outputFs, snapshotTmpDir, (short)filesMode, true);
+        LOG.warn("Change the permission of " + needSetOwnerDir + " to " + filesMode);
+        setPermission(outputFs, needSetOwnerDir, (short)filesMode, true);
       }
-    } catch (IOException e) {
-      throw new ExportSnapshotException("Failed to copy the snapshot directory: from=" +
-        snapshotDir + " to=" + initialOutputSnapshotDir, e);
     }
 
     // Write a new .snapshotinfo if the target name is different from the source name


[32/50] [abbrv] hbase git commit: HBASE-16752 addendum. Do not retry large request for client versions less than 1.3

Posted by la...@apache.org.
HBASE-16752 addendum. Do not retry large request for client versions less than 1.3

Signed-off-by: Gary Helmling <ga...@apache.org>


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

Branch: refs/heads/branch-1
Commit: 0117ed9d78820abecf066d742782de1dd49f309e
Parents: a7a4e17
Author: Ashu Pachauri <as...@gmail.com>
Authored: Wed Oct 19 16:50:52 2016 -0700
Committer: Gary Helmling <ga...@apache.org>
Committed: Thu Oct 20 10:34:46 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/exceptions/RequestTooBigException.java     | 6 +++++-
 .../main/java/org/apache/hadoop/hbase/ipc/RpcServer.java    | 9 ++++++++-
 2 files changed, 13 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/0117ed9d/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/RequestTooBigException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/RequestTooBigException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/RequestTooBigException.java
index 31baebb..0021f4a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/RequestTooBigException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/RequestTooBigException.java
@@ -26,13 +26,17 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  * Thrown when the size of the rpc request received by the server is too large.
  *
  * On receiving such an exception, the client does not retry the offending rpc.
+ * @since 1.3.0
  */
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 public class RequestTooBigException extends DoNotRetryIOException {
-
   private static final long serialVersionUID = -1593339239809586516L;
 
+  // Recognized only in HBase version 1.3 and higher.
+  public static final int MAJOR_VERSION = 1;
+  public static final int MINOR_VERSION = 3;
+
   public RequestTooBigException() {
     super();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/0117ed9d/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
index 9c21bbe..c2698b1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
@@ -1690,7 +1690,14 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
             Call reqTooBig = new Call(header.getCallId(), this.service, null, null, null,
                 null, this, responder, 0, null, this.addr,0);
             metrics.exception(REQUEST_TOO_BIG_EXCEPTION);
-            setupResponse(null, reqTooBig, REQUEST_TOO_BIG_EXCEPTION, msg);
+            // Make sure the client recognizes the underlying exception
+            // Otherwise, throw a DoNotRetryIOException.
+            if (VersionInfoUtil.hasMinimumVersion(connectionHeader.getVersionInfo(),
+                RequestTooBigException.MAJOR_VERSION, RequestTooBigException.MINOR_VERSION)) {
+              setupResponse(null, reqTooBig, REQUEST_TOO_BIG_EXCEPTION, msg);
+            } else {
+              setupResponse(null, reqTooBig, new DoNotRetryIOException(), msg);
+            }
             // We are going to close the connection, make sure we process the response
             // before that. In rare case when this fails, we still close the connection.
             responseWriteLock.lock();


[50/50] [abbrv] hbase git commit: HBASE-16970 Clarify misleading Scan.java comment about caching

Posted by la...@apache.org.
HBASE-16970 Clarify misleading Scan.java comment about caching

Signed-off-by: Yu Li <li...@apache.org>


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

Branch: refs/heads/branch-1
Commit: 5fdddae55e7220afbc5f21d90792915dd25da39f
Parents: b3ddea9
Author: Jim Kleckner <ji...@cloudphysics.com>
Authored: Tue Nov 1 17:31:23 2016 +0800
Committer: Yu Li <li...@apache.org>
Committed: Tue Nov 1 17:48:56 2016 +0800

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hbase/client/Scan.java        | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/5fdddae5/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java
index a518ffb..ac7e190 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java
@@ -136,7 +136,8 @@ public class Scan extends Query {
   public static final String HINT_LOOKAHEAD = "_look_ahead_";
 
   /*
-   * -1 means no caching
+   * -1 means no caching specified and the value of {@link HConstants#HBASE_CLIENT_SCANNER_CACHING}
+   * (default to {@link HConstants#DEFAULT_HBASE_CLIENT_SCANNER_CACHING}) will be used
    */
   private int caching = -1;
   private long maxResultSize = -1;


[43/50] [abbrv] hbase git commit: HBASE-16886 hbase-client: scanner with reversed=true and small=true gets no result (huzheng)

Posted by la...@apache.org.
HBASE-16886 hbase-client: scanner with reversed=true and small=true gets no result (huzheng)


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

Branch: refs/heads/branch-1
Commit: d4826e1665085b0ef697db548f8b6277be256591
Parents: 24a92ed
Author: tedyu <yu...@gmail.com>
Authored: Thu Oct 27 07:47:49 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Thu Oct 27 07:47:49 2016 -0700

----------------------------------------------------------------------
 .../client/ClientSmallReversedScanner.java      | 115 ++++++++++++++--
 .../client/TestClientSmallReversedScanner.java  |  16 +--
 .../hbase/client/TestSmallReversedScanner.java  | 138 +++++++++++++++++++
 3 files changed, 248 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d4826e16/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallReversedScanner.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallReversedScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallReversedScanner.java
index d4de6a0..bd5575a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallReversedScanner.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallReversedScanner.java
@@ -20,6 +20,7 @@
 package org.apache.hadoop.hbase.client;
 
 
+import com.google.protobuf.ServiceException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -29,13 +30,18 @@ import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.ClientSmallScanner.SmallScannerCallable;
-import org.apache.hadoop.hbase.client.ClientSmallScanner.SmallScannerCallableFactory;
+import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.RequestConverter;
+import org.apache.hadoop.hbase.protobuf.ResponseConverter;
+import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.util.Bytes;
 
 import com.google.common.annotations.VisibleForTesting;
 
 import java.io.IOException;
+import java.io.InterruptedIOException;
 import java.util.concurrent.ExecutorService;
 
 /**
@@ -49,8 +55,8 @@ import java.util.concurrent.ExecutorService;
 @InterfaceAudience.Private
 public class ClientSmallReversedScanner extends ReversedClientScanner {
   private static final Log LOG = LogFactory.getLog(ClientSmallReversedScanner.class);
-  private ScannerCallableWithReplicas smallScanCallable = null;
-  private SmallScannerCallableFactory callableFactory;
+  private ScannerCallableWithReplicas smallReversedScanCallable = null;
+  private SmallReversedScannerCallableFactory callableFactory;
 
   /**
    * Create a new ReversibleClientScanner for the specified table. Take note that the passed
@@ -80,7 +86,7 @@ public class ClientSmallReversedScanner extends ReversedClientScanner {
       RpcControllerFactory controllerFactory, ExecutorService pool, int primaryOperationTimeout)
       throws IOException {
     this(conf, scan, tableName, connection, rpcFactory, controllerFactory, pool,
-        primaryOperationTimeout, new SmallScannerCallableFactory());
+        primaryOperationTimeout, new SmallReversedScannerCallableFactory());
   }
 
   /**
@@ -112,7 +118,7 @@ public class ClientSmallReversedScanner extends ReversedClientScanner {
   ClientSmallReversedScanner(final Configuration conf, final Scan scan, final TableName tableName,
       ClusterConnection connection, RpcRetryingCallerFactory rpcFactory,
       RpcControllerFactory controllerFactory, ExecutorService pool, int primaryOperationTimeout,
-      SmallScannerCallableFactory callableFactory) throws IOException {
+      SmallReversedScannerCallableFactory callableFactory) throws IOException {
     super(conf, scan, tableName, connection, rpcFactory, controllerFactory, pool,
         primaryOperationTimeout);
     this.callableFactory = callableFactory;
@@ -136,6 +142,7 @@ public class ClientSmallReversedScanner extends ReversedClientScanner {
     byte[] localStartKey;
     int cacheNum = nbRows;
     boolean regionChanged = true;
+    boolean isFirstRegionToLocate = false;
     // if we're at end of table, close and return false to stop iterating
     if (this.currentRegion != null && currentRegionDone) {
       byte[] startKey = this.currentRegion.getStartKey();
@@ -158,6 +165,14 @@ public class ClientSmallReversedScanner extends ReversedClientScanner {
       localStartKey = createClosestRowBefore(lastResult.getRow());
     } else {
       localStartKey = this.scan.getStartRow();
+      isFirstRegionToLocate = true;
+    }
+
+    if (!isFirstRegionToLocate
+        && (localStartKey == null || Bytes.equals(localStartKey, HConstants.EMPTY_BYTE_ARRAY))) {
+      // when non-firstRegion & localStartKey is empty bytes, no more rowKey should scan.
+      // otherwise, maybe infinity results with RowKey=0x00 will return.
+      return false;
     }
 
     if (LOG.isTraceEnabled()) {
@@ -165,9 +180,10 @@ public class ClientSmallReversedScanner extends ReversedClientScanner {
           + Bytes.toStringBinary(localStartKey) + "'");
     }
 
-    smallScanCallable = callableFactory.getCallable(getConnection(), getTable(), scan,
-        getScanMetrics(), localStartKey, cacheNum, rpcControllerFactory, getPool(),
-        getPrimaryOperationTimeout(), getRetries(), getScannerTimeout(), getConf(), caller);
+    smallReversedScanCallable =
+        callableFactory.getCallable(getConnection(), getTable(), scan, getScanMetrics(),
+          localStartKey, cacheNum, rpcControllerFactory, getPool(), getPrimaryOperationTimeout(),
+          getRetries(), getScannerTimeout(), getConf(), caller, isFirstRegionToLocate);
 
     if (this.scanMetrics != null && regionChanged) {
       this.scanMetrics.countOfRegions.incrementAndGet();
@@ -209,8 +225,8 @@ public class ClientSmallReversedScanner extends ReversedClientScanner {
       // exhausted current region.
       // callWithoutRetries is at this layer. Within the ScannerCallableWithReplicas,
       // we do a callWithRetries
-      values = this.caller.callWithoutRetries(smallScanCallable, scannerTimeout);
-      this.currentRegion = smallScanCallable.getHRegionInfo();
+      values = this.caller.callWithoutRetries(smallReversedScanCallable, scannerTimeout);
+      this.currentRegion = smallReversedScanCallable.getHRegionInfo();
       long currentTime = System.currentTimeMillis();
       if (this.scanMetrics != null) {
         this.scanMetrics.sumOfMillisSecBetweenNexts.addAndGet(currentTime
@@ -229,8 +245,8 @@ public class ClientSmallReversedScanner extends ReversedClientScanner {
           this.lastResult = rs;
         }
       }
-      if (smallScanCallable.hasMoreResultsContext()) {
-        currentRegionDone = !smallScanCallable.getServerHasMoreResults();
+      if (smallReversedScanCallable.hasMoreResultsContext()) {
+        currentRegionDone = !smallReversedScanCallable.getServerHasMoreResults();
       } else {
         currentRegionDone = countdown > 0;
       }
@@ -250,7 +266,80 @@ public class ClientSmallReversedScanner extends ReversedClientScanner {
   }
 
   @VisibleForTesting
-  protected void setScannerCallableFactory(SmallScannerCallableFactory callableFactory) {
+  protected void setScannerCallableFactory(SmallReversedScannerCallableFactory callableFactory) {
     this.callableFactory = callableFactory;
   }
+
+  /**
+   * A reversed ScannerCallable which supports backward small scanning.
+   */
+  static class SmallReversedScannerCallable extends ReversedScannerCallable {
+
+    public SmallReversedScannerCallable(ClusterConnection connection, TableName table, Scan scan,
+        ScanMetrics scanMetrics, byte[] locateStartRow, RpcControllerFactory controllerFactory,
+        int caching, int replicaId) {
+      super(connection, table, scan, scanMetrics, locateStartRow, controllerFactory, replicaId);
+      this.setCaching(caching);
+    }
+
+    @Override
+    public Result[] call(int timeout) throws IOException {
+      if (this.closed) return null;
+      if (Thread.interrupted()) {
+        throw new InterruptedIOException();
+      }
+      ClientProtos.ScanRequest request = RequestConverter.buildScanRequest(
+        getLocation().getRegionInfo().getRegionName(), getScan(), getCaching(), true);
+      ClientProtos.ScanResponse response = null;
+      controller = controllerFactory.newController();
+      try {
+        controller.setPriority(getTableName());
+        controller.setCallTimeout(timeout);
+        response = getStub().scan(controller, request);
+        Result[] results = ResponseConverter.getResults(controller.cellScanner(), response);
+        if (response.hasMoreResultsInRegion()) {
+          setHasMoreResultsContext(true);
+          setServerHasMoreResults(response.getMoreResultsInRegion());
+        } else {
+          setHasMoreResultsContext(false);
+        }
+        // We need to update result metrics since we are overriding call()
+        updateResultsMetrics(results);
+        return results;
+      } catch (ServiceException se) {
+        throw ProtobufUtil.getRemoteException(se);
+      }
+    }
+
+    @Override
+    public ScannerCallable getScannerCallableForReplica(int id) {
+      return new SmallReversedScannerCallable(getConnection(), getTableName(), getScan(),
+          scanMetrics, locateStartRow, controllerFactory, getCaching(), id);
+    }
+  }
+
+  protected static class SmallReversedScannerCallableFactory {
+
+    public ScannerCallableWithReplicas getCallable(ClusterConnection connection, TableName table,
+        Scan scan, ScanMetrics scanMetrics, byte[] localStartKey, int cacheNum,
+        RpcControllerFactory controllerFactory, ExecutorService pool, int primaryOperationTimeout,
+        int retries, int scannerTimeout, Configuration conf, RpcRetryingCaller<Result[]> caller,
+        boolean isFirstRegionToLocate) {
+      byte[] locateStartRow = null;
+      if (isFirstRegionToLocate
+          && (localStartKey == null || Bytes.equals(localStartKey, HConstants.EMPTY_BYTE_ARRAY))) {
+        // HBASE-16886: if not setting startRow, then we will use a range [MAX_BYTE_ARRAY, +oo) to
+        // locate a region list, and the last one in region list is the region where our scan start.
+        locateStartRow = ClientScanner.MAX_BYTE_ARRAY;
+      }
+
+      scan.setStartRow(localStartKey);
+      SmallReversedScannerCallable s = new SmallReversedScannerCallable(connection, table, scan,
+          scanMetrics, locateStartRow, controllerFactory, cacheNum, 0);
+      ScannerCallableWithReplicas scannerCallableWithReplicas =
+          new ScannerCallableWithReplicas(table, connection, s, pool, primaryOperationTimeout, scan,
+              retries, scannerTimeout, cacheNum, conf, caller);
+      return scannerCallableWithReplicas;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/d4826e16/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientSmallReversedScanner.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientSmallReversedScanner.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientSmallReversedScanner.java
index 4611d08..57b52e6 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientSmallReversedScanner.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientSmallReversedScanner.java
@@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.ClientSmallScanner.SmallScannerCallableFactory;
+import org.apache.hadoop.hbase.client.ClientSmallReversedScanner.SmallReversedScannerCallableFactory;
 import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
@@ -102,15 +102,15 @@ public class TestClientSmallReversedScanner {
     };
   }
 
-  private SmallScannerCallableFactory getFactory(
+  private SmallReversedScannerCallableFactory getFactory(
       final ScannerCallableWithReplicas callableWithReplicas) {
-    return new SmallScannerCallableFactory() {
+    return new SmallReversedScannerCallableFactory() {
       @Override
       public ScannerCallableWithReplicas getCallable(ClusterConnection connection, TableName table,
           Scan scan, ScanMetrics scanMetrics, byte[] localStartKey, int cacheNum,
           RpcControllerFactory controllerFactory, ExecutorService pool,
           int primaryOperationTimeout, int retries, int scannerTimeout, Configuration conf,
-          RpcRetryingCaller<Result[]> caller) {
+          RpcRetryingCaller<Result[]> caller, boolean isFirstRegionToLocate) {
         return callableWithReplicas;
       }
     };
@@ -135,7 +135,7 @@ public class TestClientSmallReversedScanner {
     // Intentionally leave a "default" caching size in the Scan. No matter the value, we
     // should continue based on the server context
 
-    SmallScannerCallableFactory factory = getFactory(callableWithReplicas);
+    SmallReversedScannerCallableFactory factory = getFactory(callableWithReplicas);
 
     try (ClientSmallReversedScanner csrs = new ClientSmallReversedScanner(conf, scan,
         TableName.valueOf("table"), clusterConn, rpcFactory, controllerFactory, pool,
@@ -204,7 +204,7 @@ public class TestClientSmallReversedScanner {
     // While the server returns 2 records per batch, we expect more records.
     scan.setCaching(2);
 
-    SmallScannerCallableFactory factory = getFactory(callableWithReplicas);
+    SmallReversedScannerCallableFactory factory = getFactory(callableWithReplicas);
 
     try (ClientSmallReversedScanner csrs = new ClientSmallReversedScanner(conf, scan,
         TableName.valueOf("table"), clusterConn, rpcFactory, controllerFactory, pool,
@@ -280,7 +280,7 @@ public class TestClientSmallReversedScanner {
     // While the server return 2 records per RPC, we expect there to be more records.
     scan.setCaching(2);
 
-    SmallScannerCallableFactory factory = getFactory(callableWithReplicas);
+    SmallReversedScannerCallableFactory factory = getFactory(callableWithReplicas);
 
     try (ClientSmallReversedScanner csrs = new ClientSmallReversedScanner(conf, scan,
         TableName.valueOf("table"), clusterConn, rpcFactory, controllerFactory, pool,
@@ -316,7 +316,7 @@ public class TestClientSmallReversedScanner {
     ScannerCallableWithReplicas callableWithReplicas = Mockito
         .mock(ScannerCallableWithReplicas.class);
 
-    SmallScannerCallableFactory factory = getFactory(callableWithReplicas);
+    SmallReversedScannerCallableFactory factory = getFactory(callableWithReplicas);
 
     try (ClientSmallReversedScanner csrs = new ClientSmallReversedScanner(conf, scan,
         TableName.valueOf("table"), clusterConn, rpcFactory, controllerFactory, pool,

http://git-wip-us.apache.org/repos/asf/hbase/blob/d4826e16/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSmallReversedScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSmallReversedScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSmallReversedScanner.java
new file mode 100644
index 0000000..3a4e92b
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSmallReversedScanner.java
@@ -0,0 +1,138 @@
+/**
+ * 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.client;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.io.IOException;
+
+@Category(MediumTests.class)
+public class TestSmallReversedScanner {
+  public static final Log LOG = LogFactory.getLog(TestSmallReversedScanner.class);
+  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+  private static final TableName TABLE_NAME = TableName.valueOf("testReversedSmall");
+  private static final byte[] COLUMN_FAMILY = Bytes.toBytes("columnFamily");
+
+  private static Table htable = null;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    TEST_UTIL.startMiniCluster(1);
+
+    // create a table with 4 region: (-oo, b),[b,c),[c,d),[d,+oo)
+    byte[] bytes = Bytes.toBytes("bcd");
+    byte[][] splitKeys = new byte[bytes.length][];
+
+    for (int i = 0; i < bytes.length; i++) {
+      splitKeys[i] = new byte[] { bytes[i] };
+    }
+    htable = TEST_UTIL.createTable(TABLE_NAME, COLUMN_FAMILY, splitKeys);
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    TEST_UTIL.deleteTableData(TABLE_NAME);
+  }
+
+  /**
+   * all rowKeys are fit in the last region.
+   * @throws IOException
+   */
+  @Test
+  public void testSmallReversedScan01() throws IOException {
+    String[][] keysCases = new String[][] {
+            { "d0", "d1", "d2", "d3" }, // all rowKeys fit in the last region.
+            { "a0", "a1", "a2", "a3" }, // all rowKeys fit in the first region.
+            { "a0", "b1", "c2", "d3" }, // each region with a rowKey
+    };
+
+    for (int caseIndex = 0; caseIndex < keysCases.length; caseIndex++) {
+      testSmallReversedScanInternal(keysCases[caseIndex]);
+      TEST_UTIL.deleteTableData(TABLE_NAME);
+    }
+  }
+
+  private void testSmallReversedScanInternal(String[] inputRowKeys) throws IOException {
+    int rowCount = inputRowKeys.length;
+
+    for (int i = 0; i < rowCount; i++) {
+      Put put = new Put(Bytes.toBytes(inputRowKeys[i]));
+      put.addColumn(COLUMN_FAMILY, null, Bytes.toBytes(i));
+      htable.put(put);
+    }
+
+    Scan scan = new Scan();
+    scan.setReversed(true);
+    scan.setSmall(true);
+
+    ResultScanner scanner = htable.getScanner(scan);
+    Result r;
+    int value = rowCount;
+    while ((r = scanner.next()) != null) {
+      Assert.assertArrayEquals(r.getValue(COLUMN_FAMILY, null), Bytes.toBytes(--value));
+      Assert.assertArrayEquals(r.getRow(), Bytes.toBytes(inputRowKeys[value]));
+    }
+
+    Assert.assertEquals(value, 0);
+  }
+
+  /**
+   * Corner case:
+   *  HBase has 4 regions, (-oo,b),[b,c),[c,d),[d,+oo), and only rowKey with byte[]={0x00} locate in region (-oo,b) .
+   *  test whether reversed small scanner will return infinity results with RowKey={0x00}.
+   * @throws IOException
+   */
+  @Test
+  public void testSmallReversedScan02() throws IOException {
+    Put put = new Put(new byte[] { (char) 0x00 });
+    put.addColumn(COLUMN_FAMILY, null, Bytes.toBytes(0));
+    htable.put(put);
+
+    Scan scan = new Scan();
+    scan.setCaching(1);
+    scan.setReversed(true);
+    scan.setSmall(true);
+
+    ResultScanner scanner = htable.getScanner(scan);
+    Result r;
+    int count = 1;
+    while ((r = scanner.next()) != null) {
+      Assert.assertArrayEquals(r.getValue(COLUMN_FAMILY, null), Bytes.toBytes(0));
+      Assert.assertArrayEquals(r.getRow(), new byte[] { (char) 0x00 });
+      Assert.assertTrue(--count >= 0);
+    }
+    Assert.assertEquals(count, 0);
+  }
+}


[06/50] [abbrv] hbase git commit: HBASE-16788 addendum Account for HStore archiveLock in heap size

Posted by la...@apache.org.
HBASE-16788 addendum Account for HStore archiveLock in heap size


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

Branch: refs/heads/branch-1
Commit: f13a21696f2bbd4f572eb35c15282835998d4b34
Parents: 89bef67
Author: Gary Helmling <ga...@apache.org>
Authored: Tue Oct 11 10:32:45 2016 -0700
Committer: Gary Helmling <ga...@apache.org>
Committed: Tue Oct 11 11:46:12 2016 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/f13a2169/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
index 74f5a1c..7f00b7d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
@@ -2533,7 +2533,7 @@ public class HStore implements Store {
   }
 
   public static final long FIXED_OVERHEAD =
-      ClassSize.align(ClassSize.OBJECT + (16 * ClassSize.REFERENCE) + (11 * Bytes.SIZEOF_LONG)
+      ClassSize.align(ClassSize.OBJECT + (17 * ClassSize.REFERENCE) + (11 * Bytes.SIZEOF_LONG)
               + (5 * Bytes.SIZEOF_INT) + (2 * Bytes.SIZEOF_BOOLEAN));
 
   public static final long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD


[47/50] [abbrv] hbase git commit: HBASE-16969 RegionCoprocessorServiceExec should override the toString() for debugging (ChiaPing Tsai)

Posted by la...@apache.org.
HBASE-16969 RegionCoprocessorServiceExec should override the toString() for debugging (ChiaPing Tsai)


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

Branch: refs/heads/branch-1
Commit: a969f8d7903d07ca1f6672db4b2a739c227e70a0
Parents: ea5b0a0
Author: Michael Stack <st...@apache.org>
Authored: Mon Oct 31 14:37:51 2016 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Mon Oct 31 14:37:51 2016 -0700

----------------------------------------------------------------------
 .../hbase/client/RegionCoprocessorServiceExec.java    | 14 ++++++++++++++
 .../apache/hadoop/hbase/CoordinatedStateManager.java  |  2 +-
 2 files changed, 15 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a969f8d7/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorServiceExec.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorServiceExec.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorServiceExec.java
index 2d62332..cd4716e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorServiceExec.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorServiceExec.java
@@ -101,4 +101,18 @@ public class RegionCoprocessorServiceExec implements Row {
     Row other = (Row) obj;
     return compareTo(other) == 0;
   }
+
+  @Override
+  public String toString() {
+    StringBuilder builder = new StringBuilder();
+    builder.append("region:")
+           .append(Bytes.toStringBinary(region))
+           .append(", startKey:")
+           .append(Bytes.toStringBinary(startKey))
+           .append(", method:")
+           .append(method.getFullName())
+           .append(", request:")
+           .append(request);
+    return builder.toString();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a969f8d7/hbase-server/src/main/java/org/apache/hadoop/hbase/CoordinatedStateManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/CoordinatedStateManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/CoordinatedStateManager.java
index b7bfa75..bdb202d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/CoordinatedStateManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/CoordinatedStateManager.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.hbase;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 
 /**
- * Implementations of this interface will keep and return to clients 
+ * Implementations of this interface will keep and return to clients
  * implementations of classes providing API to execute
  * coordinated operations. This interface is client-side, so it does NOT
  * include methods to retrieve the particular interface implementations.


[02/50] [abbrv] hbase git commit: HBASE-16701 rely on test category timeout instead of defining one on a specific test.

Posted by la...@apache.org.
HBASE-16701 rely on test category timeout instead of defining one on a specific test.

Signed-off-by: Umesh Agashe <ua...@cloudera.com>
Signed-off-by: Yu Li <li...@apache.org>


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

Branch: refs/heads/branch-1
Commit: acb1392b1533b8ebedf2e45b6f133516cdbf99ee
Parents: 364a57a
Author: Sean Busbey <bu...@apache.org>
Authored: Wed Oct 5 17:23:20 2016 -0500
Committer: Sean Busbey <bu...@apache.org>
Committed: Mon Oct 10 00:24:24 2016 -0500

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/regionserver/TestHRegion.java     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/acb1392b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
index 1265468..7cf76fc 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
@@ -6639,7 +6639,7 @@ public class TestHRegion {
    * HBASE-16429 Make sure no stuck if roll writer when ring buffer is filled with appends
    * @throws IOException if IO error occurred during test
    */
-  @Test(timeout = 60000)
+  @Test
   public void testWritesWhileRollWriter() throws IOException {
     int testCount = 10;
     int numRows = 1024;


[09/50] [abbrv] hbase git commit: HBASE-16146 Remove thread local usage in Counter

Posted by la...@apache.org.
HBASE-16146 Remove thread local usage in Counter


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

Branch: refs/heads/branch-1
Commit: 4f29c230384b82b64ef4ad9ba61497747436799f
Parents: b47ded3
Author: Gary Helmling <ga...@apache.org>
Authored: Wed Sep 21 17:43:41 2016 -0700
Committer: Gary Helmling <ga...@apache.org>
Committed: Tue Oct 11 14:47:02 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/util/Counter.java   | 21 +-------------------
 .../org/apache/hadoop/hbase/ipc/RpcServer.java  |  1 -
 2 files changed, 1 insertion(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/4f29c230/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Counter.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Counter.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Counter.java
index 92c0a8f..ce0fddd 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Counter.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Counter.java
@@ -108,25 +108,12 @@ public class Counter {
     return h;
   }
 
-  private static class IndexHolder {
-    int index = hash();
-  }
-
-  private final ThreadLocal<IndexHolder> indexHolderThreadLocal =
-      new ThreadLocal<IndexHolder>() {
-    @Override
-    protected IndexHolder initialValue() {
-      return new IndexHolder();
-    }
-  };
-
   public void add(long delta) {
     Container container = containerRef.get();
     Cell[] cells = container.cells;
     int mask = cells.length - 1;
 
-    IndexHolder indexHolder = indexHolderThreadLocal.get();
-    int baseIndex = indexHolder.index;
+    int baseIndex = hash();
     if(cells[baseIndex & mask].add(delta)) {
       return;
     }
@@ -139,8 +126,6 @@ public class Counter {
       index++;
     }
 
-    indexHolder.index = index;
-
     if(index - baseIndex >= cells.length &&
         cells.length < MAX_CELLS_LENGTH &&
         container.demoted.compareAndSet(false, true)) {
@@ -181,10 +166,6 @@ public class Counter {
     return sum;
   }
 
-  public void destroy() {
-    indexHolderThreadLocal.remove();
-  }
-
   @Override
   public String toString() {
     Cell[] cells = containerRef.get().cells;

http://git-wip-us.apache.org/repos/asf/hbase/blob/4f29c230/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
index f7cc85f..564639f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
@@ -2011,7 +2011,6 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
           LOG.trace(ignored);
         }
       }
-      rpcCount.destroy();
     }
 
     private UserGroupInformation createUser(ConnectionHeader head) {


[29/50] [abbrv] hbase git commit: HBASE-16345 RpcRetryingCallerWithReadReplicas#call() should catch some RegionServer Exceptions

Posted by la...@apache.org.
HBASE-16345 RpcRetryingCallerWithReadReplicas#call() should catch some RegionServer Exceptions

Fix logic for
1). how to handle exception while waiting for reply from the primary replica.
2). handle exception from replicas while waiting for a correct response.

Signed-off-by: Esteban Gutierrez <es...@apache.org>


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

Branch: refs/heads/branch-1
Commit: a97aef51635539ea382699495613ebe1bf89e475
Parents: ae15133
Author: Huaxiang Sun <hs...@cloudera.com>
Authored: Wed Oct 19 14:15:31 2016 -0700
Committer: Esteban Gutierrez <es...@apache.org>
Committed: Wed Oct 19 14:22:42 2016 -0700

----------------------------------------------------------------------
 .../client/ResultBoundedCompletionService.java  | 118 +++++++++++++--
 .../RpcRetryingCallerWithReadReplicas.java      |  29 ++--
 .../client/ScannerCallableWithReplicas.java     |  60 +++++---
 .../hbase/client/TestReplicaWithCluster.java    | 143 ++++++++++++++++++-
 4 files changed, 304 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a97aef51/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ResultBoundedCompletionService.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ResultBoundedCompletionService.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ResultBoundedCompletionService.java
index 9b32e93..2848c9d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ResultBoundedCompletionService.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ResultBoundedCompletionService.java
@@ -18,13 +18,18 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import java.util.ArrayList;
+import java.util.concurrent.CancellationException;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Executor;
 import java.util.concurrent.RunnableFuture;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.htrace.Trace;
 
 /**
@@ -32,13 +37,21 @@ import org.apache.htrace.Trace;
  * Keeps the list of the futures, and allows to cancel them all.
  * This means as well that it can be used for a small set of tasks only.
  * <br>Implementation is not Thread safe.
+ *
+ * CompletedTasks is implemented as a queue, the entry is added based on the time order. I.e,
+ * when the first task completes (whether it is a success or failure), it is added as a first
+ * entry in the queue, the next completed task is added as a second entry in the queue, ...
+ * When iterating through the queue, we know it is based on time order. If the first
+ * completed task succeeds, it is returned. If it is failure, the iteration goes on until it
+ * finds a success.
  */
 @InterfaceAudience.Private
 public class ResultBoundedCompletionService<V> {
+  private static final Log LOG = LogFactory.getLog(ResultBoundedCompletionService.class);
   private final RpcRetryingCallerFactory retryingCallerFactory;
   private final Executor executor;
   private final QueueingFuture<V>[] tasks; // all the tasks
-  private volatile QueueingFuture<V> completed = null;
+  private final ArrayList<QueueingFuture> completedTasks; // completed tasks
   private volatile boolean cancelled = false;
   
   class QueueingFuture<T> implements RunnableFuture<T> {
@@ -49,12 +62,14 @@ public class ResultBoundedCompletionService<V> {
     private final int callTimeout;
     private final RpcRetryingCaller<T> retryingCaller;
     private boolean resultObtained = false;
+    private final int replicaId;  // replica id
 
 
-    public QueueingFuture(RetryingCallable<T> future, int callTimeout) {
+    public QueueingFuture(RetryingCallable<T> future, int callTimeout, int id) {
       this.future = future;
       this.callTimeout = callTimeout;
       this.retryingCaller = retryingCallerFactory.<T>newCaller();
+      this.replicaId = id;
     }
 
     @SuppressWarnings("unchecked")
@@ -70,8 +85,8 @@ public class ResultBoundedCompletionService<V> {
       } finally {
         synchronized (tasks) {
           // If this wasn't canceled then store the result.
-          if (!cancelled && completed == null) {
-            completed = (QueueingFuture<V>) QueueingFuture.this;
+          if (!cancelled) {
+            completedTasks.add(QueueingFuture.this);
           }
 
           // Notify just in case there was someone waiting and this was canceled.
@@ -80,6 +95,7 @@ public class ResultBoundedCompletionService<V> {
         }
       }
     }
+
     @Override
     public boolean cancel(boolean mayInterruptIfRunning) {
       if (resultObtained || exeEx != null) return false;
@@ -129,6 +145,14 @@ public class ResultBoundedCompletionService<V> {
 
       throw new TimeoutException("timeout=" + timeout + ", " + unit);
     }
+
+    public int getReplicaId() {
+      return replicaId;
+    }
+
+    public ExecutionException getExeEx() {
+      return exeEx;
+    }
   }
 
   @SuppressWarnings("unchecked")
@@ -138,27 +162,103 @@ public class ResultBoundedCompletionService<V> {
     this.retryingCallerFactory = retryingCallerFactory;
     this.executor = executor;
     this.tasks = new QueueingFuture[maxTasks];
+    this.completedTasks = new ArrayList<>(maxTasks);
   }
 
 
   public void submit(RetryingCallable<V> task, int callTimeout, int id) {
-    QueueingFuture<V> newFuture = new QueueingFuture<V>(task, callTimeout);
+    QueueingFuture<V> newFuture = new QueueingFuture<V>(task, callTimeout, id);
     executor.execute(Trace.wrap(newFuture));
     tasks[id] = newFuture;
   }
 
   public QueueingFuture<V> take() throws InterruptedException {
     synchronized (tasks) {
-      while (completed == null && !cancelled) tasks.wait();
+      while (!cancelled && (completedTasks.size() < 1)) tasks.wait();
     }
-    return completed;
+    return completedTasks.get(0);
   }
 
+  /**
+   * Poll for the first completed task whether it is a success or execution exception.
+   *
+   * @param timeout  - time to wait before it times out
+   * @param unit  - time unit for timeout
+   */
   public QueueingFuture<V> poll(long timeout, TimeUnit unit) throws InterruptedException {
+    return pollForSpecificCompletedTask(timeout, unit, 0);
+  }
+
+  /**
+   * Poll for the first successfully completed task whose completed order is in startIndex,
+   * endIndex(exclusive) range
+   *
+   * @param timeout  - time to wait before it times out
+   * @param unit  - time unit for timeout
+   * @param startIndex - start index, starting from 0, inclusive
+   * @param endIndex - end index, exclusive
+   *
+   * @return If within timeout time, there is no successfully completed task, return null; If all
+   *         tasks get execution exception, it will throw out the last execution exception,
+   *         otherwise return the first successfully completed task's result.
+   */
+  public QueueingFuture<V> pollForFirstSuccessfullyCompletedTask(long timeout, TimeUnit unit,
+      int startIndex, int endIndex)
+      throws InterruptedException, CancellationException, ExecutionException {
+
+    QueueingFuture<V>  f;
+    long start, duration;
+    for (int i = startIndex; i < endIndex; i ++) {
+
+      start = EnvironmentEdgeManager.currentTime();
+      f = pollForSpecificCompletedTask(timeout, unit, i);
+      duration = EnvironmentEdgeManager.currentTime() - start;
+
+      // Even with operationTimeout less than 0, still loop through the rest as there could
+      // be other completed tasks before operationTimeout.
+      timeout -= duration;
+
+      if (f == null) {
+        return null;
+      } else if (f.getExeEx() != null) {
+        // we continue here as we need to loop through all the results.
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Replica " + ((f == null) ? 0 : f.getReplicaId()) + " returns " +
+              f.getExeEx().getCause());
+        }
+
+        if (i == (endIndex - 1)) {
+          // Rethrow this exception
+          throw f.getExeEx();
+        }
+        continue;
+      }
+
+      return f;
+    }
+
+    // impossible to reach
+    return null;
+  }
+
+  /**
+   * Poll for the Nth completed task (index starts from 0 (the 1st), 1 (the second)...)
+   *
+   * @param timeout  - time to wait before it times out
+   * @param unit  - time unit for timeout
+   * @param index - the index(th) completed task, index starting from 0
+   */
+  private QueueingFuture<V> pollForSpecificCompletedTask(long timeout, TimeUnit unit, int index)
+      throws InterruptedException {
+    if (index < 0) {
+      return null;
+    }
+
     synchronized (tasks) {
-      if (completed == null && !cancelled) unit.timedWait(tasks, timeout);
+      if (!cancelled && (completedTasks.size() <= index)) unit.timedWait(tasks, timeout);
+      if (completedTasks.size() <= index) return null;
     }
-    return completed;
+    return completedTasks.get(index);
   }
 
   public void cancelAll() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a97aef51/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java
index aefa3bc..d1c40ab 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java
@@ -200,9 +200,12 @@ public class RpcRetryingCallerWithReadReplicas {
         : RegionReplicaUtil.DEFAULT_REPLICA_ID), cConnection, tableName, get.getRow());
     ResultBoundedCompletionService<Result> cs =
         new ResultBoundedCompletionService<Result>(this.rpcRetryingCallerFactory, pool, rl.size());
+    int startIndex = 0;
+    int endIndex = rl.size();
 
     if(isTargetReplicaSpecified) {
       addCallsForReplica(cs, rl, get.getReplicaId(), get.getReplicaId());
+      endIndex = 1;
     } else {
       addCallsForReplica(cs, rl, 0, 0);
       try {
@@ -212,7 +215,13 @@ public class RpcRetryingCallerWithReadReplicas {
           return f.get(); //great we got a response
         }
       } catch (ExecutionException e) {
-        throwEnrichedException(e, retries);
+        // We ignore the ExecutionException and continue with the secondary replicas
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Primary replica returns " + e.getCause());
+        }
+
+        // Skip the result from the primary as we know that there is something wrong
+        startIndex = 1;
       } catch (CancellationException e) {
         throw new InterruptedIOException();
       } catch (InterruptedException e) {
@@ -224,19 +233,14 @@ public class RpcRetryingCallerWithReadReplicas {
     }
 
     try {
-      try {
-        long start = EnvironmentEdgeManager.currentTime();
-        Future<Result> f = cs.poll(operationTimeout, TimeUnit.MILLISECONDS);
-        long duration = EnvironmentEdgeManager.currentTime() - start;
-        if (f == null) {
-          throw new RetriesExhaustedException("timed out after " + duration + " ms");
-        }
-        return f.get(operationTimeout - duration, TimeUnit.MILLISECONDS);
-      } catch (ExecutionException e) {
-        throwEnrichedException(e, retries);
-      } catch (TimeoutException te) {
+      Future<Result> f = cs.pollForFirstSuccessfullyCompletedTask(operationTimeout,
+          TimeUnit.MILLISECONDS, startIndex, endIndex);
+      if (f == null) {
         throw new RetriesExhaustedException("timed out after " + operationTimeout + " ms");
       }
+      return f.get();
+    } catch (ExecutionException e) {
+      throwEnrichedException(e, retries);
     } catch (CancellationException e) {
       throw new InterruptedIOException();
     } catch (InterruptedException e) {
@@ -247,6 +251,7 @@ public class RpcRetryingCallerWithReadReplicas {
       cs.cancelAll();
     }
 
+    LOG.error("Imposible? Arrive at an unreachable line..."); // unreachable
     return null; // unreachable
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a97aef51/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java
index 4e8647f..a030e67 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java
@@ -169,54 +169,70 @@ class ScannerCallableWithReplicas implements RetryingCallable<Result[]> {
     replicaSwitched.set(false);
     // submit call for the primary replica.
     addCallsForCurrentReplica(cs, rl);
+    int startIndex = 0;
 
     try {
       // wait for the timeout to see whether the primary responds back
       Future<Pair<Result[], ScannerCallable>> f = cs.poll(timeBeforeReplicas,
           TimeUnit.MICROSECONDS); // Yes, microseconds
       if (f != null) {
-        Pair<Result[], ScannerCallable> r = f.get(timeout, TimeUnit.MILLISECONDS);
+        // After poll, if f is not null, there must be a completed task
+        Pair<Result[], ScannerCallable> r = f.get();
         if (r != null && r.getSecond() != null) {
           updateCurrentlyServingReplica(r.getSecond(), r.getFirst(), done, pool);
         }
         return r == null ? null : r.getFirst(); //great we got a response
       }
     } catch (ExecutionException e) {
-      RpcRetryingCallerWithReadReplicas.throwEnrichedException(e, retries);
+      // We ignore the ExecutionException and continue with the replicas
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Scan with primary region returns " + e.getCause());
+      }
+
+      // If rl's size is 1 or scan's consitency is strong, it needs to throw
+      // out the exception from the primary replica
+      if ((rl.size() == 1) || (scan.getConsistency() == Consistency.STRONG)) {
+        // Rethrow the first exception
+        RpcRetryingCallerWithReadReplicas.throwEnrichedException(e, retries);
+      }
+
+      startIndex = 1;
     } catch (CancellationException e) {
       throw new InterruptedIOException(e.getMessage());
     } catch (InterruptedException e) {
       throw new InterruptedIOException(e.getMessage());
-    } catch (TimeoutException e) {
-      throw new InterruptedIOException(e.getMessage());
     }
 
     // submit call for the all of the secondaries at once
-    // TODO: this may be an overkill for large region replication
-    addCallsForOtherReplicas(cs, rl, 0, rl.size() - 1);
+    int endIndex = rl.size();
+    if (scan.getConsistency() == Consistency.STRONG) {
+      // When scan's consistency is strong, do not send to the secondaries
+      endIndex = 1;
+    } else {
+      // TODO: this may be an overkill for large region replication
+      addCallsForOtherReplicas(cs, rl, 0, rl.size() - 1);
+    }
 
     try {
-      long start = EnvironmentEdgeManager.currentTime();
-      Future<Pair<Result[], ScannerCallable>> f = cs.poll(timeout, TimeUnit.MILLISECONDS);
-      long duration = EnvironmentEdgeManager.currentTime() - start;
-      if (f != null) {
-        Pair<Result[], ScannerCallable> r = f.get(timeout - duration, TimeUnit.MILLISECONDS);
-        if (r != null && r.getSecond() != null) {
-          updateCurrentlyServingReplica(r.getSecond(), r.getFirst(), done, pool);
-        }
-        return r == null ? null : r.getFirst(); // great we got an answer
-      } else {
-        throw new IOException("Failed to get result within timeout, timeout="
-            + timeout + "ms");
+      Future<Pair<Result[], ScannerCallable>> f = cs.pollForFirstSuccessfullyCompletedTask(timeout,
+          TimeUnit.MILLISECONDS, startIndex, endIndex);
+
+      if (f == null) {
+        throw new IOException("Failed to get result within timeout, timeout=" + timeout + "ms");
+      }
+      Pair<Result[], ScannerCallable> r = f.get();
+
+      if (r != null && r.getSecond() != null) {
+        updateCurrentlyServingReplica(r.getSecond(), r.getFirst(), done, pool);
       }
+      return r == null ? null : r.getFirst(); // great we got an answer
+
     } catch (ExecutionException e) {
       RpcRetryingCallerWithReadReplicas.throwEnrichedException(e, retries);
     } catch (CancellationException e) {
       throw new InterruptedIOException(e.getMessage());
     } catch (InterruptedException e) {
       throw new InterruptedIOException(e.getMessage());
-    } catch (TimeoutException e) {
-      throw new InterruptedIOException(e.getMessage());
     } finally {
       // We get there because we were interrupted or because one or more of the
       // calls succeeded or failed. In all case, we stop all our tasks.
@@ -293,9 +309,7 @@ class ScannerCallableWithReplicas implements RetryingCallable<Result[]> {
   private void addCallsForOtherReplicas(
       ResultBoundedCompletionService<Pair<Result[], ScannerCallable>> cs, RegionLocations rl,
       int min, int max) {
-    if (scan.getConsistency() == Consistency.STRONG) {
-      return; // not scheduling on other replicas for strong consistency
-    }
+
     for (int id = min; id <= max; id++) {
       if (currentScannerCallable.id == id) {
         continue; //this was already scheduled earlier

http://git-wip-us.apache.org/repos/asf/hbase/blob/a97aef51/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
index 5967a69..617a8f7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
@@ -38,6 +38,8 @@ import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
+import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
 import org.apache.hadoop.hbase.regionserver.StorefileRefresherChore;
 import org.apache.hadoop.hbase.regionserver.TestHRegionServerBulkLoad;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
@@ -63,7 +65,7 @@ import java.util.concurrent.atomic.AtomicReference;
 public class TestReplicaWithCluster {
   private static final Log LOG = LogFactory.getLog(TestReplicaWithCluster.class);
 
-  private static final int NB_SERVERS = 2;
+  private static final int NB_SERVERS = 3;
   private static final byte[] row = TestReplicaWithCluster.class.getName().getBytes();
   private static final HBaseTestingUtility HTU = new HBaseTestingUtility();
 
@@ -110,6 +112,51 @@ public class TestReplicaWithCluster {
     }
   }
 
+  /**
+   * This copro is used to simulate region server down exception for Get and Scan
+   */
+  public static class RegionServerStoppedCopro extends BaseRegionObserver {
+
+    public RegionServerStoppedCopro() {
+    }
+
+    @Override
+    public void preGetOp(final ObserverContext<RegionCoprocessorEnvironment> e,
+        final Get get, final List<Cell> results) throws IOException {
+
+      int replicaId = e.getEnvironment().getRegion().getRegionInfo().getReplicaId();
+
+      // Fail for the primary replica and replica 1
+      if (e.getEnvironment().getRegion().getRegionInfo().getReplicaId() <= 1) {
+        LOG.info("Throw Region Server Stopped Exceptoin for replica id " + replicaId);
+        throw new RegionServerStoppedException("Server " +
+            e.getEnvironment().getRegionServerServices().getServerName()
+            + " not running");
+      } else {
+        LOG.info("We're replica region " + replicaId);
+      }
+    }
+
+    @Override
+    public RegionScanner preScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> e,
+        final Scan scan, final RegionScanner s) throws IOException {
+
+      int replicaId = e.getEnvironment().getRegion().getRegionInfo().getReplicaId();
+
+      // Fail for the primary replica and replica 1
+      if (e.getEnvironment().getRegion().getRegionInfo().getReplicaId() <= 1) {
+        LOG.info("Throw Region Server Stopped Exceptoin for replica id " + replicaId);
+        throw new RegionServerStoppedException("Server " +
+            e.getEnvironment().getRegionServerServices().getServerName()
+            + " not running");
+      } else {
+        LOG.info("We're replica region " + replicaId);
+      }
+
+      return null;
+    }
+  }
+
   @BeforeClass
   public static void beforeClass() throws Exception {
     // enable store file refreshing
@@ -124,13 +171,19 @@ public class TestReplicaWithCluster {
     HTU.getConfiguration().setInt("zookeeper.recovery.retry", 1);
     HTU.getConfiguration().setInt("zookeeper.recovery.retry.intervalmill", 10);
 
+    // Wait for primary call longer so make sure that it will get exception from the primary call
+    HTU.getConfiguration().setInt("hbase.client.primaryCallTimeout.get", 1000000);
+    HTU.getConfiguration().setInt("hbase.client.primaryCallTimeout.scan", 1000000);
+
     HTU.startMiniCluster(NB_SERVERS);
     HTU.getHBaseCluster().startMaster();
   }
 
   @AfterClass
   public static void afterClass() throws Exception {
-    HTU2.shutdownMiniCluster();
+    if (HTU2 != null) {
+      HTU2.shutdownMiniCluster();
+    }
     HTU.shutdownMiniCluster();
   }
 
@@ -382,4 +435,90 @@ public class TestReplicaWithCluster {
     HTU.getHBaseAdmin().disableTable(hdt.getTableName());
     HTU.deleteTable(hdt.getTableName());
   }
+
+  @Test
+  public void testReplicaGetWithPrimaryDown() throws IOException {
+    // Create table then get the single region for our new table.
+    HTableDescriptor hdt = HTU.createTableDescriptor("testCreateDeleteTable");
+    hdt.setRegionReplication(NB_SERVERS);
+    hdt.addCoprocessor(RegionServerStoppedCopro.class.getName());
+    try {
+      // Retry less so it can fail faster
+      HTU.getConfiguration().setInt("hbase.client.retries.number", 1);
+
+      Table table = HTU.createTable(hdt, new byte[][] { f }, null);
+
+      Put p = new Put(row);
+      p.addColumn(f, row, row);
+      table.put(p);
+
+      // Flush so it can be picked by the replica refresher thread
+      HTU.flush(table.getName());
+
+      // Sleep for some time until data is picked up by replicas
+      try {
+        Thread.sleep(2 * REFRESH_PERIOD);
+      } catch (InterruptedException e1) {
+        LOG.error(e1);
+      }
+
+      // But if we ask for stale we will get it
+      Get g = new Get(row);
+      g.setConsistency(Consistency.TIMELINE);
+      Result r = table.get(g);
+      Assert.assertTrue(r.isStale());
+    } finally {
+      HTU.getConfiguration().unset("hbase.client.retries.number");
+      HTU.getHBaseAdmin().disableTable(hdt.getTableName());
+      HTU.deleteTable(hdt.getTableName());
+    }
+  }
+
+  @Test
+  public void testReplicaScanWithPrimaryDown() throws IOException {
+    // Create table then get the single region for our new table.
+    HTableDescriptor hdt = HTU.createTableDescriptor("testCreateDeleteTable");
+    hdt.setRegionReplication(NB_SERVERS);
+    hdt.addCoprocessor(RegionServerStoppedCopro.class.getName());
+
+    try {
+      // Retry less so it can fail faster
+      HTU.getConfiguration().setInt("hbase.client.retries.number", 1);
+
+      Table table = HTU.createTable(hdt, new byte[][] { f }, null);
+
+      Put p = new Put(row);
+      p.addColumn(f, row, row);
+      table.put(p);
+
+      // Flush so it can be picked by the replica refresher thread
+      HTU.flush(table.getName());
+
+      // Sleep for some time until data is picked up by replicas
+      try {
+        Thread.sleep(2 * REFRESH_PERIOD);
+      } catch (InterruptedException e1) {
+        LOG.error(e1);
+      }
+
+      // But if we ask for stale we will get it
+      // Instantiating the Scan class
+      Scan scan = new Scan();
+
+      // Scanning the required columns
+      scan.addFamily(f);
+      scan.setConsistency(Consistency.TIMELINE);
+
+      // Getting the scan result
+      ResultScanner scanner = table.getScanner(scan);
+
+      Result r = scanner.next();
+
+      Assert.assertTrue(r.isStale());
+    } finally {
+      HTU.getConfiguration().unset("hbase.client.retries.number");
+      HTU.getHBaseAdmin().disableTable(hdt.getTableName());
+      HTU.deleteTable(hdt.getTableName());
+    }
+  }
 }


[26/50] [abbrv] hbase git commit: HBASE-16653 Backport HBASE-11393 to branches which support namespace

Posted by la...@apache.org.
HBASE-16653 Backport HBASE-11393 to branches which support namespace

Signed-off-by: chenheng <ch...@apache.org>


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

Branch: refs/heads/branch-1
Commit: 66941910bd07462fe496c5bbb591f4071f77b8fb
Parents: 6df7554
Author: Guanghao Zhang <zg...@gmail.com>
Authored: Mon Sep 26 19:33:43 2016 +0800
Committer: chenheng <ch...@apache.org>
Committed: Tue Oct 18 09:12:47 2016 +0800

----------------------------------------------------------------------
 .../client/replication/ReplicationAdmin.java    |   84 +-
 .../replication/ReplicationPeerConfig.java      |   16 +-
 .../replication/ReplicationPeerZKImpl.java      |   80 +-
 .../hbase/replication/ReplicationPeers.java     |   15 +-
 .../replication/ReplicationPeersZKImpl.java     |   60 +-
 .../replication/ReplicationSerDeHelper.java     |  189 +++
 .../replication/ReplicationStateZKBase.java     |   17 +
 .../protobuf/generated/ZooKeeperProtos.java     | 1155 +++++++++++++++++-
 .../src/main/protobuf/ZooKeeper.proto           |    8 +-
 .../org/apache/hadoop/hbase/master/HMaster.java |    8 +
 .../replication/master/TableCFsUpdater.java     |  120 ++
 .../hbase/client/TestReplicaWithCluster.java    |    5 +-
 .../replication/TestReplicationAdmin.java       |  193 +--
 .../cleaner/TestReplicationHFileCleaner.java    |    2 +-
 .../replication/TestMasterReplication.java      |    9 +-
 .../replication/TestMultiSlaveReplication.java  |    8 +-
 .../replication/TestPerTableCFReplication.java  |  153 ++-
 .../hbase/replication/TestReplicationBase.java  |    4 +-
 .../replication/TestReplicationSmallTests.java  |    4 +-
 .../replication/TestReplicationStateBasic.java  |   20 +-
 .../replication/TestReplicationSyncUpTool.java  |    4 +-
 .../TestReplicationTrackerZKImpl.java           |   10 +-
 .../replication/TestReplicationWithTags.java    |    4 +-
 .../replication/master/TestTableCFsUpdater.java |  164 +++
 .../TestReplicationSourceManager.java           |    2 +-
 ...sibilityLabelReplicationWithExpAsString.java |    5 +-
 .../TestVisibilityLabelsReplication.java        |    5 +-
 .../apache/hadoop/hbase/util/TestHBaseFsck.java |    5 +-
 .../src/main/ruby/hbase/replication_admin.rb    |   49 +-
 .../src/main/ruby/shell/commands/add_peer.rb    |    4 +-
 .../ruby/shell/commands/append_peer_tableCFs.rb |    2 +-
 .../src/main/ruby/shell/commands/list_peers.rb  |    6 +-
 .../ruby/shell/commands/remove_peer_tableCFs.rb |    4 +-
 .../ruby/shell/commands/set_peer_tableCFs.rb    |    4 +-
 .../hbase/client/TestReplicationShell.java      |    2 +-
 .../test/ruby/hbase/replication_admin_test.rb   |  118 +-
 36 files changed, 2167 insertions(+), 371 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/66941910/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
index 1304396..9fca28b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
@@ -55,6 +55,7 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerZKImpl;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
+import org.apache.hadoop.hbase.replication.ReplicationSerDeHelper;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 
@@ -184,8 +185,8 @@ public class ReplicationAdmin implements Closeable {
   @Deprecated
   public void addPeer(String id, String clusterKey, String tableCFs)
     throws ReplicationException {
-    this.replicationPeers.addPeer(id,
-      new ReplicationPeerConfig().setClusterKey(clusterKey), tableCFs);
+    this.addPeer(id, new ReplicationPeerConfig().setClusterKey(clusterKey),
+      parseTableCFsFromConfig(tableCFs));
   }
 
   /**
@@ -199,7 +200,19 @@ public class ReplicationAdmin implements Closeable {
    */
   public void addPeer(String id, ReplicationPeerConfig peerConfig,
       Map<TableName, ? extends Collection<String>> tableCfs) throws ReplicationException {
-    this.replicationPeers.addPeer(id, peerConfig, getTableCfsStr(tableCfs));
+    if (tableCfs != null) {
+      peerConfig.setTableCFsMap(tableCfs);
+    }
+    this.replicationPeers.addPeer(id, peerConfig);
+  }
+
+  /**
+   * Add a new remote slave cluster for replication.
+   * @param id a short name that identifies the cluster
+   * @param peerConfig configuration for the replication slave cluster
+   */
+  public void addPeer(String id, ReplicationPeerConfig peerConfig) throws ReplicationException {
+    this.replicationPeers.addPeer(id, peerConfig);
   }
 
   public void updatePeerConfig(String id, ReplicationPeerConfig peerConfig)
@@ -208,52 +221,7 @@ public class ReplicationAdmin implements Closeable {
   }
 
   public static Map<TableName, List<String>> parseTableCFsFromConfig(String tableCFsConfig) {
-    if (tableCFsConfig == null || tableCFsConfig.trim().length() == 0) {
-      return null;
-    }
-
-    Map<TableName, List<String>> tableCFsMap = null;
-    // TODO: This should be a PB object rather than a String to be parsed!! See HBASE-11393
-    // parse out (table, cf-list) pairs from tableCFsConfig
-    // format: "table1:cf1,cf2;table2:cfA,cfB"
-    String[] tables = tableCFsConfig.split(";");
-    for (String tab : tables) {
-      // 1 ignore empty table config
-      tab = tab.trim();
-      if (tab.length() == 0) {
-        continue;
-      }
-      // 2 split to "table" and "cf1,cf2"
-      //   for each table: "table:cf1,cf2" or "table"
-      String[] pair = tab.split(":");
-      String tabName = pair[0].trim();
-      if (pair.length > 2 || tabName.length() == 0) {
-        LOG.error("ignore invalid tableCFs setting: " + tab);
-        continue;
-      }
-
-      // 3 parse "cf1,cf2" part to List<cf>
-      List<String> cfs = null;
-      if (pair.length == 2) {
-        String[] cfsList = pair[1].split(",");
-        for (String cf : cfsList) {
-          String cfName = cf.trim();
-          if (cfName.length() > 0) {
-            if (cfs == null) {
-              cfs = new ArrayList<String>();
-            }
-            cfs.add(cfName);
-          }
-        }
-      }
-
-      // 4 put <table, List<cf>> to map
-      if (tableCFsMap == null) {
-        tableCFsMap = new HashMap<TableName, List<String>>();
-      }
-      tableCFsMap.put(TableName.valueOf(tabName), cfs);
-    }
-    return tableCFsMap;
+    return ReplicationSerDeHelper.parseTableCFsFromConfig(tableCFsConfig);
   }
 
   @VisibleForTesting
@@ -338,7 +306,7 @@ public class ReplicationAdmin implements Closeable {
    * @param id a short name that identifies the cluster
    */
   public String getPeerTableCFs(String id) throws ReplicationException {
-    return this.replicationPeers.getPeerTableCFsConfig(id);
+    return ReplicationSerDeHelper.convertToString(this.replicationPeers.getPeerTableCFsConfig(id));
   }
 
   /**
@@ -348,7 +316,7 @@ public class ReplicationAdmin implements Closeable {
    */
   @Deprecated
   public void setPeerTableCFs(String id, String tableCFs) throws ReplicationException {
-    this.replicationPeers.setPeerTableCFsConfig(id, tableCFs);
+    this.setPeerTableCFs(id, parseTableCFsFromConfig(tableCFs));
   }
 
   /**
@@ -357,7 +325,7 @@ public class ReplicationAdmin implements Closeable {
    * @param tableCfs table-cfs config str
    */
   public void appendPeerTableCFs(String id, String tableCfs) throws ReplicationException {
-    appendPeerTableCFs(id, parseTableCFsFromConfig(tableCfs));
+    appendPeerTableCFs(id, ReplicationSerDeHelper.parseTableCFsFromConfig(tableCfs));
   }
 
   /**
@@ -370,7 +338,7 @@ public class ReplicationAdmin implements Closeable {
     if (tableCfs == null) {
       throw new ReplicationException("tableCfs is null");
     }
-    Map<TableName, List<String>> preTableCfs = parseTableCFsFromConfig(getPeerTableCFs(id));
+    Map<TableName, List<String>> preTableCfs = this.replicationPeers.getPeerTableCFsConfig(id);
     if (preTableCfs == null) {
       setPeerTableCFs(id, tableCfs);
       return;
@@ -406,7 +374,7 @@ public class ReplicationAdmin implements Closeable {
    * @throws ReplicationException
    */
   public void removePeerTableCFs(String id, String tableCf) throws ReplicationException {
-    removePeerTableCFs(id, parseTableCFsFromConfig(tableCf));
+    removePeerTableCFs(id, ReplicationSerDeHelper.parseTableCFsFromConfig(tableCf));
   }
 
   /**
@@ -421,7 +389,7 @@ public class ReplicationAdmin implements Closeable {
       throw new ReplicationException("tableCfs is null");
     }
 
-    Map<TableName, List<String>> preTableCfs = parseTableCFsFromConfig(getPeerTableCFs(id));
+    Map<TableName, List<String>> preTableCfs = this.replicationPeers.getPeerTableCFsConfig(id);
     if (preTableCfs == null) {
       throw new ReplicationException("Table-Cfs for peer" + id + " is null");
     }
@@ -464,7 +432,7 @@ public class ReplicationAdmin implements Closeable {
    */
   public void setPeerTableCFs(String id, Map<TableName, ? extends Collection<String>> tableCfs)
       throws ReplicationException {
-    this.replicationPeers.setPeerTableCFsConfig(id, getTableCfsStr(tableCfs));
+    this.replicationPeers.setPeerTableCFsConfig(id, tableCfs);
   }
 
   /**
@@ -658,8 +626,8 @@ public class ReplicationAdmin implements Closeable {
       try {
         Pair<ReplicationPeerConfig, Configuration> pair = this.replicationPeers.getPeerConf(peerId);
         Configuration peerConf = pair.getSecond();
-        ReplicationPeer peer = new ReplicationPeerZKImpl(peerConf, peerId, pair.getFirst(),
-            parseTableCFsFromConfig(this.getPeerTableCFs(peerId)));
+        ReplicationPeer peer = new ReplicationPeerZKImpl(zkw, pair.getSecond(),
+          peerId, pair.getFirst(), this.connection);
         listOfPeers.add(peer);
       } catch (ReplicationException e) {
         LOG.warn("Failed to get valid replication peers. "

http://git-wip-us.apache.org/repos/asf/hbase/blob/66941910/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java
index 043b38f..e2c7bc7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java
@@ -18,10 +18,13 @@
 
 package org.apache.hadoop.hbase.replication;
 
+import java.util.Collection;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
 
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -37,7 +40,7 @@ public class ReplicationPeerConfig {
   private String replicationEndpointImpl;
   private final Map<byte[], byte[]> peerData;
   private final Map<String, String> configuration;
-
+  private Map<TableName, ? extends Collection<String>> tableCFsMap = null;
 
   public ReplicationPeerConfig() {
     this.peerData = new TreeMap<byte[], byte[]>(Bytes.BYTES_COMPARATOR);
@@ -78,10 +81,21 @@ public class ReplicationPeerConfig {
     return configuration;
   }
 
+  public Map<TableName, List<String>> getTableCFsMap() {
+    return (Map<TableName, List<String>>) tableCFsMap;
+  }
+
+  public void setTableCFsMap(Map<TableName, ? extends Collection<String>> tableCFsMap) {
+    this.tableCFsMap = tableCFsMap;
+  }
+
   @Override
   public String toString() {
     StringBuilder builder = new StringBuilder("clusterKey=").append(clusterKey).append(",");
     builder.append("replicationEndpointImpl=").append(replicationEndpointImpl);
+    if (tableCFsMap != null) {
+      builder.append(tableCFsMap.toString());
+    }
     return builder.toString();
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/66941910/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
index 6b10015..382545d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
@@ -42,7 +42,8 @@ import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.NodeExistsException;
 
 @InterfaceAudience.Private
-public class ReplicationPeerZKImpl implements ReplicationPeer, Abortable, Closeable {
+public class ReplicationPeerZKImpl extends ReplicationStateZKBase implements ReplicationPeer,
+    Abortable, Closeable {
   private static final Log LOG = LogFactory.getLog(ReplicationPeerZKImpl.class);
 
   private ReplicationPeerConfig peerConfig;
@@ -52,8 +53,8 @@ public class ReplicationPeerZKImpl implements ReplicationPeer, Abortable, Closea
   private final Configuration conf;
 
   private PeerStateTracker peerStateTracker;
-  private TableCFsTracker tableCFsTracker;
   private PeerConfigTracker peerConfigTracker;
+
   /**
    * Constructor that takes all the objects required to communicate with the specified peer, except
    * for the region server addresses.
@@ -61,39 +62,23 @@ public class ReplicationPeerZKImpl implements ReplicationPeer, Abortable, Closea
    * @param id string representation of this peer's identifier
    * @param peerConfig configuration for the replication peer
    */
-  public ReplicationPeerZKImpl(Configuration conf, String id, ReplicationPeerConfig peerConfig)
-      throws ReplicationException {
-    this.conf = conf;
-    this.peerConfig = peerConfig;
-    this.id = id;
-  }
-  
-  /**
-   * Constructor that takes all the objects required to communicate with the specified peer, except
-   * for the region server addresses.
-   * @param conf configuration object to this peer
-   * @param id string representation of this peer's identifier
-   * @param peerConfig configuration for the replication peer
-   * @param tableCFs table-cf configuration for this peer
-   */
-  public ReplicationPeerZKImpl(Configuration conf, String id, ReplicationPeerConfig peerConfig,
-      Map<TableName, List<String>> tableCFs) throws ReplicationException {
+  public ReplicationPeerZKImpl(ZooKeeperWatcher zkWatcher, Configuration conf, String id,
+      ReplicationPeerConfig peerConfig, Abortable abortable) throws ReplicationException {
+    super(zkWatcher, conf, abortable);
     this.conf = conf;
     this.peerConfig = peerConfig;
     this.id = id;
-    this.tableCFs = tableCFs;
   }
 
   /**
    * start a state tracker to check whether this peer is enabled or not
    *
-   * @param zookeeper zk watcher for the local cluster
    * @param peerStateNode path to zk node which stores peer state
    * @throws KeeperException
    */
-  public void startStateTracker(ZooKeeperWatcher zookeeper, String peerStateNode)
+  public void startStateTracker(String peerStateNode)
       throws KeeperException {
-    ensurePeerEnabled(zookeeper, peerStateNode);
+    ensurePeerEnabled(peerStateNode);
     this.peerStateTracker = new PeerStateTracker(peerStateNode, zookeeper, this);
     this.peerStateTracker.start();
     try {
@@ -112,25 +97,6 @@ public class ReplicationPeerZKImpl implements ReplicationPeer, Abortable, Closea
 
   /**
    * start a table-cfs tracker to listen the (table, cf-list) map change
-   *
-   * @param zookeeper zk watcher for the local cluster
-   * @param tableCFsNode path to zk node which stores table-cfs
-   * @throws KeeperException
-   */
-  public void startTableCFsTracker(ZooKeeperWatcher zookeeper, String tableCFsNode)
-    throws KeeperException {
-    this.tableCFsTracker = new TableCFsTracker(tableCFsNode, zookeeper,
-        this);
-    this.tableCFsTracker.start();
-    this.readTableCFsZnode();
-  }
-
-  private void readTableCFsZnode() {
-    String currentTableCFs = Bytes.toString(tableCFsTracker.getData(false));
-    this.tableCFs = ReplicationAdmin.parseTableCFsFromConfig(currentTableCFs);
-  }
-  /**
-   * start a table-cfs tracker to listen the (table, cf-list) map change
    * @param zookeeper
    * @param peerConfigNode path to zk node which stores table-cfs
    * @throws KeeperException
@@ -154,6 +120,7 @@ public class ReplicationPeerZKImpl implements ReplicationPeer, Abortable, Closea
     }
     return this.peerConfig;
   }
+
   @Override
   public PeerState getPeerState() {
     return peerState;
@@ -192,6 +159,7 @@ public class ReplicationPeerZKImpl implements ReplicationPeer, Abortable, Closea
    */
   @Override
   public Map<TableName, List<String>> getTableCFs() {
+    this.tableCFs = peerConfig.getTableCFsMap();
     return this.tableCFs;
   }
 
@@ -260,7 +228,7 @@ public class ReplicationPeerZKImpl implements ReplicationPeer, Abortable, Closea
    * @throws NodeExistsException
    * @throws KeeperException
    */
-  private static boolean ensurePeerEnabled(final ZooKeeperWatcher zookeeper, final String path)
+  private boolean ensurePeerEnabled(final String path)
       throws NodeExistsException, KeeperException {
     if (ZKUtil.checkExists(zookeeper, path) == -1) {
       // There is a race b/w PeerWatcher and ReplicationZookeeper#add method to create the
@@ -297,32 +265,6 @@ public class ReplicationPeerZKImpl implements ReplicationPeer, Abortable, Closea
   }
 
   /**
-   * Tracker for (table, cf-list) map of this peer
-   */
-  public class TableCFsTracker extends ZooKeeperNodeTracker {
-
-    public TableCFsTracker(String tableCFsZNode, ZooKeeperWatcher watcher,
-        Abortable abortable) {
-      super(watcher, tableCFsZNode, abortable);
-    }
-    
-    @Override
-    public synchronized void nodeCreated(String path) {
-      if (path.equals(node)) {
-        super.nodeCreated(path);
-        readTableCFsZnode();
-      }
-    }
-
-    @Override
-    public synchronized void nodeDataChanged(String path) {
-      if (path.equals(node)) {
-        super.nodeDataChanged(path);
-      }
-    }
-  }
-
-  /**
    * Tracker for PeerConfigNode of this peer
    */
   public class PeerConfigTracker extends ZooKeeperNodeTracker {

http://git-wip-us.apache.org/repos/asf/hbase/blob/66941910/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
index b8d04b4..37d157a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
@@ -18,6 +18,7 @@
  */
 package org.apache.hadoop.hbase.replication;
 
+import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -49,10 +50,8 @@ public interface ReplicationPeers {
    * Add a new remote slave cluster for replication.
    * @param peerId a short that identifies the cluster
    * @param peerConfig configuration for the replication slave cluster
-   * @param tableCFs the table and column-family list which will be replicated for this peer or null
-   *          for all table and column families
    */
-  void addPeer(String peerId, ReplicationPeerConfig peerConfig, String tableCFs)
+  void addPeer(String peerId, ReplicationPeerConfig peerConfig)
       throws ReplicationException;
 
   /**
@@ -78,17 +77,19 @@ public interface ReplicationPeers {
   void disablePeer(String peerId) throws ReplicationException;
 
   /**
-   * Get the table and column-family list string of the peer from ZK.
+   * Get the table and column-family list of the peer from ZK.
    * @param peerId a short that identifies the cluster
    */
-  public String getPeerTableCFsConfig(String peerId) throws ReplicationException;
+  public Map<TableName, List<String>> getPeerTableCFsConfig(String peerId)
+      throws ReplicationException;
 
   /**
-   * Set the table and column-family list string of the peer to ZK.
+   * Set the table and column-family list of the peer to ZK.
    * @param peerId a short that identifies the cluster
    * @param tableCFs the table and column-family list which will be replicated for this peer
    */
-  public void setPeerTableCFsConfig(String peerId, String tableCFs) throws ReplicationException;
+  public void setPeerTableCFsConfig(String peerId,
+      Map<TableName, ? extends Collection<String>> tableCFs) throws ReplicationException;
 
   /**
    * Get the table and column-family-list map of the peer.

http://git-wip-us.apache.org/repos/asf/hbase/blob/66941910/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
index d717b0b..bb9842b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.replication;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -78,15 +79,15 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
 
   // Map of peer clusters keyed by their id
   private Map<String, ReplicationPeerZKImpl> peerClusters;
-  private final String tableCFsNodeName;
   private final ReplicationQueuesClient queuesClient;
+  private Abortable abortable;
 
   private static final Log LOG = LogFactory.getLog(ReplicationPeersZKImpl.class);
 
   public ReplicationPeersZKImpl(final ZooKeeperWatcher zk, final Configuration conf,
       final ReplicationQueuesClient queuesClient, Abortable abortable) {
     super(zk, conf, abortable);
-    this.tableCFsNodeName = conf.get("zookeeper.znode.replication.peers.tableCFs", "tableCFs");
+    this.abortable = abortable;
     this.peerClusters = new ConcurrentHashMap<String, ReplicationPeerZKImpl>();
     this.queuesClient = queuesClient;
   }
@@ -104,7 +105,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
   }
 
   @Override
-  public void addPeer(String id, ReplicationPeerConfig peerConfig, String tableCFs)
+  public void addPeer(String id, ReplicationPeerConfig peerConfig)
       throws ReplicationException {
     try {
       if (peerExists(id)) {
@@ -129,18 +130,15 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
       ZKUtil.createWithParents(this.zookeeper, this.peersZNode);
 
       List<ZKUtilOp> listOfOps = new ArrayList<ZKUtil.ZKUtilOp>();
-      ZKUtilOp op1 = ZKUtilOp.createAndFailSilent(ZKUtil.joinZNode(this.peersZNode, id),
+      ZKUtilOp op1 = ZKUtilOp.createAndFailSilent(getPeerNode(id),
         ReplicationSerDeHelper.toByteArray(peerConfig));
       // There is a race (if hbase.zookeeper.useMulti is false)
       // b/w PeerWatcher and ReplicationZookeeper#add method to create the
       // peer-state znode. This happens while adding a peer
       // The peer state data is set as "ENABLED" by default.
       ZKUtilOp op2 = ZKUtilOp.createAndFailSilent(getPeerStateNode(id), ENABLED_ZNODE_BYTES);
-      String tableCFsStr = (tableCFs == null) ? "" : tableCFs;
-      ZKUtilOp op3 = ZKUtilOp.createAndFailSilent(getTableCFsNode(id), Bytes.toBytes(tableCFsStr));
       listOfOps.add(op1);
       listOfOps.add(op2);
-      listOfOps.add(op3);
       ZKUtil.multiOrSequential(this.zookeeper, listOfOps, false);
       // A peer is enabled by default
     } catch (KeeperException e) {
@@ -175,13 +173,17 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
   }
 
   @Override
-  public String getPeerTableCFsConfig(String id) throws ReplicationException {
+  public Map<TableName, List<String>> getPeerTableCFsConfig(String id) throws ReplicationException {
     try {
       if (!peerExists(id)) {
         throw new IllegalArgumentException("peer " + id + " doesn't exist");
       }
       try {
-        return Bytes.toString(ZKUtil.getData(this.zookeeper, getTableCFsNode(id)));
+        ReplicationPeerConfig rpc = getReplicationPeerConfig(id);
+        if (rpc == null) {
+          throw new ReplicationException("Unable to get tableCFs of the peer with id=" + id);
+        }
+        return rpc.getTableCFsMap();
       } catch (Exception e) {
         throw new ReplicationException(e);
       }
@@ -191,20 +193,22 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
   }
 
   @Override
-  public void setPeerTableCFsConfig(String id, String tableCFsStr) throws ReplicationException {
+  public void setPeerTableCFsConfig(String id,
+      Map<TableName, ? extends Collection<String>> tableCFs) throws ReplicationException {
     try {
       if (!peerExists(id)) {
         throw new IllegalArgumentException("Cannot set peer tableCFs because id=" + id
             + " does not exist.");
       }
-      String tableCFsZKNode = getTableCFsNode(id);
-      byte[] tableCFs = Bytes.toBytes(tableCFsStr);
-      if (ZKUtil.checkExists(this.zookeeper, tableCFsZKNode) != -1) {
-        ZKUtil.setData(this.zookeeper, tableCFsZKNode, tableCFs);
-      } else {
-        ZKUtil.createAndWatch(this.zookeeper, tableCFsZKNode, tableCFs);
+      ReplicationPeerConfig rpc = getReplicationPeerConfig(id);
+      if (rpc == null) {
+        throw new ReplicationException("Unable to get tableCFs of the peer with id=" + id);
       }
-      LOG.info("Peer tableCFs with id= " + id + " is now " + tableCFsStr);
+      rpc.setTableCFsMap(tableCFs);
+      ZKUtil.setData(this.zookeeper, getPeerNode(id),
+          ReplicationSerDeHelper.toByteArray(rpc));
+      LOG.info("Peer tableCFs with id= " + id + " is now "
+          + ReplicationSerDeHelper.convertToString(tableCFs));
     } catch (KeeperException e) {
       throw new ReplicationException("Unable to change tableCFs of the peer with id=" + id, e);
     }
@@ -289,7 +293,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
   @Override
   public ReplicationPeerConfig getReplicationPeerConfig(String peerId)
       throws ReplicationException {
-    String znode = ZKUtil.joinZNode(this.peersZNode, peerId);
+    String znode = getPeerNode(peerId);
     byte[] data = null;
     try {
       data = ZKUtil.getData(this.zookeeper, znode);
@@ -458,14 +462,6 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
     return true;
   }
 
-  private String getTableCFsNode(String id) {
-    return ZKUtil.joinZNode(this.peersZNode, ZKUtil.joinZNode(id, this.tableCFsNodeName));
-  }
-
-  private String getPeerStateNode(String id) {
-    return ZKUtil.joinZNode(this.peersZNode, ZKUtil.joinZNode(id, this.peerStateNodeName));
-  }
-
   /**
    * Update the state znode of a peer cluster.
    * @param id
@@ -506,22 +502,16 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
     }
     Configuration peerConf = pair.getSecond();
 
-    ReplicationPeerZKImpl peer = new ReplicationPeerZKImpl(peerConf, peerId, pair.getFirst());
+    ReplicationPeerZKImpl peer = new ReplicationPeerZKImpl(zookeeper, peerConf, peerId,
+        pair.getFirst(), abortable);
     try {
-      peer.startStateTracker(this.zookeeper, this.getPeerStateNode(peerId));
+      peer.startStateTracker(getPeerStateNode(peerId));
     } catch (KeeperException e) {
       throw new ReplicationException("Error starting the peer state tracker for peerId=" +
           peerId, e);
     }
 
     try {
-      peer.startTableCFsTracker(this.zookeeper, this.getTableCFsNode(peerId));
-    } catch (KeeperException e) {
-      throw new ReplicationException("Error starting the peer tableCFs tracker for peerId=" +
-          peerId, e);
-    }
-
-    try {
       peer.startPeerConfigTracker(this.zookeeper, this.getPeerNode(peerId));
     }
     catch(KeeperException e) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/66941910/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationSerDeHelper.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationSerDeHelper.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationSerDeHelper.java
index 05f909d..cdb95f7f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationSerDeHelper.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationSerDeHelper.java
@@ -19,8 +19,10 @@
 package org.apache.hadoop.hbase.replication;
 
 import com.google.protobuf.ByteString;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
@@ -28,8 +30,13 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Strings;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 
 @InterfaceAudience.Private
@@ -39,6 +46,175 @@ public final class ReplicationSerDeHelper {
 
   private ReplicationSerDeHelper() {}
 
+  /** convert map to TableCFs Object */
+  public static ZooKeeperProtos.TableCF[] convert(
+      Map<TableName, ? extends Collection<String>> tableCfs) {
+    if (tableCfs == null) {
+      return null;
+    }
+    List<ZooKeeperProtos.TableCF> tableCFList = new ArrayList<>();
+    ZooKeeperProtos.TableCF.Builder tableCFBuilder =  ZooKeeperProtos.TableCF.newBuilder();
+    for (Map.Entry<TableName, ? extends Collection<String>> entry : tableCfs.entrySet()) {
+      tableCFBuilder.clear();
+      tableCFBuilder.setTableName(ProtobufUtil.toProtoTableName(entry.getKey()));
+      Collection<String> v = entry.getValue();
+      if (v != null && !v.isEmpty()) {
+        for (String value : entry.getValue()) {
+          tableCFBuilder.addFamilies(ByteString.copyFromUtf8(value));
+        }
+      }
+      tableCFList.add(tableCFBuilder.build());
+    }
+    return tableCFList.toArray(new ZooKeeperProtos.TableCF[tableCFList.size()]);
+  }
+
+  public static String convertToString(Map<TableName, ? extends Collection<String>> tableCfs) {
+    if (tableCfs == null) {
+      return null;
+    }
+    return convert(convert(tableCfs));
+  }
+
+  /**
+   *  Convert string to TableCFs Object.
+   *  This is only for read TableCFs information from TableCF node.
+   *  Input String Format: ns1.table1:cf1,cf2;ns2.table2:cfA,cfB;ns3.table3.
+   * */
+  public static ZooKeeperProtos.TableCF[] convert(String tableCFsConfig) {
+    if (tableCFsConfig == null || tableCFsConfig.trim().length() == 0) {
+      return null;
+    }
+    List<ZooKeeperProtos.TableCF> tableCFList = new ArrayList<>();
+    ZooKeeperProtos.TableCF.Builder tableCFBuilder = ZooKeeperProtos.TableCF.newBuilder();
+
+    String[] tables = tableCFsConfig.split(";");
+    for (String tab : tables) {
+      // 1 ignore empty table config
+      tab = tab.trim();
+      if (tab.length() == 0) {
+        continue;
+      }
+      // 2 split to "table" and "cf1,cf2"
+      //   for each table: "table#cf1,cf2" or "table"
+      String[] pair = tab.split(":");
+      String tabName = pair[0].trim();
+      if (pair.length > 2 || tabName.length() == 0) {
+        LOG.info("incorrect format:" + tableCFsConfig);
+        continue;
+      }
+
+      tableCFBuilder.clear();
+      // split namespace from tableName
+      String ns = "default";
+      String tName = tabName;
+      String[] dbs = tabName.split("\\.");
+      if (dbs != null && dbs.length == 2) {
+        ns = dbs[0];
+        tName = dbs[1];
+      }
+      tableCFBuilder.setTableName(
+        ProtobufUtil.toProtoTableName(TableName.valueOf(ns, tName)));
+
+      // 3 parse "cf1,cf2" part to List<cf>
+      if (pair.length == 2) {
+        String[] cfsList = pair[1].split(",");
+        for (String cf : cfsList) {
+          String cfName = cf.trim();
+          if (cfName.length() > 0) {
+            tableCFBuilder.addFamilies(ByteString.copyFromUtf8(cfName));
+          }
+        }
+      }
+      tableCFList.add(tableCFBuilder.build());
+    }
+    return tableCFList.toArray(new ZooKeeperProtos.TableCF[tableCFList.size()]);
+  }
+
+  /**
+   *  Convert TableCFs Object to String.
+   *  Output String Format: ns1.table1:cf1,cf2;ns2.table2:cfA,cfB;table3
+   * */
+  public static String convert(ZooKeeperProtos.TableCF[] tableCFs) {
+    StringBuilder sb = new StringBuilder();
+    for (int i = 0, n = tableCFs.length; i < n; i++) {
+      ZooKeeperProtos.TableCF tableCF = tableCFs[i];
+      String namespace = tableCF.getTableName().getNamespace().toStringUtf8();
+      if (!Strings.isEmpty(namespace)) {
+        sb.append(namespace).append(".").
+            append(tableCF.getTableName().getQualifier().toStringUtf8())
+            .append(":");
+      } else {
+        sb.append(tableCF.getTableName().toString()).append(":");
+      }
+      for (int j = 0; j < tableCF.getFamiliesCount(); j++) {
+        sb.append(tableCF.getFamilies(j).toStringUtf8()).append(",");
+      }
+      sb.deleteCharAt(sb.length() - 1).append(";");
+    }
+    if (sb.length() > 0) {
+      sb.deleteCharAt(sb.length() - 1);
+    }
+    return sb.toString();
+  }
+
+  /**
+   *  Get TableCF in TableCFs, if not exist, return null.
+   * */
+  public static ZooKeeperProtos.TableCF getTableCF(ZooKeeperProtos.TableCF[] tableCFs,
+                                           String table) {
+    for (int i = 0, n = tableCFs.length; i < n; i++) {
+      ZooKeeperProtos.TableCF tableCF = tableCFs[i];
+      if (tableCF.getTableName().getQualifier().toStringUtf8().equals(table)) {
+        return tableCF;
+      }
+    }
+    return null;
+  }
+
+  /**
+   *  Parse bytes into TableCFs.
+   *  It is used for backward compatibility.
+   *  Old format bytes have no PB_MAGIC Header
+   * */
+  public static ZooKeeperProtos.TableCF[] parseTableCFs(byte[] bytes) throws IOException {
+    if (bytes == null) {
+      return null;
+    }
+    return ReplicationSerDeHelper.convert(Bytes.toString(bytes));
+  }
+
+  /**
+   *  Convert tableCFs string into Map.
+   * */
+  public static Map<TableName, List<String>> parseTableCFsFromConfig(String tableCFsConfig) {
+    ZooKeeperProtos.TableCF[] tableCFs = convert(tableCFsConfig);
+    return convert2Map(tableCFs);
+  }
+
+  /**
+   *  Convert tableCFs Object to Map.
+   * */
+  public static Map<TableName, List<String>> convert2Map(ZooKeeperProtos.TableCF[] tableCFs) {
+    if (tableCFs == null || tableCFs.length == 0) {
+      return null;
+    }
+    Map<TableName, List<String>> tableCFsMap = new HashMap<TableName, List<String>>();
+    for (int i = 0, n = tableCFs.length; i < n; i++) {
+      ZooKeeperProtos.TableCF tableCF = tableCFs[i];
+      List<String> families = new ArrayList<>();
+      for (int j = 0, m = tableCF.getFamiliesCount(); j < m; j++) {
+        families.add(tableCF.getFamilies(j).toStringUtf8());
+      }
+      if (families.size() > 0) {
+        tableCFsMap.put(ProtobufUtil.toTableName(tableCF.getTableName()), families);
+      } else {
+        tableCFsMap.put(ProtobufUtil.toTableName(tableCF.getTableName()), null);
+      }
+    }
+
+    return tableCFsMap;
+  }
+
   /**
    * @param bytes Content of a peer znode.
    * @return ClusterKey parsed from the passed bytes.
@@ -82,6 +258,12 @@ public final class ReplicationSerDeHelper {
     for (HBaseProtos.NameStringPair pair : peer.getConfigurationList()) {
       peerConfig.getConfiguration().put(pair.getName(), pair.getValue());
     }
+
+    Map<TableName, ? extends Collection<String>> tableCFsMap = convert2Map(
+      peer.getTableCfsList().toArray(new ZooKeeperProtos.TableCF[peer.getTableCfsCount()]));
+    if (tableCFsMap != null) {
+      peerConfig.setTableCFsMap(tableCFsMap);
+    }
     return peerConfig;
   }
 
@@ -119,6 +301,13 @@ public final class ReplicationSerDeHelper {
           .build());
     }
 
+    ZooKeeperProtos.TableCF[] tableCFs = convert(peerConfig.getTableCFsMap());
+    if (tableCFs != null) {
+      for (int i = 0; i < tableCFs.length; i++) {
+        builder.addTableCfs(tableCFs[i]);
+      }
+    }
+
     return builder.build();
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/66941910/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
index ed9359d..d0c3513 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
@@ -30,6 +30,8 @@ import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.zookeeper.KeeperException;
 
+import com.google.common.annotations.VisibleForTesting;
+
 
 /**
  * This is a base class for maintaining replication state in zookeeper.
@@ -52,6 +54,9 @@ public abstract class ReplicationStateZKBase {
   protected final String hfileRefsZNode;
   /** The cluster key of the local cluster */
   protected final String ourClusterKey;
+  /** The name of the znode that contains tableCFs */
+  protected final String tableCFsNodeName;
+
   protected final ZooKeeperWatcher zookeeper;
   protected final Configuration conf;
   protected final Abortable abortable;
@@ -77,6 +82,7 @@ public abstract class ReplicationStateZKBase {
     String hfileRefsZNodeName = conf.get(ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_KEY,
       ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_DEFAULT);
     this.peerStateNodeName = conf.get("zookeeper.znode.replication.peers.state", "peer-state");
+    this.tableCFsNodeName = conf.get("zookeeper.znode.replication.peers.tableCFs", "tableCFs");
     this.ourClusterKey = ZKConfig.getZooKeeperClusterKey(this.conf);
     this.replicationZNode = ZKUtil.joinZNode(this.zookeeper.baseZNode, replicationZNodeName);
     this.peersZNode = ZKUtil.joinZNode(replicationZNode, peersZNodeName);
@@ -119,6 +125,17 @@ public abstract class ReplicationStateZKBase {
     return path.split("/").length == peersZNode.split("/").length + 1;
   }
 
+  @VisibleForTesting
+  protected String getTableCFsNode(String id) {
+    return ZKUtil.joinZNode(this.peersZNode, ZKUtil.joinZNode(id, this.tableCFsNodeName));
+  }
+
+  @VisibleForTesting
+  protected String getPeerStateNode(String id) {
+    return ZKUtil.joinZNode(this.peersZNode, ZKUtil.joinZNode(id, this.peerStateNodeName));
+  }
+
+  @VisibleForTesting
   protected String getPeerNode(String id) {
     return ZKUtil.joinZNode(this.peersZNode, id);
   }


[11/50] [abbrv] hbase git commit: HBASE-15109 HM/RS failed to start when "fs.hdfs.impl.disable.cache" is set to true (Pankaj Kumar)

Posted by la...@apache.org.
HBASE-15109 HM/RS failed to start when "fs.hdfs.impl.disable.cache" is set to true (Pankaj Kumar)


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

Branch: refs/heads/branch-1
Commit: bf0382719698de6368dea3064c322a20c126fcf4
Parents: 6db4ef8
Author: tedyu <yu...@gmail.com>
Authored: Wed Oct 12 07:35:50 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Wed Oct 12 07:35:50 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/regionserver/ShutdownHook.java  | 9 +++++----
 1 file changed, 5 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/bf038271/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ShutdownHook.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ShutdownHook.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ShutdownHook.java
index b5f37e8..83e4ea8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ShutdownHook.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ShutdownHook.java
@@ -199,10 +199,11 @@ public class ShutdownHook {
         throw new RuntimeException("Client finalizer is null, can't suppress!");
       }
       synchronized (fsShutdownHooks) {
-        if (!fsShutdownHooks.containsKey(hdfsClientFinalizer) &&
-            !ShutdownHookManager.deleteShutdownHook(hdfsClientFinalizer)) {
-          throw new RuntimeException("Failed suppression of fs shutdown hook: " +
-            hdfsClientFinalizer);
+        boolean isFSCacheDisabled = fs.getConf().getBoolean("fs.hdfs.impl.disable.cache", false);
+        if (!isFSCacheDisabled && !fsShutdownHooks.containsKey(hdfsClientFinalizer)
+            && !ShutdownHookManager.deleteShutdownHook(hdfsClientFinalizer)) {
+          throw new RuntimeException(
+              "Failed suppression of fs shutdown hook: " + hdfsClientFinalizer);
         }
         Integer refs = fsShutdownHooks.get(hdfsClientFinalizer);
         fsShutdownHooks.put(hdfsClientFinalizer, refs == null ? 1 : refs + 1);


[18/50] [abbrv] hbase git commit: HBASE-16816 HMaster.move() should throw exception if region to move is not online (Allan Yang)

Posted by la...@apache.org.
HBASE-16816 HMaster.move() should throw exception if region to move is not online (Allan Yang)


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

Branch: refs/heads/branch-1
Commit: 57d3e9e7562b3e7410293d1462075758b1793f88
Parents: ca58187
Author: tedyu <yu...@gmail.com>
Authored: Thu Oct 13 11:11:11 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Thu Oct 13 11:11:11 2016 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hbase/master/HMaster.java   | 5 +++++
 .../java/org/apache/hadoop/hbase/master/TestWarmupRegion.java   | 2 ++
 2 files changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/57d3e9e7/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index b1051f5..8db8324 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -1501,6 +1501,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
       getRegionState(Bytes.toString(encodedRegionName));
     if (regionState == null) {
       throw new UnknownRegionException(Bytes.toStringBinary(encodedRegionName));
+    } else if (!assignmentManager.getRegionStates()
+        .isRegionOnline(regionState.getRegion())) {
+      throw new HBaseIOException(
+          "moving region not onlined: " + regionState.getRegion() + ", "
+              + regionState);
     }
 
     HRegionInfo hri = regionState.getRegion();

http://git-wip-us.apache.org/repos/asf/hbase/blob/57d3e9e7/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestWarmupRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestWarmupRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestWarmupRegion.java
index 9046397..533363d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestWarmupRegion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestWarmupRegion.java
@@ -157,6 +157,8 @@ public class TestWarmupRegion {
        HRegionServer rs = TEST_UTIL.getMiniHBaseCluster().getRegionServer(serverid);
        byte [] destName = Bytes.toBytes(rs.getServerName().toString());
        TEST_UTIL.getMiniHBaseCluster().getMaster().move(info.getEncodedNameAsBytes(), destName);
+       //wait region online
+       TEST_UTIL.waitUntilNoRegionsInTransition(1000);
        serverid = (serverid + 1) % 2;
      }
    }


[25/50] [abbrv] hbase git commit: HBASE-16653 Backport HBASE-11393 to branches which support namespace

Posted by la...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/66941910/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java
index 09479c4..955995f 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java
@@ -5032,6 +5032,719 @@ public final class ZooKeeperProtos {
     // @@protoc_insertion_point(class_scope:hbase.pb.Table)
   }
 
+  public interface TableCFOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // optional .hbase.pb.TableName table_name = 1;
+    /**
+     * <code>optional .hbase.pb.TableName table_name = 1;</code>
+     */
+    boolean hasTableName();
+    /**
+     * <code>optional .hbase.pb.TableName table_name = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName();
+    /**
+     * <code>optional .hbase.pb.TableName table_name = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder();
+
+    // repeated bytes families = 2;
+    /**
+     * <code>repeated bytes families = 2;</code>
+     */
+    java.util.List<com.google.protobuf.ByteString> getFamiliesList();
+    /**
+     * <code>repeated bytes families = 2;</code>
+     */
+    int getFamiliesCount();
+    /**
+     * <code>repeated bytes families = 2;</code>
+     */
+    com.google.protobuf.ByteString getFamilies(int index);
+  }
+  /**
+   * Protobuf type {@code hbase.pb.TableCF}
+   */
+  public static final class TableCF extends
+      com.google.protobuf.GeneratedMessage
+      implements TableCFOrBuilder {
+    // Use TableCF.newBuilder() to construct.
+    private TableCF(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private TableCF(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final TableCF defaultInstance;
+    public static TableCF getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public TableCF getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private TableCF(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000001) == 0x00000001)) {
+                subBuilder = tableName_.toBuilder();
+              }
+              tableName_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(tableName_);
+                tableName_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000001;
+              break;
+            }
+            case 18: {
+              if (!((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
+                families_ = new java.util.ArrayList<com.google.protobuf.ByteString>();
+                mutable_bitField0_ |= 0x00000002;
+              }
+              families_.add(input.readBytes());
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
+          families_ = java.util.Collections.unmodifiableList(families_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_TableCF_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_TableCF_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.class, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<TableCF> PARSER =
+        new com.google.protobuf.AbstractParser<TableCF>() {
+      public TableCF parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new TableCF(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<TableCF> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // optional .hbase.pb.TableName table_name = 1;
+    public static final int TABLE_NAME_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName tableName_;
+    /**
+     * <code>optional .hbase.pb.TableName table_name = 1;</code>
+     */
+    public boolean hasTableName() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional .hbase.pb.TableName table_name = 1;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName() {
+      return tableName_;
+    }
+    /**
+     * <code>optional .hbase.pb.TableName table_name = 1;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
+      return tableName_;
+    }
+
+    // repeated bytes families = 2;
+    public static final int FAMILIES_FIELD_NUMBER = 2;
+    private java.util.List<com.google.protobuf.ByteString> families_;
+    /**
+     * <code>repeated bytes families = 2;</code>
+     */
+    public java.util.List<com.google.protobuf.ByteString>
+        getFamiliesList() {
+      return families_;
+    }
+    /**
+     * <code>repeated bytes families = 2;</code>
+     */
+    public int getFamiliesCount() {
+      return families_.size();
+    }
+    /**
+     * <code>repeated bytes families = 2;</code>
+     */
+    public com.google.protobuf.ByteString getFamilies(int index) {
+      return families_.get(index);
+    }
+
+    private void initFields() {
+      tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+      families_ = java.util.Collections.emptyList();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (hasTableName()) {
+        if (!getTableName().isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeMessage(1, tableName_);
+      }
+      for (int i = 0; i < families_.size(); i++) {
+        output.writeBytes(2, families_.get(i));
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, tableName_);
+      }
+      {
+        int dataSize = 0;
+        for (int i = 0; i < families_.size(); i++) {
+          dataSize += com.google.protobuf.CodedOutputStream
+            .computeBytesSizeNoTag(families_.get(i));
+        }
+        size += dataSize;
+        size += 1 * getFamiliesList().size();
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF other = (org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF) obj;
+
+      boolean result = true;
+      result = result && (hasTableName() == other.hasTableName());
+      if (hasTableName()) {
+        result = result && getTableName()
+            .equals(other.getTableName());
+      }
+      result = result && getFamiliesList()
+          .equals(other.getFamiliesList());
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasTableName()) {
+        hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER;
+        hash = (53 * hash) + getTableName().hashCode();
+      }
+      if (getFamiliesCount() > 0) {
+        hash = (37 * hash) + FAMILIES_FIELD_NUMBER;
+        hash = (53 * hash) + getFamiliesList().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code hbase.pb.TableCF}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_TableCF_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_TableCF_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.class, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getTableNameFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        if (tableNameBuilder_ == null) {
+          tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+        } else {
+          tableNameBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        families_ = java.util.Collections.emptyList();
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_TableCF_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF build() {
+        org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF result = new org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (tableNameBuilder_ == null) {
+          result.tableName_ = tableName_;
+        } else {
+          result.tableName_ = tableNameBuilder_.build();
+        }
+        if (((bitField0_ & 0x00000002) == 0x00000002)) {
+          families_ = java.util.Collections.unmodifiableList(families_);
+          bitField0_ = (bitField0_ & ~0x00000002);
+        }
+        result.families_ = families_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.getDefaultInstance()) return this;
+        if (other.hasTableName()) {
+          mergeTableName(other.getTableName());
+        }
+        if (!other.families_.isEmpty()) {
+          if (families_.isEmpty()) {
+            families_ = other.families_;
+            bitField0_ = (bitField0_ & ~0x00000002);
+          } else {
+            ensureFamiliesIsMutable();
+            families_.addAll(other.families_);
+          }
+          onChanged();
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (hasTableName()) {
+          if (!getTableName().isInitialized()) {
+            
+            return false;
+          }
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // optional .hbase.pb.TableName table_name = 1;
+      private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public boolean hasTableName() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName() {
+        if (tableNameBuilder_ == null) {
+          return tableName_;
+        } else {
+          return tableNameBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public Builder setTableName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
+        if (tableNameBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          tableName_ = value;
+          onChanged();
+        } else {
+          tableNameBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public Builder setTableName(
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) {
+        if (tableNameBuilder_ == null) {
+          tableName_ = builderForValue.build();
+          onChanged();
+        } else {
+          tableNameBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public Builder mergeTableName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
+        if (tableNameBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              tableName_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance()) {
+            tableName_ =
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.newBuilder(tableName_).mergeFrom(value).buildPartial();
+          } else {
+            tableName_ = value;
+          }
+          onChanged();
+        } else {
+          tableNameBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public Builder clearTableName() {
+        if (tableNameBuilder_ == null) {
+          tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+          onChanged();
+        } else {
+          tableNameBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder getTableNameBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getTableNameFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
+        if (tableNameBuilder_ != null) {
+          return tableNameBuilder_.getMessageOrBuilder();
+        } else {
+          return tableName_;
+        }
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
+          getTableNameFieldBuilder() {
+        if (tableNameBuilder_ == null) {
+          tableNameBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
+                  tableName_,
+                  getParentForChildren(),
+                  isClean());
+          tableName_ = null;
+        }
+        return tableNameBuilder_;
+      }
+
+      // repeated bytes families = 2;
+      private java.util.List<com.google.protobuf.ByteString> families_ = java.util.Collections.emptyList();
+      private void ensureFamiliesIsMutable() {
+        if (!((bitField0_ & 0x00000002) == 0x00000002)) {
+          families_ = new java.util.ArrayList<com.google.protobuf.ByteString>(families_);
+          bitField0_ |= 0x00000002;
+         }
+      }
+      /**
+       * <code>repeated bytes families = 2;</code>
+       */
+      public java.util.List<com.google.protobuf.ByteString>
+          getFamiliesList() {
+        return java.util.Collections.unmodifiableList(families_);
+      }
+      /**
+       * <code>repeated bytes families = 2;</code>
+       */
+      public int getFamiliesCount() {
+        return families_.size();
+      }
+      /**
+       * <code>repeated bytes families = 2;</code>
+       */
+      public com.google.protobuf.ByteString getFamilies(int index) {
+        return families_.get(index);
+      }
+      /**
+       * <code>repeated bytes families = 2;</code>
+       */
+      public Builder setFamilies(
+          int index, com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureFamiliesIsMutable();
+        families_.set(index, value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated bytes families = 2;</code>
+       */
+      public Builder addFamilies(com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureFamiliesIsMutable();
+        families_.add(value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated bytes families = 2;</code>
+       */
+      public Builder addAllFamilies(
+          java.lang.Iterable<? extends com.google.protobuf.ByteString> values) {
+        ensureFamiliesIsMutable();
+        super.addAll(values, families_);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated bytes families = 2;</code>
+       */
+      public Builder clearFamilies() {
+        families_ = java.util.Collections.emptyList();
+        bitField0_ = (bitField0_ & ~0x00000002);
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:hbase.pb.TableCF)
+    }
+
+    static {
+      defaultInstance = new TableCF(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.TableCF)
+  }
+
   public interface ReplicationPeerOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
 
@@ -5129,6 +5842,31 @@ public final class ZooKeeperProtos {
      */
     org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPairOrBuilder getConfigurationOrBuilder(
         int index);
+
+    // repeated .hbase.pb.TableCF table_cfs = 5;
+    /**
+     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF> 
+        getTableCfsList();
+    /**
+     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF getTableCfs(int index);
+    /**
+     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+     */
+    int getTableCfsCount();
+    /**
+     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFOrBuilder> 
+        getTableCfsOrBuilderList();
+    /**
+     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFOrBuilder getTableCfsOrBuilder(
+        int index);
   }
   /**
    * Protobuf type {@code hbase.pb.ReplicationPeer}
@@ -5212,6 +5950,14 @@ public final class ZooKeeperProtos {
               configuration_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair.PARSER, extensionRegistry));
               break;
             }
+            case 42: {
+              if (!((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
+                tableCfs_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF>();
+                mutable_bitField0_ |= 0x00000010;
+              }
+              tableCfs_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.PARSER, extensionRegistry));
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -5226,6 +5972,9 @@ public final class ZooKeeperProtos {
         if (((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
           configuration_ = java.util.Collections.unmodifiableList(configuration_);
         }
+        if (((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
+          tableCfs_ = java.util.Collections.unmodifiableList(tableCfs_);
+        }
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
@@ -5431,11 +6180,48 @@ public final class ZooKeeperProtos {
       return configuration_.get(index);
     }
 
+    // repeated .hbase.pb.TableCF table_cfs = 5;
+    public static final int TABLE_CFS_FIELD_NUMBER = 5;
+    private java.util.List<org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF> tableCfs_;
+    /**
+     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+     */
+    public java.util.List<org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF> getTableCfsList() {
+      return tableCfs_;
+    }
+    /**
+     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+     */
+    public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFOrBuilder> 
+        getTableCfsOrBuilderList() {
+      return tableCfs_;
+    }
+    /**
+     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+     */
+    public int getTableCfsCount() {
+      return tableCfs_.size();
+    }
+    /**
+     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF getTableCfs(int index) {
+      return tableCfs_.get(index);
+    }
+    /**
+     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFOrBuilder getTableCfsOrBuilder(
+        int index) {
+      return tableCfs_.get(index);
+    }
+
     private void initFields() {
       clusterkey_ = "";
       replicationEndpointImpl_ = "";
       data_ = java.util.Collections.emptyList();
       configuration_ = java.util.Collections.emptyList();
+      tableCfs_ = java.util.Collections.emptyList();
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -5458,6 +6244,12 @@ public final class ZooKeeperProtos {
           return false;
         }
       }
+      for (int i = 0; i < getTableCfsCount(); i++) {
+        if (!getTableCfs(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
       memoizedIsInitialized = 1;
       return true;
     }
@@ -5477,6 +6269,9 @@ public final class ZooKeeperProtos {
       for (int i = 0; i < configuration_.size(); i++) {
         output.writeMessage(4, configuration_.get(i));
       }
+      for (int i = 0; i < tableCfs_.size(); i++) {
+        output.writeMessage(5, tableCfs_.get(i));
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -5502,6 +6297,10 @@ public final class ZooKeeperProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeMessageSize(4, configuration_.get(i));
       }
+      for (int i = 0; i < tableCfs_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(5, tableCfs_.get(i));
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -5539,6 +6338,8 @@ public final class ZooKeeperProtos {
           .equals(other.getDataList());
       result = result && getConfigurationList()
           .equals(other.getConfigurationList());
+      result = result && getTableCfsList()
+          .equals(other.getTableCfsList());
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -5568,6 +6369,10 @@ public final class ZooKeeperProtos {
         hash = (37 * hash) + CONFIGURATION_FIELD_NUMBER;
         hash = (53 * hash) + getConfigurationList().hashCode();
       }
+      if (getTableCfsCount() > 0) {
+        hash = (37 * hash) + TABLE_CFS_FIELD_NUMBER;
+        hash = (53 * hash) + getTableCfsList().hashCode();
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -5676,6 +6481,7 @@ public final class ZooKeeperProtos {
         if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
           getDataFieldBuilder();
           getConfigurationFieldBuilder();
+          getTableCfsFieldBuilder();
         }
       }
       private static Builder create() {
@@ -5700,6 +6506,12 @@ public final class ZooKeeperProtos {
         } else {
           configurationBuilder_.clear();
         }
+        if (tableCfsBuilder_ == null) {
+          tableCfs_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000010);
+        } else {
+          tableCfsBuilder_.clear();
+        }
         return this;
       }
 
@@ -5754,6 +6566,15 @@ public final class ZooKeeperProtos {
         } else {
           result.configuration_ = configurationBuilder_.build();
         }
+        if (tableCfsBuilder_ == null) {
+          if (((bitField0_ & 0x00000010) == 0x00000010)) {
+            tableCfs_ = java.util.Collections.unmodifiableList(tableCfs_);
+            bitField0_ = (bitField0_ & ~0x00000010);
+          }
+          result.tableCfs_ = tableCfs_;
+        } else {
+          result.tableCfs_ = tableCfsBuilder_.build();
+        }
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -5832,6 +6653,32 @@ public final class ZooKeeperProtos {
             }
           }
         }
+        if (tableCfsBuilder_ == null) {
+          if (!other.tableCfs_.isEmpty()) {
+            if (tableCfs_.isEmpty()) {
+              tableCfs_ = other.tableCfs_;
+              bitField0_ = (bitField0_ & ~0x00000010);
+            } else {
+              ensureTableCfsIsMutable();
+              tableCfs_.addAll(other.tableCfs_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.tableCfs_.isEmpty()) {
+            if (tableCfsBuilder_.isEmpty()) {
+              tableCfsBuilder_.dispose();
+              tableCfsBuilder_ = null;
+              tableCfs_ = other.tableCfs_;
+              bitField0_ = (bitField0_ & ~0x00000010);
+              tableCfsBuilder_ = 
+                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+                   getTableCfsFieldBuilder() : null;
+            } else {
+              tableCfsBuilder_.addAllMessages(other.tableCfs_);
+            }
+          }
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -5853,6 +6700,12 @@ public final class ZooKeeperProtos {
             return false;
           }
         }
+        for (int i = 0; i < getTableCfsCount(); i++) {
+          if (!getTableCfs(i).isInitialized()) {
+            
+            return false;
+          }
+        }
         return true;
       }
 
@@ -6533,6 +7386,246 @@ public final class ZooKeeperProtos {
         return configurationBuilder_;
       }
 
+      // repeated .hbase.pb.TableCF table_cfs = 5;
+      private java.util.List<org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF> tableCfs_ =
+        java.util.Collections.emptyList();
+      private void ensureTableCfsIsMutable() {
+        if (!((bitField0_ & 0x00000010) == 0x00000010)) {
+          tableCfs_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF>(tableCfs_);
+          bitField0_ |= 0x00000010;
+         }
+      }
+
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.Builder, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFOrBuilder> tableCfsBuilder_;
+
+      /**
+       * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF> getTableCfsList() {
+        if (tableCfsBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(tableCfs_);
+        } else {
+          return tableCfsBuilder_.getMessageList();
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+       */
+      public int getTableCfsCount() {
+        if (tableCfsBuilder_ == null) {
+          return tableCfs_.size();
+        } else {
+          return tableCfsBuilder_.getCount();
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF getTableCfs(int index) {
+        if (tableCfsBuilder_ == null) {
+          return tableCfs_.get(index);
+        } else {
+          return tableCfsBuilder_.getMessage(index);
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+       */
+      public Builder setTableCfs(
+          int index, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF value) {
+        if (tableCfsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureTableCfsIsMutable();
+          tableCfs_.set(index, value);
+          onChanged();
+        } else {
+          tableCfsBuilder_.setMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+       */
+      public Builder setTableCfs(
+          int index, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.Builder builderForValue) {
+        if (tableCfsBuilder_ == null) {
+          ensureTableCfsIsMutable();
+          tableCfs_.set(index, builderForValue.build());
+          onChanged();
+        } else {
+          tableCfsBuilder_.setMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+       */
+      public Builder addTableCfs(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF value) {
+        if (tableCfsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureTableCfsIsMutable();
+          tableCfs_.add(value);
+          onChanged();
+        } else {
+          tableCfsBuilder_.addMessage(value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+       */
+      public Builder addTableCfs(
+          int index, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF value) {
+        if (tableCfsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureTableCfsIsMutable();
+          tableCfs_.add(index, value);
+          onChanged();
+        } else {
+          tableCfsBuilder_.addMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+       */
+      public Builder addTableCfs(
+          org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.Builder builderForValue) {
+        if (tableCfsBuilder_ == null) {
+          ensureTableCfsIsMutable();
+          tableCfs_.add(builderForValue.build());
+          onChanged();
+        } else {
+          tableCfsBuilder_.addMessage(builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+       */
+      public Builder addTableCfs(
+          int index, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.Builder builderForValue) {
+        if (tableCfsBuilder_ == null) {
+          ensureTableCfsIsMutable();
+          tableCfs_.add(index, builderForValue.build());
+          onChanged();
+        } else {
+          tableCfsBuilder_.addMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+       */
+      public Builder addAllTableCfs(
+          java.lang.Iterable<? extends org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF> values) {
+        if (tableCfsBuilder_ == null) {
+          ensureTableCfsIsMutable();
+          super.addAll(values, tableCfs_);
+          onChanged();
+        } else {
+          tableCfsBuilder_.addAllMessages(values);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+       */
+      public Builder clearTableCfs() {
+        if (tableCfsBuilder_ == null) {
+          tableCfs_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000010);
+          onChanged();
+        } else {
+          tableCfsBuilder_.clear();
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+       */
+      public Builder removeTableCfs(int index) {
+        if (tableCfsBuilder_ == null) {
+          ensureTableCfsIsMutable();
+          tableCfs_.remove(index);
+          onChanged();
+        } else {
+          tableCfsBuilder_.remove(index);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.Builder getTableCfsBuilder(
+          int index) {
+        return getTableCfsFieldBuilder().getBuilder(index);
+      }
+      /**
+       * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFOrBuilder getTableCfsOrBuilder(
+          int index) {
+        if (tableCfsBuilder_ == null) {
+          return tableCfs_.get(index);  } else {
+          return tableCfsBuilder_.getMessageOrBuilder(index);
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+       */
+      public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFOrBuilder> 
+           getTableCfsOrBuilderList() {
+        if (tableCfsBuilder_ != null) {
+          return tableCfsBuilder_.getMessageOrBuilderList();
+        } else {
+          return java.util.Collections.unmodifiableList(tableCfs_);
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.Builder addTableCfsBuilder() {
+        return getTableCfsFieldBuilder().addBuilder(
+            org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.Builder addTableCfsBuilder(
+          int index) {
+        return getTableCfsFieldBuilder().addBuilder(
+            index, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.Builder> 
+           getTableCfsBuilderList() {
+        return getTableCfsFieldBuilder().getBuilderList();
+      }
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.Builder, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFOrBuilder> 
+          getTableCfsFieldBuilder() {
+        if (tableCfsBuilder_ == null) {
+          tableCfsBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.Builder, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFOrBuilder>(
+                  tableCfs_,
+                  ((bitField0_ & 0x00000010) == 0x00000010),
+                  getParentForChildren(),
+                  isClean());
+          tableCfs_ = null;
+        }
+        return tableCfsBuilder_;
+      }
+
       // @@protoc_insertion_point(builder_scope:hbase.pb.ReplicationPeer)
     }
 
@@ -9756,6 +10849,11 @@ public final class ZooKeeperProtos {
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
       internal_static_hbase_pb_Table_fieldAccessorTable;
   private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_hbase_pb_TableCF_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_hbase_pb_TableCF_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ReplicationPeer_descriptor;
   private static
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
@@ -9815,23 +10913,26 @@ public final class ZooKeeperProtos {
       "LOG_SPLITTING\020\001\022\016\n\nLOG_REPLAY\020\002\"w\n\005Table" +
       "\022-\n\005state\030\001 \002(\0162\025.hbase.pb.Table.State:\007",
       "ENABLED\"?\n\005State\022\013\n\007ENABLED\020\000\022\014\n\010DISABLE" +
-      "D\020\001\022\r\n\tDISABLING\020\002\022\014\n\010ENABLING\020\003\"\237\001\n\017Rep" +
-      "licationPeer\022\022\n\nclusterkey\030\001 \002(\t\022\037\n\027repl" +
-      "icationEndpointImpl\030\002 \001(\t\022&\n\004data\030\003 \003(\0132" +
-      "\030.hbase.pb.BytesBytesPair\022/\n\rconfigurati" +
-      "on\030\004 \003(\0132\030.hbase.pb.NameStringPair\"g\n\020Re" +
-      "plicationState\022/\n\005state\030\001 \002(\0162 .hbase.pb" +
-      ".ReplicationState.State\"\"\n\005State\022\013\n\007ENAB" +
-      "LED\020\000\022\014\n\010DISABLED\020\001\"+\n\027ReplicationHLogPo" +
-      "sition\022\020\n\010position\030\001 \002(\003\"%\n\017ReplicationL",
-      "ock\022\022\n\nlock_owner\030\001 \002(\t\"\252\001\n\tTableLock\022\'\n" +
-      "\ntable_name\030\001 \001(\0132\023.hbase.pb.TableName\022(" +
-      "\n\nlock_owner\030\002 \001(\0132\024.hbase.pb.ServerName" +
-      "\022\021\n\tthread_id\030\003 \001(\003\022\021\n\tis_shared\030\004 \001(\010\022\017" +
-      "\n\007purpose\030\005 \001(\t\022\023\n\013create_time\030\006 \001(\003\"\036\n\013" +
-      "SwitchState\022\017\n\007enabled\030\001 \001(\010BE\n*org.apac" +
-      "he.hadoop.hbase.protobuf.generatedB\017ZooK" +
-      "eeperProtosH\001\210\001\001\240\001\001"
+      "D\020\001\022\r\n\tDISABLING\020\002\022\014\n\010ENABLING\020\003\"D\n\007Tabl" +
+      "eCF\022\'\n\ntable_name\030\001 \001(\0132\023.hbase.pb.Table" +
+      "Name\022\020\n\010families\030\002 \003(\014\"\305\001\n\017ReplicationPe" +
+      "er\022\022\n\nclusterkey\030\001 \002(\t\022\037\n\027replicationEnd" +
+      "pointImpl\030\002 \001(\t\022&\n\004data\030\003 \003(\0132\030.hbase.pb" +
+      ".BytesBytesPair\022/\n\rconfiguration\030\004 \003(\0132\030" +
+      ".hbase.pb.NameStringPair\022$\n\ttable_cfs\030\005 " +
+      "\003(\0132\021.hbase.pb.TableCF\"g\n\020ReplicationSta" +
+      "te\022/\n\005state\030\001 \002(\0162 .hbase.pb.Replication",
+      "State.State\"\"\n\005State\022\013\n\007ENABLED\020\000\022\014\n\010DIS" +
+      "ABLED\020\001\"+\n\027ReplicationHLogPosition\022\020\n\010po" +
+      "sition\030\001 \002(\003\"%\n\017ReplicationLock\022\022\n\nlock_" +
+      "owner\030\001 \002(\t\"\252\001\n\tTableLock\022\'\n\ntable_name\030" +
+      "\001 \001(\0132\023.hbase.pb.TableName\022(\n\nlock_owner" +
+      "\030\002 \001(\0132\024.hbase.pb.ServerName\022\021\n\tthread_i" +
+      "d\030\003 \001(\003\022\021\n\tis_shared\030\004 \001(\010\022\017\n\007purpose\030\005 " +
+      "\001(\t\022\023\n\013create_time\030\006 \001(\003\"\036\n\013SwitchState\022" +
+      "\017\n\007enabled\030\001 \001(\010BE\n*org.apache.hadoop.hb" +
+      "ase.protobuf.generatedB\017ZooKeeperProtosH",
+      "\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -9874,38 +10975,44 @@ public final class ZooKeeperProtos {
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_Table_descriptor,
               new java.lang.String[] { "State", });
-          internal_static_hbase_pb_ReplicationPeer_descriptor =
+          internal_static_hbase_pb_TableCF_descriptor =
             getDescriptor().getMessageTypes().get(6);
+          internal_static_hbase_pb_TableCF_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_hbase_pb_TableCF_descriptor,
+              new java.lang.String[] { "TableName", "Families", });
+          internal_static_hbase_pb_ReplicationPeer_descriptor =
+            getDescriptor().getMessageTypes().get(7);
           internal_static_hbase_pb_ReplicationPeer_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_ReplicationPeer_descriptor,
-              new java.lang.String[] { "Clusterkey", "ReplicationEndpointImpl", "Data", "Configuration", });
+              new java.lang.String[] { "Clusterkey", "ReplicationEndpointImpl", "Data", "Configuration", "TableCfs", });
           internal_static_hbase_pb_ReplicationState_descriptor =
-            getDescriptor().getMessageTypes().get(7);
+            getDescriptor().getMessageTypes().get(8);
           internal_static_hbase_pb_ReplicationState_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_ReplicationState_descriptor,
               new java.lang.String[] { "State", });
           internal_static_hbase_pb_ReplicationHLogPosition_descriptor =
-            getDescriptor().getMessageTypes().get(8);
+            getDescriptor().getMessageTypes().get(9);
           internal_static_hbase_pb_ReplicationHLogPosition_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_ReplicationHLogPosition_descriptor,
               new java.lang.String[] { "Position", });
           internal_static_hbase_pb_ReplicationLock_descriptor =
-            getDescriptor().getMessageTypes().get(9);
+            getDescriptor().getMessageTypes().get(10);
           internal_static_hbase_pb_ReplicationLock_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_ReplicationLock_descriptor,
               new java.lang.String[] { "LockOwner", });
           internal_static_hbase_pb_TableLock_descriptor =
-            getDescriptor().getMessageTypes().get(10);
+            getDescriptor().getMessageTypes().get(11);
           internal_static_hbase_pb_TableLock_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_TableLock_descriptor,
               new java.lang.String[] { "TableName", "LockOwner", "ThreadId", "IsShared", "Purpose", "CreateTime", });
           internal_static_hbase_pb_SwitchState_descriptor =
-            getDescriptor().getMessageTypes().get(11);
+            getDescriptor().getMessageTypes().get(12);
           internal_static_hbase_pb_SwitchState_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_SwitchState_descriptor,

http://git-wip-us.apache.org/repos/asf/hbase/blob/66941910/hbase-protocol/src/main/protobuf/ZooKeeper.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/ZooKeeper.proto b/hbase-protocol/src/main/protobuf/ZooKeeper.proto
index b408db9..60ed229 100644
--- a/hbase-protocol/src/main/protobuf/ZooKeeper.proto
+++ b/hbase-protocol/src/main/protobuf/ZooKeeper.proto
@@ -119,6 +119,11 @@ message Table {
   required State state = 1 [default = ENABLED];
 }
 
+message TableCF {
+  optional TableName table_name = 1;
+  repeated bytes families = 2;
+}
+
 /**
  * Used by replication. Holds a replication peer key.
  */
@@ -129,6 +134,7 @@ message ReplicationPeer {
   optional string replicationEndpointImpl = 2;
   repeated BytesBytesPair data = 3;
   repeated NameStringPair configuration = 4;
+  repeated TableCF table_cfs = 5;
 }
 
 /**
@@ -173,4 +179,4 @@ message TableLock {
  */
 message SwitchState {
   optional bool enabled = 1;
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/66941910/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 8db8324..e079b3b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -143,6 +143,7 @@ import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
 import org.apache.hadoop.hbase.regionserver.RegionSplitPolicy;
 import org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy;
 import org.apache.hadoop.hbase.regionserver.compactions.FIFOCompactionPolicy;
+import org.apache.hadoop.hbase.replication.master.TableCFsUpdater;
 import org.apache.hadoop.hbase.replication.regionserver.Replication;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.UserProvider;
@@ -712,6 +713,13 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
     status.setStatus("Initializing ZK system trackers");
     initializeZKBasedSystemTrackers();
 
+    // This is for backwards compatibility
+    // See HBASE-11393
+    status.setStatus("Update TableCFs node in ZNode");
+    TableCFsUpdater tableCFsUpdater = new TableCFsUpdater(zooKeeper,
+            conf, this.clusterConnection);
+    tableCFsUpdater.update();
+
     // initialize master side coprocessors before we start handling requests
     status.setStatus("Initializing master coprocessors");
     this.cpHost = new MasterCoprocessorHost(this, this.conf);

http://git-wip-us.apache.org/repos/asf/hbase/blob/66941910/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/TableCFsUpdater.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/TableCFsUpdater.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/TableCFsUpdater.java
new file mode 100644
index 0000000..ce07868
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/TableCFsUpdater.java
@@ -0,0 +1,120 @@
+/*
+ *
+ * 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.replication.master;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.replication.ReplicationSerDeHelper;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.ReplicationStateZKBase;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.zookeeper.KeeperException;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * This class is used to upgrade TableCFs from HBase 1.x to HBase 2.x.
+ * It will be removed in HBase 3.x.  See HBASE-11393
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class TableCFsUpdater extends ReplicationStateZKBase {
+
+  private static final Log LOG = LogFactory.getLog(TableCFsUpdater.class);
+
+  public TableCFsUpdater(ZooKeeperWatcher zookeeper,
+                         Configuration conf, Abortable abortable) {
+    super(zookeeper, conf, abortable);
+  }
+
+  public void update() {
+    List<String> znodes = null;
+    try {
+      znodes = ZKUtil.listChildrenNoWatch(this.zookeeper, this.peersZNode);
+    } catch (KeeperException e) {
+      LOG.warn("", e);
+    }
+    if (znodes != null) {
+      for (String peerId : znodes) {
+        if (!update(peerId)) {
+          LOG.error("upgrade tableCFs failed for peerId=" + peerId);
+        }
+      }
+    }
+  }
+
+  public boolean update(String peerId) {
+    String tableCFsNode = getTableCFsNode(peerId);
+    try {
+      if (ZKUtil.checkExists(zookeeper, tableCFsNode) != -1) {
+        String peerNode = getPeerNode(peerId);
+        ReplicationPeerConfig rpc = getReplicationPeerConig(peerNode);
+        // We only need to copy data from tableCFs node to rpc Node the first time hmaster start.
+        if (rpc.getTableCFsMap() == null || rpc.getTableCFsMap().size() == 0) {
+          // we copy TableCFs node into PeerNode
+          LOG.info("copy tableCFs into peerNode:" + peerId);
+          ZooKeeperProtos.TableCF[] tableCFs =
+                  ReplicationSerDeHelper.parseTableCFs(
+                          ZKUtil.getData(this.zookeeper, tableCFsNode));
+          rpc.setTableCFsMap(ReplicationSerDeHelper.convert2Map(tableCFs));
+          ZKUtil.setData(this.zookeeper, peerNode,
+                  ReplicationSerDeHelper.toByteArray(rpc));
+        } else {
+          LOG.info("No tableCFs in peerNode:" + peerId);
+        }
+      }
+    } catch (KeeperException e) {
+      LOG.warn("NOTICE!! Update peerId failed, peerId=" + peerId, e);
+      return false;
+    } catch (InterruptedException e) {
+      LOG.warn("NOTICE!! Update peerId failed, peerId=" + peerId, e);
+      return false;
+    } catch (IOException e) {
+      LOG.warn("NOTICE!! Update peerId failed, peerId=" + peerId, e);
+      return false;
+    }
+    return true;
+  }
+
+  private ReplicationPeerConfig getReplicationPeerConig(String peerNode)
+          throws KeeperException, InterruptedException {
+    byte[] data = null;
+    data = ZKUtil.getData(this.zookeeper, peerNode);
+    if (data == null) {
+      LOG.error("Could not get configuration for " +
+              "peer because it doesn't exist. peer=" + peerNode);
+      return null;
+    }
+    try {
+      return ReplicationSerDeHelper.parsePeerFrom(data);
+    } catch (DeserializationException e) {
+      LOG.warn("Failed to parse cluster key from peer=" + peerNode);
+      return null;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/66941910/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
index a771c21..5967a69 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequ
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.regionserver.StorefileRefresherChore;
 import org.apache.hadoop.hbase.regionserver.TestHRegionServerBulkLoad;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
@@ -249,7 +250,9 @@ public class TestReplicaWithCluster {
     HTU2.getHBaseAdmin().createTable(hdt, HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE);
 
     ReplicationAdmin admin = new ReplicationAdmin(HTU.getConfiguration());
-    admin.addPeer("2", HTU2.getClusterKey());
+    ReplicationPeerConfig rpc = new ReplicationPeerConfig();
+    rpc.setClusterKey(HTU2.getClusterKey());
+    admin.addPeer("2", rpc);
     admin.close();
 
     Put p = new Put(row);

http://git-wip-us.apache.org/repos/asf/hbase/blob/66941910/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
index 03d7aee..775a6d8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hbase.client.replication;
 
+import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
@@ -44,6 +46,7 @@ import com.google.common.collect.Lists;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -92,11 +95,15 @@ public class TestReplicationAdmin {
    */
   @Test
   public void testAddRemovePeer() throws Exception {
+    ReplicationPeerConfig rpc1 = new ReplicationPeerConfig();
+    rpc1.setClusterKey(KEY_ONE);
+    ReplicationPeerConfig rpc2 = new ReplicationPeerConfig();
+    rpc2.setClusterKey(KEY_SECOND);
     // Add a valid peer
-    admin.addPeer(ID_ONE, KEY_ONE);
+    admin.addPeer(ID_ONE, rpc1);
     // try adding the same (fails)
     try {
-      admin.addPeer(ID_ONE, KEY_ONE);
+      admin.addPeer(ID_ONE, rpc1);
     } catch (IllegalArgumentException iae) {
       // OK!
     }
@@ -111,7 +118,7 @@ public class TestReplicationAdmin {
     assertEquals(1, admin.getPeersCount());
     // Add a second since multi-slave is supported
     try {
-      admin.addPeer(ID_SECOND, KEY_SECOND);
+      admin.addPeer(ID_SECOND, rpc2);
     } catch (IllegalStateException iae) {
       fail();
     }
@@ -125,6 +132,10 @@ public class TestReplicationAdmin {
   
   @Test
   public void testAddPeerWithUnDeletedQueues() throws Exception {
+    ReplicationPeerConfig rpc1 = new ReplicationPeerConfig();
+    rpc1.setClusterKey(KEY_ONE);
+    ReplicationPeerConfig rpc2 = new ReplicationPeerConfig();
+    rpc2.setClusterKey(KEY_SECOND);
     Configuration conf = TEST_UTIL.getConfiguration();
     ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "Test HBaseAdmin", null);
     ReplicationQueues repQueues =
@@ -134,7 +145,7 @@ public class TestReplicationAdmin {
     // add queue for ID_ONE
     repQueues.addLog(ID_ONE, "file1");
     try {
-      admin.addPeer(ID_ONE, KEY_ONE);
+      admin.addPeer(ID_ONE, rpc1);
       fail();
     } catch (ReplicationException e) {
       // OK!
@@ -145,7 +156,7 @@ public class TestReplicationAdmin {
     // add recovered queue for ID_ONE
     repQueues.addLog(ID_ONE + "-server2", "file1");
     try {
-      admin.addPeer(ID_ONE, KEY_ONE);
+      admin.addPeer(ID_ONE, rpc2);
       fail();
     } catch (ReplicationException e) {
       // OK!
@@ -182,7 +193,9 @@ public class TestReplicationAdmin {
    */
   @Test
   public void testEnableDisable() throws Exception {
-    admin.addPeer(ID_ONE, KEY_ONE);
+    ReplicationPeerConfig rpc1 = new ReplicationPeerConfig();
+    rpc1.setClusterKey(KEY_ONE);
+    admin.addPeer(ID_ONE, rpc1);
     assertEquals(1, admin.getPeersCount());
     assertTrue(admin.getPeerState(ID_ONE));
     admin.disablePeer(ID_ONE);
@@ -197,100 +210,140 @@ public class TestReplicationAdmin {
   }
 
   @Test
-  public void testGetTableCfsStr() {
-    // opposite of TestPerTableCFReplication#testParseTableCFsFromConfig()
-
-    Map<TableName, List<String>> tabCFsMap = null;
-
-    // 1. null or empty string, result should be null
-    assertEquals(null, ReplicationAdmin.getTableCfsStr(tabCFsMap));
-
-
-    // 2. single table: "tab1" / "tab2:cf1" / "tab3:cf1,cf3"
-    tabCFsMap = new TreeMap<TableName, List<String>>();
-    tabCFsMap.put(TableName.valueOf("tab1"), null);   // its table name is "tab1"
-    assertEquals("tab1", ReplicationAdmin.getTableCfsStr(tabCFsMap));
-
-    tabCFsMap = new TreeMap<TableName, List<String>>();
-    tabCFsMap.put(TableName.valueOf("tab1"), Lists.newArrayList("cf1"));
-    assertEquals("tab1:cf1", ReplicationAdmin.getTableCfsStr(tabCFsMap));
-
-    tabCFsMap = new TreeMap<TableName, List<String>>();
-    tabCFsMap.put(TableName.valueOf("tab1"), Lists.newArrayList("cf1", "cf3"));
-    assertEquals("tab1:cf1,cf3", ReplicationAdmin.getTableCfsStr(tabCFsMap));
-
-    // 3. multiple tables: "tab1 ; tab2:cf1 ; tab3:cf1,cf3"
-    tabCFsMap = new TreeMap<TableName, List<String>>();
-    tabCFsMap.put(TableName.valueOf("tab1"), null);
-    tabCFsMap.put(TableName.valueOf("tab2"), Lists.newArrayList("cf1"));
-    tabCFsMap.put(TableName.valueOf("tab3"), Lists.newArrayList("cf1", "cf3"));
-    assertEquals("tab1;tab2:cf1;tab3:cf1,cf3", ReplicationAdmin.getTableCfsStr(tabCFsMap));
-  }
-
-  @Test
   public void testAppendPeerTableCFs() throws Exception {
+    ReplicationPeerConfig rpc1 = new ReplicationPeerConfig();
+    rpc1.setClusterKey(KEY_ONE);
+    TableName tab1 = TableName.valueOf("t1");
+    TableName tab2 = TableName.valueOf("t2");
+    TableName tab3 = TableName.valueOf("t3");
+    TableName tab4 = TableName.valueOf("t4");
+
     // Add a valid peer
-    admin.addPeer(ID_ONE, KEY_ONE);
+    admin.addPeer(ID_ONE, rpc1);
 
-    admin.appendPeerTableCFs(ID_ONE, "t1");
-    assertEquals("t1", admin.getPeerTableCFs(ID_ONE));
+    Map<TableName, List<String>> tableCFs = new HashMap<>();
 
-    // append table t2 to replication
-    admin.appendPeerTableCFs(ID_ONE, "t2");
-    String peerTablesOne = admin.getPeerTableCFs(ID_ONE);
+    tableCFs.put(tab1, null);
+    admin.appendPeerTableCFs(ID_ONE, tableCFs);
+    Map<TableName, List<String>> result = admin.getPeerConfig(ID_ONE).getTableCFsMap();
+    assertEquals(1, result.size());
+    assertEquals(true, result.containsKey(tab1));
+    assertNull(result.get(tab1));
 
-    // Different jdk's return different sort order for the tables. ( Not sure on why exactly )
-    //
-    // So instead of asserting that the string is exactly we
-    // assert that the string contains all tables and the needed separator.
-    assertTrue("Should contain t1", peerTablesOne.contains("t1"));
-    assertTrue("Should contain t2", peerTablesOne.contains("t2"));
-    assertTrue("Should contain ; as the seperator", peerTablesOne.contains(";"));
+    // append table t2 to replication
+    tableCFs.clear();
+    tableCFs.put(tab2, null);
+    admin.appendPeerTableCFs(ID_ONE, tableCFs);
+    result = admin.getPeerConfig(ID_ONE).getTableCFsMap();
+    assertEquals(2, result.size());
+    assertTrue("Should contain t1", result.containsKey(tab1));
+    assertTrue("Should contain t2", result.containsKey(tab2));
+    assertNull(result.get(tab1));
+    assertNull(result.get(tab2));
 
     // append table column family: f1 of t3 to replication
-    admin.appendPeerTableCFs(ID_ONE, "t3:f1");
-    String peerTablesTwo = admin.getPeerTableCFs(ID_ONE);
-    assertTrue("Should contain t1", peerTablesTwo.contains("t1"));
-    assertTrue("Should contain t2", peerTablesTwo.contains("t2"));
-    assertTrue("Should contain t3:f1", peerTablesTwo.contains("t3:f1"));
-    assertTrue("Should contain ; as the seperator", peerTablesTwo.contains(";"));
+    tableCFs.clear();
+    tableCFs.put(tab3, new ArrayList<String>());
+    tableCFs.get(tab3).add("f1");
+    admin.appendPeerTableCFs(ID_ONE, tableCFs);
+    result = admin.getPeerConfig(ID_ONE).getTableCFsMap();
+    assertEquals(3, result.size());
+    assertTrue("Should contain t1", result.containsKey(tab1));
+    assertTrue("Should contain t2", result.containsKey(tab2));
+    assertTrue("Should contain t3", result.containsKey(tab3));
+    assertNull(result.get(tab1));
+    assertNull(result.get(tab2));
+    assertEquals(1, result.get(tab3).size());
+    assertEquals("f1", result.get(tab3).get(0));
+
+    tableCFs.clear();
+    tableCFs.put(tab4, new ArrayList<String>());
+    tableCFs.get(tab4).add("f1");
+    tableCFs.get(tab4).add("f2");
+    admin.appendPeerTableCFs(ID_ONE, tableCFs);
+    result = admin.getPeerConfig(ID_ONE).getTableCFsMap();
+    assertEquals(4, result.size());
+    assertTrue("Should contain t1", result.containsKey(tab1));
+    assertTrue("Should contain t2", result.containsKey(tab2));
+    assertTrue("Should contain t3", result.containsKey(tab3));
+    assertTrue("Should contain t4", result.containsKey(tab4));
+    assertNull(result.get(tab1));
+    assertNull(result.get(tab2));
+    assertEquals(1, result.get(tab3).size());
+    assertEquals("f1", result.get(tab3).get(0));
+    assertEquals(2, result.get(tab4).size());
+    assertEquals("f1", result.get(tab4).get(0));
+    assertEquals("f2", result.get(tab4).get(1));
+
     admin.removePeer(ID_ONE);
   }
 
   @Test
   public void testRemovePeerTableCFs() throws Exception {
+    ReplicationPeerConfig rpc1 = new ReplicationPeerConfig();
+    rpc1.setClusterKey(KEY_ONE);
+    TableName tab1 = TableName.valueOf("t1");
+    TableName tab2 = TableName.valueOf("t2");
+    TableName tab3 = TableName.valueOf("t3");
     // Add a valid peer
-    admin.addPeer(ID_ONE, KEY_ONE);
+    admin.addPeer(ID_ONE, rpc1);
+    Map<TableName, List<String>> tableCFs = new HashMap<>();
     try {
-      admin.removePeerTableCFs(ID_ONE, "t3");
+      tableCFs.put(tab3, null);
+      admin.removePeerTableCFs(ID_ONE, tableCFs);
       assertTrue(false);
     } catch (ReplicationException e) {
     }
-    assertEquals("", admin.getPeerTableCFs(ID_ONE));
+    assertNull(admin.getPeerConfig(ID_ONE).getTableCFsMap());
 
-    admin.setPeerTableCFs(ID_ONE, "t1;t2:cf1");
+    tableCFs.clear();
+    tableCFs.put(tab1, null);
+    tableCFs.put(tab2, new ArrayList<String>());
+    tableCFs.get(tab2).add("cf1");
+    admin.setPeerTableCFs(ID_ONE, tableCFs);
     try {
-      admin.removePeerTableCFs(ID_ONE, "t3");
+      tableCFs.clear();
+      tableCFs.put(tab3, null);
+      admin.removePeerTableCFs(ID_ONE, tableCFs);
       assertTrue(false);
     } catch (ReplicationException e) {
     }
-    assertEquals("t1;t2:cf1", admin.getPeerTableCFs(ID_ONE));
-
+    Map<TableName, List<String>> result = admin.getPeerConfig(ID_ONE).getTableCFsMap();
+    assertEquals(2, result.size());
+    assertTrue("Should contain t1", result.containsKey(tab1));
+    assertTrue("Should contain t2", result.containsKey(tab2));
+    assertNull(result.get(tab1));
+    assertEquals(1, result.get(tab2).size());
+    assertEquals("cf1", result.get(tab2).get(0));
+
+    tableCFs.clear();
+    tableCFs.put(tab1, new ArrayList<String>());
+    tableCFs.get(tab1).add("f1");
     try {
-      admin.removePeerTableCFs(ID_ONE, "t1:f1");
+      admin.removePeerTableCFs(ID_ONE, tableCFs);
       assertTrue(false);
     } catch (ReplicationException e) {
     }
-    admin.removePeerTableCFs(ID_ONE, "t1");
-    assertEquals("t2:cf1", admin.getPeerTableCFs(ID_ONE));
-
+    tableCFs.clear();
+    tableCFs.put(tab1, null);
+    admin.removePeerTableCFs(ID_ONE, tableCFs);
+    result = admin.getPeerConfig(ID_ONE).getTableCFsMap();
+    assertEquals(1, result.size());
+    assertEquals(1, result.get(tab2).size());
+    assertEquals("cf1", result.get(tab2).get(0));
+
+    tableCFs.clear();
+    tableCFs.put(tab2, null);
     try {
-      admin.removePeerTableCFs(ID_ONE, "t2");
+      admin.removePeerTableCFs(ID_ONE, tableCFs);
       assertTrue(false);
     } catch (ReplicationException e) {
     }
-    admin.removePeerTableCFs(ID_ONE, "t2:cf1");
-    assertEquals("", admin.getPeerTableCFs(ID_ONE));
+    tableCFs.clear();
+    tableCFs.put(tab2, new ArrayList<String>());
+    tableCFs.get(tab2).add("cf1");
+    admin.removePeerTableCFs(ID_ONE, tableCFs);
+    assertNull(admin.getPeerConfig(ID_ONE).getTableCFsMap());
     admin.removePeer(ID_ONE);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/66941910/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
index eb793dc..ad3f3c8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
@@ -110,7 +110,7 @@ public class TestReplicationHFileCleaner {
   @Before
   public void setup() throws ReplicationException, IOException {
     root = TEST_UTIL.getDataTestDirOnTestFS();
-    rp.addPeer(peerId, new ReplicationPeerConfig().setClusterKey(TEST_UTIL.getClusterKey()), null);
+    rp.addPeer(peerId, new ReplicationPeerConfig().setClusterKey(TEST_UTIL.getClusterKey()));
     rq.addPeerToHFileRefs(peerId);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/66941910/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java
index ffa3e42..2a1ef6a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java
@@ -508,8 +508,9 @@ public class TestMasterReplication {
     try {
       replicationAdmin = new ReplicationAdmin(
           configurations[masterClusterNumber]);
-      replicationAdmin.addPeer(id,
-          utilities[slaveClusterNumber].getClusterKey());
+      ReplicationPeerConfig rpc = new ReplicationPeerConfig();
+      rpc.setClusterKey(utilities[slaveClusterNumber].getClusterKey());
+      replicationAdmin.addPeer(id, rpc);
     } finally {
       close(replicationAdmin);
     }
@@ -520,7 +521,9 @@ public class TestMasterReplication {
     ReplicationAdmin replicationAdmin = null;
     try {
       replicationAdmin = new ReplicationAdmin(configurations[masterClusterNumber]);
-      replicationAdmin.addPeer(id, utilities[slaveClusterNumber].getClusterKey(), tableCfs);
+      ReplicationPeerConfig rpc = new ReplicationPeerConfig();
+      rpc.setClusterKey(utilities[slaveClusterNumber].getClusterKey());
+      replicationAdmin.addPeer(id, rpc, ReplicationSerDeHelper.parseTableCFsFromConfig(tableCfs));
     } finally {
       close(replicationAdmin);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/66941910/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java
index 4480dd2..8a59661 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java
@@ -152,7 +152,9 @@ public class TestMultiSlaveReplication {
     Table htable3 = new HTable(conf3, tableName);
     htable3.setWriteBufferSize(1024);
 
-    admin1.addPeer("1", utility2.getClusterKey());
+    ReplicationPeerConfig rpc = new ReplicationPeerConfig();
+    rpc.setClusterKey(utility2.getClusterKey());
+    admin1.addPeer("1", rpc);
 
     // put "row" and wait 'til it got around, then delete
     putAndWait(row, famName, htable1, htable2);
@@ -168,7 +170,9 @@ public class TestMultiSlaveReplication {
     // after the log was rolled put a new row
     putAndWait(row3, famName, htable1, htable2);
 
-    admin1.addPeer("2", utility3.getClusterKey());
+    rpc = new ReplicationPeerConfig();
+    rpc.setClusterKey(utility3.getClusterKey());
+    admin1.addPeer("2", rpc);
 
     // put a row, check it was replicated to all clusters
     putAndWait(row1, famName, htable1, htable2, htable3);


[08/50] [abbrv] hbase git commit: HBASE-16803 Make hbase:acl table unsplittable - revert pending review comments

Posted by la...@apache.org.
HBASE-16803 Make hbase:acl table unsplittable - revert pending review comments


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

Branch: refs/heads/branch-1
Commit: b47ded3b4226e290953cf433829fdc1ee25c08fd
Parents: 408a9eb
Author: tedyu <yu...@gmail.com>
Authored: Tue Oct 11 12:51:58 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Tue Oct 11 12:51:58 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hbase/security/access/AccessControlLists.java   | 3 ---
 1 file changed, 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b47ded3b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
index b20c110..50d575e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
@@ -62,7 +62,6 @@ import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
 import org.apache.hadoop.hbase.regionserver.BloomType;
-import org.apache.hadoop.hbase.regionserver.DisabledRegionSplitPolicy;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.security.User;
@@ -137,8 +136,6 @@ public class AccessControlLists {
         // Set cache data blocks in L1 if more than one cache tier deployed; e.g. this will
         // be the case if we are using CombinedBlockCache (Bucket Cache).
         .setCacheDataInL1(true));
-    ACL_TABLEDESC.setValue(HTableDescriptor.SPLIT_POLICY,
-        DisabledRegionSplitPolicy.class.getName());
     master.createSystemTable(ACL_TABLEDESC);
   }
 


[33/50] [abbrv] hbase git commit: HBASE-16889 Proc-V2: verifyTables in the IntegrationTestDDLMasterFailover test after each table DDL is incorrect (Stephen Yuan Jiang)

Posted by la...@apache.org.
HBASE-16889 Proc-V2: verifyTables in the IntegrationTestDDLMasterFailover test after each table DDL is incorrect (Stephen Yuan Jiang)


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

Branch: refs/heads/branch-1
Commit: 42e7a4acd7112fde71ca939eaf8225db64a422d2
Parents: 0117ed9
Author: Stephen Yuan Jiang <sy...@gmail.com>
Authored: Thu Oct 20 18:25:29 2016 -0700
Committer: Stephen Yuan Jiang <sy...@gmail.com>
Committed: Thu Oct 20 18:25:29 2016 -0700

----------------------------------------------------------------------
 .../hbase/IntegrationTestDDLMasterFailover.java | 54 ++++++++++++--------
 1 file changed, 33 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/42e7a4ac/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestDDLMasterFailover.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestDDLMasterFailover.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestDDLMasterFailover.java
index 52a118a..adf0f46 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestDDLMasterFailover.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestDDLMasterFailover.java
@@ -153,7 +153,7 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase {
       admin.disableTables("ittable-\\d+");
       admin.deleteTables("ittable-\\d+");
       NamespaceDescriptor [] nsds = admin.listNamespaceDescriptors();
-      for(NamespaceDescriptor nsd:nsds ) {
+      for(NamespaceDescriptor nsd: nsds) {
         if(nsd.getName().matches("itnamespace\\d+")) {
           LOG.info("Removing namespace="+nsd.getName());
           admin.deleteNamespace(nsd.getName());
@@ -294,15 +294,14 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase {
         admin.createNamespace(nsd);
         NamespaceDescriptor freshNamespaceDesc = admin.getNamespaceDescriptor(nsd.getName());
         Assert.assertTrue("Namespace: " + nsd + " was not created", freshNamespaceDesc != null);
-        LOG.info("Created namespace:" + freshNamespaceDesc);
         namespaceMap.put(nsd.getName(), freshNamespaceDesc);
+        LOG.info("Created namespace:" + freshNamespaceDesc);
       } catch (Exception e){
         LOG.warn("Caught exception in action: " + this.getClass());
         throw e;
       } finally {
         admin.close();
       }
-      verifyNamespaces();
     }
 
     private NamespaceDescriptor createNamespaceDesc() {
@@ -340,15 +339,17 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase {
         Assert.assertTrue(
           "Namespace: " + selected + " was not modified",
           freshNamespaceDesc.getConfigurationValue(nsTestConfigKey).equals(nsValueNew));
-        LOG.info("Modified namespace :" + freshNamespaceDesc);
+        Assert.assertTrue(
+          "Namespace: " + namespaceName + " does not exist",
+          admin.getNamespaceDescriptor(namespaceName) != null);
         namespaceMap.put(namespaceName, freshNamespaceDesc);
+        LOG.info("Modified namespace :" + freshNamespaceDesc);
       } catch (Exception e){
         LOG.warn("Caught exception in action: " + this.getClass());
         throw e;
       } finally {
         admin.close();
       }
-      verifyNamespaces();
     }
   }
 
@@ -382,7 +383,6 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase {
       } finally {
         admin.close();
       }
-      verifyNamespaces();
     }
   }
 
@@ -425,6 +425,8 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase {
         admin.createTable(htd, startKey, endKey, numRegions);
         Assert.assertTrue("Table: " + htd + " was not created", admin.tableExists(tableName));
         HTableDescriptor freshTableDesc = admin.getTableDescriptor(tableName);
+        Assert.assertTrue(
+          "After create, Table: " + tableName + " in not enabled", admin.isTableEnabled(tableName));
         enabledTables.put(tableName, freshTableDesc);
         LOG.info("Created table:" + freshTableDesc);
       } catch (Exception e) {
@@ -433,7 +435,6 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase {
       } finally {
         admin.close();
       }
-      verifyTables();
     }
 
     private HTableDescriptor createTableDesc() {
@@ -465,6 +466,9 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase {
         Assert.assertTrue("Table: " + selected + " was not disabled",
             admin.isTableDisabled(tableName));
         HTableDescriptor freshTableDesc = admin.getTableDescriptor(tableName);
+        Assert.assertTrue(
+          "After disable, Table: " + tableName + " is not disabled",
+          admin.isTableDisabled(tableName));
         disabledTables.put(tableName, freshTableDesc);
         LOG.info("Disabled table :" + freshTableDesc);
       } catch (Exception e){
@@ -490,7 +494,6 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase {
       } finally {
         admin.close();
       }
-      verifyTables();
     }
   }
 
@@ -512,6 +515,8 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase {
         Assert.assertTrue("Table: " + selected + " was not enabled",
             admin.isTableEnabled(tableName));
         HTableDescriptor freshTableDesc = admin.getTableDescriptor(tableName);
+        Assert.assertTrue(
+          "After enable, Table: " + tableName + " in not enabled", admin.isTableEnabled(tableName));
         enabledTables.put(tableName, freshTableDesc);
         LOG.info("Enabled table :" + freshTableDesc);
       } catch (Exception e){
@@ -537,7 +542,6 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase {
       } finally {
         admin.close();
       }
-      verifyTables();
     }
   }
 
@@ -566,7 +570,6 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase {
       } finally {
         admin.close();
       }
-      verifyTables();
     }
   }
 
@@ -611,15 +614,17 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase {
         HTableDescriptor freshTableDesc = admin.getTableDescriptor(tableName);
         Assert.assertTrue("Column family: " + cfd + " was not added",
             freshTableDesc.hasFamily(cfd.getName()));
-        LOG.info("Added column family: " + cfd + " to table: " + tableName);
+        Assert.assertTrue(
+          "After add column family, Table: " + tableName + " is not disabled",
+          admin.isTableDisabled(tableName));
         disabledTables.put(tableName, freshTableDesc);
+        LOG.info("Added column family: " + cfd + " to table: " + tableName);
       } catch (Exception e) {
         LOG.warn("Caught exception in action: " + this.getClass());
         throw e;
       } finally {
         admin.close();
       }
-      verifyTables();
     }
 
     private HColumnDescriptor createFamilyDesc() {
@@ -658,16 +663,18 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase {
             freshColumnDesc.getMaxVersions(), versions);
         Assert.assertEquals("Column family: " + freshColumnDesc + " was not altered",
             freshColumnDesc.getMinVersions(), versions);
-        LOG.info("Altered versions of column family: " + columnDesc + " to: " + versions +
-            " in table: " + tableName);
+        Assert.assertTrue(
+          "After alter versions of column family, Table: " + tableName + " is not disabled",
+          admin.isTableDisabled(tableName));
         disabledTables.put(tableName, freshTableDesc);
+        LOG.info("Altered versions of column family: " + columnDesc + " to: " + versions +
+          " in table: " + tableName);
       } catch (Exception e) {
         LOG.warn("Caught exception in action: " + this.getClass());
         throw e;
       } finally {
         admin.close();
       }
-      verifyTables();
     }
   }
 
@@ -699,16 +706,18 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase {
         HColumnDescriptor freshColumnDesc = freshTableDesc.getFamily(columnDesc.getName());
         Assert.assertEquals("Encoding of column family: " + columnDesc + " was not altered",
             freshColumnDesc.getDataBlockEncoding().getId(), id);
-        LOG.info("Altered encoding of column family: " + freshColumnDesc + " to: " + id +
-            " in table: " + tableName);
+        Assert.assertTrue(
+          "After alter encoding of column family, Table: " + tableName + " is not disabled",
+          admin.isTableDisabled(tableName));
         disabledTables.put(tableName, freshTableDesc);
+        LOG.info("Altered encoding of column family: " + freshColumnDesc + " to: " + id +
+          " in table: " + tableName);
       } catch (Exception e) {
         LOG.warn("Caught exception in action: " + this.getClass());
         throw e;
       } finally {
         admin.close();
       }
-      verifyTables();
     }
   }
 
@@ -735,15 +744,17 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase {
         HTableDescriptor freshTableDesc = admin.getTableDescriptor(tableName);
         Assert.assertFalse("Column family: " + cfd + " was not added",
             freshTableDesc.hasFamily(cfd.getName()));
-        LOG.info("Deleted column family: " + cfd + " from table: " + tableName);
+        Assert.assertTrue(
+          "After delete column family, Table: " + tableName + " is not disabled",
+          admin.isTableDisabled(tableName));
         disabledTables.put(tableName, freshTableDesc);
+        LOG.info("Deleted column family: " + cfd + " from table: " + tableName);
       } catch (Exception e) {
         LOG.warn("Caught exception in action: " + this.getClass());
         throw e;
       } finally {
         admin.close();
       }
-      verifyTables();
     }
   }
 
@@ -782,6 +793,8 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase {
           table.put(put);
         }
         HTableDescriptor freshTableDesc = admin.getTableDescriptor(tableName);
+        Assert.assertTrue(
+          "After insert, Table: " + tableName + " in not enabled", admin.isTableEnabled(tableName));
         enabledTables.put(tableName, freshTableDesc);
         LOG.info("Added " + numRows + " rows to table: " + selected);
       } catch (Exception e) {
@@ -790,7 +803,6 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase {
       } finally {
         admin.close();
       }
-      verifyTables();
     }
   }
 


[44/50] [abbrv] hbase git commit: HBASE-16450 Shell tool to dump replication queues

Posted by la...@apache.org.
HBASE-16450 Shell tool to dump replication queues

New tool to dump existing replication peers, configurations and
queues when using HBase Replication. The tool provides two flags:

 --distributed  This flag will poll each RS for information about
                the replication queues being processed on this RS.
                By default this is not enabled and the information
                about the replication queues and configuration will
                be obtained from ZooKeeper.
 --hdfs         When --distributed is used, this flag will attempt
                to calculate the total size of the WAL files used
                by the replication queues. Since its possible that
                multiple peers can be configured this value can be
                overestimated.

Signed-off-by: Matteo Bertozzi <ma...@cloudera.com>


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

Branch: refs/heads/branch-1
Commit: 0a41493ac4e5ad0e0feac82cd17531c3d2a27089
Parents: d4826e1
Author: Esteban Gutierrez <es...@apache.org>
Authored: Mon Aug 22 19:53:29 2016 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Thu Oct 27 21:38:42 2016 -0700

----------------------------------------------------------------------
 .../replication/ReplicationQueuesZKImpl.java    |   2 +-
 .../apache/hadoop/hbase/zookeeper/ZKUtil.java   |  11 +
 .../regionserver/DumpReplicationQueues.java     | 355 +++++++++++++++++++
 3 files changed, 367 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/0a41493a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
index 37d23ef..c7af78e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
@@ -162,7 +162,7 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
       return ZKUtil.parseWALPositionFrom(bytes);
     } catch (DeserializationException de) {
       LOG.warn("Failed to parse WALPosition for queueId=" + queueId + " and wal=" + filename
-          + "znode content, continuing.");
+          + " znode content, continuing.");
     }
     // if we can not parse the position, start at the beginning of the wal file
     // again

http://git-wip-us.apache.org/repos/asf/hbase/blob/0a41493a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
index ff837cb..66ddb62 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
@@ -1825,6 +1825,17 @@ public class ZKUtil {
     }
   }
 
+  /**
+   * Returns a string with replication znodes and position of the replication log
+   * @param zkw
+   * @return aq string of replication znodes and log positions
+   */
+  public static String getReplicationZnodesDump(ZooKeeperWatcher zkw) throws KeeperException {
+    StringBuilder sb = new StringBuilder();
+    getReplicationZnodesDump(zkw, sb);
+    return sb.toString();
+  }
+
   private static void appendRSZnodes(ZooKeeperWatcher zkw, String znode, StringBuilder sb)
       throws KeeperException {
     List<String> stack = new LinkedList<String>();

http://git-wip-us.apache.org/repos/asf/hbase/blob/0a41493a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java
new file mode 100644
index 0000000..bf38d6f
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java
@@ -0,0 +1,355 @@
+/**
+ * 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.replication.regionserver;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.*;
+import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
+import org.apache.hadoop.hbase.io.WALLink;
+import org.apache.hadoop.hbase.procedure2.util.StringUtils;
+import org.apache.hadoop.hbase.replication.*;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.zookeeper.KeeperException;
+import org.mortbay.util.IO;
+
+import java.io.IOException;
+import java.util.*;
+
+/**
+ * Provides information about the existing states of replication, replication peers and queues.
+ *
+ * Usage: hbase org.apache.hadoop.hbase.replication.regionserver.DumpReplicationQueues [args]
+ * Arguments: --distributed    Polls each RS to dump information about the queue
+ *            --hdfs           Reports HDFS usage by the replication queues (note: can be overestimated).
+ */
+
+public class DumpReplicationQueues extends Configured implements Tool {
+
+  private static final Log LOG = LogFactory.getLog(DumpReplicationQueues.class.getName());
+
+  static class DumpOptions {
+    boolean hdfs = false;
+    boolean distributed = false;
+
+    public DumpOptions() {
+    }
+
+    public DumpOptions(DumpOptions that) {
+      this.hdfs = that.hdfs;
+      this.distributed = that.distributed;
+    }
+
+    boolean isHdfs () {
+      return hdfs;
+    }
+
+    boolean isDistributed() {
+      return distributed;
+    }
+
+    void setHdfs (boolean hdfs) {
+      this.hdfs = hdfs;
+    }
+
+    void setDistributed(boolean distributed) {
+      this.distributed = distributed;
+    }
+  }
+
+  static DumpOptions parseOpts(Queue<String> args) {
+    DumpOptions opts = new DumpOptions();
+
+    String cmd = null;
+    while ((cmd = args.poll()) != null) {
+      if (cmd.equals("-h") || cmd.equals("--h") || cmd.equals("--help")) {
+        // place item back onto queue so that caller knows parsing was incomplete
+        args.add(cmd);
+        break;
+      }
+      final String hdfs = "--hdfs";
+      if (cmd.equals(hdfs)) {
+        opts.setHdfs(true);
+        continue;
+      }
+      final String distributed = "--distributed";
+      if (cmd.equals(distributed)) {
+        opts.setDistributed(true);
+        continue;
+      } else {
+        printUsageAndExit("ERROR: Unrecognized option/command: " + cmd, -1);
+      }
+      // check that --distributed is present when --hdfs is in the arguments
+      if (!opts.isDistributed()  && opts.isHdfs()) {
+        printUsageAndExit("ERROR: --hdfs option can only be used with --distributed: " + cmd, -1);
+      }
+    }
+    return opts;
+  }
+
+  /**
+   * Main
+   *
+   * @param args
+   * @throws Exception
+   */
+  public static void main(String[] args) throws Exception {
+    Configuration conf = HBaseConfiguration.create();
+    int ret = ToolRunner.run(conf, new DumpReplicationQueues(), args);
+    System.exit(ret);
+  }
+
+  @Override
+  public int run(String[] args) throws Exception {
+
+    int errCode = -1;
+    LinkedList<String> argv = new LinkedList<String>();
+    argv.addAll(Arrays.asList(args));
+    DumpOptions opts = parseOpts(argv);
+
+    // args remaining, print help and exit
+    if (!argv.isEmpty()) {
+      errCode = 0;
+      printUsage();
+      return errCode;
+    }
+    return dumpReplicationQueues(opts);
+  }
+
+  protected void printUsage() {
+    printUsage(this.getClass().getName(), null);
+  }
+
+  protected static void printUsage(final String message) {
+    printUsage(DumpReplicationQueues.class.getName(), message);
+  }
+
+  protected static void printUsage(final String className, final String message) {
+    if (message != null && message.length() > 0) {
+      System.err.println(message);
+    }
+    System.err.println("Usage: java " + className + " \\");
+    System.err.println("  <OPTIONS> [-D<property=value>]*");
+    System.err.println();
+    System.err.println("General Options:");
+    System.err.println(" distributed  Poll each RS and print its own replication queue. "
+        + "Default only polls ZooKeeper");
+    System.err.println(" hdfs         Use HDFS to calculate usage of WALs by replication. It could be overestimated"
+        + " if replicating to multiple peers. --distributed flag is also needed.");
+  }
+
+  protected static void printUsageAndExit(final String message, final int exitCode) {
+    printUsage(message);
+    System.exit(exitCode);
+  }
+
+  private int dumpReplicationQueues(DumpOptions opts) throws Exception {
+
+    Configuration conf = getConf();
+    HBaseAdmin.checkHBaseAvailable(conf);
+    ReplicationAdmin replicationAdmin = new ReplicationAdmin(conf);
+    ClusterConnection connection = (ClusterConnection) ConnectionFactory.createConnection(conf);
+    long deleted = 0;
+
+    ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "DumpReplicationQueues" + System.currentTimeMillis(),
+        new WarnOnlyAbortable(), true);
+
+    try {
+      // Our zk watcher
+      LOG.info("Our Quorum: " + zkw.getQuorum());
+      List<HashMap<String, String>> replicatedTables = replicationAdmin.listReplicated();
+      if (replicatedTables.isEmpty()) {
+        LOG.info("No tables with a configured replication peer were found.");
+        return(0);
+      } else {
+        LOG.info("Replicated Tables: " + replicatedTables);
+      }
+
+      Map<String, ReplicationPeerConfig> peerConfigs = replicationAdmin.listPeerConfigs();
+
+      if (peerConfigs.isEmpty()) {
+        LOG.info("Replication is enabled but no peer configuration was found.");
+      }
+
+      System.out.println("Dumping replication peers and configurations:");
+      System.out.println(dumpPeersState(replicationAdmin, peerConfigs));
+
+      if (opts.isDistributed()) {
+        LOG.info("Found [--distributed], will poll each RegionServer.");
+        System.out.println(dumpQueues(connection, zkw, opts.isHdfs(), deleted));
+        if (deleted > 0) {
+          LOG.warn("Found " + deleted +" deleted queues"
+              + ", run hbck -fixReplication in order to remove the deleted replication queues");
+        }
+      } else {
+        // use ZK instead
+        System.out.print("Dumping replication znodes via ZooKeeper:");
+        System.out.println(ZKUtil.getReplicationZnodesDump(zkw));
+      }
+      return (0);
+    } catch (IOException e) {
+      return (-1);
+    } finally {
+      zkw.close();
+    }
+  }
+
+  public String dumpPeersState(ReplicationAdmin replicationAdmin,
+                              Map<String, ReplicationPeerConfig> peerConfigs) throws Exception {
+    Map<String, String> currentConf;
+    StringBuilder sb = new StringBuilder();
+    for (Map.Entry<String, ReplicationPeerConfig> peer : peerConfigs.entrySet()) {
+      try {
+        sb.append("Peer: " + peer.getKey() + "\n");
+        sb.append("    " + "State: " + (replicationAdmin.getPeerState(peer.getKey()) ? "ENABLED" : "DISABLED") + "\n");
+        sb.append("    " + "Cluster Name: " + peer.getValue() + "\n");
+        currentConf = peer.getValue().getConfiguration();
+        // Only show when we have a custom configuration for the peer
+        if (currentConf.size() > 1) {
+          sb.append("    " + "Peer Configuration: " + currentConf + "\n");
+        }
+        sb.append("    " + "Peer Table CFs: " + replicationAdmin.getPeerTableCFs(peer.getKey()) + "\n");
+      } catch (ReplicationException re) {
+        sb.append("Got an exception while invoking ReplicationAdmin: " + re + "\n");
+      }
+    }
+    return sb.toString();
+  }
+
+  public String dumpQueues(ClusterConnection connection, ZooKeeperWatcher zkw, boolean hdfs, long deleted)
+      throws Exception {
+    ReplicationQueuesClient queuesClient;
+    ReplicationPeers replicationPeers;
+    ReplicationQueues replicationQueues;
+
+    StringBuilder sb = new StringBuilder();
+
+    queuesClient = ReplicationFactory.getReplicationQueuesClient(zkw, getConf(), connection);
+    queuesClient.init();
+    replicationQueues = ReplicationFactory.getReplicationQueues(zkw, getConf(), connection);
+    replicationPeers = ReplicationFactory.getReplicationPeers(zkw, getConf(), queuesClient, connection);
+    replicationPeers.init();
+
+    // Loops each peer on each RS and dumps the queues
+
+    Set<String> peerIds = new HashSet<String>(replicationPeers.getAllPeerIds());
+    try {
+      List<String> regionservers = queuesClient.getListOfReplicators();
+      for (String regionserver : regionservers) {
+        List<String> queueIds = queuesClient.getAllQueues(regionserver);
+        replicationQueues.init(regionserver);
+        for (String queueId : queueIds) {
+          ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId);
+          List<String> wals = queuesClient.getLogsInQueue(regionserver, queueId);
+          if (!peerIds.contains(queueInfo.getPeerId())) {
+            deleted++;
+            sb.append(formatQueue(regionserver, replicationQueues, queueInfo, queueId, wals, true, hdfs));
+          } else {
+            sb.append(formatQueue(regionserver, replicationQueues, queueInfo, queueId, wals, false, hdfs));
+          }
+        }
+      }
+    } catch (KeeperException ke) {
+      throw new IOException(ke);
+    }
+    return sb.toString();
+  }
+
+  private String formatQueue(String regionserver, ReplicationQueues replicationQueues, ReplicationQueueInfo queueInfo,
+                           String queueId, List<String> wals, boolean isDeleted, boolean hdfs) throws Exception {
+
+    StringBuilder sb = new StringBuilder();
+
+    List<String> deadServers ;
+
+    sb.append("Dumping replication queue info for RegionServer: [" + regionserver + "]" + "\n");
+    sb.append("    PeerID: " + queueInfo.getPeerId() + "\n");
+    sb.append("    Recovered: " + queueInfo.isQueueRecovered() + "\n");
+    deadServers = queueInfo.getDeadRegionServers();
+    if (deadServers.isEmpty()) {
+      sb.append("    No dead RegionServers found in this queue." + "\n");
+    } else {
+      sb.append("    Dead RegionServers: " + deadServers + "\n");
+    }
+    sb.append("    Was deleted: " + isDeleted + "\n");
+    sb.append("    Number of WALs in replication queue: " + wals.size() + "\n");
+    for (String wal : wals) {
+      long position = replicationQueues.getLogPosition(queueInfo.getPeerId(), wal);
+      sb.append("    Replication position for " + wal + ": " + (position > 0 ? position : "0"
+          + " (not started or nothing to replicate)") + "\n");
+    }
+
+    if (hdfs) {
+      FileSystem fs = FileSystem.get(getConf());
+      sb.append("    Total size of WALs on HDFS for this queue: "
+          + StringUtils.humanSize(getTotalWALSize(fs, wals, regionserver)) + "\n");
+    }
+    return sb.toString();
+  }
+  /**
+   *  return total size in bytes from a list of WALs
+   */
+  private long getTotalWALSize(FileSystem fs, List<String> wals, String server) throws IOException {
+    int size = 0;
+    FileStatus fileStatus;
+
+    for (String wal : wals) {
+      try {
+        fileStatus = (new WALLink(getConf(), server, wal)).getFileStatus(fs);
+      } catch (IOException e) {
+        LOG.warn("WAL " + wal + " couldn't be found, skipping");
+        break;
+      }
+      size += fileStatus.getLen();
+    }
+    return size;
+  }
+
+  private static class WarnOnlyAbortable implements Abortable {
+    @Override
+    public void abort(String why, Throwable e) {
+      LOG.warn("DumpReplicationQueue received abort, ignoring.  Reason: " + why);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(e);
+      }
+    }
+
+    @Override
+    public boolean isAborted() {
+      return false;
+    }
+  }
+
+  private static void usage(final String errorMsg) {
+    if (errorMsg != null && errorMsg.length() > 0) {
+      LOG.error(errorMsg);
+    }
+  }
+}


[31/50] [abbrv] hbase git commit: HBASE-16698 Performance issue: handlers stuck waiting for CountDownLatch inside WALKey#getWriteEntry under high writing workload

Posted by la...@apache.org.
HBASE-16698 Performance issue: handlers stuck waiting for CountDownLatch inside WALKey#getWriteEntry under high writing workload


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

Branch: refs/heads/branch-1
Commit: a7a4e17f1d04d389f87ad22da96d72cd3be050d9
Parents: 33e89fa
Author: Yu Li <li...@apache.org>
Authored: Thu Oct 20 15:32:59 2016 +0800
Committer: Yu Li <li...@apache.org>
Committed: Thu Oct 20 15:32:59 2016 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/HRegion.java      | 86 +++++++++++++++-----
 .../hbase/regionserver/wal/FSWALEntry.java      | 21 +++--
 .../org/apache/hadoop/hbase/wal/WALKey.java     | 44 +++++++---
 .../hadoop/hbase/regionserver/TestHRegion.java  |  7 +-
 4 files changed, 115 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a7a4e17f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index c18564d..ca37eb1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -62,6 +62,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import org.apache.commons.lang.RandomStringUtils;
@@ -206,6 +207,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   public static final String LOAD_CFS_ON_DEMAND_CONFIG_KEY =
     "hbase.hregion.scan.loadColumnFamiliesOnDemand";
 
+  /** Config key for using mvcc pre-assign feature for put */
+  public static final String HREGION_MVCC_PRE_ASSIGN = "hbase.hregion.mvcc.preassign";
+  public static final boolean DEFAULT_HREGION_MVCC_PRE_ASSIGN = true;
+
   /**
    * Longest time we'll wait on a sequenceid.
    * Sequenceid comes up out of the WAL subsystem. WAL subsystem can go bad or a test might use
@@ -604,6 +609,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   private final Durability durability;
   private final boolean regionStatsEnabled;
 
+  // flag and lock for MVCC preassign
+  private final boolean mvccPreAssign;
+  private final ReentrantLock preAssignMvccLock;
+
   /**
    * HRegion constructor. This constructor should only be used for testing and
    * extensions.  Instances of HRegion should be instantiated with the
@@ -753,6 +762,14 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
           false :
           conf.getBoolean(HConstants.ENABLE_CLIENT_BACKPRESSURE,
               HConstants.DEFAULT_ENABLE_CLIENT_BACKPRESSURE);
+
+    // get mvcc pre-assign flag and lock
+    this.mvccPreAssign = conf.getBoolean(HREGION_MVCC_PRE_ASSIGN, DEFAULT_HREGION_MVCC_PRE_ASSIGN);
+    if (this.mvccPreAssign) {
+      this.preAssignMvccLock = new ReentrantLock();
+    } else {
+      this.preAssignMvccLock = null;
+    }
   }
 
   void setHTableSpecificConf() {
@@ -2576,7 +2593,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     // a timeout. May happen in tests after we tightened the semantic via HBASE-14317.
     // Also, the getSequenceId blocks on a latch. There is no global list of outstanding latches
     // so if an abort or stop, there is no way to call them in.
-    WALKey key = this.appendEmptyEdit(wal);
+    WALKey key = this.appendEmptyEdit(wal, null);
     mvcc.complete(key.getWriteEntry());
     return key.getSequenceId(this.maxWaitForSeqId);
   }
@@ -3283,25 +3300,44 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
           mutation.getClusterIds(), currentNonceGroup, currentNonce, mvcc);
         long replaySeqId = batchOp.getReplaySequenceId();
         walKey.setOrigLogSeqNum(replaySeqId);
-      }
-      if (walEdit.size() > 0) {
-        if (!isInReplay) {
-        // we use HLogKey here instead of WALKey directly to support legacy coprocessors.
-        walKey = new HLogKey(this.getRegionInfo().getEncodedNameAsBytes(),
-            this.htableDescriptor.getTableName(), WALKey.NO_SEQUENCE_ID, now,
-            mutation.getClusterIds(), currentNonceGroup, currentNonce, mvcc);
+        if (walEdit.size() > 0) {
+          txid =
+              this.wal.append(this.htableDescriptor, this.getRegionInfo(), walKey, walEdit, true);
+        }
+      } else {
+        try {
+          if (mvccPreAssign) {
+            preAssignMvccLock.lock();
+            writeEntry = mvcc.begin();
+          }
+          if (walEdit.size() > 0) {
+            // we use HLogKey here instead of WALKey directly to support legacy coprocessors.
+            walKey = new HLogKey(this.getRegionInfo().getEncodedNameAsBytes(),
+                this.htableDescriptor.getTableName(), WALKey.NO_SEQUENCE_ID, now,
+                mutation.getClusterIds(), currentNonceGroup, currentNonce, mvcc);
+            if (mvccPreAssign) {
+              walKey.setPreAssignedWriteEntry(writeEntry);
+            }
+            txid =
+                this.wal.append(this.htableDescriptor, this.getRegionInfo(), walKey, walEdit, true);
+          } else {
+            // If this is a skip wal operation just get the read point from mvcc
+            walKey = this.appendEmptyEdit(this.wal, writeEntry);
+          }
+        } finally {
+          if (mvccPreAssign) {
+            preAssignMvccLock.unlock();
+          }
         }
-        txid = this.wal.append(this.htableDescriptor, this.getRegionInfo(), walKey, walEdit, true);
       }
       // ------------------------------------
       // Acquire the latest mvcc number
       // ----------------------------------
-      if (walKey == null) {
-        // If this is a skip wal operation just get the read point from mvcc
-        walKey = this.appendEmptyEdit(this.wal);
-      }
       if (!isInReplay) {
-        writeEntry = walKey.getWriteEntry();
+        if (writeEntry == null) {
+          // we need to wait for mvcc to be assigned here if not preassigned
+          writeEntry = walKey.getWriteEntry();
+        }
         mvccNum = writeEntry.getWriteNumber();
       } else {
         mvccNum = batchOp.getReplaySequenceId();
@@ -3324,7 +3360,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         // We need to update the sequence id for following reasons.
         // 1) If the op is in replay mode, FSWALEntry#stampRegionSequenceId won't stamp sequence id.
         // 2) If no WAL, FSWALEntry won't be used
-        boolean updateSeqId = isInReplay || batchOp.getMutation(i).getDurability() == Durability.SKIP_WAL;
+        // 3) If mvcc preassigned, the asynchronous append may still hasn't run to
+        // FSWALEntry#stampRegionSequenceId and the cell seqId will be 0. So we need to update
+        // before apply to memstore to avoid scan return incorrect value
+        boolean updateSeqId = isInReplay
+            || batchOp.getMutation(i).getDurability() == Durability.SKIP_WAL || mvccPreAssign;
         if (updateSeqId) {
           updateSequenceId(familyMaps[i].values(), mvccNum);
         }
@@ -7206,7 +7246,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
           if(walKey == null){
             // since we use wal sequence Id as mvcc, for SKIP_WAL changes we need a "faked" WALEdit
             // to get a sequence id assigned which is done by FSWALEntry#stampRegionSequenceId
-            walKey = this.appendEmptyEdit(this.wal);
+            walKey = this.appendEmptyEdit(this.wal, null);
           }
 
           // 7. Start mvcc transaction
@@ -7531,7 +7571,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
           boolean updateSeqId = false;
           if (walKey == null) {
             // Append a faked WALEdit in order for SKIP_WAL updates to get mvcc assigned
-            walKey = this.appendEmptyEdit(this.wal);
+            walKey = this.appendEmptyEdit(this.wal, null);
             // If no WAL, FSWALEntry won't be used and no update for sequence id
             updateSeqId = true;
           }
@@ -7770,7 +7810,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
               this.wal.append(this.htableDescriptor, this.getRegionInfo(), walKey, walEdits, true);
           } else {
             // Append a faked WALEdit in order for SKIP_WAL updates to get mvccNum assigned
-            walKey = this.appendEmptyEdit(this.wal);
+            walKey = this.appendEmptyEdit(this.wal, null);
             // If no WAL, FSWALEntry won't be used and no update for sequence id
             updateSeqId = true;
           }
@@ -7991,9 +8031,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   public static final long FIXED_OVERHEAD = ClassSize.align(
       ClassSize.OBJECT +
       ClassSize.ARRAY +
-      46 * ClassSize.REFERENCE + 3 * Bytes.SIZEOF_INT +
+      47 * ClassSize.REFERENCE + 3 * Bytes.SIZEOF_INT +
       (14 * Bytes.SIZEOF_LONG) +
-      5 * Bytes.SIZEOF_BOOLEAN);
+      6 * Bytes.SIZEOF_BOOLEAN);
 
   // woefully out of date - currently missing:
   // 1 x HashMap - coprocessorServiceHandlers
@@ -8575,15 +8615,19 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    * Append a faked WALEdit in order to get a long sequence number and wal syncer will just ignore
    * the WALEdit append later.
    * @param wal
+   * @param writeEntry Preassigned writeEntry, if any
    * @return Return the key used appending with no sync and no append.
    * @throws IOException
    */
-  private WALKey appendEmptyEdit(final WAL wal) throws IOException {
+  private WALKey appendEmptyEdit(final WAL wal, WriteEntry writeEntry) throws IOException {
     // we use HLogKey here instead of WALKey directly to support legacy coprocessors.
     @SuppressWarnings("deprecation")
     WALKey key = new HLogKey(getRegionInfo().getEncodedNameAsBytes(),
       getRegionInfo().getTable(), WALKey.NO_SEQUENCE_ID, 0, null,
       HConstants.NO_NONCE, HConstants.NO_NONCE, getMVCC());
+    if (writeEntry != null) {
+      key.setPreAssignedWriteEntry(writeEntry);
+    }
 
     // Call append but with an empty WALEdit.  The returned sequence id will not be associated
     // with any edit and we can be sure it went in after all outstanding appends.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a7a4e17f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java
index 7f3eb61..f55e185 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java
@@ -111,11 +111,16 @@ class FSWALEntry extends Entry {
    */
   long stampRegionSequenceId() throws IOException {
     long regionSequenceId = WALKey.NO_SEQUENCE_ID;
-    MultiVersionConcurrencyControl mvcc = getKey().getMvcc();
-    MultiVersionConcurrencyControl.WriteEntry we = null;
-
-    if (mvcc != null) {
-      we = mvcc.begin();
+    WALKey key = getKey();
+    MultiVersionConcurrencyControl.WriteEntry we = key.getPreAssignedWriteEntry();
+    boolean preAssigned = (we != null);
+    if (!preAssigned) {
+      MultiVersionConcurrencyControl mvcc = key.getMvcc();
+      if (mvcc != null) {
+        we = mvcc.begin();
+      }
+    }
+    if (we != null) {
       regionSequenceId = we.getWriteNumber();
     }
 
@@ -126,9 +131,9 @@ class FSWALEntry extends Entry {
     }
 
     // This has to stay in this order
-    WALKey key = getKey();
-    key.setLogSeqNum(regionSequenceId);
-    key.setWriteEntry(we);
+    if (!preAssigned) {
+      key.setWriteEntry(we);
+    }
     return regionSequenceId;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a7a4e17f/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java
index 05acd72..585c8f4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java
@@ -33,6 +33,7 @@ import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
+import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl.WriteEntry;
 import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -94,6 +95,10 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
    */
   @InterfaceAudience.Private // For internal use only.
   public MultiVersionConcurrencyControl.WriteEntry getWriteEntry() throws InterruptedIOException {
+    if (this.preAssignedWriteEntry != null) {
+      // don't wait for seqNumAssignedLatch if writeEntry is preassigned
+      return this.preAssignedWriteEntry;
+    }
     try {
       this.seqNumAssignedLatch.await();
     } catch (InterruptedException ie) {
@@ -114,7 +119,12 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
 
   @InterfaceAudience.Private // For internal use only.
   public void setWriteEntry(MultiVersionConcurrencyControl.WriteEntry writeEntry) {
+    assert this.writeEntry == null : "Non-null writeEntry when trying to set one";
     this.writeEntry = writeEntry;
+    // Set our sequenceid now using WriteEntry.
+    if (this.writeEntry != null) {
+      this.logSeqNum = this.writeEntry.getWriteNumber();
+    }
     this.seqNumAssignedLatch.countDown();
   }
 
@@ -196,6 +206,7 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
   private long nonce = HConstants.NO_NONCE;
   private MultiVersionConcurrencyControl mvcc;
   private MultiVersionConcurrencyControl.WriteEntry writeEntry;
+  private MultiVersionConcurrencyControl.WriteEntry preAssignedWriteEntry = null;
   public static final List<UUID> EMPTY_UUIDS = Collections.unmodifiableList(new ArrayList<UUID>());
 
   // visible for deprecated HLogKey
@@ -360,17 +371,6 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
   }
 
   /**
-   * Allow that the log sequence id to be set post-construction
-   * Only public for org.apache.hadoop.hbase.regionserver.wal.FSWALEntry
-   * @param sequence
-   */
-  @InterfaceAudience.Private
-  public void setLogSeqNum(final long sequence) {
-    this.logSeqNum = sequence;
-
-  }
-
-  /**
    * Used to set original seq Id for WALKey during wal replay
    * @param seqId
    */
@@ -666,4 +666,24 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
       this.origLogSeqNum = walKey.getOrigSequenceNumber();
     }
   }
-}
+
+  /**
+   * @return The preassigned writeEntry, if any
+   */
+  @InterfaceAudience.Private // For internal use only.
+  public MultiVersionConcurrencyControl.WriteEntry getPreAssignedWriteEntry() {
+    return this.preAssignedWriteEntry;
+  }
+
+  /**
+   * Preassign writeEntry
+   * @param writeEntry the entry to assign
+   */
+  @InterfaceAudience.Private // For internal use only.
+  public void setPreAssignedWriteEntry(WriteEntry writeEntry) {
+    if (writeEntry != null) {
+      this.preAssignedWriteEntry = writeEntry;
+      this.logSeqNum = writeEntry.getWriteNumber();
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a7a4e17f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
index 7cf76fc..6ba0351 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
@@ -6200,8 +6200,11 @@ public class TestHRegion {
         @Override
         public Long answer(InvocationOnMock invocation) throws Throwable {
           WALKey key = invocation.getArgumentAt(2, WALKey.class);
-          MultiVersionConcurrencyControl.WriteEntry we = key.getMvcc().begin();
-          key.setWriteEntry(we);
+          MultiVersionConcurrencyControl.WriteEntry we = key.getPreAssignedWriteEntry();
+          if (we == null) {
+            we = key.getMvcc().begin();
+            key.setWriteEntry(we);
+          }
           return 1L;
         }
 


[15/50] [abbrv] hbase git commit: HBASE-16664 Timeout logic in AsyncProcess is broken

Posted by la...@apache.org.
HBASE-16664 Timeout logic in AsyncProcess is broken

Signed-off-by: chenheng <ch...@apache.org>


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

Branch: refs/heads/branch-1
Commit: 8f9fadf0216977996564ec56347a91e5a0a8b945
Parents: e2278f9
Author: Phil Yang <ud...@gmail.com>
Authored: Sun Oct 9 19:31:45 2016 +0800
Committer: chenheng <ch...@apache.org>
Committed: Thu Oct 13 17:14:52 2016 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/client/AsyncProcess.java       |  73 +++++---
 .../hbase/client/BufferedMutatorImpl.java       |  15 +-
 .../org/apache/hadoop/hbase/client/HTable.java  |  29 ++-
 .../hbase/client/MultiServerCallable.java       |  15 +-
 .../hbase/client/RetryingTimeTracker.java       |   3 +-
 .../hadoop/hbase/client/TestAsyncProcess.java   |  13 +-
 .../org/apache/hadoop/hbase/client/TestHCM.java | 182 ++++++++++++++++---
 7 files changed, 259 insertions(+), 71 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/8f9fadf0/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
index 647a466..b0652a7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
@@ -259,7 +259,8 @@ class AsyncProcess {
   protected final long pause;
   protected int numTries;
   protected int serverTrackerTimeout;
-  protected int timeout;
+  protected int rpcTimeout;
+  protected int operationTimeout;
   protected long primaryCallTimeoutMicroseconds;
   /** Whether to log details for batch errors */
   private final boolean logBatchErrorDetails;
@@ -322,7 +323,9 @@ class AsyncProcess {
         HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
     this.numTries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
         HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
-    this.timeout = rpcTimeout;
+    this.rpcTimeout = rpcTimeout;
+    this.operationTimeout = conf.getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,
+        HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
     this.primaryCallTimeoutMicroseconds = conf.getInt(PRIMARY_CALL_TIMEOUT_KEY, 10000);
 
     this.maxTotalConcurrentTasks = conf.getInt(HConstants.HBASE_CLIENT_MAX_TOTAL_TASKS,
@@ -378,6 +381,14 @@ class AsyncProcess {
           DEFAULT_THRESHOLD_TO_LOG_UNDONE_TASK_DETAILS);
   }
 
+  public void setRpcTimeout(int rpcTimeout) {
+    this.rpcTimeout = rpcTimeout;
+  }
+
+  public void setOperationTimeout(int operationTimeout) {
+    this.operationTimeout = operationTimeout;
+  }
+
   /**
    * @return pool if non null, otherwise returns this.pool if non null, otherwise throws
    *         RuntimeException
@@ -570,12 +581,12 @@ class AsyncProcess {
    */
   public <CResult> AsyncRequestFuture submitAll(TableName tableName,
       List<? extends Row> rows, Batch.Callback<CResult> callback, Object[] results) {
-    return submitAll(null, tableName, rows, callback, results, null, timeout);
+    return submitAll(null, tableName, rows, callback, results, null, operationTimeout, rpcTimeout);
   }
 
   public <CResult> AsyncRequestFuture submitAll(ExecutorService pool, TableName tableName,
       List<? extends Row> rows, Batch.Callback<CResult> callback, Object[] results) {
-    return submitAll(pool, tableName, rows, callback, results, null, timeout);
+    return submitAll(pool, tableName, rows, callback, results, null, operationTimeout, rpcTimeout);
   }
   /**
    * Submit immediately the list of rows, whatever the server status. Kept for backward
@@ -589,7 +600,7 @@ class AsyncProcess {
    */
   public <CResult> AsyncRequestFuture submitAll(ExecutorService pool, TableName tableName,
       List<? extends Row> rows, Batch.Callback<CResult> callback, Object[] results,
-      PayloadCarryingServerCallable callable, int curTimeout) {
+      PayloadCarryingServerCallable callable, int operationTimeout, int rpcTimeout) {
     List<Action<Row>> actions = new ArrayList<Action<Row>>(rows.size());
 
     // The position will be used by the processBatch to match the object array returned.
@@ -609,7 +620,7 @@ class AsyncProcess {
     }
     AsyncRequestFutureImpl<CResult> ars = createAsyncRequestFuture(
         tableName, actions, ng.getNonceGroup(), getPool(pool), callback, results, results != null,
-        callable, curTimeout);
+        callable, operationTimeout, rpcTimeout);
     ars.groupAndSendMultiAction(actions, 1);
     return ars;
   }
@@ -779,12 +790,12 @@ class AsyncProcess {
           if (callable == null) {
             callable = createCallable(server, tableName, multiAction);
           }
-          RpcRetryingCaller<MultiResponse> caller = createCaller(callable);
+          RpcRetryingCaller<MultiResponse> caller = createCaller(callable, rpcTimeout);
           try {
             if (callsInProgress != null) {
               callsInProgress.add(callable);
             }
-            res = caller.callWithoutRetries(callable, currentCallTotalTimeout);
+            res = caller.callWithoutRetries(callable, operationTimeout);
             if (res == null) {
               // Cancelled
               return;
@@ -850,11 +861,15 @@ class AsyncProcess {
     private final boolean hasAnyReplicaGets;
     private final long nonceGroup;
     private PayloadCarryingServerCallable currentCallable;
-    private int currentCallTotalTimeout;
+    private int operationTimeout;
+    private int rpcTimeout;
     private final Map<ServerName, List<Long>> heapSizesByServer = new HashMap<>();
+    private RetryingTimeTracker tracker;
+
     public AsyncRequestFutureImpl(TableName tableName, List<Action<Row>> actions, long nonceGroup,
         ExecutorService pool, boolean needResults, Object[] results,
-        Batch.Callback<CResult> callback, PayloadCarryingServerCallable callable, int timeout) {
+        Batch.Callback<CResult> callback, PayloadCarryingServerCallable callable,
+        int operationTimeout, int rpcTimeout) {
       this.pool = pool;
       this.callback = callback;
       this.nonceGroup = nonceGroup;
@@ -924,7 +939,12 @@ class AsyncProcess {
       this.errorsByServer = createServerErrorTracker();
       this.errors = (globalErrors != null) ? globalErrors : new BatchErrors();
       this.currentCallable = callable;
-      this.currentCallTotalTimeout = timeout;
+      this.operationTimeout = operationTimeout;
+      this.rpcTimeout = rpcTimeout;
+      if (callable == null) {
+        tracker = new RetryingTimeTracker();
+        tracker.start();
+      }
     }
 
     public Set<PayloadCarryingServerCallable> getCallsInProgress() {
@@ -1759,6 +1779,16 @@ class AsyncProcess {
       waitUntilDone();
       return results;
     }
+
+    /**
+     * Create a callable. Isolated to be easily overridden in the tests.
+     */
+    @VisibleForTesting
+    protected MultiServerCallable<Row> createCallable(final ServerName server,
+        TableName tableName, final MultiAction<Row> multi) {
+      return new MultiServerCallable<Row>(connection, tableName, server,
+          AsyncProcess.this.rpcFactory, multi, rpcTimeout, tracker);
+    }
   }
 
   @VisibleForTesting
@@ -1781,10 +1811,10 @@ class AsyncProcess {
   protected <CResult> AsyncRequestFutureImpl<CResult> createAsyncRequestFuture(
       TableName tableName, List<Action<Row>> actions, long nonceGroup, ExecutorService pool,
       Batch.Callback<CResult> callback, Object[] results, boolean needResults,
-      PayloadCarryingServerCallable callable, int curTimeout) {
+      PayloadCarryingServerCallable callable, int operationTimeout, int rpcTimeout) {
     return new AsyncRequestFutureImpl<CResult>(
         tableName, actions, nonceGroup, getPool(pool), needResults,
-        results, callback, callable, curTimeout);
+        results, callback, callable, operationTimeout, rpcTimeout);
   }
 
   @VisibleForTesting
@@ -1793,24 +1823,17 @@ class AsyncProcess {
       TableName tableName, List<Action<Row>> actions, long nonceGroup, ExecutorService pool,
       Batch.Callback<CResult> callback, Object[] results, boolean needResults) {
     return createAsyncRequestFuture(
-        tableName, actions, nonceGroup, pool, callback, results, needResults, null, timeout);
-  }
-
-  /**
-   * Create a callable. Isolated to be easily overridden in the tests.
-   */
-  @VisibleForTesting
-  protected MultiServerCallable<Row> createCallable(final ServerName server,
-      TableName tableName, final MultiAction<Row> multi) {
-    return new MultiServerCallable<Row>(connection, tableName, server, this.rpcFactory, multi);
+        tableName, actions, nonceGroup, pool, callback, results, needResults, null,
+        operationTimeout, rpcTimeout);
   }
 
   /**
    * Create a caller. Isolated to be easily overridden in the tests.
    */
   @VisibleForTesting
-  protected RpcRetryingCaller<MultiResponse> createCaller(PayloadCarryingServerCallable callable) {
-    return rpcCallerFactory.<MultiResponse> newCaller();
+  protected RpcRetryingCaller<MultiResponse> createCaller(PayloadCarryingServerCallable callable,
+      int rpcTimeout) {
+    return rpcCallerFactory.<MultiResponse> newCaller(rpcTimeout);
   }
 
   @VisibleForTesting

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f9fadf0/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java
index e12b34d..1974be3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java
@@ -81,6 +81,7 @@ public class BufferedMutatorImpl implements BufferedMutator {
   private boolean closed = false;
   private final ExecutorService pool;
   private int writeRpcTimeout; // needed to pass in through AsyncProcess constructor
+  private int operationTimeout;
 
   @VisibleForTesting
   protected AsyncProcess ap; // non-final so can be overridden in test
@@ -106,7 +107,9 @@ public class BufferedMutatorImpl implements BufferedMutator {
     this.writeRpcTimeout = conn.getConfiguration().getInt(HConstants.HBASE_RPC_WRITE_TIMEOUT_KEY,
         conn.getConfiguration().getInt(HConstants.HBASE_RPC_TIMEOUT_KEY,
             HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
-
+    this.operationTimeout = conn.getConfiguration().getInt(
+        HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,
+        HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
     // puts need to track errors globally due to how the APIs currently work.
     ap = new AsyncProcess(connection, conf, pool, rpcCallerFactory, true, rpcFactory, writeRpcTimeout);
   }
@@ -281,6 +284,16 @@ public class BufferedMutatorImpl implements BufferedMutator {
     return this.writeBufferSize;
   }
 
+  public void setRpcTimeout(int writeRpcTimeout) {
+    this.writeRpcTimeout = writeRpcTimeout;
+    this.ap.setRpcTimeout(writeRpcTimeout);
+  }
+
+  public void setOperationTimeout(int operationTimeout) {
+    this.operationTimeout = operationTimeout;
+    this.ap.setOperationTimeout(operationTimeout);
+  }
+
   /**
    * This is used for legacy purposes in {@link HTable#getWriteBuffer()} only. This should not be�
    * called from production uses.

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f9fadf0/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
index 5fc2d65..e8a969f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
@@ -911,9 +911,10 @@ public class HTable implements HTableInterface, RegionLocator {
     }
   }
 
-  public void batch(final List<? extends Row> actions, final Object[] results, int timeout)
+  public void batch(final List<? extends Row> actions, final Object[] results, int rpcTimeout)
       throws InterruptedException, IOException {
-    AsyncRequestFuture ars = multiAp.submitAll(pool, tableName, actions, null, results, null, timeout);
+    AsyncRequestFuture ars = multiAp.submitAll(pool, tableName, actions, null, results, null,
+        operationTimeout, rpcTimeout);
     ars.waitUntilDone();
     if (ars.hasError()) {
       throw ars.getErrors();
@@ -1055,13 +1056,12 @@ public class HTable implements HTableInterface, RegionLocator {
    */
   @Override
   public void mutateRow(final RowMutations rm) throws IOException {
-    final RetryingTimeTracker tracker = new RetryingTimeTracker();
+    final RetryingTimeTracker tracker = new RetryingTimeTracker().start();
     PayloadCarryingServerCallable<MultiResponse> callable =
       new PayloadCarryingServerCallable<MultiResponse>(connection, getName(), rm.getRow(),
           rpcControllerFactory) {
         @Override
         public MultiResponse call(int callTimeout) throws IOException {
-          tracker.start();
           controller.setPriority(tableName);
           int remainingTime = tracker.getRemainingTime(callTimeout);
           if (remainingTime == 0) {
@@ -1091,7 +1091,7 @@ public class HTable implements HTableInterface, RegionLocator {
         }
       };
     AsyncRequestFuture ars = multiAp.submitAll(pool, tableName, rm.getMutations(),
-        null, null, callable, operationTimeout);
+        null, null, callable, operationTimeout, writeRpcTimeout);
     ars.waitUntilDone();
     if (ars.hasError()) {
       throw ars.getErrors();
@@ -1364,13 +1364,12 @@ public class HTable implements HTableInterface, RegionLocator {
   public boolean checkAndMutate(final byte [] row, final byte [] family, final byte [] qualifier,
     final CompareOp compareOp, final byte [] value, final RowMutations rm)
     throws IOException {
-    final RetryingTimeTracker tracker = new RetryingTimeTracker();
+    final RetryingTimeTracker tracker = new RetryingTimeTracker().start();
     PayloadCarryingServerCallable<MultiResponse> callable =
       new PayloadCarryingServerCallable<MultiResponse>(connection, getName(), rm.getRow(),
         rpcControllerFactory) {
         @Override
         public MultiResponse call(int callTimeout) throws IOException {
-          tracker.start();
           controller.setPriority(tableName);
           int remainingTime = tracker.getRemainingTime(callTimeout);
           if (remainingTime == 0) {
@@ -1404,7 +1403,7 @@ public class HTable implements HTableInterface, RegionLocator {
      * */
     Object[] results = new Object[rm.getMutations().size()];
     AsyncRequestFuture ars = multiAp.submitAll(pool, tableName, rm.getMutations(),
-      null, results, callable, operationTimeout);
+      null, results, callable, operationTimeout, writeRpcTimeout);
     ars.waitUntilDone();
     if (ars.hasError()) {
       throw ars.getErrors();
@@ -1809,6 +1808,10 @@ public class HTable implements HTableInterface, RegionLocator {
 
   public void setOperationTimeout(int operationTimeout) {
     this.operationTimeout = operationTimeout;
+    if (mutator != null) {
+      mutator.setOperationTimeout(operationTimeout);
+    }
+    multiAp.setOperationTimeout(operationTimeout);
   }
 
   public int getOperationTimeout() {
@@ -1824,8 +1827,8 @@ public class HTable implements HTableInterface, RegionLocator {
   @Override
   @Deprecated
   public void setRpcTimeout(int rpcTimeout) {
-    this.readRpcTimeout = rpcTimeout;
-    this.writeRpcTimeout = rpcTimeout;
+    setWriteRpcTimeout(rpcTimeout);
+    setReadRpcTimeout(rpcTimeout);
   }
 
   @Override
@@ -1836,6 +1839,10 @@ public class HTable implements HTableInterface, RegionLocator {
   @Override
   public void setWriteRpcTimeout(int writeRpcTimeout) {
     this.writeRpcTimeout = writeRpcTimeout;
+    if (mutator != null) {
+      mutator.setRpcTimeout(writeRpcTimeout);
+    }
+    multiAp.setRpcTimeout(writeRpcTimeout);
   }
 
   @Override
@@ -1973,6 +1980,8 @@ public class HTable implements HTableInterface, RegionLocator {
               .writeBufferSize(connConfiguration.getWriteBufferSize())
               .maxKeyValueSize(connConfiguration.getMaxKeyValueSize())
       );
+      mutator.setRpcTimeout(writeRpcTimeout);
+      mutator.setOperationTimeout(operationTimeout);
     }
     return mutator;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f9fadf0/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java
index d0b4c81..115ba33 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java
@@ -52,9 +52,12 @@ import com.google.protobuf.ServiceException;
 class MultiServerCallable<R> extends PayloadCarryingServerCallable<MultiResponse> {
   private final MultiAction<R> multiAction;
   private final boolean cellBlock;
+  private final RetryingTimeTracker tracker;
+  private final int rpcTimeout;
 
   MultiServerCallable(final ClusterConnection connection, final TableName tableName,
-      final ServerName location, RpcControllerFactory rpcFactory, final MultiAction<R> multi) {
+      final ServerName location, RpcControllerFactory rpcFactory, final MultiAction<R> multi,
+      int rpcTimeout, RetryingTimeTracker tracker) {
     super(connection, tableName, null, rpcFactory);
     this.multiAction = multi;
     // RegionServerCallable has HRegionLocation field, but this is a multi-region request.
@@ -62,6 +65,8 @@ class MultiServerCallable<R> extends PayloadCarryingServerCallable<MultiResponse
     // we will store the server here, and throw if someone tries to obtain location/regioninfo.
     this.location = new HRegionLocation(null, location);
     this.cellBlock = isCellBlock();
+    this.tracker = tracker;
+    this.rpcTimeout = rpcTimeout;
   }
 
   @Override
@@ -79,7 +84,13 @@ class MultiServerCallable<R> extends PayloadCarryingServerCallable<MultiResponse
   }
 
   @Override
-  public MultiResponse call(int callTimeout) throws IOException {
+  public MultiResponse call(int operationTimeout) throws IOException {
+    int remainingTime = tracker.getRemainingTime(operationTimeout);
+    if (remainingTime <= 1) {
+      // "1" is a special return value in RetryingTimeTracker, see its implementation.
+      throw new DoNotRetryIOException("Operation Timeout");
+    }
+    int callTimeout = Math.min(rpcTimeout, remainingTime);
     int countOfActions = this.multiAction.size();
     if (countOfActions <= 0) throw new DoNotRetryIOException("No Actions");
     MultiRequest.Builder multiRequestBuilder = MultiRequest.newBuilder();

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f9fadf0/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingTimeTracker.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingTimeTracker.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingTimeTracker.java
index 24288e6..406928a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingTimeTracker.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingTimeTracker.java
@@ -25,10 +25,11 @@ class RetryingTimeTracker {
 
   private long globalStartTime = -1;
 
-  public void start() {
+  public RetryingTimeTracker start() {
     if (this.globalStartTime < 0) {
       this.globalStartTime = EnvironmentEdgeManager.currentTime();
     }
+    return this;
   }
 
   public int getRemainingTime(int callTimeout) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f9fadf0/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
index 9a566e8..3b7f395 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
@@ -211,9 +211,10 @@ public class TestAsyncProcess {
     @Override
     public <CResult> AsyncRequestFuture submitAll(ExecutorService pool, TableName tableName,
       List<? extends Row> rows, Batch.Callback<CResult> callback, Object[] results,
-      PayloadCarryingServerCallable callable, int curTimeout) {
-      previousTimeout = curTimeout;
-      return super.submitAll(pool, tableName, rows, callback, results, callable, curTimeout);
+      PayloadCarryingServerCallable callable, int operationTimeout, int rpcTimeout) {
+      previousTimeout = rpcTimeout;
+      return super.submitAll(pool, tableName, rows, callback, results, callable, operationTimeout,
+          rpcTimeout);
     }
 
     @Override
@@ -222,7 +223,7 @@ public class TestAsyncProcess {
     }
     @Override
     protected RpcRetryingCaller<MultiResponse> createCaller(
-        PayloadCarryingServerCallable callable) {
+        PayloadCarryingServerCallable callable, int rpcTimeout) {
       callsCt.incrementAndGet();
       MultiServerCallable callable1 = (MultiServerCallable) callable;
       final MultiResponse mr = createMultiResponse(
@@ -285,7 +286,7 @@ public class TestAsyncProcess {
 
     @Override
     protected RpcRetryingCaller<MultiResponse> createCaller(
-      PayloadCarryingServerCallable callable) {
+      PayloadCarryingServerCallable callable, int rpcTimeout) {
       callsCt.incrementAndGet();
       return new CallerWithFailure(ioe);
     }
@@ -336,7 +337,7 @@ public class TestAsyncProcess {
 
     @Override
     protected RpcRetryingCaller<MultiResponse> createCaller(
-        PayloadCarryingServerCallable payloadCallable) {
+        PayloadCarryingServerCallable payloadCallable, int rpcTimeout) {
       MultiServerCallable<Row> callable = (MultiServerCallable) payloadCallable;
       final MultiResponse mr = createMultiResponse(
           callable.getMulti(), nbMultiResponse, nbActions, new ResponseGenerator() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f9fadf0/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java
index 8436563..f468c16 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java
@@ -61,6 +61,7 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.exceptions.RegionMovedException;
 import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.filter.FilterBase;
+import org.apache.hadoop.hbase.ipc.CallTimeoutException;
 import org.apache.hadoop.hbase.ipc.RpcClient;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.ipc.ServerTooBusyException;
@@ -123,14 +124,14 @@ public class TestHCM {
 * This copro sleeps 20 second. The first call it fails. The second time, it works.
 */
   public static class SleepAndFailFirstTime extends BaseRegionObserver {
-    static final AtomicLong ct = new AtomicLong(0);
-    static final String SLEEP_TIME_CONF_KEY =
-        "hbase.coprocessor.SleepAndFailFirstTime.sleepTime";
-    static final long DEFAULT_SLEEP_TIME = 20000;
-    static final AtomicLong sleepTime = new AtomicLong(DEFAULT_SLEEP_TIME);
+  static final AtomicLong ct = new AtomicLong(0);
+  static final String SLEEP_TIME_CONF_KEY =
+      "hbase.coprocessor.SleepAndFailFirstTime.sleepTime";
+  static final long DEFAULT_SLEEP_TIME = 20000;
+  static final AtomicLong sleepTime = new AtomicLong(DEFAULT_SLEEP_TIME);
 
-    public SleepAndFailFirstTime() {
-    }
+  public SleepAndFailFirstTime() {
+  }
 
     @Override
     public void postOpen(ObserverContext<RegionCoprocessorEnvironment> c) {
@@ -141,12 +142,42 @@ public class TestHCM {
 
     @Override
     public void preGetOp(final ObserverContext<RegionCoprocessorEnvironment> e,
-              final Get get, final List<Cell> results) throws IOException {
+        final Get get, final List<Cell> results) throws IOException {
+      Threads.sleep(sleepTime.get());
+      if (ct.incrementAndGet() == 1) {
+        throw new IOException("first call I fail");
+      }
+    }
+
+    @Override
+    public void prePut(final ObserverContext<RegionCoprocessorEnvironment> e,
+        final Put put, final WALEdit edit, final Durability durability) throws IOException {
+      Threads.sleep(sleepTime.get());
+      if (ct.incrementAndGet() == 1) {
+        throw new IOException("first call I fail");
+      }
+    }
+
+    @Override
+    public void preDelete(final ObserverContext<RegionCoprocessorEnvironment> e,
+        final Delete delete,
+        final WALEdit edit, final Durability durability) throws IOException {
+      Threads.sleep(sleepTime.get());
+      if (ct.incrementAndGet() == 1) {
+        throw new IOException("first call I fail");
+      }
+    }
+
+    @Override
+    public Result preIncrement(final ObserverContext<RegionCoprocessorEnvironment> e,
+        final Increment increment) throws IOException {
       Threads.sleep(sleepTime.get());
-      if (ct.incrementAndGet() == 1){
+      if (ct.incrementAndGet() == 1) {
         throw new IOException("first call I fail");
       }
+      return super.preIncrement(e, increment);
     }
+
   }
 
   public static class SleepCoprocessor extends BaseRegionObserver {
@@ -162,16 +193,20 @@ public class TestHCM {
         final Put put, final WALEdit edit, final Durability durability) throws IOException {
       Threads.sleep(SLEEP_TIME);
     }
-  }
 
-  public static class SleepWriteCoprocessor extends BaseRegionObserver {
-    public static final int SLEEP_TIME = 5000;
     @Override
     public Result preIncrement(final ObserverContext<RegionCoprocessorEnvironment> e,
-                               final Increment increment) throws IOException {
+        final Increment increment) throws IOException {
       Threads.sleep(SLEEP_TIME);
       return super.preIncrement(e, increment);
     }
+
+    @Override
+    public void preDelete(final ObserverContext<RegionCoprocessorEnvironment> e, final Delete delete,
+        final WALEdit edit, final Durability durability) throws IOException {
+      Threads.sleep(SLEEP_TIME);
+    }
+
   }
 
   public static class SleepLongerAtFirstCoprocessor extends BaseRegionObserver {
@@ -365,11 +400,12 @@ public class TestHCM {
    * timeouted when the server answers.
    */
   @Test
-  public void testOperationTimeout() throws Exception {
-    HTableDescriptor hdt = TEST_UTIL.createTableDescriptor("HCM-testOperationTimeout");
+  public void testGetOperationTimeout() throws Exception {
+    HTableDescriptor hdt = TEST_UTIL.createTableDescriptor("HCM-testGetOperationTimeout");
     hdt.addCoprocessor(SleepAndFailFirstTime.class.getName());
-    HTable table = TEST_UTIL.createTable(hdt, new byte[][]{FAM_NAM}, TEST_UTIL.getConfiguration());
+    Table table = TEST_UTIL.createTable(hdt, new byte[][]{FAM_NAM}, TEST_UTIL.getConfiguration());
     table.setRpcTimeout(Integer.MAX_VALUE);
+    SleepAndFailFirstTime.ct.set(0);
     // Check that it works if the timeout is big enough
     table.setOperationTimeout(120 * 1000);
     table.get(new Get(FAM_NAM));
@@ -393,6 +429,62 @@ public class TestHCM {
   }
 
   @Test
+  public void testPutOperationTimeout() throws Exception {
+    HTableDescriptor hdt = TEST_UTIL.createTableDescriptor("HCM-testPutOperationTimeout");
+    hdt.addCoprocessor(SleepAndFailFirstTime.class.getName());
+    Table table = TEST_UTIL.createTable(hdt, new byte[][] { FAM_NAM },TEST_UTIL.getConfiguration());
+    table.setRpcTimeout(Integer.MAX_VALUE);
+    SleepAndFailFirstTime.ct.set(0);
+    // Check that it works if the timeout is big enough
+    table.setOperationTimeout(120 * 1000);
+    table.put(new Put(FAM_NAM).addColumn(FAM_NAM, FAM_NAM, FAM_NAM));
+
+    // Resetting and retrying. Will fail this time, not enough time for the second try
+    SleepAndFailFirstTime.ct.set(0);
+    try {
+      table.setOperationTimeout(30 * 1000);
+      table.put(new Put(FAM_NAM).addColumn(FAM_NAM, FAM_NAM, FAM_NAM));
+      Assert.fail("We expect an exception here");
+    } catch (RetriesExhaustedWithDetailsException e) {
+      // The client has a CallTimeout class, but it's not shared.We're not very clean today,
+      //  in the general case you can expect the call to stop, but the exception may vary.
+      // In this test however, we're sure that it will be a socket timeout.
+      LOG.info("We received an exception, as expected ", e);
+    } catch (IOException e) {
+      Assert.fail("Wrong exception:" + e.getMessage());
+    } finally {
+      table.close();
+    }
+  }
+
+  @Test
+  public void testDeleteOperationTimeout() throws Exception {
+    HTableDescriptor hdt = TEST_UTIL.createTableDescriptor("HCM-testDeleteOperationTimeout");
+    hdt.addCoprocessor(SleepAndFailFirstTime.class.getName());
+    Table table = TEST_UTIL.createTable(hdt, new byte[][] { FAM_NAM },TEST_UTIL.getConfiguration());
+    table.setRpcTimeout(Integer.MAX_VALUE);
+    SleepAndFailFirstTime.ct.set(0);
+    // Check that it works if the timeout is big enough
+    table.setOperationTimeout(120 * 1000);
+    table.delete(new Delete(FAM_NAM));
+
+    // Resetting and retrying. Will fail this time, not enough time for the second try
+    SleepAndFailFirstTime.ct.set(0);
+    try {
+      table.setOperationTimeout(30 * 1000);
+      table.delete(new Delete(FAM_NAM));
+      Assert.fail("We expect an exception here");
+    } catch (IOException e) {
+      // The client has a CallTimeout class, but it's not shared.We're not very clean today,
+      //  in the general case you can expect the call to stop, but the exception may vary.
+      // In this test however, we're sure that it will be a socket timeout.
+      LOG.info("We received an exception, as expected ", e);
+    } finally {
+      table.close();
+    }
+  }
+
+  @Test
   public void testRpcTimeout() throws Exception {
     HTableDescriptor hdt = TEST_UTIL.createTableDescriptor("HCM-testRpcTimeout");
     hdt.addCoprocessor(SleepCoprocessor.class.getName());
@@ -420,14 +512,14 @@ public class TestHCM {
   }
 
   @Test
-  public void testWriteRpcTimeout() throws Exception {
-    HTableDescriptor hdt = TEST_UTIL.createTableDescriptor("HCM-testWriteRpcTimeout");
-    hdt.addCoprocessor(SleepWriteCoprocessor.class.getName());
+  public void testIncrementRpcTimeout() throws Exception {
+    HTableDescriptor hdt = TEST_UTIL.createTableDescriptor("HCM-testIncrementRpcTimeout");
+    hdt.addCoprocessor(SleepCoprocessor.class.getName());
     Configuration c = new Configuration(TEST_UTIL.getConfiguration());
 
     try (Table t = TEST_UTIL.createTable(hdt, new byte[][] { FAM_NAM }, c)) {
-      t.setWriteRpcTimeout(SleepWriteCoprocessor.SLEEP_TIME / 2);
-      t.setOperationTimeout(SleepWriteCoprocessor.SLEEP_TIME * 100);
+      t.setWriteRpcTimeout(SleepCoprocessor.SLEEP_TIME / 2);
+      t.setOperationTimeout(SleepCoprocessor.SLEEP_TIME * 100);
       Increment i = new Increment(FAM_NAM);
       i.addColumn(FAM_NAM, FAM_NAM, 1);
       t.increment(i);
@@ -437,7 +529,7 @@ public class TestHCM {
     }
 
     // Again, with configuration based override
-    c.setInt(HConstants.HBASE_RPC_WRITE_TIMEOUT_KEY, SleepWriteCoprocessor.SLEEP_TIME / 2);
+    c.setInt(HConstants.HBASE_RPC_WRITE_TIMEOUT_KEY, SleepCoprocessor.SLEEP_TIME / 2);
     try (Connection conn = ConnectionFactory.createConnection(c)) {
       try (Table t = conn.getTable(hdt.getTableName())) {
         Increment i = new Increment(FAM_NAM);
@@ -451,8 +543,46 @@ public class TestHCM {
   }
 
   @Test
-  public void testReadRpcTimeout() throws Exception {
-    HTableDescriptor hdt = TEST_UTIL.createTableDescriptor("HCM-testReadRpcTimeout");
+  public void testDeleteRpcTimeout() throws Exception {
+    HTableDescriptor hdt = TEST_UTIL.createTableDescriptor("HCM-testDeleteRpcTimeout");
+    hdt.addCoprocessor(SleepCoprocessor.class.getName());
+    Configuration c = new Configuration(TEST_UTIL.getConfiguration());
+
+    try (Table t = TEST_UTIL.createTable(hdt, new byte[][] { FAM_NAM }, c)) {
+      t.setWriteRpcTimeout(SleepCoprocessor.SLEEP_TIME / 2);
+      t.setOperationTimeout(SleepCoprocessor.SLEEP_TIME * 100);
+      Delete d = new Delete(FAM_NAM);
+      d.addColumn(FAM_NAM, FAM_NAM, 1);
+      t.delete(d);
+      fail("Write should not have succeeded");
+    } catch (RetriesExhaustedException e) {
+      // expected
+    }
+
+  }
+
+  @Test
+  public void testPutRpcTimeout() throws Exception {
+    HTableDescriptor hdt = TEST_UTIL.createTableDescriptor("HCM-testPutRpcTimeout");
+    hdt.addCoprocessor(SleepCoprocessor.class.getName());
+    Configuration c = new Configuration(TEST_UTIL.getConfiguration());
+
+    try (Table t = TEST_UTIL.createTable(hdt, new byte[][] { FAM_NAM }, c)) {
+      t.setWriteRpcTimeout(SleepCoprocessor.SLEEP_TIME / 2);
+      t.setOperationTimeout(SleepCoprocessor.SLEEP_TIME * 100);
+      Put p = new Put(FAM_NAM);
+      p.addColumn(FAM_NAM, FAM_NAM, FAM_NAM);
+      t.put(p);
+      fail("Write should not have succeeded");
+    } catch (RetriesExhaustedException e) {
+      // expected
+    }
+
+  }
+
+  @Test
+  public void testGetRpcTimeout() throws Exception {
+    HTableDescriptor hdt = TEST_UTIL.createTableDescriptor("HCM-testGetRpcTimeout");
     hdt.addCoprocessor(SleepCoprocessor.class.getName());
     Configuration c = new Configuration(TEST_UTIL.getConfiguration());
 
@@ -503,6 +633,7 @@ public class TestHCM {
     TEST_UTIL.createTable(hdt, new byte[][] { FAM_NAM }).close();
 
     Configuration c = new Configuration(TEST_UTIL.getConfiguration());
+    SleepAndFailFirstTime.ct.set(0);
     c.setInt(HConstants.HBASE_CLIENT_PAUSE, 3000);
     c.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, 4000);
 
@@ -1009,8 +1140,7 @@ public class TestHCM {
       curServer.getServerName().getPort(),
       conn.getCachedLocation(TABLE_NAME, ROW).getRegionLocation().getPort());
 
-    TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
-        HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
+    TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, RPC_RETRY);
     table.close();
   }
 


[23/50] [abbrv] hbase git commit: HBASE-16855 Avoid NPE in MetricsConnection’s construction (ChiaPing Tsai)

Posted by la...@apache.org.
HBASE-16855 Avoid NPE in MetricsConnection\u2019s construction (ChiaPing Tsai)


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

Branch: refs/heads/branch-1
Commit: 6df7554d299ce72e6d8dca35c5de119a47f5594d
Parents: 08498c6
Author: tedyu <yu...@gmail.com>
Authored: Mon Oct 17 09:35:02 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Mon Oct 17 09:35:02 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/client/MetricsConnection.java  | 38 +++++++++++++++++---
 .../hbase/client/TestMetricsConnection.java     | 36 ++++++++++++++++---
 2 files changed, 65 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/6df7554d/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetricsConnection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetricsConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetricsConnection.java
index b3c8180..9dd803a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetricsConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetricsConnection.java
@@ -22,6 +22,7 @@ import com.google.protobuf.Descriptors.MethodDescriptor;
 import com.google.protobuf.Message;
 import com.yammer.metrics.core.Counter;
 import com.yammer.metrics.core.Histogram;
+import com.yammer.metrics.core.MetricName;
 import com.yammer.metrics.core.MetricsRegistry;
 import com.yammer.metrics.core.Timer;
 import com.yammer.metrics.reporting.JmxReporter;
@@ -294,24 +295,38 @@ public class MetricsConnection implements StatisticTrackable {
   public MetricsConnection(final ConnectionManager.HConnectionImplementation conn) {
     this.scope = conn.toString();
     this.registry = new MetricsRegistry();
-    final ThreadPoolExecutor batchPool = (ThreadPoolExecutor) conn.getCurrentBatchPool();
-    final ThreadPoolExecutor metaPool = (ThreadPoolExecutor) conn.getCurrentMetaLookupPool();
 
-    this.registry.newGauge(this.getClass(), "executorPoolActiveThreads", scope,
+    this.registry.newGauge(getExecutorPoolName(),
         new RatioGauge() {
           @Override protected double getNumerator() {
+            ThreadPoolExecutor batchPool = (ThreadPoolExecutor) conn.getCurrentBatchPool();
+            if (batchPool == null) {
+              return 0;
+            }
             return batchPool.getActiveCount();
           }
           @Override protected double getDenominator() {
+            ThreadPoolExecutor batchPool = (ThreadPoolExecutor) conn.getCurrentBatchPool();
+            if (batchPool == null) {
+              return 0;
+            }
             return batchPool.getMaximumPoolSize();
           }
         });
-    this.registry.newGauge(this.getClass(), "metaPoolActiveThreads", scope,
+    this.registry.newGauge(getMetaPoolName(),
         new RatioGauge() {
           @Override protected double getNumerator() {
+            ThreadPoolExecutor metaPool = (ThreadPoolExecutor) conn.getCurrentMetaLookupPool();
+            if (metaPool == null) {
+              return 0;
+            }
             return metaPool.getActiveCount();
           }
           @Override protected double getDenominator() {
+            ThreadPoolExecutor metaPool = (ThreadPoolExecutor) conn.getCurrentMetaLookupPool();
+            if (metaPool == null) {
+              return 0;
+            }
             return metaPool.getMaximumPoolSize();
           }
         });
@@ -334,6 +349,21 @@ public class MetricsConnection implements StatisticTrackable {
     this.reporter.start();
   }
 
+  @VisibleForTesting
+  final MetricName getExecutorPoolName() {
+    return new MetricName(getClass(), "executorPoolActiveThreads", scope);
+  }
+
+  @VisibleForTesting
+  final MetricName getMetaPoolName() {
+    return new MetricName(getClass(), "metaPoolActiveThreads", scope);
+  }
+
+  @VisibleForTesting
+  MetricsRegistry getMetricsRegistry() {
+    return registry;
+  }
+
   public void shutdown() {
     this.reporter.shutdown();
     this.registry.shutdown();

http://git-wip-us.apache.org/repos/asf/hbase/blob/6df7554d/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMetricsConnection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMetricsConnection.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMetricsConnection.java
index 10a913e..b9a9a73 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMetricsConnection.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMetricsConnection.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hbase.client;
 
 import com.google.protobuf.ByteString;
+import com.yammer.metrics.util.RatioGauge;
 import org.apache.hadoop.hbase.client.ConnectionManager.HConnectionImplementation;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
@@ -37,24 +38,43 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.mockito.Mockito;
-
 import java.io.IOException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import static org.junit.Assert.assertEquals;
 
 @Category({MetricsTests.class, SmallTests.class})
 public class TestMetricsConnection {
-
+  private static final ExecutorService BATCH_POOL = Executors.newFixedThreadPool(2);
   private static MetricsConnection METRICS;
-
+  private static final AtomicBoolean closed = new AtomicBoolean(false);
+  private static final Runnable RUNNER = new Runnable() {
+    @Override
+    public void run() {
+      try {
+        while (!closed.get() && !Thread.interrupted()) {
+          TimeUnit.MILLISECONDS.sleep(10);
+        }
+      } catch (InterruptedException e) {
+      }
+    }
+  };
   @BeforeClass
   public static void beforeClass() {
     HConnectionImplementation mocked = Mockito.mock(HConnectionImplementation.class);
     Mockito.when(mocked.toString()).thenReturn("mocked-connection");
-    METRICS = new MetricsConnection(Mockito.mock(HConnectionImplementation.class));
+    Mockito.when(mocked.getCurrentBatchPool()).thenReturn(BATCH_POOL);
+    BATCH_POOL.submit(RUNNER);
+    METRICS = new MetricsConnection(mocked);
   }
 
   @AfterClass
-  public static void afterClass() {
+  public static void afterClass() throws InterruptedException {
     METRICS.shutdown();
+    BATCH_POOL.shutdownNow();
+    BATCH_POOL.awaitTermination(Long.MAX_VALUE, TimeUnit.DAYS);
   }
 
   @Test
@@ -116,5 +136,11 @@ public class TestMetricsConnection {
       Assert.assertEquals("Failed to invoke reqHist on " + t, loop, t.reqHist.count());
       Assert.assertEquals("Failed to invoke respHist on " + t, loop, t.respHist.count());
     }
+    RatioGauge executorMetrics = (RatioGauge) METRICS.getMetricsRegistry()
+            .allMetrics().get(METRICS.getExecutorPoolName());
+    RatioGauge metaMetrics = (RatioGauge) METRICS.getMetricsRegistry()
+            .allMetrics().get(METRICS.getMetaPoolName());
+    assertEquals((double) 0.5, executorMetrics.value(), 0);
+    assertEquals(Double.NaN, metaMetrics.value(), 0);
   }
 }


[03/50] [abbrv] hbase git commit: HBASE-16699 Overflows in AverageIntervalRateLimiter's refill() and getWaitInterval()

Posted by la...@apache.org.
HBASE-16699 Overflows in AverageIntervalRateLimiter's refill() and getWaitInterval()

Signed-off-by: Matteo Bertozzi <ma...@cloudera.com>


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

Branch: refs/heads/branch-1
Commit: 66038b8c1a4ad775fdb16c7a70ceb3d86ff149bb
Parents: acb1392
Author: Huaxiang Sun <hs...@cloudera.com>
Authored: Mon Oct 10 14:12:03 2016 -0700
Committer: Matteo Bertozzi <ma...@cloudera.com>
Committed: Mon Oct 10 14:12:22 2016 -0700

----------------------------------------------------------------------
 .../quotas/AverageIntervalRateLimiter.java      |  20 ++-
 .../hbase/quotas/DefaultOperationQuota.java     |  47 +++----
 .../hadoop/hbase/quotas/NoopOperationQuota.java |   5 -
 .../hadoop/hbase/quotas/NoopQuotaLimiter.java   |   9 --
 .../hadoop/hbase/quotas/OperationQuota.java     |  59 +--------
 .../hadoop/hbase/quotas/QuotaLimiter.java       |   9 --
 .../apache/hadoop/hbase/quotas/RateLimiter.java |  38 +++++-
 .../hadoop/hbase/quotas/TimeBasedLimiter.java   |  13 --
 .../hadoop/hbase/quotas/TestRateLimiter.java    | 130 +++++++++++++++++++
 9 files changed, 196 insertions(+), 134 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/66038b8c/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/AverageIntervalRateLimiter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/AverageIntervalRateLimiter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/AverageIntervalRateLimiter.java
index 75e6aea..9320d7c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/AverageIntervalRateLimiter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/AverageIntervalRateLimiter.java
@@ -34,12 +34,21 @@ public class AverageIntervalRateLimiter extends RateLimiter {
       return limit;
     }
 
-    long delta = (limit * (now - nextRefillTime)) / super.getTimeUnitInMillis();
+    long timeInterval = now - nextRefillTime;
+    long delta = 0;
+    long timeUnitInMillis = super.getTimeUnitInMillis();
+    if (timeInterval >= timeUnitInMillis) {
+      delta = limit;
+    } else if (timeInterval > 0) {
+      double r = ((double)timeInterval / (double)timeUnitInMillis) * limit;
+      delta = (long)r;
+    }
+
     if (delta > 0) {
       this.nextRefillTime = now;
-      return Math.min(limit, delta);
     }
-    return 0;
+
+    return delta;
   }
 
   @Override
@@ -47,8 +56,9 @@ public class AverageIntervalRateLimiter extends RateLimiter {
     if (nextRefillTime == -1) {
       return 0;
     }
-    long timeUnitInMillis = super.getTimeUnitInMillis();
-    return ((amount * timeUnitInMillis) / limit) - ((available * timeUnitInMillis) / limit);
+
+    double r = ((double)(amount - available)) * super.getTimeUnitInMillis() / limit;
+    return (long)r;
   }
 
   // This method is for strictly testing purpose only

http://git-wip-us.apache.org/repos/asf/hbase/blob/66038b8c/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/DefaultOperationQuota.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/DefaultOperationQuota.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/DefaultOperationQuota.java
index 654e8fa..6caac74 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/DefaultOperationQuota.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/DefaultOperationQuota.java
@@ -31,8 +31,7 @@ public class DefaultOperationQuota implements OperationQuota {
   private long readAvailable = 0;
   private long writeConsumed = 0;
   private long readConsumed = 0;
-
-  private AvgOperationSize avgOpSize = new AvgOperationSize();
+  private final long[] operationSize;
 
   public DefaultOperationQuota(final QuotaLimiter... limiters) {
     this(Arrays.asList(limiters));
@@ -43,6 +42,12 @@ public class DefaultOperationQuota implements OperationQuota {
    */
   public DefaultOperationQuota(final List<QuotaLimiter> limiters) {
     this.limiters = limiters;
+    int size = OperationType.values().length;
+    operationSize = new long[size];
+
+    for (int i = 0; i < size; ++i) {
+      operationSize[i] = 0;
+    }
   }
 
   @Override
@@ -68,22 +73,12 @@ public class DefaultOperationQuota implements OperationQuota {
 
   @Override
   public void close() {
-    // Calculate and set the average size of get, scan and mutate for the current operation
-    long getSize = avgOpSize.getAvgOperationSize(OperationType.GET);
-    long scanSize = avgOpSize.getAvgOperationSize(OperationType.SCAN);
-    long mutationSize = avgOpSize.getAvgOperationSize(OperationType.MUTATE);
-    for (final QuotaLimiter limiter : limiters) {
-      limiter.addOperationSize(OperationType.GET, getSize);
-      limiter.addOperationSize(OperationType.SCAN, scanSize);
-      limiter.addOperationSize(OperationType.MUTATE, mutationSize);
-    }
-
     // Adjust the quota consumed for the specified operation
-    long writeDiff = avgOpSize.getOperationSize(OperationType.MUTATE) - writeConsumed;
-    long readDiff =
-        (avgOpSize.getOperationSize(OperationType.GET) + avgOpSize
-            .getOperationSize(OperationType.SCAN)) - readConsumed;
-    for (final QuotaLimiter limiter : limiters) {
+    long writeDiff = operationSize[OperationType.MUTATE.ordinal()] - writeConsumed;
+    long readDiff = operationSize[OperationType.GET.ordinal()] +
+        operationSize[OperationType.SCAN.ordinal()] - readConsumed;
+
+    for (final QuotaLimiter limiter: limiters) {
       if (writeDiff != 0) limiter.consumeWrite(writeDiff);
       if (readDiff != 0) limiter.consumeRead(readDiff);
     }
@@ -101,33 +96,21 @@ public class DefaultOperationQuota implements OperationQuota {
 
   @Override
   public void addGetResult(final Result result) {
-    avgOpSize.addGetResult(result);
+    operationSize[OperationType.GET.ordinal()] += QuotaUtil.calculateResultSize(result);
   }
 
   @Override
   public void addScanResult(final List<Result> results) {
-    avgOpSize.addScanResult(results);
+    operationSize[OperationType.SCAN.ordinal()] += QuotaUtil.calculateResultSize(results);
   }
 
   @Override
   public void addMutation(final Mutation mutation) {
-    avgOpSize.addMutation(mutation);
-  }
-
-  @Override
-  public long getAvgOperationSize(OperationType type) {
-    return avgOpSize.getAvgOperationSize(type);
+    operationSize[OperationType.MUTATE.ordinal()] += QuotaUtil.calculateMutationSize(mutation);
   }
 
   private long estimateConsume(final OperationType type, int numReqs, long avgSize) {
     if (numReqs > 0) {
-      for (final QuotaLimiter limiter : limiters) {
-        long size = limiter.getAvgOperationSize(type);
-        if (size > 0) {
-          avgSize = size;
-          break;
-        }
-      }
       return avgSize * numReqs;
     }
     return 0;

http://git-wip-us.apache.org/repos/asf/hbase/blob/66038b8c/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/NoopOperationQuota.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/NoopOperationQuota.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/NoopOperationQuota.java
index 2463ef7..9d37141 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/NoopOperationQuota.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/NoopOperationQuota.java
@@ -68,9 +68,4 @@ final class NoopOperationQuota implements OperationQuota {
   public long getWriteAvailable() {
     return Long.MAX_VALUE;
   }
-
-  @Override
-  public long getAvgOperationSize(OperationType type) {
-    return -1;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/66038b8c/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/NoopQuotaLimiter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/NoopQuotaLimiter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/NoopQuotaLimiter.java
index 699fd1a..e72f3d2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/NoopQuotaLimiter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/NoopQuotaLimiter.java
@@ -63,15 +63,6 @@ final class NoopQuotaLimiter implements QuotaLimiter {
   }
 
   @Override
-  public void addOperationSize(OperationType type, long size) {
-  }
-
-  @Override
-  public long getAvgOperationSize(OperationType type) {
-    return -1;
-  }
-
-  @Override
   public String toString() {
     return "NoopQuotaLimiter";
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/66038b8c/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/OperationQuota.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/OperationQuota.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/OperationQuota.java
index 6010c13..ee38256 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/OperationQuota.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/OperationQuota.java
@@ -29,58 +29,10 @@ public interface OperationQuota {
   }
 
   /**
-   * Keeps track of the average data size of operations like get, scan, mutate
-   */
-  public class AvgOperationSize {
-    private final long[] sizeSum;
-    private final long[] count;
-
-    public AvgOperationSize() {
-      int size = OperationType.values().length;
-      sizeSum = new long[size];
-      count = new long[size];
-      for (int i = 0; i < size; ++i) {
-        sizeSum[i] = 0;
-        count[i] = 0;
-      }
-    }
-
-    public void addOperationSize(OperationType type, long size) {
-      if (size > 0) {
-        int index = type.ordinal();
-        sizeSum[index] += size;
-        count[index]++;
-      }
-    }
-
-    public long getAvgOperationSize(OperationType type) {
-      int index = type.ordinal();
-      return count[index] > 0 ? sizeSum[index] / count[index] : 0;
-    }
-
-    public long getOperationSize(OperationType type) {
-      return sizeSum[type.ordinal()];
-    }
-
-    public void addGetResult(final Result result) {
-      long size = QuotaUtil.calculateResultSize(result);
-      addOperationSize(OperationType.GET, size);
-    }
-
-    public void addScanResult(final List<Result> results) {
-      long size = QuotaUtil.calculateResultSize(results);
-      addOperationSize(OperationType.SCAN, size);
-    }
-
-    public void addMutation(final Mutation mutation) {
-      long size = QuotaUtil.calculateMutationSize(mutation);
-      addOperationSize(OperationType.MUTATE, size);
-    }
-  }
-
-  /**
-   * Checks if it is possible to execute the specified operation. The quota will be estimated based
-   * on the number of operations to perform and the average size accumulated during time.
+   * Checks if it is possible to execute the specified operation.
+   * The quota will be estimated based on the number of operations to perform
+   * and the average size accumulated during time.
+   *
    * @param numWrites number of write operation that will be performed
    * @param numReads number of small-read operation that will be performed
    * @param numScans number of long-read operation that will be performed
@@ -114,7 +66,4 @@ public interface OperationQuota {
 
   /** @return the number of bytes available to write to avoid exceeding the quota */
   long getWriteAvailable();
-
-  /** @return the average data size of the specified operation */
-  long getAvgOperationSize(OperationType type);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/66038b8c/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaLimiter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaLimiter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaLimiter.java
index ffacbc0..a27211c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaLimiter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaLimiter.java
@@ -68,13 +68,4 @@ public interface QuotaLimiter {
 
   /** @return the number of bytes available to write to avoid exceeding the quota */
   long getWriteAvailable();
-
-  /**
-   * Add the average size of the specified operation type.
-   * The average will be used as estimate for the next operations.
-   */
-  void addOperationSize(OperationType type, long size);
-
-  /** @return the average data size of the specified operation */
-  long getAvgOperationSize(OperationType type);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/66038b8c/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RateLimiter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RateLimiter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RateLimiter.java
index 3b407bd..356afd3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RateLimiter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RateLimiter.java
@@ -111,7 +111,15 @@ public abstract class RateLimiter {
   public synchronized void update(final RateLimiter other) {
     this.tunit = other.tunit;
     if (this.limit < other.limit) {
-      this.avail += (other.limit - this.limit);
+      // If avail is capped to this.limit, it will never overflow,
+      // otherwise, avail may overflow, just be careful here.
+      long diff = other.limit - this.limit;
+      if (this.avail <= Long.MAX_VALUE - diff) {
+        this.avail += diff;
+        this.avail = Math.min(this.avail, other.limit);
+      } else {
+        this.avail = other.limit;
+      }
     }
     this.limit = other.limit;
   }
@@ -142,10 +150,14 @@ public abstract class RateLimiter {
 
   /**
    * Are there enough available resources to allow execution?
-   * @param amount the number of required resources
+   * @param amount the number of required resources, a non-negative number
    * @return true if there are enough available resources, otherwise false
    */
   public synchronized boolean canExecute(final long amount) {
+    if (isBypass()) {
+      return true;
+    }
+
     long refillAmount = refill(limit);
     if (refillAmount == 0 && avail < amount) {
       return false;
@@ -170,13 +182,27 @@ public abstract class RateLimiter {
   }
 
   /**
-   * consume amount available units.
+   * consume amount available units, amount could be a negative number
    * @param amount the number of units to consume
    */
   public synchronized void consume(final long amount) {
-    this.avail -= amount;
-    if (this.avail < 0) {
-      this.avail = 0;
+
+    if (isBypass()) {
+      return;
+    }
+
+    if (amount >= 0 ) {
+      this.avail -= amount;
+      if (this.avail < 0) {
+        this.avail = 0;
+      }
+    } else {
+      if (this.avail <= Long.MAX_VALUE + amount) {
+        this.avail -= amount;
+        this.avail = Math.min(this.avail, this.limit);
+      } else {
+        this.avail = this.limit;
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/66038b8c/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TimeBasedLimiter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TimeBasedLimiter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TimeBasedLimiter.java
index beb4147..1563878 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TimeBasedLimiter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TimeBasedLimiter.java
@@ -26,8 +26,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Throttle;
 import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuota;
-import org.apache.hadoop.hbase.quotas.OperationQuota.AvgOperationSize;
-import org.apache.hadoop.hbase.quotas.OperationQuota.OperationType;
 
 /**
  * Simple time based limiter that checks the quota Throttle
@@ -42,7 +40,6 @@ public class TimeBasedLimiter implements QuotaLimiter {
   private RateLimiter writeSizeLimiter = null;
   private RateLimiter readReqsLimiter = null;
   private RateLimiter readSizeLimiter = null;
-  private AvgOperationSize avgOpSize = new AvgOperationSize();
 
   private TimeBasedLimiter() {
     if (FixedIntervalRateLimiter.class.getName().equals(
@@ -186,16 +183,6 @@ public class TimeBasedLimiter implements QuotaLimiter {
   }
 
   @Override
-  public void addOperationSize(OperationType type, long size) {
-    avgOpSize.addOperationSize(type, size);
-  }
-
-  @Override
-  public long getAvgOperationSize(OperationType type) {
-    return avgOpSize.getAvgOperationSize(type);
-  }
-
-  @Override
   public String toString() {
     StringBuilder builder = new StringBuilder();
     builder.append("TimeBasedLimiter(");

http://git-wip-us.apache.org/repos/asf/hbase/blob/66038b8c/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRateLimiter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRateLimiter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRateLimiter.java
index 4c5a658..1ca6643 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRateLimiter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRateLimiter.java
@@ -19,6 +19,7 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.ManualEnvironmentEdge;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -277,4 +278,133 @@ public class TestRateLimiter {
     limiter.setNextRefillTime(limiter.getNextRefillTime() - 5000);
     assertEquals(60, limiter.refill(limiter.getLimit()));
   }
+
+  @Test
+  public void testUnconfiguredLimiters() throws InterruptedException {
+
+    ManualEnvironmentEdge testEdge = new ManualEnvironmentEdge();
+    EnvironmentEdgeManager.injectEdge(testEdge);
+    long limit = Long.MAX_VALUE;
+
+    // For unconfigured limiters, it is supposed to use as much as possible
+    RateLimiter avgLimiter = new AverageIntervalRateLimiter();
+    RateLimiter fixLimiter = new FixedIntervalRateLimiter();
+
+    assertEquals(limit, avgLimiter.getAvailable());
+    assertEquals(limit, fixLimiter.getAvailable());
+
+    assertTrue(avgLimiter.canExecute(limit));
+    avgLimiter.consume(limit);
+
+    assertTrue(fixLimiter.canExecute(limit));
+    fixLimiter.consume(limit);
+
+    // Make sure that available is Long.MAX_VALUE
+    assertTrue(limit == avgLimiter.getAvailable());
+    assertTrue(limit == fixLimiter.getAvailable());
+
+    // after 100 millseconds, it should be able to execute limit as well
+    testEdge.incValue(100);
+
+    assertTrue(avgLimiter.canExecute(limit));
+    avgLimiter.consume(limit);
+
+    assertTrue(fixLimiter.canExecute(limit));
+    fixLimiter.consume(limit);
+
+    // Make sure that available is Long.MAX_VALUE
+    assertTrue(limit == avgLimiter.getAvailable());
+    assertTrue(limit == fixLimiter.getAvailable());
+
+    EnvironmentEdgeManager.reset();
+  }
+
+  @Test
+  public void testExtremeLimiters() throws InterruptedException {
+
+    ManualEnvironmentEdge testEdge = new ManualEnvironmentEdge();
+    EnvironmentEdgeManager.injectEdge(testEdge);
+    long limit = Long.MAX_VALUE - 1;
+
+    RateLimiter avgLimiter = new AverageIntervalRateLimiter();
+    avgLimiter.set(limit, TimeUnit.SECONDS);
+    RateLimiter fixLimiter = new FixedIntervalRateLimiter();
+    fixLimiter.set(limit, TimeUnit.SECONDS);
+
+    assertEquals(limit, avgLimiter.getAvailable());
+    assertEquals(limit, fixLimiter.getAvailable());
+
+    assertTrue(avgLimiter.canExecute(limit / 2));
+    avgLimiter.consume(limit / 2);
+
+    assertTrue(fixLimiter.canExecute(limit / 2));
+    fixLimiter.consume(limit / 2);
+
+    // Make sure that available is whatever left
+    assertTrue((limit - (limit / 2)) == avgLimiter.getAvailable());
+    assertTrue((limit - (limit / 2)) == fixLimiter.getAvailable());
+
+    // after 100 millseconds, both should not be able to execute the limit
+    testEdge.incValue(100);
+
+    assertFalse(avgLimiter.canExecute(limit));
+    assertFalse(fixLimiter.canExecute(limit));
+
+    // after 500 millseconds, average interval limiter should be able to execute the limit
+    testEdge.incValue(500);
+    assertTrue(avgLimiter.canExecute(limit));
+    assertFalse(fixLimiter.canExecute(limit));
+
+    // Make sure that available is correct
+    assertTrue(limit == avgLimiter.getAvailable());
+    assertTrue((limit - (limit / 2)) == fixLimiter.getAvailable());
+
+    // after 500 millseconds, both should be able to execute
+    testEdge.incValue(500);
+    assertTrue(avgLimiter.canExecute(limit));
+    assertTrue(fixLimiter.canExecute(limit));
+
+    // Make sure that available is Long.MAX_VALUE
+    assertTrue(limit == avgLimiter.getAvailable());
+    assertTrue(limit == fixLimiter.getAvailable());
+
+    EnvironmentEdgeManager.reset();
+  }
+
+  /*
+   * This test case is tricky. Basically, it simulates the following events:
+   *           Thread-1                             Thread-2
+   * t0:  canExecute(100) and consume(100)
+   * t1:                                         canExecute(100), avail may be increased by 80
+   * t2:  consume(-80) as actual size is 20
+   * It will check if consume(-80) can handle overflow correctly.
+   */
+  @Test
+  public void testLimiterCompensationOverflow() throws InterruptedException {
+
+    long limit = Long.MAX_VALUE - 1;
+    long guessNumber = 100;
+
+    // For unconfigured limiters, it is supposed to use as much as possible
+    RateLimiter avgLimiter = new AverageIntervalRateLimiter();
+    avgLimiter.set(limit, TimeUnit.SECONDS);
+
+    assertEquals(limit, avgLimiter.getAvailable());
+
+    // The initial guess is that 100 bytes.
+    assertTrue(avgLimiter.canExecute(guessNumber));
+    avgLimiter.consume(guessNumber);
+
+    // Make sure that available is whatever left
+    assertTrue((limit - guessNumber) == avgLimiter.getAvailable());
+
+    // Manually set avil to simulate that another thread call canExecute().
+    // It is simulated by consume().
+    avgLimiter.consume(-80);
+    assertTrue((limit - guessNumber + 80) == avgLimiter.getAvailable());
+
+    // Now thread1 compensates 80
+    avgLimiter.consume(-80);
+    assertTrue(limit == avgLimiter.getAvailable());
+  }
 }


[37/50] [abbrv] hbase git commit: Revert "HBASE-16562 ITBLL should fail to start if misconfigured, addendum"

Posted by la...@apache.org.
Revert "HBASE-16562 ITBLL should fail to start if misconfigured, addendum"

This reverts commit 6f73ef2dff46692ede976621f3e4b9e5cfae01fa.

See discussion on JIRA.


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

Branch: refs/heads/branch-1
Commit: 65c2dd489f2c1773c9e253fb950daf223eb9a154
Parents: d76cc4c
Author: Sean Busbey <bu...@apache.org>
Authored: Mon Oct 24 09:16:40 2016 -0500
Committer: Sean Busbey <bu...@apache.org>
Committed: Mon Oct 24 09:16:40 2016 -0500

----------------------------------------------------------------------
 .../test/IntegrationTestBigLinkedList.java      | 21 ++++++++++----------
 1 file changed, 11 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/65c2dd48/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
index 87ac1f7..366fc02 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
@@ -678,6 +678,17 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase {
         Integer width = (args.length < 4) ? null : Integer.parseInt(args[3]);
         Integer wrapMultiplier = (args.length < 5) ? null : Integer.parseInt(args[4]);
         Integer numWalkers = (args.length < 6) ? null : Integer.parseInt(args[5]);
+
+        long wrap = (long)width*wrapMultiplier;
+        if (wrap < numNodes && numNodes % wrap != 0) {
+          /**
+           *  numNodes should be a multiple of width*wrapMultiplier.
+           *  If numNodes less than wrap, wrap will be set to be equal with numNodes,
+           *  See {@link GeneratorMapper#setup(Mapper.Context)}
+           * */
+          System.err.println(USAGE);
+          return 1;
+        }
         return run(numMappers, numNodes, tmpOutput, width, wrapMultiplier, numWalkers);
       } catch (NumberFormatException e) {
         System.err.println("Parsing generator arguments failed: " + e.getMessage());
@@ -802,16 +813,6 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase {
     public int run(int numMappers, long numNodes, Path tmpOutput,
         Integer width, Integer wrapMultiplier, Integer numWalkers)
         throws Exception {
-      long wrap = (long)width*wrapMultiplier;
-      if (wrap < numNodes && numNodes % wrap != 0) {
-        /**
-         *  numNodes should be a multiple of width*wrapMultiplier.
-         *  If numNodes less than wrap, wrap will be set to be equal with numNodes,
-         *  See {@link GeneratorMapper#setup(Mapper.Context)}
-         * */
-        System.err.println(USAGE);
-        return 1;
-      }
       int ret = runRandomInputGenerator(numMappers, numNodes, tmpOutput, width, wrapMultiplier,
           numWalkers);
       if (ret > 0) {


[21/50] [abbrv] hbase git commit: HBASE-16853 Regions are assigned to Region Servers in /hbase/draining after HBase Master failover (David Pope)

Posted by la...@apache.org.
HBASE-16853 Regions are assigned to Region Servers in /hbase/draining after HBase Master failover (David Pope)


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

Branch: refs/heads/branch-1
Commit: a4cbdede31f97bb988daf944f603f9f25b368044
Parents: b7f283c
Author: tedyu <yu...@gmail.com>
Authored: Sun Oct 16 18:54:59 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Sun Oct 16 18:54:59 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/master/ServerManager.java      |   2 +-
 .../hbase/zookeeper/DrainingServerTracker.java  |  18 ++++
 .../hbase/master/TestAssignmentListener.java    | 100 +++++++++++++++++++
 3 files changed, 119 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a4cbdede/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
index bcaa4d0..1817d6e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
@@ -533,7 +533,6 @@ public class ServerManager {
     }
   }
 
-
   public DeadServer getDeadServers() {
     return this.deadservers;
   }
@@ -744,6 +743,7 @@ public class ServerManager {
                "Ignoring request to add it again.");
       return false;
     }
+    LOG.info("Server " + sn + " added to draining server list.");
     return this.drainingServers.add(sn);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a4cbdede/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/DrainingServerTracker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/DrainingServerTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/DrainingServerTracker.java
index 5969143..413f226 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/DrainingServerTracker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/DrainingServerTracker.java
@@ -27,6 +27,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.master.ServerListener;
 import org.apache.hadoop.hbase.master.ServerManager;
 import org.apache.zookeeper.KeeperException;
 
@@ -68,6 +69,23 @@ public class DrainingServerTracker extends ZooKeeperListener {
    */
   public void start() throws KeeperException, IOException {
     watcher.registerListener(this);
+    // Add a ServerListener to check if a server is draining when it's added.
+    serverManager.registerListener(
+        new ServerListener() {
+
+          @Override
+          public void serverAdded(ServerName sn) {
+            if (drainingServers.contains(sn)){
+              serverManager.addServerToDrainList(sn);
+            }
+          }
+
+          @Override
+          public void serverRemoved(ServerName serverName) {
+            // no-op
+          }
+        }
+    );
     List<String> servers =
       ZKUtil.listChildrenAndWatchThem(watcher, watcher.drainingZNode);
     add(servers);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a4cbdede/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java
index 6257fe5..f171821 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java
@@ -21,16 +21,22 @@ package org.apache.hadoop.hbase.master;
 import static org.junit.Assert.assertEquals;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.ServerLoad;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.HTable;
@@ -40,16 +46,32 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.JVMClusterUtil;
+import org.apache.hadoop.hbase.zookeeper.DrainingServerTracker;
+import org.apache.hadoop.hbase.zookeeper.RegionServerTracker;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.junit.AfterClass;
+import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
 
 @Category(MediumTests.class)
 public class TestAssignmentListener {
   private static final Log LOG = LogFactory.getLog(TestAssignmentListener.class);
 
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static final Abortable abortable = new Abortable() {
+    @Override
+    public boolean isAborted() {
+      return false;
+    }
+
+    @Override
+    public void abort(String why, Throwable e) {
+    }
+  };
 
   static class DummyListener {
     protected AtomicInteger modified = new AtomicInteger(0);
@@ -260,4 +282,82 @@ public class TestAssignmentListener {
       am.unregisterListener(listener);
     }
   }
+
+  @Test
+  public void testAddNewServerThatExistsInDraining() throws Exception {
+    // Under certain circumstances, such as when we failover to the Backup
+    // HMaster, the DrainingServerTracker is started with existing servers in
+    // draining before all of the Region Servers register with the
+    // ServerManager as "online".  This test is to ensure that Region Servers
+    // are properly added to the ServerManager.drainingServers when they
+    // register with the ServerManager under these circumstances.
+    Configuration conf = TEST_UTIL.getConfiguration();
+    ZooKeeperWatcher zooKeeper = new ZooKeeperWatcher(conf,
+        "zkWatcher-NewServerDrainTest", abortable, true);
+    String baseZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
+        HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
+    String drainingZNode = ZKUtil.joinZNode(baseZNode,
+        conf.get("zookeeper.znode.draining.rs", "draining"));
+
+    MasterServices services = Mockito.mock(MasterServices.class);
+    HMaster master = Mockito.mock(HMaster.class);
+    Mockito.when(master.getConfiguration()).thenReturn(conf);
+
+    ServerName SERVERNAME_A = ServerName.valueOf("mockserverbulk_a.org", 1000, 8000);
+    ServerName SERVERNAME_B = ServerName.valueOf("mockserverbulk_b.org", 1001, 8000);
+    ServerName SERVERNAME_C = ServerName.valueOf("mockserverbulk_c.org", 1002, 8000);
+
+    // We'll start with 2 servers in draining that existed before the
+    // HMaster started.
+    ArrayList<ServerName> drainingServers = new ArrayList<ServerName>();
+    drainingServers.add(SERVERNAME_A);
+    drainingServers.add(SERVERNAME_B);
+
+    // We'll have 2 servers that come online AFTER the DrainingServerTracker
+    // is started (just as we see when we failover to the Backup HMaster).
+    // One of these will already be a draining server.
+    HashMap<ServerName, ServerLoad> onlineServers = new HashMap<ServerName, ServerLoad>();
+    onlineServers.put(SERVERNAME_A, ServerLoad.EMPTY_SERVERLOAD);
+    onlineServers.put(SERVERNAME_C, ServerLoad.EMPTY_SERVERLOAD);
+
+    // Create draining znodes for the draining servers, which would have been
+    // performed when the previous HMaster was running.
+    for (ServerName sn : drainingServers) {
+      String znode = ZKUtil.joinZNode(drainingZNode, sn.getServerName());
+      ZKUtil.createAndFailSilent(zooKeeper, znode);
+    }
+
+    // Now, we follow the same order of steps that the HMaster does to setup
+    // the ServerManager, RegionServerTracker, and DrainingServerTracker.
+    ServerManager serverManager = new ServerManager(master, services);
+
+    RegionServerTracker regionServerTracker = new RegionServerTracker(
+        zooKeeper, master, serverManager);
+    regionServerTracker.start();
+
+    DrainingServerTracker drainingServerTracker = new DrainingServerTracker(
+        zooKeeper, master, serverManager);
+    drainingServerTracker.start();
+
+    // Confirm our ServerManager lists are empty.
+    Assert.assertEquals(serverManager.getOnlineServers(),
+        new HashMap<ServerName, ServerLoad>());
+    Assert.assertEquals(serverManager.getDrainingServersList(),
+        new ArrayList<ServerName>());
+
+    // checkAndRecordNewServer() is how servers are added to the ServerManager.
+    ArrayList<ServerName> onlineDrainingServers = new ArrayList<ServerName>();
+    for (ServerName sn : onlineServers.keySet()){
+      // Here's the actual test.
+      serverManager.checkAndRecordNewServer(sn, onlineServers.get(sn));
+      if (drainingServers.contains(sn)){
+        onlineDrainingServers.add(sn);  // keeping track for later verification
+      }
+    }
+
+    // Verify the ServerManager lists are correctly updated.
+    Assert.assertEquals(serverManager.getOnlineServers(), onlineServers);
+    Assert.assertEquals(serverManager.getDrainingServersList(),
+        onlineDrainingServers);
+  }
 }


[07/50] [abbrv] hbase git commit: HBASE-16803 Make hbase:acl table unsplittable

Posted by la...@apache.org.
HBASE-16803 Make hbase:acl table unsplittable


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

Branch: refs/heads/branch-1
Commit: 408a9eb8a303e9f0db5eecaa77c30f28ff521d07
Parents: f13a216
Author: tedyu <yu...@gmail.com>
Authored: Tue Oct 11 12:50:05 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Tue Oct 11 12:50:05 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hbase/security/access/AccessControlLists.java   | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/408a9eb8/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
index 50d575e..b20c110 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
@@ -62,6 +62,7 @@ import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
 import org.apache.hadoop.hbase.regionserver.BloomType;
+import org.apache.hadoop.hbase.regionserver.DisabledRegionSplitPolicy;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.security.User;
@@ -136,6 +137,8 @@ public class AccessControlLists {
         // Set cache data blocks in L1 if more than one cache tier deployed; e.g. this will
         // be the case if we are using CombinedBlockCache (Bucket Cache).
         .setCacheDataInL1(true));
+    ACL_TABLEDESC.setValue(HTableDescriptor.SPLIT_POLICY,
+        DisabledRegionSplitPolicy.class.getName());
     master.createSystemTable(ACL_TABLEDESC);
   }
 


[49/50] [abbrv] hbase git commit: HBASE-16983 TestMultiTableSnapshotInputFormat failing with Unable to create region directory: /tmp/...

Posted by la...@apache.org.
HBASE-16983 TestMultiTableSnapshotInputFormat failing with Unable to
create region directory: /tmp/...


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

Branch: refs/heads/branch-1
Commit: b3ddea9fc4f9e5ee7017642b57f05d0a40256f2f
Parents: 722be3a
Author: Michael Stack <st...@apache.org>
Authored: Mon Oct 31 21:48:49 2016 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Mon Oct 31 21:48:49 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/mapreduce/TestMultiTableSnapshotInputFormat.java  | 2 +-
 .../org/apache/hadoop/hbase/regionserver/TestStoreFileInfo.java    | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b3ddea9f/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableSnapshotInputFormat.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableSnapshotInputFormat.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableSnapshotInputFormat.java
index 93bb820..d8c4be7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableSnapshotInputFormat.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableSnapshotInputFormat.java
@@ -63,7 +63,7 @@ public class TestMultiTableSnapshotInputFormat extends MultiTableInputFormatTest
 
   @Before
   public void setUp() throws Exception {
-    this.restoreDir = new Path("/tmp");
+    this.restoreDir = TEST_UTIL.getDataTestDir();
 
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b3ddea9f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileInfo.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileInfo.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileInfo.java
index 5aa96c1..c5200fb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileInfo.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileInfo.java
@@ -66,7 +66,7 @@ public class TestStoreFileInfo {
   @Test
   public void testEqualsWithLink() throws IOException {
     Path origin = new Path("/origin");
-    Path tmp = new Path("/tmp");
+    Path tmp = TEST_UTIL.getDataTestDir();
     Path archive = new Path("/archive");
     HFileLink link1 = new HFileLink(new Path(origin, "f1"), new Path(tmp, "f1"),
       new Path(archive, "f1"));


[24/50] [abbrv] hbase git commit: HBASE-16653 Backport HBASE-11393 to branches which support namespace

Posted by la...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/66941910/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java
index 2c9fc0f..dd15e4c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java
@@ -22,10 +22,14 @@ package org.apache.hadoop.hbase.replication;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
@@ -37,6 +41,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
@@ -183,13 +188,13 @@ public class TestPerTableCFReplication {
     Map<TableName, List<String>> tabCFsMap = null;
 
     // 1. null or empty string, result should be null
-    tabCFsMap = ReplicationAdmin.parseTableCFsFromConfig(null);
+    tabCFsMap = ReplicationSerDeHelper.parseTableCFsFromConfig(null);
     assertEquals(null, tabCFsMap);
 
-    tabCFsMap = ReplicationAdmin.parseTableCFsFromConfig("");
+    tabCFsMap = ReplicationSerDeHelper.parseTableCFsFromConfig("");
     assertEquals(null, tabCFsMap);
 
-    tabCFsMap = ReplicationAdmin.parseTableCFsFromConfig("   ");
+    tabCFsMap = ReplicationSerDeHelper.parseTableCFsFromConfig("   ");
     assertEquals(null, tabCFsMap);
 
     TableName tab1 = TableName.valueOf("tab1");
@@ -197,20 +202,20 @@ public class TestPerTableCFReplication {
     TableName tab3 = TableName.valueOf("tab3");
 
     // 2. single table: "tab1" / "tab2:cf1" / "tab3:cf1,cf3"
-    tabCFsMap = ReplicationAdmin.parseTableCFsFromConfig("tab1");
+    tabCFsMap = ReplicationSerDeHelper.parseTableCFsFromConfig("tab1");
     assertEquals(1, tabCFsMap.size()); // only one table
     assertTrue(tabCFsMap.containsKey(tab1));   // its table name is "tab1"
     assertFalse(tabCFsMap.containsKey(tab2));  // not other table
     assertEquals(null, tabCFsMap.get(tab1));   // null cf-list,
 
-    tabCFsMap = ReplicationAdmin.parseTableCFsFromConfig("tab2:cf1");
+    tabCFsMap = ReplicationSerDeHelper.parseTableCFsFromConfig("tab2:cf1");
     assertEquals(1, tabCFsMap.size()); // only one table
     assertTrue(tabCFsMap.containsKey(tab2));   // its table name is "tab2"
     assertFalse(tabCFsMap.containsKey(tab1));  // not other table
     assertEquals(1, tabCFsMap.get(tab2).size());   // cf-list contains only 1 cf
     assertEquals("cf1", tabCFsMap.get(tab2).get(0));// the only cf is "cf1"
 
-    tabCFsMap = ReplicationAdmin.parseTableCFsFromConfig("tab3 : cf1 , cf3");
+    tabCFsMap = ReplicationSerDeHelper.parseTableCFsFromConfig("tab3 : cf1 , cf3");
     assertEquals(1, tabCFsMap.size()); // only one table
     assertTrue(tabCFsMap.containsKey(tab3));   // its table name is "tab2"
     assertFalse(tabCFsMap.containsKey(tab1));  // not other table
@@ -219,7 +224,7 @@ public class TestPerTableCFReplication {
     assertTrue(tabCFsMap.get(tab3).contains("cf3"));// contains "cf3"
 
     // 3. multiple tables: "tab1 ; tab2:cf1 ; tab3:cf1,cf3"
-    tabCFsMap = ReplicationAdmin.parseTableCFsFromConfig("tab1 ; tab2:cf1 ; tab3:cf1,cf3");
+    tabCFsMap = ReplicationSerDeHelper.parseTableCFsFromConfig("tab1 ; tab2:cf1 ; tab3:cf1,cf3");
     // 3.1 contains 3 tables : "tab1", "tab2" and "tab3"
     assertEquals(3, tabCFsMap.size());
     assertTrue(tabCFsMap.containsKey(tab1));
@@ -237,7 +242,7 @@ public class TestPerTableCFReplication {
 
     // 4. contiguous or additional ";"(table delimiter) or ","(cf delimiter) can be tolerated
     // still use the example of multiple tables: "tab1 ; tab2:cf1 ; tab3:cf1,cf3"
-    tabCFsMap = ReplicationAdmin.parseTableCFsFromConfig(
+    tabCFsMap = ReplicationSerDeHelper.parseTableCFsFromConfig(
       "tab1 ; ; tab2:cf1 ; tab3:cf1,,cf3 ;");
     // 4.1 contains 3 tables : "tab1", "tab2" and "tab3"
     assertEquals(3, tabCFsMap.size());
@@ -256,7 +261,7 @@ public class TestPerTableCFReplication {
 
     // 5. invalid format "tab1:tt:cf1 ; tab2::cf1 ; tab3:cf1,cf3"
     //    "tab1:tt:cf1" and "tab2::cf1" are invalid and will be ignored totally
-    tabCFsMap = ReplicationAdmin.parseTableCFsFromConfig(
+    tabCFsMap = ReplicationSerDeHelper.parseTableCFsFromConfig(
       "tab1:tt:cf1 ; tab2::cf1 ; tab3:cf1,cf3");
     // 5.1 no "tab1" and "tab2", only "tab3"
     assertEquals(1, tabCFsMap.size()); // only one table
@@ -267,7 +272,100 @@ public class TestPerTableCFReplication {
     assertEquals(2, tabCFsMap.get(tab3).size());
     assertTrue(tabCFsMap.get(tab3).contains("cf1"));
     assertTrue(tabCFsMap.get(tab3).contains("cf3"));
- }
+  }
+
+  @Test
+  public void testTableCFsHelperConverter() {
+
+    ZooKeeperProtos.TableCF[] tableCFs = null;
+    Map<TableName, List<String>> tabCFsMap = null;
+
+    // 1. null or empty string, result should be null
+    assertNull(ReplicationSerDeHelper.convert(tabCFsMap));
+
+    tabCFsMap = new HashMap<TableName, List<String>>();
+    tableCFs = ReplicationSerDeHelper.convert(tabCFsMap);
+    assertEquals(0, tableCFs.length);
+
+    TableName tab1 = TableName.valueOf("tab1");
+    TableName tab2 = TableName.valueOf("tab2");
+    TableName tab3 = TableName.valueOf("tab3");
+
+    // 2. single table: "tab1" / "tab2:cf1" / "tab3:cf1,cf3"
+    tabCFsMap.clear();
+    tabCFsMap.put(tab1, null);
+    tableCFs = ReplicationSerDeHelper.convert(tabCFsMap);
+    assertEquals(1, tableCFs.length); // only one table
+    assertEquals(tab1.toString(),
+        tableCFs[0].getTableName().getQualifier().toStringUtf8());
+    assertEquals(0, tableCFs[0].getFamiliesCount());
+
+    tabCFsMap.clear();
+    tabCFsMap.put(tab2, new ArrayList<String>());
+    tabCFsMap.get(tab2).add("cf1");
+    tableCFs = ReplicationSerDeHelper.convert(tabCFsMap);
+    assertEquals(1, tableCFs.length); // only one table
+    assertEquals(tab2.toString(),
+        tableCFs[0].getTableName().getQualifier().toStringUtf8());
+    assertEquals(1, tableCFs[0].getFamiliesCount());
+    assertEquals("cf1", tableCFs[0].getFamilies(0).toStringUtf8());
+
+    tabCFsMap.clear();
+    tabCFsMap.put(tab3, new ArrayList<String>());
+    tabCFsMap.get(tab3).add("cf1");
+    tabCFsMap.get(tab3).add("cf3");
+    tableCFs = ReplicationSerDeHelper.convert(tabCFsMap);
+    assertEquals(1, tableCFs.length);
+    assertEquals(tab3.toString(),
+        tableCFs[0].getTableName().getQualifier().toStringUtf8());
+    assertEquals(2, tableCFs[0].getFamiliesCount());
+    assertEquals("cf1", tableCFs[0].getFamilies(0).toStringUtf8());
+    assertEquals("cf3", tableCFs[0].getFamilies(1).toStringUtf8());
+
+    tabCFsMap.clear();
+    tabCFsMap.put(tab1, null);
+    tabCFsMap.put(tab2, new ArrayList<String>());
+    tabCFsMap.get(tab2).add("cf1");
+    tabCFsMap.put(tab3, new ArrayList<String>());
+    tabCFsMap.get(tab3).add("cf1");
+    tabCFsMap.get(tab3).add("cf3");
+
+    tableCFs = ReplicationSerDeHelper.convert(tabCFsMap);
+    assertEquals(3, tableCFs.length);
+    assertNotNull(ReplicationSerDeHelper.getTableCF(tableCFs, tab1.toString()));
+    assertNotNull(ReplicationSerDeHelper.getTableCF(tableCFs, tab2.toString()));
+    assertNotNull(ReplicationSerDeHelper.getTableCF(tableCFs, tab3.toString()));
+
+    assertEquals(0,
+        ReplicationSerDeHelper.getTableCF(tableCFs, tab1.toString()).getFamiliesCount());
+
+    assertEquals(1,
+        ReplicationSerDeHelper.getTableCF(tableCFs, tab2.toString()).getFamiliesCount());
+    assertEquals("cf1",
+        ReplicationSerDeHelper.getTableCF(tableCFs, tab2.toString()).getFamilies(0).toStringUtf8());
+
+    assertEquals(2,
+        ReplicationSerDeHelper.getTableCF(tableCFs, tab3.toString()).getFamiliesCount());
+    assertEquals("cf1",
+        ReplicationSerDeHelper.getTableCF(tableCFs, tab3.toString()).getFamilies(0).toStringUtf8());
+    assertEquals("cf3",
+        ReplicationSerDeHelper.getTableCF(tableCFs, tab3.toString()).getFamilies(1).toStringUtf8());
+
+    tabCFsMap = ReplicationSerDeHelper.convert2Map(tableCFs);
+    assertEquals(3, tabCFsMap.size());
+    assertTrue(tabCFsMap.containsKey(tab1));
+    assertTrue(tabCFsMap.containsKey(tab2));
+    assertTrue(tabCFsMap.containsKey(tab3));
+    // 3.2 table "tab1" : null cf-list
+    assertEquals(null, tabCFsMap.get(tab1));
+    // 3.3 table "tab2" : cf-list contains a single cf "cf1"
+    assertEquals(1, tabCFsMap.get(tab2).size());
+    assertEquals("cf1", tabCFsMap.get(tab2).get(0));
+    // 3.4 table "tab3" : cf-list contains "cf1" and "cf3"
+    assertEquals(2, tabCFsMap.get(tab3).size());
+    assertTrue(tabCFsMap.get(tab3).contains("cf1"));
+    assertTrue(tabCFsMap.get(tab3).contains("cf3"));
+  }
 
   @Test(timeout=300000)
   public void testPerTableCFReplication() throws Exception {
@@ -304,8 +402,23 @@ public class TestPerTableCFReplication {
       Table htab3C = connection3.getTable(tabCName);
 
       // A. add cluster2/cluster3 as peers to cluster1
-      replicationAdmin.addPeer("2", utility2.getClusterKey(), "TC;TB:f1,f3");
-      replicationAdmin.addPeer("3", utility3.getClusterKey(), "TA;TB:f1,f2");
+      ReplicationPeerConfig rpc2 = new ReplicationPeerConfig();
+      rpc2.setClusterKey(utility2.getClusterKey());
+      Map<TableName, List<String>> tableCFs = new HashMap<>();
+      tableCFs.put(tabCName, null);
+      tableCFs.put(tabBName, new ArrayList<String>());
+      tableCFs.get(tabBName).add("f1");
+      tableCFs.get(tabBName).add("f3");
+      replicationAdmin.addPeer("2", rpc2, tableCFs);
+
+      ReplicationPeerConfig rpc3 = new ReplicationPeerConfig();
+      rpc3.setClusterKey(utility3.getClusterKey());
+      tableCFs.clear();
+      tableCFs.put(tabAName, null);
+      tableCFs.put(tabBName, new ArrayList<String>());
+      tableCFs.get(tabBName).add("f1");
+      tableCFs.get(tabBName).add("f2");
+      replicationAdmin.addPeer("3", rpc3, tableCFs);
 
       // A1. tableA can only replicated to cluster3
       putAndWaitWithFamily(row1, f1Name, htab1A, htab3A);
@@ -348,8 +461,20 @@ public class TestPerTableCFReplication {
       deleteAndWaitWithFamily(row1, f3Name, htab1C, htab2C);
 
       // B. change peers' replicable table-cf config
-      replicationAdmin.setPeerTableCFs("2", "TA:f1,f2; TC:f2,f3");
-      replicationAdmin.setPeerTableCFs("3", "TB; TC:f3");
+      tableCFs.clear();
+      tableCFs.put(tabAName, new ArrayList<String>());
+      tableCFs.get(tabAName).add("f1");
+      tableCFs.get(tabAName).add("f2");
+      tableCFs.put(tabCName, new ArrayList<String>());
+      tableCFs.get(tabCName).add("f2");
+      tableCFs.get(tabCName).add("f3");
+      replicationAdmin.setPeerTableCFs("2", tableCFs);
+
+      tableCFs.clear();
+      tableCFs.put(tabBName, null);
+      tableCFs.put(tabCName, new ArrayList<String>());
+      tableCFs.get(tabCName).add("f3");
+      replicationAdmin.setPeerTableCFs("3", tableCFs);
 
       // B1. cf 'f1' of tableA can only replicated to cluster2
       putAndWaitWithFamily(row2, f1Name, htab1A, htab2A);

http://git-wip-us.apache.org/repos/asf/hbase/blob/66941910/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java
index e52a600..5283433 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java
@@ -124,7 +124,9 @@ public class TestReplicationBase {
     utility2.setZkCluster(miniZK);
     zkw2 = new ZooKeeperWatcher(conf2, "cluster2", null, true);
 
-    admin.addPeer("2", utility2.getClusterKey());
+    ReplicationPeerConfig rpc = new ReplicationPeerConfig();
+    rpc.setClusterKey(utility2.getClusterKey());
+    admin.addPeer("2", rpc);
 
     LOG.info("Setup second Zk");
     CONF_WITH_LOCALFS = HBaseConfiguration.create(conf1);

http://git-wip-us.apache.org/repos/asf/hbase/blob/66941910/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
index c293444..ba634dd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
@@ -369,7 +369,9 @@ public class TestReplicationSmallTests extends TestReplicationBase {
       }
     }
 
-    admin.addPeer("2", utility2.getClusterKey());
+    ReplicationPeerConfig rpc = new ReplicationPeerConfig();
+    rpc.setClusterKey(utility2.getClusterKey());
+    admin.addPeer("2", rpc);
     Thread.sleep(SLEEP_TIME);
     rowKey = Bytes.toBytes("do rep");
     put = new Put(rowKey);

http://git-wip-us.apache.org/repos/asf/hbase/blob/66941910/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
index 7f7ee98..001f147 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
@@ -170,7 +170,7 @@ public abstract class TestReplicationStateBasic {
 
     try {
       rp.addPeer(ID_ONE,
-        new ReplicationPeerConfig().setClusterKey("hostname1.example.org:1234:hbase"), null);
+        new ReplicationPeerConfig().setClusterKey("hostname1.example.org:1234:hbase"));
       fail("Should throw an IllegalArgumentException because "
             + "zookeeper.znode.parent is missing leading '/'.");
     } catch (IllegalArgumentException e) {
@@ -179,7 +179,7 @@ public abstract class TestReplicationStateBasic {
 
     try {
       rp.addPeer(ID_ONE,
-        new ReplicationPeerConfig().setClusterKey("hostname1.example.org:1234:/"), null);
+        new ReplicationPeerConfig().setClusterKey("hostname1.example.org:1234:/"));
       fail("Should throw an IllegalArgumentException because zookeeper.znode.parent is missing.");
     } catch (IllegalArgumentException e) {
       // Expected.
@@ -187,7 +187,7 @@ public abstract class TestReplicationStateBasic {
 
     try {
       rp.addPeer(ID_ONE,
-        new ReplicationPeerConfig().setClusterKey("hostname1.example.org::/hbase"), null);
+        new ReplicationPeerConfig().setClusterKey("hostname1.example.org::/hbase"));
       fail("Should throw an IllegalArgumentException because "
           + "hbase.zookeeper.property.clientPort is missing.");
     } catch (IllegalArgumentException e) {
@@ -207,7 +207,7 @@ public abstract class TestReplicationStateBasic {
     files1.add("file_3");
     assertNull(rqc.getReplicableHFiles(ID_ONE));
     assertEquals(0, rqc.getAllPeersFromHFileRefsQueue().size());
-    rp.addPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE), null);
+    rp.addPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE));
     rq1.addPeerToHFileRefs(ID_ONE);
     rq1.addHFileRefs(ID_ONE, files1);
     assertEquals(1, rqc.getAllPeersFromHFileRefsQueue().size());
@@ -229,8 +229,8 @@ public abstract class TestReplicationStateBasic {
     rqc.init();
 
     rp.init();
-    rp.addPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE), null);
-    rp.addPeer(ID_TWO, new ReplicationPeerConfig().setClusterKey(KEY_TWO), null);
+    rp.addPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE));
+    rp.addPeer(ID_TWO, new ReplicationPeerConfig().setClusterKey(KEY_TWO));
 
     List<String> files1 = new ArrayList<String>(3);
     files1.add("file_1");
@@ -288,9 +288,9 @@ public abstract class TestReplicationStateBasic {
     assertNumberOfPeers(0);
 
     // Add some peers
-    rp.addPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE), null);
+    rp.addPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE));
     assertNumberOfPeers(1);
-    rp.addPeer(ID_TWO, new ReplicationPeerConfig().setClusterKey(KEY_TWO), null);
+    rp.addPeer(ID_TWO, new ReplicationPeerConfig().setClusterKey(KEY_TWO));
     assertNumberOfPeers(2);
 
     // Test methods with a peer that is added but not connected
@@ -305,7 +305,7 @@ public abstract class TestReplicationStateBasic {
     assertNumberOfPeers(1);
 
     // Add one peer
-    rp.addPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE), null);
+    rp.addPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE));
     rp.peerAdded(ID_ONE);
     assertNumberOfPeers(2);
     assertTrue(rp.getStatusOfPeer(ID_ONE));
@@ -365,7 +365,7 @@ public abstract class TestReplicationStateBasic {
         rq3.addLog("qId" + i, "filename" + j);
       }
       //Add peers for the corresponding queues so they are not orphans
-      rp.addPeer("qId" + i, new ReplicationPeerConfig().setClusterKey("localhost:2818:/bogus" + i), null);
+      rp.addPeer("qId" + i, new ReplicationPeerConfig().setClusterKey("localhost:2818:/bogus" + i));
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/66941910/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java
index fd02d1a..a949e92 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java
@@ -203,7 +203,9 @@ public class TestReplicationSyncUpTool extends TestReplicationBase {
     /**
      * set M-S : Master: utility1 Slave1: utility2
      */
-    admin1.addPeer("1", utility2.getClusterKey());
+    ReplicationPeerConfig rpc = new ReplicationPeerConfig();
+    rpc.setClusterKey(utility2.getClusterKey());
+    admin1.addPeer("1", rpc);
 
     admin1.close();
     admin2.close();

http://git-wip-us.apache.org/repos/asf/hbase/blob/66941910/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
index a5df432..7b2e1fd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
@@ -146,7 +146,7 @@ public class TestReplicationTrackerZKImpl {
 
   @Test(timeout = 30000)
   public void testPeerRemovedEvent() throws Exception {
-    rp.addPeer("5", new ReplicationPeerConfig().setClusterKey(utility.getClusterKey()), null);
+    rp.addPeer("5", new ReplicationPeerConfig().setClusterKey(utility.getClusterKey()));
     rt.registerListener(new DummyReplicationListener());
     rp.removePeer("5");
     // wait for event
@@ -159,7 +159,7 @@ public class TestReplicationTrackerZKImpl {
   @Test(timeout = 30000)
   public void testPeerListChangedEvent() throws Exception {
     // add a peer
-    rp.addPeer("5", new ReplicationPeerConfig().setClusterKey(utility.getClusterKey()), null);
+    rp.addPeer("5", new ReplicationPeerConfig().setClusterKey(utility.getClusterKey()));
     zkw.getRecoverableZooKeeper().getZooKeeper().getChildren("/hbase/replication/peers/5", true);
     rt.registerListener(new DummyReplicationListener());
     rp.disablePeer("5");
@@ -183,16 +183,16 @@ public class TestReplicationTrackerZKImpl {
   public void testPeerNameControl() throws Exception {
     int exists = 0;
     int hyphen = 0;
-    rp.addPeer("6", new ReplicationPeerConfig().setClusterKey(utility.getClusterKey()), null);
+    rp.addPeer("6", new ReplicationPeerConfig().setClusterKey(utility.getClusterKey()));
 
     try{
-      rp.addPeer("6", new ReplicationPeerConfig().setClusterKey(utility.getClusterKey()), null);
+      rp.addPeer("6", new ReplicationPeerConfig().setClusterKey(utility.getClusterKey()));
     }catch(IllegalArgumentException e){
       exists++;
     }
 
     try{
-      rp.addPeer("6-ec2", new ReplicationPeerConfig().setClusterKey(utility.getClusterKey()), null);
+      rp.addPeer("6-ec2", new ReplicationPeerConfig().setClusterKey(utility.getClusterKey()));
     }catch(IllegalArgumentException e){
       hyphen++;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/66941910/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java
index 5010365..a246241 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java
@@ -130,7 +130,9 @@ public class TestReplicationWithTags {
     utility2 = new HBaseTestingUtility(conf2);
     utility2.setZkCluster(miniZK);
 
-    replicationAdmin.addPeer("2", utility2.getClusterKey());
+    ReplicationPeerConfig rpc = new ReplicationPeerConfig();
+    rpc.setClusterKey(utility2.getClusterKey());
+    replicationAdmin.addPeer("2", rpc);
 
     LOG.info("Setup second Zk");
     utility1.startMiniCluster(2);

http://git-wip-us.apache.org/repos/asf/hbase/blob/66941910/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestTableCFsUpdater.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestTableCFsUpdater.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestTableCFsUpdater.java
new file mode 100644
index 0000000..f53aef3
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestTableCFsUpdater.java
@@ -0,0 +1,164 @@
+/**
+ * Copyright The Apache Software Foundation
+ *
+ * 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.replication.master;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.replication.ReplicationSerDeHelper;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.zookeeper.KeeperException;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+@Category({ReplicationTests.class, SmallTests.class})
+public class TestTableCFsUpdater extends TableCFsUpdater {
+
+  private static final Log LOG = LogFactory.getLog(TestTableCFsUpdater.class);
+  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+  private static ZooKeeperWatcher zkw = null;
+  private static Abortable abortable = null;
+
+  public TestTableCFsUpdater() {
+    super(zkw, TEST_UTIL.getConfiguration(), abortable);
+  }
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    TEST_UTIL.startMiniZKCluster();
+    Configuration conf = TEST_UTIL.getConfiguration();
+    abortable = new Abortable() {
+      @Override
+      public void abort(String why, Throwable e) {
+        LOG.info(why, e);
+      }
+
+      @Override
+      public boolean isAborted() {
+        return false;
+      }
+    };
+    zkw = new ZooKeeperWatcher(conf, "TableCFs", abortable, true);
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    TEST_UTIL.shutdownMiniZKCluster();
+  }
+
+  @Test
+  public void testUpgrade() throws KeeperException, InterruptedException,
+      DeserializationException {
+    String peerId = "1";
+    TableName tab1 = TableName.valueOf("table1");
+    TableName tab2 = TableName.valueOf("table2");
+    TableName tab3 = TableName.valueOf("table3");
+
+    ReplicationPeerConfig rpc = new ReplicationPeerConfig();
+    rpc.setClusterKey(zkw.getQuorum());
+    String peerNode = getPeerNode(peerId);
+    ZKUtil.createWithParents(zkw, peerNode, ReplicationSerDeHelper.toByteArray(rpc));
+
+    String tableCFs = "table1:cf1,cf2;table2:cf3;table3";
+    String tableCFsNode = getTableCFsNode(peerId);
+    LOG.info("create tableCFs :" + tableCFsNode + " for peerId=" + peerId);
+    ZKUtil.createWithParents(zkw, tableCFsNode , Bytes.toBytes(tableCFs));
+
+    ReplicationPeerConfig actualRpc = ReplicationSerDeHelper.parsePeerFrom(ZKUtil.getData(zkw, peerNode));
+    String actualTableCfs = Bytes.toString(ZKUtil.getData(zkw, tableCFsNode));
+
+    assertEquals(rpc.getClusterKey(), actualRpc.getClusterKey());
+    assertNull(actualRpc.getTableCFsMap());
+    assertEquals(tableCFs, actualTableCfs);
+
+    peerId = "2";
+    rpc = new ReplicationPeerConfig();
+    rpc.setClusterKey(zkw.getQuorum());
+    peerNode = getPeerNode(peerId);
+    ZKUtil.createWithParents(zkw, peerNode, ReplicationSerDeHelper.toByteArray(rpc));
+
+    tableCFs = "table1:cf1,cf3;table2:cf2";
+    tableCFsNode = getTableCFsNode(peerId);
+    LOG.info("create tableCFs :" + tableCFsNode + " for peerId=" + peerId);
+    ZKUtil.createWithParents(zkw, tableCFsNode , Bytes.toBytes(tableCFs));
+
+    actualRpc = ReplicationSerDeHelper.parsePeerFrom(ZKUtil.getData(zkw, peerNode));
+    actualTableCfs = Bytes.toString(ZKUtil.getData(zkw, tableCFsNode));
+
+    assertEquals(rpc.getClusterKey(), actualRpc.getClusterKey());
+    assertNull(actualRpc.getTableCFsMap());
+    assertEquals(tableCFs, actualTableCfs);
+
+
+    update();
+
+    peerId = "1";
+    peerNode = getPeerNode(peerId);
+    actualRpc = ReplicationSerDeHelper.parsePeerFrom(ZKUtil.getData(zkw, peerNode));
+    assertEquals(rpc.getClusterKey(), actualRpc.getClusterKey());
+    Map<TableName, List<String>> tableNameListMap = actualRpc.getTableCFsMap();
+    assertEquals(3, tableNameListMap.size());
+    assertTrue(tableNameListMap.containsKey(tab1));
+    assertTrue(tableNameListMap.containsKey(tab2));
+    assertTrue(tableNameListMap.containsKey(tab3));
+    assertEquals(2, tableNameListMap.get(tab1).size());
+    assertEquals("cf1", tableNameListMap.get(tab1).get(0));
+    assertEquals("cf2", tableNameListMap.get(tab1).get(1));
+    assertEquals(1, tableNameListMap.get(tab2).size());
+    assertEquals("cf3", tableNameListMap.get(tab2).get(0));
+    assertNull(tableNameListMap.get(tab3));
+
+
+    peerId = "2";
+    peerNode = getPeerNode(peerId);
+    actualRpc = ReplicationSerDeHelper.parsePeerFrom(ZKUtil.getData(zkw, peerNode));
+    assertEquals(rpc.getClusterKey(), actualRpc.getClusterKey());
+    tableNameListMap = actualRpc.getTableCFsMap();
+    assertEquals(2, tableNameListMap.size());
+    assertTrue(tableNameListMap.containsKey(tab1));
+    assertTrue(tableNameListMap.containsKey(tab2));
+    assertEquals(2, tableNameListMap.get(tab1).size());
+    assertEquals("cf1", tableNameListMap.get(tab1).get(0));
+    assertEquals("cf3", tableNameListMap.get(tab1).get(1));
+    assertEquals(1, tableNameListMap.get(tab2).size());
+    assertEquals("cf2", tableNameListMap.get(tab2).get(0));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/66941910/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
index 7614b0f..24c6ef3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
@@ -535,7 +535,7 @@ public class TestReplicationSourceManager {
           FailInitializeDummyReplicationSource.class.getName());
       final ReplicationPeers rp = manager.getReplicationPeers();
       // Set up the znode and ReplicationPeer for the fake peer
-      rp.addPeer("FakePeer", new ReplicationPeerConfig().setClusterKey("localhost:1:/hbase"), null);
+      rp.addPeer("FakePeer", new ReplicationPeerConfig().setClusterKey("localhost:1:/hbase"));
       // Wait for the peer to get created and connected
       Waiter.waitFor(conf, 20000, new Waiter.Predicate<Exception>() {
         @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/66941910/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java
index dc4a340..f9ae011 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java
@@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
 import org.apache.hadoop.hbase.codec.KeyValueCodecWithTags;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.visibility.VisibilityController.VisibilityReplication;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -131,7 +132,9 @@ public class TestVisibilityLabelReplicationWithExpAsString extends TestVisibilit
     TEST_UTIL1 = new HBaseTestingUtility(conf1);
     TEST_UTIL1.setZkCluster(miniZK);
     zkw2 = new ZooKeeperWatcher(conf1, "cluster2", null, true);
-    replicationAdmin.addPeer("2", TEST_UTIL1.getClusterKey());
+    ReplicationPeerConfig rpc = new ReplicationPeerConfig();
+    rpc.setClusterKey(TEST_UTIL1.getClusterKey());
+    replicationAdmin.addPeer("2", rpc);
 
     TEST_UTIL.startMiniCluster(1);
     // Wait for the labels table to become available

http://git-wip-us.apache.org/repos/asf/hbase/blob/66941910/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java
index 419ad91..79cf0ac 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java
@@ -65,6 +65,7 @@ import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.visibility.VisibilityController.VisibilityReplication;
 import org.junit.experimental.categories.Category;
@@ -180,7 +181,9 @@ public class TestVisibilityLabelsReplication {
     TEST_UTIL1 = new HBaseTestingUtility(conf1);
     TEST_UTIL1.setZkCluster(miniZK);
     zkw2 = new ZooKeeperWatcher(conf1, "cluster2", null, true);
-    replicationAdmin.addPeer("2", TEST_UTIL1.getClusterKey());
+    ReplicationPeerConfig rpc = new ReplicationPeerConfig();
+    rpc.setClusterKey(TEST_UTIL1.getClusterKey());
+    replicationAdmin.addPeer("2", rpc);
 
     TEST_UTIL.startMiniCluster(1);
     // Wait for the labels table to become available

http://git-wip-us.apache.org/repos/asf/hbase/blob/66941910/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java
index d8087f5..2a3e7f4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java
@@ -108,6 +108,7 @@ import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.SplitTransactionImpl;
 import org.apache.hadoop.hbase.regionserver.TestEndToEndSplitTransaction;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationQueues;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter;
@@ -2260,7 +2261,9 @@ public class TestHBaseFsck {
     ReplicationAdmin replicationAdmin = new ReplicationAdmin(conf);
     Assert.assertEquals(0, replicationAdmin.getPeersCount());
     String zkPort =  conf.get(HConstants.ZOOKEEPER_CLIENT_PORT);
-    replicationAdmin.addPeer("1", "127.0.0.1:2181" + zkPort + ":/hbase");
+    ReplicationPeerConfig rpc = new ReplicationPeerConfig();
+    rpc.setClusterKey("127.0.0.1:2181" + zkPort + ":/hbase");
+    replicationAdmin.addPeer("1", rpc);
     replicationAdmin.getPeersCount();
     Assert.assertEquals(1, replicationAdmin.getPeersCount());
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/66941910/hbase-shell/src/main/ruby/hbase/replication_admin.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/replication_admin.rb b/hbase-shell/src/main/ruby/hbase/replication_admin.rb
index d0719d8..2e240e1 100644
--- a/hbase-shell/src/main/ruby/hbase/replication_admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/replication_admin.rb
@@ -39,11 +39,7 @@ module Hbase
     #----------------------------------------------------------------------------------------------
     # Add a new peer cluster to replicate to
     def add_peer(id, args = {}, peer_tableCFs = nil)
-      # make add_peer backwards compatible to take in string for clusterKey and peer_tableCFs
-      if args.is_a?(String)
-        cluster_key = args
-        @replication_admin.addPeer(id, cluster_key, peer_tableCFs)
-      elsif args.is_a?(Hash)
+      if args.is_a?(Hash)
         unless peer_tableCFs.nil?
           raise(ArgumentError, "peer_tableCFs should be specified as TABLE_CFS in args")
         end
@@ -87,9 +83,18 @@ module Hbase
           }
         end
 
-        @replication_admin.add_peer(id, replication_peer_config, table_cfs)
+        unless table_cfs.nil?
+          # convert table_cfs to TableName
+          map = java.util.HashMap.new
+          table_cfs.each{|key, val|
+            map.put(org.apache.hadoop.hbase.TableName.valueOf(key), val)
+          }
+          replication_peer_config.set_table_cfs_map(map)
+        end
+
+        @replication_admin.add_peer(id, replication_peer_config)
       else
-        raise(ArgumentError, "args must be either a String or Hash")
+        raise(ArgumentError, "args must be a Hash")
       end
     end
 
@@ -111,7 +116,7 @@ module Hbase
     #----------------------------------------------------------------------------------------------
     # List all peer clusters
     def list_peers
-      @replication_admin.listPeers
+      @replication_admin.listPeerConfigs
     end
 
     #----------------------------------------------------------------------------------------------
@@ -141,20 +146,42 @@ module Hbase
     #----------------------------------------------------------------------------------------------
     # Set new tableCFs config for the specified peer
     def set_peer_tableCFs(id, tableCFs)
-      @replication_admin.setPeerTableCFs(id, tableCFs)
+      unless tableCFs.nil?
+        # convert tableCFs to TableName
+        map = java.util.HashMap.new
+        tableCFs.each{|key, val|
+          map.put(org.apache.hadoop.hbase.TableName.valueOf(key), val)
+        }
+      end
+      @replication_admin.setPeerTableCFs(id, map)
     end
 
     #----------------------------------------------------------------------------------------------
     # Append a tableCFs config for the specified peer
     def append_peer_tableCFs(id, tableCFs)
-      @replication_admin.appendPeerTableCFs(id, tableCFs)
+      unless tableCFs.nil?
+        # convert tableCFs to TableName
+        map = java.util.HashMap.new
+        tableCFs.each{|key, val|
+          map.put(org.apache.hadoop.hbase.TableName.valueOf(key), val)
+        }
+      end
+      @replication_admin.appendPeerTableCFs(id, map)
     end
 
     #----------------------------------------------------------------------------------------------
     # Remove some tableCFs from the tableCFs config of the specified peer
     def remove_peer_tableCFs(id, tableCFs)
-      @replication_admin.removePeerTableCFs(id, tableCFs)
+      unless tableCFs.nil?
+        # convert tableCFs to TableName
+        map = java.util.HashMap.new
+        tableCFs.each{|key, val|
+          map.put(org.apache.hadoop.hbase.TableName.valueOf(key), val)
+        }
+      end
+      @replication_admin.removePeerTableCFs(id, map)
     end
+
     #----------------------------------------------------------------------------------------------
     # Enables a table's replication switch
     def enable_tablerep(table_name)

http://git-wip-us.apache.org/repos/asf/hbase/blob/66941910/hbase-shell/src/main/ruby/shell/commands/add_peer.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/add_peer.rb b/hbase-shell/src/main/ruby/shell/commands/add_peer.rb
index cf9862a..d209a37 100644
--- a/hbase-shell/src/main/ruby/shell/commands/add_peer.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/add_peer.rb
@@ -33,7 +33,7 @@ Examples:
 
   hbase> add_peer '1', CLUSTER_KEY => "server1.cie.com:2181:/hbase"
   hbase> add_peer '2', CLUSTER_KEY => "zk1,zk2,zk3:2182:/hbase-prod",
-    TABLE_CFS => { "table1" => [], "table2" => ["cf1"], "table3" => ["cf1", "cf2"] }
+    TABLE_CFS => { "table1" => [], "ns2:table2" => ["cf1"], "ns3:table3" => ["cf1", "cf2"] }
 
 For a custom replication endpoint, the ENDPOINT_CLASSNAME can be provided. Two optional arguments
 are DATA and CONFIG which can be specified to set different either the peer_data or configuration
@@ -48,7 +48,7 @@ the key TABLE_CFS.
   hbase> add_peer '9', ENDPOINT_CLASSNAME => 'org.apache.hadoop.hbase.MyReplicationEndpoint',
     DATA => { "key1" => 1 }, CONFIG => { "config1" => "value1", "config2" => "value2" },
   hbase> add_peer '10', ENDPOINT_CLASSNAME => 'org.apache.hadoop.hbase.MyReplicationEndpoint',
-    TABLE_CFS => { "table1" => [], "table2" => ["cf1"], "table3" => ["cf1", "cf2"] }
+    TABLE_CFS => { "table1" => [], "ns2:table2" => ["cf1"], "ns3:table3" => ["cf1", "cf2"] }
   hbase> add_peer '11', ENDPOINT_CLASSNAME => 'org.apache.hadoop.hbase.MyReplicationEndpoint',
     DATA => { "key1" => 1 }, CONFIG => { "config1" => "value1", "config2" => "value2" },
     TABLE_CFS => { "table1" => [], "ns2:table2" => ["cf1"], "ns3:table3" => ["cf1", "cf2"] }

http://git-wip-us.apache.org/repos/asf/hbase/blob/66941910/hbase-shell/src/main/ruby/shell/commands/append_peer_tableCFs.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/append_peer_tableCFs.rb b/hbase-shell/src/main/ruby/shell/commands/append_peer_tableCFs.rb
index 3919b20..24a9976 100644
--- a/hbase-shell/src/main/ruby/shell/commands/append_peer_tableCFs.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/append_peer_tableCFs.rb
@@ -26,7 +26,7 @@ Append a replicable table-cf config for the specified peer
 Examples:
 
   # append a table / table-cf to be replicable for a peer
-  hbase> append_peer_tableCFs '2', "table4:cfA,cfB"
+  hbase> append_peer_tableCFs '2',  { "ns1:table4" => ["cfA", "cfB"] }
 
 EOF
       end

http://git-wip-us.apache.org/repos/asf/hbase/blob/66941910/hbase-shell/src/main/ruby/shell/commands/list_peers.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/list_peers.rb b/hbase-shell/src/main/ruby/shell/commands/list_peers.rb
index cc1be04..6444c79 100644
--- a/hbase-shell/src/main/ruby/shell/commands/list_peers.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/list_peers.rb
@@ -33,12 +33,14 @@ EOF
         now = Time.now
         peers = replication_admin.list_peers
 
-        formatter.header(["PEER_ID", "CLUSTER_KEY", "STATE", "TABLE_CFS"])
+        formatter.header(["PEER_ID", "CLUSTER_KEY", "ENDPOINT_CLASSNAME",
+          "STATE", "TABLE_CFS"])
 
         peers.entrySet().each do |e|
           state = replication_admin.get_peer_state(e.key)
           tableCFs = replication_admin.show_peer_tableCFs(e.key)
-          formatter.row([ e.key, e.value, state, tableCFs ])
+          formatter.row([ e.key, e.value.getClusterKey,
+            e.value.getReplicationEndpointImpl, state, tableCFs ])
         end
 
         formatter.footer(now)

http://git-wip-us.apache.org/repos/asf/hbase/blob/66941910/hbase-shell/src/main/ruby/shell/commands/remove_peer_tableCFs.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/remove_peer_tableCFs.rb b/hbase-shell/src/main/ruby/shell/commands/remove_peer_tableCFs.rb
index 5b15b52..af64bda 100644
--- a/hbase-shell/src/main/ruby/shell/commands/remove_peer_tableCFs.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/remove_peer_tableCFs.rb
@@ -26,8 +26,8 @@ Remove a table / table-cf from the table-cfs config for the specified peer
 Examples:
 
   # Remove a table / table-cf from the replicable table-cfs for a peer
-  hbase> remove_peer_tableCFs '2', "table1"
-  hbase> remove_peer_tableCFs '2', "table1:cf1"
+  hbase> remove_peer_tableCFs '2', { "ns1:table1" => [] }
+  hbase> remove_peer_tableCFs '2', { "ns1:table1" => ["cf1"] }
 
 EOF
       end

http://git-wip-us.apache.org/repos/asf/hbase/blob/66941910/hbase-shell/src/main/ruby/shell/commands/set_peer_tableCFs.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/set_peer_tableCFs.rb b/hbase-shell/src/main/ruby/shell/commands/set_peer_tableCFs.rb
index 3a88dbb..5599aee 100644
--- a/hbase-shell/src/main/ruby/shell/commands/set_peer_tableCFs.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/set_peer_tableCFs.rb
@@ -32,7 +32,9 @@ module Shell
     # set table / table-cf to be replicable for a peer, for a table without
     # an explicit column-family list, all replicable column-families (with
     # replication_scope == 1) will be replicated
-    hbase> set_peer_tableCFs '2', "table1; table2:cf1,cf2; table3:cfA,cfB"
+    hbase> set_peer_tableCFs '2', { "ns1:table1" => [],
+                                    "ns2:table2" => ["cf1", "cf2"],
+                                    "ns3:table3" => ["cfA", "cfB"] }
 
   EOF
       end

http://git-wip-us.apache.org/repos/asf/hbase/blob/66941910/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestReplicationShell.java
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestReplicationShell.java b/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestReplicationShell.java
index 3f4af05..04fbc7a 100644
--- a/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestReplicationShell.java
+++ b/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestReplicationShell.java
@@ -28,7 +28,7 @@ import org.junit.experimental.categories.Category;
 
 @Category({ ClientTests.class, LargeTests.class })
 public class TestReplicationShell extends AbstractTestShell {
-  @Ignore ("Disabled because hangs on occasion.. about 10% of the time") @Test
+  @Test
   public void testRunShellTests() throws IOException {
     System.setProperty("shell.test.include", "replication_admin_test.rb");
     // Start all ruby tests

http://git-wip-us.apache.org/repos/asf/hbase/blob/66941910/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb b/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb
index 5b99c37..84bdf56 100644
--- a/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb
+++ b/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb
@@ -62,108 +62,142 @@ module Hbase
       assert_raise(ArgumentError) do
         replication_admin.add_peer(@peer_id, ['test'])
       end
+      assert_raise(ArgumentError) do
+        replication_admin.add_peer(@peer_id, 'test')
+      end
     end
 
-    define_test "add_peer: single zk cluster key" do
+    define_test "add_peer: single zk cluster key - peer config" do
       cluster_key = "server1.cie.com:2181:/hbase"
 
-      replication_admin.add_peer(@peer_id, cluster_key)
+      args = { CLUSTER_KEY => cluster_key }
+      replication_admin.add_peer(@peer_id, args)
 
       assert_equal(1, replication_admin.list_peers.length)
       assert(replication_admin.list_peers.key?(@peer_id))
-      assert_equal(cluster_key, replication_admin.list_peers.fetch(@peer_id))
+      assert_equal(cluster_key, replication_admin.list_peers.fetch(@peer_id).get_cluster_key)
 
       # cleanup for future tests
       replication_admin.remove_peer(@peer_id)
     end
 
-    define_test "add_peer: multiple zk cluster key" do
+    define_test "add_peer: multiple zk cluster key - peer config" do
       cluster_key = "zk1,zk2,zk3:2182:/hbase-prod"
 
-      replication_admin.add_peer(@peer_id, cluster_key)
+      args = { CLUSTER_KEY => cluster_key }
+      replication_admin.add_peer(@peer_id, args)
 
       assert_equal(1, replication_admin.list_peers.length)
       assert(replication_admin.list_peers.key?(@peer_id))
-      assert_equal(replication_admin.list_peers.fetch(@peer_id), cluster_key)
+      assert_equal(cluster_key, replication_admin.list_peers.fetch(@peer_id).get_cluster_key)
 
       # cleanup for future tests
       replication_admin.remove_peer(@peer_id)
     end
 
-    define_test "add_peer: multiple zk cluster key and table_cfs" do
+    define_test "add_peer: multiple zk cluster key and table_cfs - peer config" do
       cluster_key = "zk4,zk5,zk6:11000:/hbase-test"
-      table_cfs_str = "table1;table2:cf1;table3:cf2,cf3"
+      table_cfs = { "table1" => [], "ns2:table2" => ["cf1"], "ns3:table3" => ["cf1", "cf2"] }
 
-      replication_admin.add_peer(@peer_id, cluster_key, table_cfs_str)
+      args = { CLUSTER_KEY => cluster_key, TABLE_CFS => table_cfs }
+      replication_admin.add_peer(@peer_id, args)
 
       assert_equal(1, replication_admin.list_peers.length)
       assert(replication_admin.list_peers.key?(@peer_id))
-      assert_equal(cluster_key, replication_admin.list_peers.fetch(@peer_id))
-      assert_equal(table_cfs_str, replication_admin.show_peer_tableCFs(@peer_id))
+      assert_equal(cluster_key, replication_admin.list_peers.fetch(@peer_id).get_cluster_key)
+
+      table_cfs_map = replication_admin.get_peer_config(@peer_id).getTableCFsMap()
+      assert_tablecfs_equal(table_cfs, table_cfs_map)
 
       # cleanup for future tests
       replication_admin.remove_peer(@peer_id)
     end
 
-    define_test "add_peer: single zk cluster key - peer config" do
-      cluster_key = "server1.cie.com:2181:/hbase"
+    def assert_tablecfs_equal(table_cfs, table_cfs_map)
+      assert_equal(table_cfs.length, table_cfs_map.length)
+      table_cfs_map.each{|key, value|
+        assert(table_cfs.has_key?(key.getNameAsString))
+        if table_cfs.fetch(key.getNameAsString).length == 0
+          assert_equal(nil, value)
+        else
+          assert_equal(table_cfs.fetch(key.getNameAsString).length, value.length)
+          value.each{|v|
+            assert(table_cfs.fetch(key.getNameAsString).include?(v))
+          }
+        end
+      }
+    end
 
-      args = { CLUSTER_KEY => cluster_key }
+    define_test "add_peer: should fail when args is a hash and peer_tableCFs provided" do
+      cluster_key = "zk4,zk5,zk6:11000:/hbase-test"
+      table_cfs_str = "table1;table2:cf1;table3:cf1,cf2"
+
+      assert_raise(ArgumentError) do
+        args = { CLUSTER_KEY => cluster_key }
+        replication_admin.add_peer(@peer_id, args, table_cfs_str)
+      end
+    end
+
+    define_test "set_peer_tableCFs: works with table-cfs map" do
+      cluster_key = "zk4,zk5,zk6:11000:/hbase-test"
+      args = { CLUSTER_KEY => cluster_key}
       replication_admin.add_peer(@peer_id, args)
 
       assert_equal(1, replication_admin.list_peers.length)
       assert(replication_admin.list_peers.key?(@peer_id))
-      assert_equal(cluster_key, replication_admin.list_peers.fetch(@peer_id))
+      assert_equal(cluster_key, replication_admin.list_peers.fetch(@peer_id).get_cluster_key)
+
+      table_cfs = { "table1" => [], "table2" => ["cf1"], "ns3:table3" => ["cf1", "cf2"] }
+      replication_admin.set_peer_tableCFs(@peer_id, table_cfs)
+      table_cfs_map = replication_admin.get_peer_config(@peer_id).getTableCFsMap()
+      assert_tablecfs_equal(table_cfs, table_cfs_map)
 
       # cleanup for future tests
       replication_admin.remove_peer(@peer_id)
     end
 
-    define_test "add_peer: multiple zk cluster key - peer config" do
-      cluster_key = "zk1,zk2,zk3:2182:/hbase-prod"
-
+    define_test "append_peer_tableCFs: works with table-cfs map" do
+      cluster_key = "zk4,zk5,zk6:11000:/hbase-test"
       args = { CLUSTER_KEY => cluster_key }
       replication_admin.add_peer(@peer_id, args)
 
       assert_equal(1, replication_admin.list_peers.length)
       assert(replication_admin.list_peers.key?(@peer_id))
-      assert_equal(cluster_key, replication_admin.list_peers.fetch(@peer_id))
+      assert_equal(cluster_key, replication_admin.list_peers.fetch(@peer_id).get_cluster_key)
+
+      table_cfs = { "table1" => [], "ns2:table2" => ["cf1"] }
+      replication_admin.append_peer_tableCFs(@peer_id, table_cfs)
+      table_cfs_map = replication_admin.get_peer_config(@peer_id).getTableCFsMap()
+      assert_tablecfs_equal(table_cfs, table_cfs_map)
+
+      table_cfs = { "table1" => [], "ns2:table2" => ["cf1"], "ns3:table3" => ["cf1", "cf2"] }
+      replication_admin.append_peer_tableCFs(@peer_id, { "ns3:table3" => ["cf1", "cf2"] })
+      table_cfs_map = replication_admin.get_peer_config(@peer_id).getTableCFsMap()
+      assert_tablecfs_equal(table_cfs, table_cfs_map)
 
       # cleanup for future tests
       replication_admin.remove_peer(@peer_id)
     end
 
-    define_test "add_peer: multiple zk cluster key and table_cfs - peer config" do
+    define_test "remove_peer_tableCFs: works with table-cfs map" do
       cluster_key = "zk4,zk5,zk6:11000:/hbase-test"
-      table_cfs = { "table1" => [], "table2" => ["cf1"], "table3" => ["cf1", "cf2"] }
-      #table_cfs_str = "default.table1;default.table3:cf1,cf2;default.table2:cf1"
-
+      table_cfs = { "table1" => [], "ns2:table2" => ["cf1"], "ns3:table3" => ["cf1", "cf2"] }
       args = { CLUSTER_KEY => cluster_key, TABLE_CFS => table_cfs }
       replication_admin.add_peer(@peer_id, args)
 
-      assert_equal(1, command(:list_peers).length)
-      assert(command(:list_peers).key?(@peer_id))
-      assert_equal(cluster_key, command(:list_peers).fetch(@peer_id).get_cluster_key)
+      assert_equal(1, replication_admin.list_peers.length)
+      assert(replication_admin.list_peers.key?(@peer_id))
+      assert_equal(cluster_key, replication_admin.list_peers.fetch(@peer_id).get_cluster_key)
 
-      # Note: below assertion is dependent on the sort order of an unordered
-      # map and hence flaky depending on JVM
-      # Commenting out until HBASE-16274 is worked.
-      # assert_equal(table_cfs_str, command(:show_peer_tableCFs, @peer_id))
+      table_cfs = { "table1" => [], "ns2:table2" => ["cf1"] }
+      replication_admin.remove_peer_tableCFs(@peer_id, { "ns3:table3" => ["cf1", "cf2"] })
+      table_cfs_map = replication_admin.get_peer_config(@peer_id).getTableCFsMap()
+      assert_tablecfs_equal(table_cfs, table_cfs_map)
 
       # cleanup for future tests
       replication_admin.remove_peer(@peer_id)
     end
 
-    define_test "add_peer: should fail when args is a hash and peer_tableCFs provided" do
-      cluster_key = "zk4,zk5,zk6:11000:/hbase-test"
-      table_cfs_str = "table1;table2:cf1;table3:cf1,cf2"
-
-      assert_raise(ArgumentError) do
-        args = { CLUSTER_KEY => cluster_key }
-        replication_admin.add_peer(@peer_id, args, table_cfs_str)
-      end
-    end
-
     define_test "get_peer_config: works with simple clusterKey peer" do
       cluster_key = "localhost:2181:/hbase-test"
       args = { CLUSTER_KEY => cluster_key }
@@ -180,8 +214,8 @@ module Hbase
       config_params = { "config1" => "value1", "config2" => "value2" }
       args = { CLUSTER_KEY => cluster_key, ENDPOINT_CLASSNAME => repl_impl,
                CONFIG => config_params }
-      command(:add_peer, @peer_id, args)
-      peer_config = command(:get_peer_config, @peer_id)
+      replication_admin.add_peer(@peer_id, args)
+      peer_config = replication_admin.get_peer_config(@peer_id)
       assert_equal(cluster_key, peer_config.get_cluster_key)
       assert_equal(repl_impl, peer_config.get_replication_endpoint_impl)
       assert_equal(2, peer_config.get_configuration.size)


[12/50] [abbrv] hbase git commit: HBASE-16801 The Append/Increment may return the data from future (ChiaPing Tsai)

Posted by la...@apache.org.
HBASE-16801 The Append/Increment may return the data from future (ChiaPing Tsai)


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

Branch: refs/heads/branch-1
Commit: 3830890635261c9eccd7ceeff8ad1b95e3714cd6
Parents: bf03827
Author: tedyu <yu...@gmail.com>
Authored: Wed Oct 12 10:07:37 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Wed Oct 12 10:07:37 2016 -0700

----------------------------------------------------------------------
 .../hbase/regionserver/ServerNonceManager.java  |  2 +-
 .../regionserver/TestServerNonceManager.java    | 21 ++++++++++++++++++++
 2 files changed, 22 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/38308906/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ServerNonceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ServerNonceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ServerNonceManager.java
index bd9dad9..1e1a9a9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ServerNonceManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ServerNonceManager.java
@@ -62,7 +62,7 @@ public class ServerNonceManager {
     private static final long WAITING_BIT = 4;
     private static final long ALL_FLAG_BITS = WAITING_BIT | STATE_BITS;
 
-    private long mvcc;
+    private volatile long mvcc;
 
     @Override
     public String toString() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/38308906/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestServerNonceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestServerNonceManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestServerNonceManager.java
index 940f715..6bf1721 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestServerNonceManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestServerNonceManager.java
@@ -45,6 +45,27 @@ import org.mockito.stubbing.Answer;
 public class TestServerNonceManager {
 
   @Test
+  public void testMvcc() throws Exception {
+    ServerNonceManager nm = createManager();
+    final long group = 100;
+    final long nonce = 1;
+    final long initMvcc = 999;
+    assertTrue(nm.startOperation(group, nonce, createStoppable()));
+    nm.addMvccToOperationContext(group, nonce, initMvcc);
+    nm.endOperation(group, nonce, true);
+    assertEquals(initMvcc, nm.getMvccFromOperationContext(group, nonce));
+    long newMvcc = initMvcc + 1;
+    for (long newNonce = nonce + 1; newNonce != (nonce + 5); ++newNonce) {
+      assertTrue(nm.startOperation(group, newNonce, createStoppable()));
+      nm.addMvccToOperationContext(group, newNonce, newMvcc);
+      nm.endOperation(group, newNonce, true);
+      assertEquals(newMvcc, nm.getMvccFromOperationContext(group, newNonce));
+      ++newMvcc;
+    }
+    assertEquals(initMvcc, nm.getMvccFromOperationContext(group, nonce));
+  }
+
+  @Test
   public void testNormalStartEnd() throws Exception {
     final long[] numbers = new long[] { NO_NONCE, 1, 2, Long.MAX_VALUE, Long.MIN_VALUE };
     ServerNonceManager nm = createManager();