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

[01/50] [abbrv] hbase git commit: HBASE-18262 name of parameter quote need update

Repository: hbase
Updated Branches:
  refs/heads/HBASE-14070.HLC 82a9cec59 -> 675b63d50


HBASE-18262 name of parameter quote need update

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/5507150a
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/5507150a
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/5507150a

Branch: refs/heads/HBASE-14070.HLC
Commit: 5507150a163f08c966f4cd55607feff8e2570c17
Parents: e4ba404
Author: dongtao.zhang <zh...@zte.com.cn>
Authored: Fri Jun 23 14:52:02 2017 +0800
Committer: Michael Stack <st...@apache.org>
Committed: Thu Aug 10 15:01:37 2017 -0700

----------------------------------------------------------------------
 hbase-common/src/main/resources/hbase-default.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/5507150a/hbase-common/src/main/resources/hbase-default.xml
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/resources/hbase-default.xml b/hbase-common/src/main/resources/hbase-default.xml
index 87a36c1..43efd4b 100644
--- a/hbase-common/src/main/resources/hbase-default.xml
+++ b/hbase-common/src/main/resources/hbase-default.xml
@@ -88,7 +88,7 @@ possible configurations would overwhelm and obscure the important.
     list of ZooKeeper ensemble servers. If HBASE_MANAGES_ZK is set in hbase-env.sh
     this is the list of servers which hbase will start/stop ZooKeeper on as
     part of cluster start/stop.  Client-side, we will take this list of
-    ensemble members and put it together with the hbase.zookeeper.clientPort
+    ensemble members and put it together with the hbase.zookeeper.property.clientPort
     config. and pass it into zookeeper constructor as the connectString
     parameter.</description>
   </property>


[14/50] [abbrv] hbase git commit: HBASE-18555: Remove redundant familyMap.put() from addxxx() of sub-classes of Mutation and Query

Posted by st...@apache.org.
HBASE-18555: Remove redundant familyMap.put() from addxxx() of sub-classes of Mutation and Query

Signed-off-by: Jerry He <je...@apache.org>


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

Branch: refs/heads/HBASE-14070.HLC
Commit: 173dce73471da005fb6780a7e7b65b43bad481e2
Parents: 95e8839
Author: Xiang Li <wa...@gmail.com>
Authored: Fri Aug 11 00:07:11 2017 +0800
Committer: Jerry He <je...@apache.org>
Committed: Fri Aug 11 22:49:38 2017 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/hbase/client/Append.java  |  2 +-
 .../main/java/org/apache/hadoop/hbase/client/Delete.java  | 10 +++++-----
 .../src/main/java/org/apache/hadoop/hbase/client/Get.java |  2 +-
 .../java/org/apache/hadoop/hbase/client/Increment.java    |  2 --
 .../java/org/apache/hadoop/hbase/client/Mutation.java     |  1 +
 .../src/main/java/org/apache/hadoop/hbase/client/Put.java |  7 -------
 .../main/java/org/apache/hadoop/hbase/client/Scan.java    |  2 +-
 7 files changed, 9 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/173dce73/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Append.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Append.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Append.java
index 02ec770..2bd0860 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Append.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Append.java
@@ -137,10 +137,10 @@ public class Append extends Mutation {
     List<Cell> list = this.familyMap.get(family);
     if (list == null) {
       list  = new ArrayList<>(1);
+      this.familyMap.put(family, list);
     }
     // find where the new entry should be placed in the List
     list.add(cell);
-    this.familyMap.put(family, list);
     return this;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/173dce73/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java
index 395c277..bf5241c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java
@@ -183,9 +183,9 @@ public class Delete extends Mutation implements Comparable<Row> {
     List<Cell> list = familyMap.get(family);
     if (list == null) {
       list = new ArrayList<>(1);
+      familyMap.put(family, list);
     }
     list.add(kv);
-    familyMap.put(family, list);
     return this;
   }
 
@@ -219,12 +219,12 @@ public class Delete extends Mutation implements Comparable<Row> {
     List<Cell> list = familyMap.get(family);
     if(list == null) {
       list = new ArrayList<>(1);
+      familyMap.put(family, list);
     } else if(!list.isEmpty()) {
       list.clear();
     }
     KeyValue kv = new KeyValue(row, family, null, timestamp, KeyValue.Type.DeleteFamily);
     list.add(kv);
-    familyMap.put(family, list);
     return this;
   }
 
@@ -239,10 +239,10 @@ public class Delete extends Mutation implements Comparable<Row> {
     List<Cell> list = familyMap.get(family);
     if(list == null) {
       list = new ArrayList<>(1);
+      familyMap.put(family, list);
     }
     list.add(new KeyValue(row, family, null, timestamp,
           KeyValue.Type.DeleteFamilyVersion));
-    familyMap.put(family, list);
     return this;
   }
 
@@ -272,10 +272,10 @@ public class Delete extends Mutation implements Comparable<Row> {
     List<Cell> list = familyMap.get(family);
     if (list == null) {
       list = new ArrayList<>(1);
+      familyMap.put(family, list);
     }
     list.add(new KeyValue(this.row, family, qualifier, timestamp,
         KeyValue.Type.DeleteColumn));
-    familyMap.put(family, list);
     return this;
   }
 
@@ -307,10 +307,10 @@ public class Delete extends Mutation implements Comparable<Row> {
     List<Cell> list = familyMap.get(family);
     if(list == null) {
       list = new ArrayList<>(1);
+      familyMap.put(family, list);
     }
     KeyValue kv = new KeyValue(this.row, family, qualifier, timestamp, KeyValue.Type.Delete);
     list.add(kv);
-    familyMap.put(family, list);
     return this;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/173dce73/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Get.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Get.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Get.java
index 086a0b4..226ca85 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Get.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Get.java
@@ -205,12 +205,12 @@ public class Get extends Query
     NavigableSet<byte []> set = familyMap.get(family);
     if(set == null) {
       set = new TreeSet<>(Bytes.BYTES_COMPARATOR);
+      familyMap.put(family, set);
     }
     if (qualifier == null) {
       qualifier = HConstants.EMPTY_BYTE_ARRAY;
     }
     set.add(qualifier);
-    familyMap.put(family, set);
     return this;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/173dce73/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java
index d323555..c144c87 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java
@@ -102,7 +102,6 @@ public class Increment extends Mutation implements Comparable<Row> {
         " doesn't match the original one " +  Bytes.toStringBinary(this.row));
     }
     list.add(cell);
-    familyMap.put(family, list);
     return this;
   }
 
@@ -123,7 +122,6 @@ public class Increment extends Mutation implements Comparable<Row> {
     List<Cell> list = getCellList(family);
     KeyValue kv = createPutKeyValue(family, qualifier, ts, Bytes.toBytes(amount));
     list.add(kv);
-    familyMap.put(CellUtil.cloneFamily(kv), list);
     return this;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/173dce73/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
index 3b60497..25b088d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
@@ -111,6 +111,7 @@ public abstract class Mutation extends OperationWithAttributes implements Row, C
     List<Cell> list = this.familyMap.get(family);
     if (list == null) {
       list = new ArrayList<>();
+      this.familyMap.put(family, list);
     }
     return list;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/173dce73/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java
index 5c1528f..b5b10d0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java
@@ -216,7 +216,6 @@ public class Put extends Mutation implements HeapSize, Comparable<Row> {
     List<Cell> list = getCellList(family);
     KeyValue kv = createPutKeyValue(family, qualifier, ts, value);
     list.add(kv);
-    familyMap.put(CellUtil.cloneFamily(kv), list);
     return this;
   }
 
@@ -238,7 +237,6 @@ public class Put extends Mutation implements HeapSize, Comparable<Row> {
 
     List<Cell> list = getCellList(family);
     list.add(new IndividualBytesFieldCell(this.row, family, qualifier, ts, KeyValue.Type.Put, value));
-    familyMap.put(family, list);
     return this;
   }
 
@@ -253,7 +251,6 @@ public class Put extends Mutation implements HeapSize, Comparable<Row> {
     List<Cell> list = getCellList(family);
     KeyValue kv = createPutKeyValue(family, qualifier, ts, value, tag);
     list.add(kv);
-    familyMap.put(family, list);
     return this;
   }
 
@@ -272,7 +269,6 @@ public class Put extends Mutation implements HeapSize, Comparable<Row> {
     List<Cell> list = getCellList(family);
     KeyValue kv = createPutKeyValue(family, qualifier, ts, value, tag);
     list.add(kv);
-    familyMap.put(family, list);
     return this;
   }
 
@@ -293,7 +289,6 @@ public class Put extends Mutation implements HeapSize, Comparable<Row> {
     List<Cell> list = getCellList(family);
     KeyValue kv = createPutKeyValue(family, qualifier, ts, value, null);
     list.add(kv);
-    familyMap.put(CellUtil.cloneFamily(kv), list);
     return this;
   }
 
@@ -309,7 +304,6 @@ public class Put extends Mutation implements HeapSize, Comparable<Row> {
     List<Cell> list = getCellList(family);
     KeyValue kv = createPutKeyValue(family, qualifier, ts, value, null);
     list.add(kv);
-    familyMap.put(family, list);
     return this;
   }
 
@@ -330,7 +324,6 @@ public class Put extends Mutation implements HeapSize, Comparable<Row> {
         " doesn't match the original one " +  Bytes.toStringBinary(this.row));
     }
     list.add(kv);
-    familyMap.put(family, list);
     return this;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/173dce73/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 5b75151..9100b45 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
@@ -340,12 +340,12 @@ public class Scan extends Query {
     NavigableSet<byte []> set = familyMap.get(family);
     if(set == null) {
       set = new TreeSet<>(Bytes.BYTES_COMPARATOR);
+      familyMap.put(family, set);
     }
     if (qualifier == null) {
       qualifier = HConstants.EMPTY_BYTE_ARRAY;
     }
     set.add(qualifier);
-    familyMap.put(family, set);
     return this;
   }
 


[18/50] [abbrv] hbase git commit: HBASE-14135 Merge backup images (Vladimir Rodionov)

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/05e6e569/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/HFileSplitterJob.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/HFileSplitterJob.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/HFileSplitterJob.java
deleted file mode 100644
index ba1b65e..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/HFileSplitterJob.java
+++ /dev/null
@@ -1,181 +0,0 @@
-/**
- * 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.backup.mapreduce;
-
-import java.io.IOException;
-
-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.Path;
-import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValue.Type;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.RegionLocator;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.mapreduce.HFileInputFormat;
-import org.apache.hadoop.hbase.mapreduce.HFileOutputFormat2;
-import org.apache.hadoop.hbase.mapreduce.KeyValueSortReducer;
-import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-
-/**
- * A tool to split HFiles into new region boundaries as a MapReduce job. The tool generates HFiles
- * for later bulk importing.
- */
-@InterfaceAudience.Private
-public class HFileSplitterJob extends Configured implements Tool {
-  private static final Log LOG = LogFactory.getLog(HFileSplitterJob.class);
-  final static String NAME = "HFileSplitterJob";
-  public final static String BULK_OUTPUT_CONF_KEY = "hfile.bulk.output";
-  public final static String TABLES_KEY = "hfile.input.tables";
-  public final static String TABLE_MAP_KEY = "hfile.input.tablesmap";
-  private final static String JOB_NAME_CONF_KEY = "mapreduce.job.name";
-
-  public HFileSplitterJob() {
-  }
-
-  protected HFileSplitterJob(final Configuration c) {
-    super(c);
-  }
-
-  /**
-   * A mapper that just writes out cells. This one can be used together with
-   * {@link KeyValueSortReducer}
-   */
-  static class HFileCellMapper extends
-      Mapper<NullWritable, KeyValue, ImmutableBytesWritable, KeyValue> {
-
-    @Override
-    public void map(NullWritable key, KeyValue value, Context context) throws IOException,
-        InterruptedException {
-      // Convert value to KeyValue if subclass
-      if (!value.getClass().equals(KeyValue.class)) {
-        value =
-            new KeyValue(value.getRowArray(), value.getRowOffset(), value.getRowLength(),
-                value.getFamilyArray(), value.getFamilyOffset(), value.getFamilyLength(),
-                value.getQualifierArray(), value.getQualifierOffset(), value.getQualifierLength(),
-                value.getTimestamp(), Type.codeToType(value.getTypeByte()), value.getValueArray(),
-                value.getValueOffset(), value.getValueLength());
-      }
-      context.write(new ImmutableBytesWritable(CellUtil.cloneRow(value)), value);
-    }
-
-    @Override
-    public void setup(Context context) throws IOException {
-      // do nothing
-    }
-  }
-
-  /**
-   * Sets up the actual job.
-   * @param args The command line parameters.
-   * @return The newly created job.
-   * @throws IOException When setting up the job fails.
-   */
-  public Job createSubmittableJob(String[] args) throws IOException {
-    Configuration conf = getConf();
-    String inputDirs = args[0];
-    String tabName = args[1];
-    conf.setStrings(TABLES_KEY, tabName);
-    conf.set(FileInputFormat.INPUT_DIR, inputDirs);
-    Job job =
-        Job.getInstance(conf,
-          conf.get(JOB_NAME_CONF_KEY, NAME + "_" + EnvironmentEdgeManager.currentTime()));
-    job.setJarByClass(HFileSplitterJob.class);
-    job.setInputFormatClass(HFileInputFormat.class);
-    job.setMapOutputKeyClass(ImmutableBytesWritable.class);
-    String hfileOutPath = conf.get(BULK_OUTPUT_CONF_KEY);
-    if (hfileOutPath != null) {
-      LOG.debug("add incremental job :" + hfileOutPath + " from " + inputDirs);
-      TableName tableName = TableName.valueOf(tabName);
-      job.setMapperClass(HFileCellMapper.class);
-      job.setReducerClass(KeyValueSortReducer.class);
-      Path outputDir = new Path(hfileOutPath);
-      FileOutputFormat.setOutputPath(job, outputDir);
-      job.setMapOutputValueClass(KeyValue.class);
-      try (Connection conn = ConnectionFactory.createConnection(conf);
-          Table table = conn.getTable(tableName);
-          RegionLocator regionLocator = conn.getRegionLocator(tableName)) {
-        HFileOutputFormat2.configureIncrementalLoad(job, table.getDescriptor(), regionLocator);
-      }
-      LOG.debug("success configuring load incremental job");
-
-      TableMapReduceUtil.addDependencyJars(job.getConfiguration(),
-        org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions.class);
-    } else {
-      throw new IOException("No bulk output directory specified");
-    }
-    return job;
-  }
-
-  /**
-   * Print usage
-   * @param errorMsg Error message. Can be null.
-   */
-  private void usage(final String errorMsg) {
-    if (errorMsg != null && errorMsg.length() > 0) {
-      System.err.println("ERROR: " + errorMsg);
-    }
-    System.err.println("Usage: " + NAME + " [options] <HFile inputdir(s)> <table>");
-    System.err.println("Read all HFile's for <table> and split them to <table> region boundaries.");
-    System.err.println("<table>  table to load.\n");
-    System.err.println("To generate HFiles for a bulk data load, pass the option:");
-    System.err.println("  -D" + BULK_OUTPUT_CONF_KEY + "=/path/for/output");
-    System.err.println("Other options:");
-    System.err.println("   -D " + JOB_NAME_CONF_KEY
-        + "=jobName - use the specified mapreduce job name for the HFile splitter");
-    System.err.println("For performance also consider the following options:\n"
-        + "  -Dmapreduce.map.speculative=false\n" + "  -Dmapreduce.reduce.speculative=false");
-  }
-
-  /**
-   * Main entry point.
-   * @param args The command line parameters.
-   * @throws Exception When running the job fails.
-   */
-  public static void main(String[] args) throws Exception {
-    int ret = ToolRunner.run(new HFileSplitterJob(HBaseConfiguration.create()), args);
-    System.exit(ret);
-  }
-
-  @Override
-  public int run(String[] args) throws Exception {
-    if (args.length < 2) {
-      usage("Wrong number of arguments: " + args.length);
-      System.exit(-1);
-    }
-    Job job = createSubmittableJob(args);
-    int result = job.waitForCompletion(true) ? 0 : 1;
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/05e6e569/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupMergeJob.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupMergeJob.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupMergeJob.java
new file mode 100644
index 0000000..00c5b83
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupMergeJob.java
@@ -0,0 +1,321 @@
+/**
+ * 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.backup.mapreduce;
+
+import static org.apache.hadoop.hbase.backup.util.BackupUtils.succeeded;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupInfo;
+import org.apache.hadoop.hbase.backup.BackupMergeJob;
+import org.apache.hadoop.hbase.backup.BackupRestoreConstants;
+import org.apache.hadoop.hbase.backup.HBackupFileSystem;
+import org.apache.hadoop.hbase.backup.impl.BackupManifest;
+import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
+import org.apache.hadoop.hbase.backup.util.BackupUtils;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.util.Tool;
+
+/**
+ * MapReduce implementation of {@link BackupMergeJob}
+ * Must be initialized with configuration of a backup destination cluster
+ *
+ */
+
+@InterfaceAudience.Private
+public class MapReduceBackupMergeJob implements BackupMergeJob {
+  public static final Log LOG = LogFactory.getLog(MapReduceBackupMergeJob.class);
+
+  protected Tool player;
+  protected Configuration conf;
+
+  public MapReduceBackupMergeJob() {
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  public void run(String[] backupIds) throws IOException {
+    String bulkOutputConfKey;
+
+    // TODO : run player on remote cluster
+    player = new MapReduceHFileSplitterJob();
+    bulkOutputConfKey = MapReduceHFileSplitterJob.BULK_OUTPUT_CONF_KEY;
+    // Player reads all files in arbitrary directory structure and creates
+    // a Map task for each file
+    String bids = StringUtils.join(backupIds, ",");
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Merge backup images " + bids);
+    }
+
+    List<Pair<TableName, Path>> processedTableList = new ArrayList<Pair<TableName, Path>>();
+    boolean finishedTables = false;
+    Connection conn = ConnectionFactory.createConnection(getConf());
+    BackupSystemTable table = new BackupSystemTable(conn);
+    FileSystem fs = FileSystem.get(getConf());
+
+    try {
+
+      // Get exclusive lock on backup system
+      table.startBackupExclusiveOperation();
+      // Start merge operation
+      table.startMergeOperation(backupIds);
+
+      // Select most recent backup id
+      String mergedBackupId = findMostRecentBackupId(backupIds);
+
+      TableName[] tableNames = getTableNamesInBackupImages(backupIds);
+      String backupRoot = null;
+
+      BackupInfo bInfo = table.readBackupInfo(backupIds[0]);
+      backupRoot = bInfo.getBackupRootDir();
+
+      for (int i = 0; i < tableNames.length; i++) {
+
+        LOG.info("Merge backup images for " + tableNames[i]);
+
+        // Find input directories for table
+
+        Path[] dirPaths = findInputDirectories(fs, backupRoot, tableNames[i], backupIds);
+        String dirs = StringUtils.join(dirPaths, ",");
+        Path bulkOutputPath =
+            BackupUtils.getBulkOutputDir(BackupUtils.getFileNameCompatibleString(tableNames[i]),
+              getConf(), false);
+        // Delete content if exists
+        if (fs.exists(bulkOutputPath)) {
+          if (!fs.delete(bulkOutputPath, true)) {
+            LOG.warn("Can not delete: " + bulkOutputPath);
+          }
+        }
+        Configuration conf = getConf();
+        conf.set(bulkOutputConfKey, bulkOutputPath.toString());
+        String[] playerArgs = { dirs, tableNames[i].getNameAsString() };
+
+        int result = 0;
+
+        player.setConf(getConf());
+        result = player.run(playerArgs);
+        if (!succeeded(result)) {
+          throw new IOException("Can not merge backup images for " + dirs
+              + " (check Hadoop/MR and HBase logs). Player return code =" + result);
+        }
+        // Add to processed table list
+        processedTableList.add(new Pair<TableName, Path>(tableNames[i], bulkOutputPath));
+        LOG.debug("Merge Job finished:" + result);
+      }
+      List<TableName> tableList = toTableNameList(processedTableList);
+      table.updateProcessedTablesForMerge(tableList);
+      finishedTables = true;
+
+      // Move data
+      for (Pair<TableName, Path> tn : processedTableList) {
+        moveData(fs, backupRoot, tn.getSecond(), tn.getFirst(), mergedBackupId);
+      }
+
+      // Delete old data and update manifest
+      List<String> backupsToDelete = getBackupIdsToDelete(backupIds, mergedBackupId);
+      deleteBackupImages(backupsToDelete, conn, fs, backupRoot);
+      updateBackupManifest(backupRoot, mergedBackupId, backupsToDelete);
+      // Finish merge session
+      table.finishMergeOperation();
+      // Release lock
+      table.finishBackupExclusiveOperation();
+    } catch (RuntimeException e) {
+
+      throw e;
+    } catch (Exception e) {
+      LOG.error(e);
+      if (!finishedTables) {
+        // cleanup bulk directories and finish merge
+        // merge MUST be repeated (no need for repair)
+        cleanupBulkLoadDirs(fs, toPathList(processedTableList));
+        table.finishMergeOperation();
+        table.finishBackupExclusiveOperation();
+        throw new IOException("Backup merge operation failed, you should try it again", e);
+      } else {
+        // backup repair must be run
+        throw new IOException(
+            "Backup merge operation failed, run backup repair tool to restore system's integrity",
+            e);
+      }
+    } finally {
+      table.close();
+      conn.close();
+    }
+  }
+
+  protected List<Path> toPathList(List<Pair<TableName, Path>> processedTableList) {
+    ArrayList<Path> list = new ArrayList<Path>();
+    for (Pair<TableName, Path> p : processedTableList) {
+      list.add(p.getSecond());
+    }
+    return list;
+  }
+
+  protected List<TableName> toTableNameList(List<Pair<TableName, Path>> processedTableList) {
+    ArrayList<TableName> list = new ArrayList<TableName>();
+    for (Pair<TableName, Path> p : processedTableList) {
+      list.add(p.getFirst());
+    }
+    return list;
+  }
+
+  protected void cleanupBulkLoadDirs(FileSystem fs, List<Path> pathList) throws IOException {
+    for (Path path : pathList) {
+
+      if (!fs.delete(path, true)) {
+        LOG.warn("Can't delete " + path);
+      }
+    }
+  }
+
+  protected void updateBackupManifest(String backupRoot, String mergedBackupId,
+      List<String> backupsToDelete) throws IllegalArgumentException, IOException {
+
+    BackupManifest manifest =
+        HBackupFileSystem.getManifest(conf, new Path(backupRoot), mergedBackupId);
+    manifest.getBackupImage().removeAncestors(backupsToDelete);
+    // save back
+    manifest.store(conf);
+
+  }
+
+  protected void deleteBackupImages(List<String> backupIds, Connection conn, FileSystem fs,
+      String backupRoot) throws IOException {
+
+    // Delete from backup system table
+    try (BackupSystemTable table = new BackupSystemTable(conn);) {
+      for (String backupId : backupIds) {
+        table.deleteBackupInfo(backupId);
+      }
+    }
+
+    // Delete from file system
+    for (String backupId : backupIds) {
+      Path backupDirPath = HBackupFileSystem.getBackupPath(backupRoot, backupId);
+
+      if (!fs.delete(backupDirPath, true)) {
+        LOG.warn("Could not delete " + backupDirPath);
+      }
+    }
+  }
+
+  protected List<String> getBackupIdsToDelete(String[] backupIds, String mergedBackupId) {
+    List<String> list = new ArrayList<String>();
+    for (String id : backupIds) {
+      if (id.equals(mergedBackupId)) {
+        continue;
+      }
+      list.add(id);
+    }
+    return list;
+  }
+
+  protected void moveData(FileSystem fs, String backupRoot, Path bulkOutputPath, TableName tableName,
+      String mergedBackupId) throws IllegalArgumentException, IOException {
+
+    Path dest =
+        new Path(HBackupFileSystem.getTableBackupDataDir(backupRoot, mergedBackupId, tableName));
+
+    // Delete all in dest
+    if (!fs.delete(dest, true)) {
+      throw new IOException("Could not delete " + dest);
+    }
+
+    FileStatus[] fsts = fs.listStatus(bulkOutputPath);
+    for (FileStatus fst : fsts) {
+      if (fst.isDirectory()) {
+        fs.rename(fst.getPath().getParent(), dest);
+      }
+    }
+
+  }
+
+  protected String findMostRecentBackupId(String[] backupIds) {
+    long recentTimestamp = Long.MIN_VALUE;
+    for (String backupId : backupIds) {
+      long ts = Long.parseLong(backupId.split("_")[1]);
+      if (ts > recentTimestamp) {
+        recentTimestamp = ts;
+      }
+    }
+    return BackupRestoreConstants.BACKUPID_PREFIX + recentTimestamp;
+  }
+
+  protected TableName[] getTableNamesInBackupImages(String[] backupIds) throws IOException {
+
+    Set<TableName> allSet = new HashSet<TableName>();
+
+    try (Connection conn = ConnectionFactory.createConnection(conf);
+        BackupSystemTable table = new BackupSystemTable(conn);) {
+      for (String backupId : backupIds) {
+        BackupInfo bInfo = table.readBackupInfo(backupId);
+
+        allSet.addAll(bInfo.getTableNames());
+      }
+    }
+
+    TableName[] ret = new TableName[allSet.size()];
+    return allSet.toArray(ret);
+  }
+
+  protected Path[] findInputDirectories(FileSystem fs, String backupRoot, TableName tableName,
+      String[] backupIds) throws IOException {
+
+    List<Path> dirs = new ArrayList<Path>();
+
+    for (String backupId : backupIds) {
+      Path fileBackupDirPath =
+          new Path(HBackupFileSystem.getTableBackupDataDir(backupRoot, backupId, tableName));
+      if (fs.exists(fileBackupDirPath)) {
+        dirs.add(fileBackupDirPath);
+      } else {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("File: " + fileBackupDirPath + " does not exist.");
+        }
+      }
+    }
+    Path[] ret = new Path[dirs.size()];
+    return dirs.toArray(ret);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/05e6e569/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceHFileSplitterJob.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceHFileSplitterJob.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceHFileSplitterJob.java
new file mode 100644
index 0000000..49e8c75
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceHFileSplitterJob.java
@@ -0,0 +1,181 @@
+/**
+ * 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.backup.mapreduce;
+
+import java.io.IOException;
+
+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.Path;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValue.Type;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.mapreduce.HFileInputFormat;
+import org.apache.hadoop.hbase.mapreduce.HFileOutputFormat2;
+import org.apache.hadoop.hbase.mapreduce.KeyValueSortReducer;
+import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+/**
+ * A tool to split HFiles into new region boundaries as a MapReduce job. The tool generates HFiles
+ * for later bulk importing.
+ */
+@InterfaceAudience.Private
+public class MapReduceHFileSplitterJob extends Configured implements Tool {
+  private static final Log LOG = LogFactory.getLog(MapReduceHFileSplitterJob.class);
+  final static String NAME = "HFileSplitterJob";
+  public final static String BULK_OUTPUT_CONF_KEY = "hfile.bulk.output";
+  public final static String TABLES_KEY = "hfile.input.tables";
+  public final static String TABLE_MAP_KEY = "hfile.input.tablesmap";
+  private final static String JOB_NAME_CONF_KEY = "mapreduce.job.name";
+
+  public MapReduceHFileSplitterJob() {
+  }
+
+  protected MapReduceHFileSplitterJob(final Configuration c) {
+    super(c);
+  }
+
+  /**
+   * A mapper that just writes out cells. This one can be used together with
+   * {@link KeyValueSortReducer}
+   */
+  static class HFileCellMapper extends
+      Mapper<NullWritable, KeyValue, ImmutableBytesWritable, KeyValue> {
+
+    @Override
+    public void map(NullWritable key, KeyValue value, Context context) throws IOException,
+        InterruptedException {
+      // Convert value to KeyValue if subclass
+      if (!value.getClass().equals(KeyValue.class)) {
+        value =
+            new KeyValue(value.getRowArray(), value.getRowOffset(), value.getRowLength(),
+                value.getFamilyArray(), value.getFamilyOffset(), value.getFamilyLength(),
+                value.getQualifierArray(), value.getQualifierOffset(), value.getQualifierLength(),
+                value.getTimestamp(), Type.codeToType(value.getTypeByte()), value.getValueArray(),
+                value.getValueOffset(), value.getValueLength());
+      }
+      context.write(new ImmutableBytesWritable(CellUtil.cloneRow(value)), value);
+    }
+
+    @Override
+    public void setup(Context context) throws IOException {
+      // do nothing
+    }
+  }
+
+  /**
+   * Sets up the actual job.
+   * @param args The command line parameters.
+   * @return The newly created job.
+   * @throws IOException When setting up the job fails.
+   */
+  public Job createSubmittableJob(String[] args) throws IOException {
+    Configuration conf = getConf();
+    String inputDirs = args[0];
+    String tabName = args[1];
+    conf.setStrings(TABLES_KEY, tabName);
+    conf.set(FileInputFormat.INPUT_DIR, inputDirs);
+    Job job =
+        Job.getInstance(conf,
+          conf.get(JOB_NAME_CONF_KEY, NAME + "_" + EnvironmentEdgeManager.currentTime()));
+    job.setJarByClass(MapReduceHFileSplitterJob.class);
+    job.setInputFormatClass(HFileInputFormat.class);
+    job.setMapOutputKeyClass(ImmutableBytesWritable.class);
+    String hfileOutPath = conf.get(BULK_OUTPUT_CONF_KEY);
+    if (hfileOutPath != null) {
+      LOG.debug("add incremental job :" + hfileOutPath + " from " + inputDirs);
+      TableName tableName = TableName.valueOf(tabName);
+      job.setMapperClass(HFileCellMapper.class);
+      job.setReducerClass(KeyValueSortReducer.class);
+      Path outputDir = new Path(hfileOutPath);
+      FileOutputFormat.setOutputPath(job, outputDir);
+      job.setMapOutputValueClass(KeyValue.class);
+      try (Connection conn = ConnectionFactory.createConnection(conf);
+          Table table = conn.getTable(tableName);
+          RegionLocator regionLocator = conn.getRegionLocator(tableName)) {
+        HFileOutputFormat2.configureIncrementalLoad(job, table.getDescriptor(), regionLocator);
+      }
+      LOG.debug("success configuring load incremental job");
+
+      TableMapReduceUtil.addDependencyJars(job.getConfiguration(),
+        org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions.class);
+    } else {
+      throw new IOException("No bulk output directory specified");
+    }
+    return job;
+  }
+
+  /**
+   * Print usage
+   * @param errorMsg Error message. Can be null.
+   */
+  private void usage(final String errorMsg) {
+    if (errorMsg != null && errorMsg.length() > 0) {
+      System.err.println("ERROR: " + errorMsg);
+    }
+    System.err.println("Usage: " + NAME + " [options] <HFile inputdir(s)> <table>");
+    System.err.println("Read all HFile's for <table> and split them to <table> region boundaries.");
+    System.err.println("<table>  table to load.\n");
+    System.err.println("To generate HFiles for a bulk data load, pass the option:");
+    System.err.println("  -D" + BULK_OUTPUT_CONF_KEY + "=/path/for/output");
+    System.err.println("Other options:");
+    System.err.println("   -D " + JOB_NAME_CONF_KEY
+        + "=jobName - use the specified mapreduce job name for the HFile splitter");
+    System.err.println("For performance also consider the following options:\n"
+        + "  -Dmapreduce.map.speculative=false\n" + "  -Dmapreduce.reduce.speculative=false");
+  }
+
+  /**
+   * Main entry point.
+   * @param args The command line parameters.
+   * @throws Exception When running the job fails.
+   */
+  public static void main(String[] args) throws Exception {
+    int ret = ToolRunner.run(new MapReduceHFileSplitterJob(HBaseConfiguration.create()), args);
+    System.exit(ret);
+  }
+
+  @Override
+  public int run(String[] args) throws Exception {
+    if (args.length < 2) {
+      usage("Wrong number of arguments: " + args.length);
+      System.exit(-1);
+    }
+    Job job = createSubmittableJob(args);
+    int result = job.waitForCompletion(true) ? 0 : 1;
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/05e6e569/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreJob.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreJob.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreJob.java
index 4161ca9..1209e7c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreJob.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreJob.java
@@ -17,31 +17,31 @@
  */
 package org.apache.hadoop.hbase.backup.mapreduce;
 
+import static org.apache.hadoop.hbase.backup.util.BackupUtils.failed;
+import static org.apache.hadoop.hbase.backup.util.BackupUtils.succeeded;
+
 import java.io.IOException;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.backup.BackupRestoreConstants;
 import org.apache.hadoop.hbase.backup.RestoreJob;
+import org.apache.hadoop.hbase.backup.util.BackupUtils;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
-import org.apache.hadoop.hbase.mapreduce.WALPlayer;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.util.Tool;
 
+
 /**
  * MapReduce implementation of {@link RestoreJob}
  *
- * For full backup restore, it runs {@link HFileSplitterJob} job and creates
+ * For backup restore, it runs {@link MapReduceHFileSplitterJob} job and creates
  * HFiles which are aligned with a region boundaries of a table being
- * restored, for incremental backup restore it runs {@link WALPlayer} in
- * bulk load mode (creates HFiles from WAL edits).
+ * restored.
  *
  * The resulting HFiles then are loaded using HBase bulk load tool
  * {@link LoadIncrementalHFiles}
@@ -62,8 +62,8 @@ public class MapReduceRestoreJob implements RestoreJob {
 
     String bulkOutputConfKey;
 
-    player = new HFileSplitterJob();
-    bulkOutputConfKey = HFileSplitterJob.BULK_OUTPUT_CONF_KEY;
+    player = new MapReduceHFileSplitterJob();
+    bulkOutputConfKey = MapReduceHFileSplitterJob.BULK_OUTPUT_CONF_KEY;
     // Player reads all files in arbitrary directory structure and creates
     // a Map task for each file
     String dirs = StringUtils.join(dirPaths, ",");
@@ -71,8 +71,8 @@ public class MapReduceRestoreJob implements RestoreJob {
     if (LOG.isDebugEnabled()) {
       LOG.debug("Restore " + (fullBackupRestore ? "full" : "incremental")
           + " backup from directory " + dirs + " from hbase tables "
-          + StringUtils.join(tableNames, BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND) +
-          " to tables "
+          + StringUtils.join(tableNames, BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND)
+          + " to tables "
           + StringUtils.join(newTableNames, BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND));
     }
 
@@ -80,13 +80,16 @@ public class MapReduceRestoreJob implements RestoreJob {
 
       LOG.info("Restore " + tableNames[i] + " into " + newTableNames[i]);
 
-      Path bulkOutputPath = getBulkOutputDir(getFileNameCompatibleString(newTableNames[i]));
+      Path bulkOutputPath =
+          BackupUtils.getBulkOutputDir(BackupUtils.getFileNameCompatibleString(newTableNames[i]),
+            getConf());
       Configuration conf = getConf();
       conf.set(bulkOutputConfKey, bulkOutputPath.toString());
       String[] playerArgs =
-        { dirs,
-          fullBackupRestore? newTableNames[i].getNameAsString():tableNames[i].getNameAsString()
-        };
+          {
+              dirs,
+              fullBackupRestore ? newTableNames[i].getNameAsString() : tableNames[i]
+                  .getNameAsString() };
 
       int result = 0;
       int loaderResult = 0;
@@ -96,7 +99,7 @@ public class MapReduceRestoreJob implements RestoreJob {
         result = player.run(playerArgs);
         if (succeeded(result)) {
           // do bulk load
-          LoadIncrementalHFiles loader = createLoader(getConf());
+          LoadIncrementalHFiles loader = BackupUtils.createLoader(getConf());
           if (LOG.isDebugEnabled()) {
             LOG.debug("Restoring HFiles from directory " + bulkOutputPath);
           }
@@ -113,58 +116,11 @@ public class MapReduceRestoreJob implements RestoreJob {
         }
         LOG.debug("Restore Job finished:" + result);
       } catch (Exception e) {
+        LOG.error(e);
         throw new IOException("Can not restore from backup directory " + dirs
             + " (check Hadoop and HBase logs) ", e);
       }
-
-    }
-  }
-
-  private String getFileNameCompatibleString(TableName table) {
-    return table.getNamespaceAsString() + "-" + table.getQualifierAsString();
-  }
-
-  private boolean failed(int result) {
-    return result != 0;
-  }
-
-  private boolean succeeded(int result) {
-    return result == 0;
-  }
-
-  public static LoadIncrementalHFiles createLoader(Configuration config) throws IOException {
-    // set configuration for restore:
-    // LoadIncrementalHFile needs more time
-    // <name>hbase.rpc.timeout</name> <value>600000</value>
-    // calculates
-    Integer milliSecInHour = 3600000;
-    Configuration conf = new Configuration(config);
-    conf.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, milliSecInHour);
-
-    // By default, it is 32 and loader will fail if # of files in any region exceed this
-    // limit. Bad for snapshot restore.
-    conf.setInt(LoadIncrementalHFiles.MAX_FILES_PER_REGION_PER_FAMILY, Integer.MAX_VALUE);
-    conf.set(LoadIncrementalHFiles.IGNORE_UNMATCHED_CF_CONF_KEY, "yes");
-    LoadIncrementalHFiles loader = null;
-    try {
-      loader = new LoadIncrementalHFiles(conf);
-    } catch (Exception e) {
-      throw new IOException(e);
     }
-    return loader;
-  }
-
-  private Path getBulkOutputDir(String tableName) throws IOException {
-    Configuration conf = getConf();
-    FileSystem fs = FileSystem.get(conf);
-    String tmp =
-        conf.get(HConstants.TEMPORARY_FS_DIRECTORY_KEY,
-          HConstants.DEFAULT_TEMPORARY_HDFS_DIRECTORY);
-    Path path =
-        new Path(tmp + Path.SEPARATOR + "bulk_output-" + tableName + "-"
-            + EnvironmentEdgeManager.currentTime());
-    fs.deleteOnExit(path);
-    return path;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/05e6e569/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java
index e32853d..ce77645 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java
@@ -56,7 +56,9 @@ import org.apache.hadoop.hbase.backup.impl.BackupManifest;
 import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
 import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
@@ -68,14 +70,15 @@ import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 public final class BackupUtils {
   protected static final Log LOG = LogFactory.getLog(BackupUtils.class);
   public static final String LOGNAME_SEPARATOR = ".";
+  public static final int MILLISEC_IN_HOUR = 3600000;
 
   private BackupUtils() {
     throw new AssertionError("Instantiating utility class...");
   }
 
   /**
-   * Loop through the RS log timestamp map for the tables, for each RS, find the min timestamp
-   * value for the RS among the tables.
+   * Loop through the RS log timestamp map for the tables, for each RS, find the min timestamp value
+   * for the RS among the tables.
    * @param rsLogTimestampMap timestamp map
    * @return the min timestamp of each RS
    */
@@ -114,16 +117,17 @@ public final class BackupUtils {
   }
 
   /**
-   * copy out Table RegionInfo into incremental backup image need to consider move this
-   * logic into HBackupFileSystem
+   * copy out Table RegionInfo into incremental backup image need to consider move this logic into
+   * HBackupFileSystem
    * @param conn connection
    * @param backupInfo backup info
    * @param conf configuration
    * @throws IOException exception
    * @throws InterruptedException exception
    */
-  public static void copyTableRegionInfo(Connection conn, BackupInfo backupInfo,
-      Configuration conf) throws IOException, InterruptedException {
+  public static void
+      copyTableRegionInfo(Connection conn, BackupInfo backupInfo, Configuration conf)
+          throws IOException, InterruptedException {
     Path rootDir = FSUtils.getRootDir(conf);
     FileSystem fs = rootDir.getFileSystem(conf);
 
@@ -152,10 +156,8 @@ public final class BackupUtils {
       LOG.debug("Starting to write region info for table " + table);
       for (HRegionInfo regionInfo : regions) {
         Path regionDir =
-            HRegion.getRegionDir(new Path(backupInfo.getTableBackupDir(table)),
-              regionInfo);
-        regionDir =
-            new Path(backupInfo.getTableBackupDir(table), regionDir.getName());
+            HRegion.getRegionDir(new Path(backupInfo.getTableBackupDir(table)), regionInfo);
+        regionDir = new Path(backupInfo.getTableBackupDir(table), regionDir.getName());
         writeRegioninfoOnFilesystem(conf, targetFs, regionDir, regionInfo);
       }
       LOG.debug("Finished writing region info for table " + table);
@@ -301,7 +303,6 @@ public final class BackupUtils {
     return ret;
   }
 
-
   /**
    * Check whether the backup path exist
    * @param backupStr backup
@@ -431,8 +432,7 @@ public final class BackupUtils {
    * @param conf configuration
    * @throws IOException exception
    */
-  private static void cleanupHLogDir(BackupInfo backupInfo, Configuration conf)
-      throws IOException {
+  private static void cleanupHLogDir(BackupInfo backupInfo, Configuration conf) throws IOException {
 
     String logDir = backupInfo.getHLogTargetDir();
     if (logDir == null) {
@@ -452,7 +452,6 @@ public final class BackupUtils {
     }
   }
 
-
   private static void cleanupTargetDir(BackupInfo backupInfo, Configuration conf) {
     try {
       // clean up the data at target directory
@@ -498,8 +497,8 @@ public final class BackupUtils {
    * @param tableName table name
    * @return backupPath String for the particular table
    */
-  public static String getTableBackupDir(String backupRootDir, String backupId,
-      TableName tableName) {
+  public static String
+      getTableBackupDir(String backupRootDir, String backupId, TableName tableName) {
     return backupRootDir + Path.SEPARATOR + backupId + Path.SEPARATOR
         + tableName.getNamespaceAsString() + Path.SEPARATOR + tableName.getQualifierAsString()
         + Path.SEPARATOR;
@@ -523,7 +522,6 @@ public final class BackupUtils {
     return list;
   }
 
-
   /**
    * Calls fs.listStatus() and treats FileNotFoundException as non-fatal This accommodates
    * differences between hadoop versions, where hadoop 1 does not throw a FileNotFoundException, and
@@ -655,19 +653,16 @@ public final class BackupUtils {
    * @param backupId backup id
    * @param check check only
    * @param fromTables table list from
-   * @param toTables   table list to
+   * @param toTables table list to
    * @param isOverwrite overwrite data
    * @return request obkect
    */
   public static RestoreRequest createRestoreRequest(String backupRootDir, String backupId,
       boolean check, TableName[] fromTables, TableName[] toTables, boolean isOverwrite) {
     RestoreRequest.Builder builder = new RestoreRequest.Builder();
-    RestoreRequest request = builder.withBackupRootDir(backupRootDir)
-                                    .withBackupId(backupId)
-                                    .withCheck(check)
-                                    .withFromTables(fromTables)
-                                    .withToTables(toTables)
-                                    .withOvewrite(isOverwrite).build();
+    RestoreRequest request =
+        builder.withBackupRootDir(backupRootDir).withBackupId(backupId).withCheck(check)
+            .withFromTables(fromTables).withToTables(toTables).withOvewrite(isOverwrite).build();
     return request;
   }
 
@@ -699,4 +694,54 @@ public final class BackupUtils {
     return isValid;
   }
 
+  public static Path getBulkOutputDir(String tableName, Configuration conf, boolean deleteOnExit)
+      throws IOException {
+    FileSystem fs = FileSystem.get(conf);
+    String tmp =
+        conf.get(HConstants.TEMPORARY_FS_DIRECTORY_KEY, HConstants.DEFAULT_TEMPORARY_HDFS_DIRECTORY);
+    Path path =
+        new Path(tmp + Path.SEPARATOR + "bulk_output-" + tableName + "-"
+            + EnvironmentEdgeManager.currentTime());
+    if (deleteOnExit) {
+      fs.deleteOnExit(path);
+    }
+    return path;
+  }
+
+  public static Path getBulkOutputDir(String tableName, Configuration conf) throws IOException {
+    return getBulkOutputDir(tableName, conf, true);
+  }
+
+  public static String getFileNameCompatibleString(TableName table) {
+    return table.getNamespaceAsString() + "-" + table.getQualifierAsString();
+  }
+
+  public static boolean failed(int result) {
+    return result != 0;
+  }
+
+  public static boolean succeeded(int result) {
+    return result == 0;
+  }
+
+  public static LoadIncrementalHFiles createLoader(Configuration config) throws IOException {
+    // set configuration for restore:
+    // LoadIncrementalHFile needs more time
+    // <name>hbase.rpc.timeout</name> <value>600000</value>
+    // calculates
+    Configuration conf = new Configuration(config);
+    conf.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, MILLISEC_IN_HOUR);
+
+    // By default, it is 32 and loader will fail if # of files in any region exceed this
+    // limit. Bad for snapshot restore.
+    conf.setInt(LoadIncrementalHFiles.MAX_FILES_PER_REGION_PER_FAMILY, Integer.MAX_VALUE);
+    conf.set(LoadIncrementalHFiles.IGNORE_UNMATCHED_CF_CONF_KEY, "yes");
+    LoadIncrementalHFiles loader = null;
+    try {
+      loader = new LoadIncrementalHFiles(conf);
+    } catch (Exception e) {
+      throw new IOException(e);
+    }
+    return loader;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/05e6e569/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupMergeWithFailures.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupMergeWithFailures.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupMergeWithFailures.java
new file mode 100644
index 0000000..7011ed3
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupMergeWithFailures.java
@@ -0,0 +1,336 @@
+/**
+ * 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.backup;
+
+import static org.apache.hadoop.hbase.backup.util.BackupUtils.succeeded;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl;
+import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
+import org.apache.hadoop.hbase.backup.mapreduce.MapReduceBackupMergeJob;
+import org.apache.hadoop.hbase.backup.mapreduce.MapReduceHFileSplitterJob;
+import org.apache.hadoop.hbase.backup.util.BackupUtils;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.util.Pair;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.google.common.collect.Lists;
+
+@Category(LargeTests.class)
+public class TestIncrementalBackupMergeWithFailures extends TestBackupBase {
+  private static final Log LOG = LogFactory.getLog(TestIncrementalBackupMergeWithFailures.class);
+
+  static enum FailurePhase {
+    PHASE1, PHASE2, PHASE3, PHASE4
+  }
+  public final static String FAILURE_PHASE_KEY = "failurePhase";
+
+  static class BackupMergeJobWithFailures extends MapReduceBackupMergeJob {
+
+    FailurePhase failurePhase;
+
+    @Override
+    public void setConf(Configuration conf) {
+      super.setConf(conf);
+      String val = conf.get(FAILURE_PHASE_KEY);
+      if (val != null) {
+        failurePhase = FailurePhase.valueOf(val);
+      } else {
+        Assert.fail("Failure phase is not set");
+      }
+    }
+
+
+    /**
+     * This is the exact copy of parent's run() with injections
+     * of different types of failures
+     */
+    @Override
+    public void run(String[] backupIds) throws IOException {
+      String bulkOutputConfKey;
+
+      // TODO : run player on remote cluster
+      player = new MapReduceHFileSplitterJob();
+      bulkOutputConfKey = MapReduceHFileSplitterJob.BULK_OUTPUT_CONF_KEY;
+      // Player reads all files in arbitrary directory structure and creates
+      // a Map task for each file
+      String bids = StringUtils.join(backupIds, ",");
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Merge backup images " + bids);
+      }
+
+      List<Pair<TableName, Path>> processedTableList = new ArrayList<Pair<TableName, Path>>();
+      boolean finishedTables = false;
+      Connection conn = ConnectionFactory.createConnection(getConf());
+      BackupSystemTable table = new BackupSystemTable(conn);
+      FileSystem fs = FileSystem.get(getConf());
+
+      try {
+
+        // Start backup exclusive operation
+        table.startBackupExclusiveOperation();
+        // Start merge operation
+        table.startMergeOperation(backupIds);
+
+        // Select most recent backup id
+        String mergedBackupId = findMostRecentBackupId(backupIds);
+
+        TableName[] tableNames = getTableNamesInBackupImages(backupIds);
+        String backupRoot = null;
+
+        BackupInfo bInfo = table.readBackupInfo(backupIds[0]);
+        backupRoot = bInfo.getBackupRootDir();
+        // PHASE 1
+        checkFailure(FailurePhase.PHASE1);
+
+        for (int i = 0; i < tableNames.length; i++) {
+
+          LOG.info("Merge backup images for " + tableNames[i]);
+
+          // Find input directories for table
+
+          Path[] dirPaths = findInputDirectories(fs, backupRoot, tableNames[i], backupIds);
+          String dirs = StringUtils.join(dirPaths, ",");
+          Path bulkOutputPath =
+              BackupUtils.getBulkOutputDir(BackupUtils.getFileNameCompatibleString(tableNames[i]),
+                getConf(), false);
+          // Delete content if exists
+          if (fs.exists(bulkOutputPath)) {
+            if (!fs.delete(bulkOutputPath, true)) {
+              LOG.warn("Can not delete: " + bulkOutputPath);
+            }
+          }
+          Configuration conf = getConf();
+          conf.set(bulkOutputConfKey, bulkOutputPath.toString());
+          String[] playerArgs = { dirs, tableNames[i].getNameAsString() };
+
+          int result = 0;
+          // PHASE 2
+          checkFailure(FailurePhase.PHASE2);
+          player.setConf(getConf());
+          result = player.run(playerArgs);
+          if (succeeded(result)) {
+            // Add to processed table list
+            processedTableList.add(new Pair<TableName, Path>(tableNames[i], bulkOutputPath));
+          } else {
+            throw new IOException("Can not merge backup images for " + dirs
+                + " (check Hadoop/MR and HBase logs). Player return code =" + result);
+          }
+          LOG.debug("Merge Job finished:" + result);
+        }
+        List<TableName> tableList = toTableNameList(processedTableList);
+        // PHASE 3
+        checkFailure(FailurePhase.PHASE3);
+        table.updateProcessedTablesForMerge(tableList);
+        finishedTables = true;
+
+        // Move data
+        for (Pair<TableName, Path> tn : processedTableList) {
+          moveData(fs, backupRoot, tn.getSecond(), tn.getFirst(), mergedBackupId);
+        }
+        // PHASE 4
+        checkFailure(FailurePhase.PHASE4);
+        // Delete old data and update manifest
+        List<String> backupsToDelete = getBackupIdsToDelete(backupIds, mergedBackupId);
+        deleteBackupImages(backupsToDelete, conn, fs, backupRoot);
+        updateBackupManifest(backupRoot, mergedBackupId, backupsToDelete);
+        // Finish merge session
+        table.finishMergeOperation();
+      } catch (RuntimeException e) {
+        throw e;
+      } catch (Exception e) {
+        LOG.error(e);
+        if (!finishedTables) {
+          // cleanup bulk directories and finish merge
+          // merge MUST be repeated (no need for repair)
+          cleanupBulkLoadDirs(fs, toPathList(processedTableList));
+          table.finishMergeOperation();
+          table.finishBackupExclusiveOperation();
+          throw new IOException("Backup merge operation failed, you should try it again", e);
+        } else {
+          // backup repair must be run
+          throw new IOException(
+              "Backup merge operation failed, run backup repair tool to restore system's integrity",
+              e);
+        }
+      } finally {
+        table.close();
+        conn.close();
+      }
+
+    }
+
+    private void checkFailure(FailurePhase phase) throws IOException {
+      if ( failurePhase != null && failurePhase == phase) {
+        throw new IOException (phase.toString());
+      }
+    }
+
+  }
+
+
+  @Test
+  public void TestIncBackupMergeRestore() throws Exception {
+
+    int ADD_ROWS = 99;
+    // #1 - create full backup for all tables
+    LOG.info("create full backup image for all tables");
+
+    List<TableName> tables = Lists.newArrayList(table1, table2);
+    // Set custom Merge Job implementation
+    conf1.setClass(BackupRestoreFactory.HBASE_BACKUP_MERGE_IMPL_CLASS,
+      BackupMergeJobWithFailures.class, BackupMergeJob.class);
+
+    Connection conn = ConnectionFactory.createConnection(conf1);
+
+    HBaseAdmin admin = null;
+    admin = (HBaseAdmin) conn.getAdmin();
+    BackupAdminImpl client = new BackupAdminImpl(conn);
+
+    BackupRequest request = createBackupRequest(BackupType.FULL, tables, BACKUP_ROOT_DIR);
+    String backupIdFull = client.backupTables(request);
+
+    assertTrue(checkSucceeded(backupIdFull));
+
+    // #2 - insert some data to table1
+    HTable t1 = insertIntoTable(conn, table1, famName, 1, ADD_ROWS);
+    LOG.debug("writing " + ADD_ROWS + " rows to " + table1);
+
+    Assert.assertEquals(TEST_UTIL.countRows(t1), NB_ROWS_IN_BATCH + ADD_ROWS);
+    t1.close();
+    LOG.debug("written " + ADD_ROWS + " rows to " + table1);
+
+    HTable t2 = insertIntoTable(conn, table2, famName, 1, ADD_ROWS);
+
+    Assert.assertEquals(TEST_UTIL.countRows(t2), NB_ROWS_IN_BATCH + ADD_ROWS);
+    t2.close();
+    LOG.debug("written " + ADD_ROWS + " rows to " + table2);
+
+    // #3 - incremental backup for multiple tables
+    tables = Lists.newArrayList(table1, table2);
+    request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR);
+    String backupIdIncMultiple = client.backupTables(request);
+
+    assertTrue(checkSucceeded(backupIdIncMultiple));
+
+    t1 = insertIntoTable(conn, table1, famName, 2, ADD_ROWS);
+    t1.close();
+
+    t2 = insertIntoTable(conn, table2, famName, 2, ADD_ROWS);
+    t2.close();
+
+    // #3 - incremental backup for multiple tables
+    request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR);
+    String backupIdIncMultiple2 = client.backupTables(request);
+    assertTrue(checkSucceeded(backupIdIncMultiple2));
+
+    // #4 Merge backup images with failures
+
+    for ( FailurePhase phase : FailurePhase.values()) {
+      Configuration conf = conn.getConfiguration();
+
+      conf.set(FAILURE_PHASE_KEY, phase.toString());
+
+      try (BackupAdmin bAdmin = new BackupAdminImpl(conn);)
+      {
+        String[] backups = new String[] { backupIdIncMultiple, backupIdIncMultiple2 };
+        bAdmin.mergeBackups(backups);
+        Assert.fail("Expected IOException");
+      } catch (IOException e) {
+        BackupSystemTable table = new BackupSystemTable(conn);
+        if(phase.ordinal() < FailurePhase.PHASE4.ordinal()) {
+          // No need to repair:
+          // Both Merge and backup exclusive operations are finished
+          assertFalse(table.isMergeInProgress());
+          try {
+            table.finishBackupExclusiveOperation();
+            Assert.fail("IOException is expected");
+          } catch(IOException ee) {
+            // Expected
+          }
+        } else {
+          // Repair is required
+          assertTrue(table.isMergeInProgress());
+          try {
+            table.startBackupExclusiveOperation();
+            Assert.fail("IOException is expected");
+          } catch(IOException ee) {
+            // Expected - clean up before proceeding
+            table.finishMergeOperation();
+            table.finishBackupExclusiveOperation();
+          }
+        }
+        table.close();
+        LOG.debug("Expected :"+ e.getMessage());
+      }
+    }
+
+    // Now merge w/o failures
+    Configuration conf = conn.getConfiguration();
+    conf.unset(FAILURE_PHASE_KEY);
+    conf.unset(BackupRestoreFactory.HBASE_BACKUP_MERGE_IMPL_CLASS);
+
+    try (BackupAdmin bAdmin = new BackupAdminImpl(conn);) {
+      String[] backups = new String[] { backupIdIncMultiple, backupIdIncMultiple2 };
+      bAdmin.mergeBackups(backups);
+    }
+
+    // #6 - restore incremental backup for multiple tables, with overwrite
+    TableName[] tablesRestoreIncMultiple = new TableName[] { table1, table2 };
+    TableName[] tablesMapIncMultiple = new TableName[] { table1_restore, table2_restore };
+    client.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupIdIncMultiple2, false,
+      tablesRestoreIncMultiple, tablesMapIncMultiple, true));
+
+    Table hTable = conn.getTable(table1_restore);
+    LOG.debug("After incremental restore: " + hTable.getTableDescriptor());
+    LOG.debug("f1 has " + TEST_UTIL.countRows(hTable, famName) + " rows");
+    Assert.assertEquals(TEST_UTIL.countRows(hTable, famName), NB_ROWS_IN_BATCH + 2 * ADD_ROWS);
+
+    hTable.close();
+
+    hTable = conn.getTable(table2_restore);
+    Assert.assertEquals(TEST_UTIL.countRows(hTable), NB_ROWS_IN_BATCH + 2 * ADD_ROWS);
+    hTable.close();
+
+    admin.close();
+    conn.close();
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/05e6e569/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRepairAfterFailedDelete.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRepairAfterFailedDelete.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRepairAfterFailedDelete.java
index 9c47641..556521f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRepairAfterFailedDelete.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRepairAfterFailedDelete.java
@@ -74,7 +74,7 @@ public class TestRepairAfterFailedDelete extends TestBackupBase {
     admin.restoreSnapshot(snapshotName);
     admin.enableTable(BackupSystemTable.getTableName(conf1));
     // Start backup session
-    table.startBackupSession();
+    table.startBackupExclusiveOperation();
     // Start delete operation
     table.startDeleteOperation(backupIds);
 


[16/50] [abbrv] hbase git commit: HBASE-18271 Shade netty Purge mention of netty-all.

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyIPC.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyIPC.java
index 3b32383..6a39e12 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyIPC.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyIPC.java
@@ -17,10 +17,10 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
-import io.netty.channel.epoll.EpollEventLoopGroup;
-import io.netty.channel.epoll.EpollSocketChannel;
-import io.netty.channel.nio.NioEventLoopGroup;
-import io.netty.channel.socket.nio.NioSocketChannel;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.epoll.EpollEventLoopGroup;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.epoll.EpollSocketChannel;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoopGroup;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.nio.NioSocketChannel;
 
 import java.util.ArrayList;
 import java.util.Collection;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncFSWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncFSWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncFSWAL.java
index 9b28975..2ae916f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncFSWAL.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncFSWAL.java
@@ -17,10 +17,10 @@
  */
 package org.apache.hadoop.hbase.regionserver.wal;
 
-import io.netty.channel.Channel;
-import io.netty.channel.EventLoopGroup;
-import io.netty.channel.nio.NioEventLoopGroup;
-import io.netty.channel.socket.nio.NioSocketChannel;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoopGroup;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoopGroup;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.nio.NioSocketChannel;
 
 import java.io.IOException;
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncProtobufLog.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncProtobufLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncProtobufLog.java
index bb67820..708d64c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncProtobufLog.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncProtobufLog.java
@@ -19,10 +19,10 @@ package org.apache.hadoop.hbase.regionserver.wal;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.base.Throwables;
 
-import io.netty.channel.Channel;
-import io.netty.channel.EventLoopGroup;
-import io.netty.channel.nio.NioEventLoopGroup;
-import io.netty.channel.socket.nio.NioSocketChannel;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoopGroup;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoopGroup;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.nio.NioSocketChannel;
 
 import java.io.IOException;
 import java.io.InterruptedIOException;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncWALReplay.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncWALReplay.java
index 17f58f8..881cf7c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncWALReplay.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncWALReplay.java
@@ -17,10 +17,10 @@
  */
 package org.apache.hadoop.hbase.regionserver.wal;
 
-import io.netty.channel.Channel;
-import io.netty.channel.EventLoopGroup;
-import io.netty.channel.nio.NioEventLoopGroup;
-import io.netty.channel.socket.nio.NioSocketChannel;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoopGroup;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoopGroup;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.nio.NioSocketChannel;
 
 import java.io.IOException;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 7925e4e..5040361 100755
--- a/pom.xml
+++ b/pom.xml
@@ -626,6 +626,17 @@
             <redirectTestOutputToFile>${test.output.tofile}</redirectTestOutputToFile>
             <systemPropertyVariables>
               <test.build.classes>${test.build.classes}</test.build.classes>
+              <!--For shaded netty, to find the relocated .so.
+                   Trick from
+                https://stackoverflow.com/questions/33825743/rename-files-inside-a-jar-using-some-maven-plugin
+
+                The netty jar has a .so in it. Shading requires rename of the .so and then passing a system
+                property so netty finds the renamed .so and associates it w/ the relocated netty files.
+
+                The relocated netty is in hbase-thirdparty dependency. Just set this propery globally rather
+                than per module.
+               -->
+              <org.apache.hadoop.hbase.shaded.io.netty.packagePrefix>org.apache.hadoop.hbase.shaded.</org.apache.hadoop.hbase.shaded.io.netty.packagePrefix>
             </systemPropertyVariables>
 
             <excludes>
@@ -1394,7 +1405,8 @@
     <clover.version>4.0.3</clover.version>
     <jamon-runtime.version>2.4.1</jamon-runtime.version>
     <jettison.version>1.3.8</jettison.version>
-    <netty.version>4.1.9.Final</netty.version>
+    <!--This property is for hadoops netty. HBase netty
+         comes in via hbase-thirdparty hbase-shaded-netty-->
     <netty.hadoop.version>3.6.2.Final</netty.hadoop.version>
     <!--Make sure these joni/jcodings are compatible with the versions used by jruby-->
     <joni.version>2.1.11</joni.version>
@@ -1790,11 +1802,6 @@
         </exclusions>
       </dependency>
       <dependency>
-        <groupId>io.netty</groupId>
-        <artifactId>netty-all</artifactId>
-        <version>${netty.version}</version>
-      </dependency>
-      <dependency>
         <groupId>org.apache.thrift</groupId>
         <artifactId>libthrift</artifactId>
         <version>${thrift.version}</version>


[50/50] [abbrv] hbase git commit: Merge in master branch

Posted by st...@apache.org.
Merge in master branch


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

Branch: refs/heads/HBASE-14070.HLC
Commit: 675b63d5055dfb27fc74b5c59a80bf8fc7b33ab5
Parents: 75a6b36 82a9cec
Author: Michael Stack <st...@apache.org>
Authored: Thu Aug 17 12:14:23 2017 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Thu Aug 17 12:14:23 2017 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hbase/HTableDescriptor.java   |  12 +
 .../apache/hadoop/hbase/MetaTableAccessor.java  | 144 ++-----
 .../hadoop/hbase/client/RawAsyncHBaseAdmin.java |   2 +-
 .../hadoop/hbase/client/TableDescriptor.java    |   9 +
 .../hbase/client/TableDescriptorBuilder.java    |  45 +++
 .../hbase/shaded/protobuf/ProtobufUtil.java     |  55 ++-
 .../hbase/shaded/protobuf/RequestConverter.java |  28 +-
 .../java/org/apache/hadoop/hbase/Clock.java     | 107 ++++++
 .../org/apache/hadoop/hbase/ClockException.java |  33 ++
 .../java/org/apache/hadoop/hbase/ClockType.java |  38 ++
 .../apache/hadoop/hbase/HybridLogicalClock.java | 134 +++++++
 .../apache/hadoop/hbase/SettableTimestamp.java  |   2 +-
 .../org/apache/hadoop/hbase/SystemClock.java    |  44 +++
 .../hadoop/hbase/SystemMonotonicClock.java      |  83 +++++
 .../org/apache/hadoop/hbase/TimestampType.java  | 312 ++++++++++++++++
 .../apache/hadoop/hbase/util/AtomicUtils.java   |   1 +
 .../java/org/apache/hadoop/hbase/TestClock.java | 307 +++++++++++++++
 .../apache/hadoop/hbase/TestTimestampType.java  | 213 +++++++++++
 .../src/main/protobuf/Admin.proto               |   8 +
 .../src/main/protobuf/HBase.proto               |  13 +
 hbase-protocol/src/main/protobuf/Admin.proto    |   8 +-
 .../org/apache/hadoop/hbase/master/HMaster.java |   2 +
 .../hadoop/hbase/master/MasterServices.java     |   8 +
 .../hadoop/hbase/master/ServerManager.java      |   1 +
 .../master/assignment/RegionStateStore.java     |   5 +-
 .../master/procedure/ModifyTableProcedure.java  |   4 +
 .../master/procedure/RSProcedureDispatcher.java |  55 ++-
 .../hadoop/hbase/regionserver/HRegion.java      |  81 +++-
 .../hbase/regionserver/HRegionServer.java       |  63 +++-
 .../hadoop/hbase/regionserver/HStore.java       |  14 +-
 .../hbase/regionserver/RSRpcServices.java       |  41 +-
 .../hadoop/hbase/regionserver/Region.java       |   6 +
 .../hbase/regionserver/RegionMergeRequest.java  |   3 +-
 .../regionserver/RegionServerServices.java      |  23 +-
 .../apache/hadoop/hbase/regionserver/Store.java |   7 +
 .../hadoop/hbase/regionserver/StoreScanner.java | 102 +++--
 .../hadoop/hbase/regionserver/StoreUtils.java   |   8 +
 .../regionserver/handler/OpenMetaHandler.java   |   4 +-
 .../handler/OpenPriorityRegionHandler.java      |   4 +-
 .../regionserver/handler/OpenRegionHandler.java |  20 +-
 .../DropDeletesCompactionScanQueryMatcher.java  |  19 +-
 .../querymatcher/LegacyScanQueryMatcher.java    |   3 +-
 .../MajorCompactionScanQueryMatcher.java        |   5 +-
 .../MinorCompactionScanQueryMatcher.java        |   3 +-
 .../NormalUserScanQueryMatcher.java             |   3 +-
 .../querymatcher/RawScanQueryMatcher.java       |   3 +-
 .../querymatcher/ScanQueryMatcher.java          |  24 +-
 .../StripeCompactionScanQueryMatcher.java       |   5 +-
 .../hbase/security/access/AccessController.java |  10 +-
 .../hadoop/hbase/util/FSTableDescriptors.java   |   1 +
 .../hadoop/hbase/util/HBaseFsckRepair.java      |   2 +-
 .../hadoop/hbase/HBaseTestingUtility.java       |  28 ++
 .../hadoop/hbase/MockRegionServerServices.java  |   5 +
 .../hadoop/hbase/TestClockWithCluster.java      | 373 +++++++++++++++++++
 .../hadoop/hbase/TestMetaTableAccessor.java     | 101 +----
 .../hadoop/hbase/client/TestReplicasClient.java |   4 +-
 .../coprocessor/TestIncrementTimeRange.java     |  41 +-
 .../hadoop/hbase/mapreduce/TestCopyTable.java   |  31 +-
 .../hbase/master/MockNoopMasterServices.java    |   5 +
 .../hadoop/hbase/master/MockRegionServer.java   |   8 +
 .../hadoop/hbase/master/TestMasterFailover.java |   2 +-
 .../hbase/master/TestMasterNoCluster.java       |   1 +
 .../regionserver/TestCompactingMemStore.java    |  46 ++-
 .../hbase/regionserver/TestDefaultMemStore.java |  62 ++-
 .../hadoop/hbase/regionserver/TestHRegion.java  | 185 ++++++++-
 .../regionserver/TestHRegionReplayEvents.java   |   3 +
 .../regionserver/TestRegionServerNoMaster.java  |  10 +-
 .../regionserver/TestRegionSplitPolicy.java     |   4 +
 .../hbase/regionserver/TestStoreScanner.java    | 168 ++++++---
 .../hbase/regionserver/TestWALLockup.java       |   3 +
 .../TestCompactionScanQueryMatcher.java         |   3 +-
 .../querymatcher/TestUserScanQueryMatcher.java  |  12 +-
 .../regionserver/wal/AbstractTestWALReplay.java |  49 ++-
 .../access/TestCellACLWithMultipleVersions.java | 179 ++++++---
 .../hbase/util/TestCoprocessorScanPolicy.java   |  30 +-
 .../hbase/util/TestHBaseFsckReplicas.java       |   2 +-
 .../apache/hadoop/hbase/util/TestTableName.java |   5 +-
 77 files changed, 2980 insertions(+), 507 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/675b63d5/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hbase/blob/675b63d5/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hbase/blob/675b63d5/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
----------------------------------------------------------------------
diff --cc hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
index ba68a96,1ae0eec..8f4e34a
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
@@@ -2897,4 -2917,4 +2897,4 @@@ public class RawAsyncHBaseAdmin impleme
          .pause(pauseNs, TimeUnit.NANOSECONDS).maxAttempts(maxAttempts)
          .startLogErrorsCnt(startLogErrorsCnt);
    }
--}
++}

http://git-wip-us.apache.org/repos/asf/hbase/blob/675b63d5/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hbase/blob/675b63d5/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
----------------------------------------------------------------------
diff --cc hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
index 08ed3dc,d57ffb7..e79b991
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
@@@ -26,7 -26,7 +26,8 @@@ import java.util.Set
  import java.util.regex.Pattern;
  
  import org.apache.hadoop.hbase.CellScannable;
 +import org.apache.hadoop.hbase.ClusterStatus.Options;
+ import org.apache.hadoop.hbase.ClockType;
  import org.apache.hadoop.hbase.DoNotRetryIOException;
  import org.apache.hadoop.hbase.HConstants;
  import org.apache.hadoop.hbase.HRegionInfo;

http://git-wip-us.apache.org/repos/asf/hbase/blob/675b63d5/hbase-protocol-shaded/src/main/protobuf/Admin.proto
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hbase/blob/675b63d5/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --cc hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 6b4d4e9,fb9d611..975f17a
--- 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
@@@ -53,8 -53,9 +53,10 @@@ import org.apache.commons.logging.LogFa
  import org.apache.hadoop.conf.Configuration;
  import org.apache.hadoop.fs.FileSystem;
  import org.apache.hadoop.fs.Path;
+ import org.apache.hadoop.hbase.Clock;
+ import org.apache.hadoop.hbase.ClockType;
  import org.apache.hadoop.hbase.ClusterStatus;
 +import org.apache.hadoop.hbase.ClusterStatus.Options;
  import org.apache.hadoop.hbase.CoordinatedStateException;
  import org.apache.hadoop.hbase.CoordinatedStateManager;
  import org.apache.hadoop.hbase.DoNotRetryIOException;

http://git-wip-us.apache.org/repos/asf/hbase/blob/675b63d5/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hbase/blob/675b63d5/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hbase/blob/675b63d5/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hbase/blob/675b63d5/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hbase/blob/675b63d5/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hbase/blob/675b63d5/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --cc hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 0c1814f,bf6dc3a..fad0d96
--- 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
@@@ -591,7 -600,14 +603,13 @@@ public class HRegionServer extends HasT
      this.abortRequested = false;
      this.stopped = false;
  
+     final long maxClockSkew =
+         conf.getLong("hbase.max.clock.skew.in.ms", Clock.DEFAULT_MAX_CLOCK_SKEW_IN_MS);
+     this.hybridLogicalClock = new HybridLogicalClock(maxClockSkew);
+     this.systemMonotonicClock = new SystemMonotonicClock(maxClockSkew);
+     this.systemClock = new SystemClock();
+ 
      rpcServices = createRpcServices();
 -    this.startcode = System.currentTimeMillis();
      if (this instanceof HMaster) {
        useThisHostnameInstead = conf.get(MASTER_HOSTNAME_KEY);
      } else {

http://git-wip-us.apache.org/repos/asf/hbase/blob/675b63d5/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
----------------------------------------------------------------------
diff --cc hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
index 882e1fc,7245597..42fea4c
--- 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
@@@ -92,14 -92,15 +92,15 @@@ import org.apache.hadoop.hbase.util.Pai
  import org.apache.hadoop.hbase.util.ReflectionUtils;
  import org.apache.hadoop.util.StringUtils;
  import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
+ import org.apache.hadoop.hbase.Clock;
  
 -import com.google.common.annotations.VisibleForTesting;
 -import com.google.common.base.Preconditions;
 -import com.google.common.collect.ImmutableCollection;
 -import com.google.common.collect.ImmutableList;
 -import com.google.common.collect.Lists;
 -import com.google.common.collect.Sets;
  import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 +import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 +import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
 +import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableCollection;
 +import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
 +import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
 +import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
  
  /**
   * A Store holds a column family in a Region.  Its a memstore and a set of zero

http://git-wip-us.apache.org/repos/asf/hbase/blob/675b63d5/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hbase/blob/675b63d5/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hbase/blob/675b63d5/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeRequest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hbase/blob/675b63d5/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
----------------------------------------------------------------------
diff --cc hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
index 5286c39,3f453fd..951f47c
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
@@@ -48,10 -42,10 +42,9 @@@ import org.apache.hadoop.hbase.regionse
  import org.apache.hadoop.hbase.regionserver.querymatcher.LegacyScanQueryMatcher;
  import org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher;
  import org.apache.hadoop.hbase.regionserver.querymatcher.UserScanQueryMatcher;
 +import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
  import org.apache.hadoop.hbase.util.CollectionUtils;
- import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
  
 -import com.google.common.annotations.VisibleForTesting;
 -
  /**
   * Scanner scans both the memstore and the Store. Coalesce KeyValue stream into List&lt;KeyValue&gt;
   * for a single row.
@@@ -603,9 -663,8 +661,9 @@@ public class StoreScanner extends NonRe
        int cellSize = CellUtil.estimatedSerializedSizeOf(cell);
        bytesRead += cellSize;
        prevCell = cell;
 +      scannerContext.setLastPeekedCell(cell);
        topChanged = false;
-       ScanQueryMatcher.MatchCode qcode = matcher.match(cell);
+       ScanQueryMatcher.MatchCode qcode = matcher.match(cell, timestampType);
        switch (qcode) {
          case INCLUDE:
          case INCLUDE_AND_SEEK_NEXT_ROW:

http://git-wip-us.apache.org/repos/asf/hbase/blob/675b63d5/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/DropDeletesCompactionScanQueryMatcher.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hbase/blob/675b63d5/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/LegacyScanQueryMatcher.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hbase/blob/675b63d5/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/MajorCompactionScanQueryMatcher.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hbase/blob/675b63d5/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/MinorCompactionScanQueryMatcher.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hbase/blob/675b63d5/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/NormalUserScanQueryMatcher.java
----------------------------------------------------------------------
diff --cc hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/NormalUserScanQueryMatcher.java
index b168034,c97614c..51489c3
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/NormalUserScanQueryMatcher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/NormalUserScanQueryMatcher.java
@@@ -22,8 -22,10 +22,9 @@@ import java.io.IOException
  import org.apache.hadoop.hbase.Cell;
  import org.apache.hadoop.hbase.CellUtil;
  import org.apache.hadoop.hbase.KeepDeletedCells;
+ import org.apache.hadoop.hbase.TimestampType;
  import org.apache.hadoop.hbase.classification.InterfaceAudience;
  import org.apache.hadoop.hbase.client.Scan;
 -import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
  import org.apache.hadoop.hbase.regionserver.ScanInfo;
  
  /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/675b63d5/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/ScanQueryMatcher.java
----------------------------------------------------------------------
diff --cc hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/ScanQueryMatcher.java
index 524d3f7,c38e408..b925668
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/ScanQueryMatcher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/ScanQueryMatcher.java
@@@ -32,16 -31,15 +32,18 @@@ import org.apache.hadoop.hbase.Tag
  import org.apache.hadoop.hbase.TagType;
  import org.apache.hadoop.hbase.TagUtil;
  import org.apache.hadoop.hbase.classification.InterfaceAudience;
 +import org.apache.hadoop.hbase.client.Scan;
+ import org.apache.hadoop.hbase.TimestampType;
  import org.apache.hadoop.hbase.filter.Filter;
 -import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
  import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
  import org.apache.hadoop.hbase.regionserver.ScanInfo;
  import org.apache.hadoop.hbase.regionserver.ShipperListener;
  import org.apache.hadoop.hbase.regionserver.querymatcher.DeleteTracker.DeleteResult;
 +import org.apache.hadoop.hbase.security.visibility.VisibilityNewVersionBehaivorTracker;
 +import org.apache.hadoop.hbase.security.visibility.VisibilityScanDeleteTracker;
+ import org.apache.hadoop.hbase.shaded.com.google.protobuf.Timestamp;
  import org.apache.hadoop.hbase.util.Bytes;
 +import org.apache.hadoop.hbase.util.Pair;
  
  /**
   * A query matcher that is specifically designed for the scan case.

http://git-wip-us.apache.org/repos/asf/hbase/blob/675b63d5/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/StripeCompactionScanQueryMatcher.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hbase/blob/675b63d5/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
----------------------------------------------------------------------
diff --cc hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
index c40d481,f58e24f..429180d
--- 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
@@@ -53,9 -55,9 +53,10 @@@ import org.apache.hadoop.hbase.Procedur
  import org.apache.hadoop.hbase.ServerName;
  import org.apache.hadoop.hbase.TableName;
  import org.apache.hadoop.hbase.Tag;
+ import org.apache.hadoop.hbase.TimestampType;
  import org.apache.hadoop.hbase.classification.InterfaceAudience;
  import org.apache.hadoop.hbase.client.Append;
 +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
  import org.apache.hadoop.hbase.client.Delete;
  import org.apache.hadoop.hbase.client.Durability;
  import org.apache.hadoop.hbase.client.Get;

http://git-wip-us.apache.org/repos/asf/hbase/blob/675b63d5/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hbase/blob/675b63d5/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hbase/blob/675b63d5/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hbase/blob/675b63d5/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hbase/blob/675b63d5/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hbase/blob/675b63d5/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hbase/blob/675b63d5/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java
----------------------------------------------------------------------
diff --cc hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java
index 8118e41,b7a5168..2e20a9a
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java
@@@ -193,11 -216,12 +216,12 @@@ public class TestCompactingMemStore ext
      Configuration conf = HBaseConfiguration.create();
      for (int startRowId = 0; startRowId < ROW_COUNT; startRowId++) {
        ScanInfo scanInfo = new ScanInfo(conf, FAMILY, 0, 1, Integer.MAX_VALUE,
 -          KeepDeletedCells.FALSE, HConstants.DEFAULT_BLOCKSIZE, 0, this.memstore.getComparator());
 +          KeepDeletedCells.FALSE, HConstants.DEFAULT_BLOCKSIZE, 0, this.memstore.getComparator(), false);
        ScanType scanType = ScanType.USER_SCAN;
-       InternalScanner scanner = new StoreScanner(new Scan(
-           Bytes.toBytes(startRowId)), scanInfo, scanType, null,
-           memstore.getScanners(0));
+       Store mockStore = mock(HStore.class);
+       when(mockStore.getClock()).thenReturn(Clock.getDummyClockOfGivenClockType(clock.getClockType()));
+       InternalScanner scanner = new StoreScanner(mockStore, new Scan(Bytes.toBytes(startRowId)),
+           scanInfo, scanType, null, memstore.getScanners(0)) ;
        List<Cell> results = new ArrayList<>();
        for (int i = 0; scanner.next(results); i++) {
          int rowId = startRowId + i;

http://git-wip-us.apache.org/repos/asf/hbase/blob/675b63d5/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
----------------------------------------------------------------------
diff --cc hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
index 7b10846,f9ad88f..efec800
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
@@@ -601,11 -627,13 +627,13 @@@ public class TestDefaultMemStore 
      Configuration conf = HBaseConfiguration.create();
      for (int startRowId = 0; startRowId < ROW_COUNT; startRowId++) {
        ScanInfo scanInfo = new ScanInfo(conf, FAMILY, 0, 1, Integer.MAX_VALUE,
 -          KeepDeletedCells.FALSE, HConstants.DEFAULT_BLOCKSIZE, 0, this.memstore.getComparator());
 +          KeepDeletedCells.FALSE, HConstants.DEFAULT_BLOCKSIZE, 0, this.memstore.getComparator(), false);
        ScanType scanType = ScanType.USER_SCAN;
-       try (InternalScanner scanner = new StoreScanner(new Scan(
-           Bytes.toBytes(startRowId)), scanInfo, scanType, null,
-           memstore.getScanners(0))) {
+       Store store = mock(HStore.class);
+       when(store.getClock()).thenReturn(Clock.getDummyClockOfGivenClockType(clock.getClockType()));
+ 
+       try (InternalScanner scanner = new StoreScanner(store, new Scan(Bytes.toBytes(startRowId)),
+           scanInfo, scanType, null, memstore.getScanners(0))) {
          List<Cell> results = new ArrayList<>();
          for (int i = 0; scanner.next(results); i++) {
            int rowId = startRowId + i;

http://git-wip-us.apache.org/repos/asf/hbase/blob/675b63d5/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hbase/blob/675b63d5/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hbase/blob/675b63d5/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java
----------------------------------------------------------------------
diff --cc hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java
index 1653728,c7ccc57..a3ceb14
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java
@@@ -831,9 -854,11 +854,11 @@@ public class TestStoreScanner 
      Scan scan = new Scan();
      scan.setMaxVersions(1);
      ScanInfo scanInfo = new ScanInfo(CONF, CF, 0, 1, 500, KeepDeletedCells.FALSE,
 -        HConstants.DEFAULT_BLOCKSIZE, 0, CellComparator.COMPARATOR);
 +        HConstants.DEFAULT_BLOCKSIZE, 0, CellComparator.COMPARATOR, false);
      ScanType scanType = ScanType.USER_SCAN;
-     try (StoreScanner scanner = new StoreScanner(scan, scanInfo, scanType, null, scanners)) {
+     Store store = mock(HStore.class);
+     when(store.getClock()).thenReturn(Clock.getDummyClockOfGivenClockType(clockType));
+     try (StoreScanner scanner = new StoreScanner(store, scan, scanInfo, scanType, null, scanners)) {
        List<Cell> results = new ArrayList<>();
        Assert.assertEquals(true, scanner.next(results));
        Assert.assertEquals(2, results.size());
@@@ -904,11 -944,12 +944,12 @@@
      scan.setMaxVersions(1);
      // scanner with ttl equal to 500
      ScanInfo scanInfo = new ScanInfo(CONF, CF, 0, 1, 500, KeepDeletedCells.FALSE,
 -        HConstants.DEFAULT_BLOCKSIZE, 0, CellComparator.COMPARATOR);
 +        HConstants.DEFAULT_BLOCKSIZE, 0, CellComparator.COMPARATOR, false);
      ScanType scanType = ScanType.USER_SCAN;
+     Store store = mock(HStore.class);
+     when(store.getClock()).thenReturn(clock);
      try (StoreScanner scanner =
-         new StoreScanner(scan, scanInfo, scanType, null, scanners)) {
- 
+         new StoreScanner(store, scan, scanInfo, scanType, null, scanners)) {
        List<Cell> results = new ArrayList<>();
        Assert.assertEquals(true, scanner.next(results));
        Assert.assertEquals(1, results.size());
@@@ -1008,4 -1080,4 +1080,4 @@@
      StoreScanner storeScanner = new StoreScanner(scan, scanInfo, scanType, null, scanners);
      assertFalse(storeScanner.isScanUsePread());
    }
--}
++}

http://git-wip-us.apache.org/repos/asf/hbase/blob/675b63d5/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestCompactionScanQueryMatcher.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hbase/blob/675b63d5/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestUserScanQueryMatcher.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hbase/blob/675b63d5/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java
----------------------------------------------------------------------
diff --cc hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java
index 993a1fd,d1f0282..2b204dc
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java
@@@ -31,7 -31,7 +31,8 @@@ import java.util.NavigableSet
  import org.apache.hadoop.conf.Configuration;
  import org.apache.hadoop.hbase.Cell;
  import org.apache.hadoop.hbase.CellUtil;
 +import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
+ import org.apache.hadoop.hbase.ClockType;
  import org.apache.hadoop.hbase.HBaseTestingUtility;
  import org.apache.hadoop.hbase.HColumnDescriptor;
  import org.apache.hadoop.hbase.HConstants;

http://git-wip-us.apache.org/repos/asf/hbase/blob/675b63d5/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplicas.java
----------------------------------------------------------------------


[33/50] [abbrv] hbase git commit: HBASE-18504 Add documentation for WAL compression

Posted by st...@apache.org.
HBASE-18504 Add documentation for WAL compression

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/63e313b5
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/63e313b5
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/63e313b5

Branch: refs/heads/HBASE-14070.HLC
Commit: 63e313b5c0d7c56d9cf9602e3c204214331189d5
Parents: 70c4f78
Author: Peter Somogyi <ps...@cloudera.com>
Authored: Wed Aug 2 17:00:52 2017 +0200
Committer: Michael Stack <st...@apache.org>
Committed: Tue Aug 15 12:54:08 2017 -0700

----------------------------------------------------------------------
 src/main/asciidoc/_chapters/architecture.adoc | 16 ++++++++++++++++
 1 file changed, 16 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/63e313b5/src/main/asciidoc/_chapters/architecture.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/architecture.adoc b/src/main/asciidoc/_chapters/architecture.adoc
index ebb0677..2ded813 100644
--- a/src/main/asciidoc/_chapters/architecture.adoc
+++ b/src/main/asciidoc/_chapters/architecture.adoc
@@ -1216,6 +1216,22 @@ This will be the default for HBase 0.99 (link:https://issues.apache.org/jira/bro
 You must also enable HFile version 3 (which is the default HFile format starting in HBase 0.99.
 See link:https://issues.apache.org/jira/browse/HBASE-10855[HBASE-10855]). Distributed log replay is unsafe for rolling upgrades.
 
+[[wal.compression]]
+==== WAL Compression ====
+
+The content of the WAL can be compressed using LRU Dictionary compression.
+This can be used to speed up WAL replication to different datanodes.
+The dictionary can store up to 2^15^ elements; eviction starts after this number is exceeded.
+
+To enable WAL compression, set the `hbase.regionserver.wal.enablecompression` property to `true`.
+The default value for this property is `false`.
+By default, WAL tag compression is turned on when WAL compression is enabled.
+You can turn off WAL tag compression by setting the `hbase.regionserver.wal.tags.enablecompression` property to 'false'.
+
+A possible downside to WAL compression is that we lose more data from the last block in the WAL if it ill-terminated
+mid-write. If entries in this last block were added with new dictionary entries but we failed persist the amended
+dictionary because of an abrupt termination, a read of this last block may not be able to resolve last-written entries. 
+
 [[wal.disable]]
 ==== Disabling the WAL
 


[28/50] [abbrv] hbase git commit: HBASE-18533 Expose BucketCache values to be configured

Posted by st...@apache.org.
HBASE-18533 Expose BucketCache values to be configured

Before this commit, BucketCache always used the default values.
This commit adds the ability to configure these values.

Signed-off-by: tedyu <yu...@gmail.com>


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

Branch: refs/heads/HBASE-14070.HLC
Commit: 0e32869f01697abf29292aa786d0cdcca10213c6
Parents: 0b26ccd
Author: Zach York <zy...@amazon.com>
Authored: Wed Aug 2 14:43:03 2017 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Mon Aug 14 13:27:26 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/io/hfile/CacheConfig.java      |   2 +-
 .../hbase/io/hfile/bucket/BucketCache.java      | 126 ++++++++++++++-----
 .../hbase/io/hfile/bucket/TestBucketCache.java  | 114 ++++++++++++++++-
 .../io/hfile/bucket/TestBucketWriterThread.java |   3 +-
 4 files changed, 214 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/0e32869f/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java
index 140009b..13f048e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java
@@ -671,7 +671,7 @@ public class CacheConfig {
       // Bucket cache logs its stats on creation internal to the constructor.
       bucketCache = new BucketCache(bucketCacheIOEngineName,
         bucketCacheSize, blockSize, bucketSizes, writerThreads, writerQueueLen, persistentPath,
-        ioErrorsTolerationDuration);
+        ioErrorsTolerationDuration, c);
     } catch (IOException ioex) {
       LOG.error("Can't instantiate bucket cache", ioex); throw new RuntimeException(ioex);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/0e32869f/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
index 1084399..79b1f4d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
@@ -52,8 +52,11 @@ import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
+import com.google.common.base.Preconditions;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.io.HeapSize;
 import org.apache.hadoop.hbase.io.hfile.BlockCache;
@@ -100,14 +103,23 @@ import org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.ThreadFa
 public class BucketCache implements BlockCache, HeapSize {
   private static final Log LOG = LogFactory.getLog(BucketCache.class);
 
+  /** Priority buckets config */
+  static final String SINGLE_FACTOR_CONFIG_NAME = "hbase.bucketcache.single.factor";
+  static final String MULTI_FACTOR_CONFIG_NAME = "hbase.bucketcache.multi.factor";
+  static final String MEMORY_FACTOR_CONFIG_NAME = "hbase.bucketcache.memory.factor";
+  static final String EXTRA_FREE_FACTOR_CONFIG_NAME = "hbase.bucketcache.extrafreefactor";
+  static final String ACCEPT_FACTOR_CONFIG_NAME = "hbase.bucketcache.acceptfactor";
+  static final String MIN_FACTOR_CONFIG_NAME = "hbase.bucketcache.minfactor";
+
   /** Priority buckets */
-  private static final float DEFAULT_SINGLE_FACTOR = 0.25f;
-  private static final float DEFAULT_MULTI_FACTOR = 0.50f;
-  private static final float DEFAULT_MEMORY_FACTOR = 0.25f;
-  private static final float DEFAULT_EXTRA_FREE_FACTOR = 0.10f;
+  @VisibleForTesting
+  static final float DEFAULT_SINGLE_FACTOR = 0.25f;
+  static final float DEFAULT_MULTI_FACTOR = 0.50f;
+  static final float DEFAULT_MEMORY_FACTOR = 0.25f;
+  static final float DEFAULT_MIN_FACTOR = 0.85f;
 
+  private static final float DEFAULT_EXTRA_FREE_FACTOR = 0.10f;
   private static final float DEFAULT_ACCEPT_FACTOR = 0.95f;
-  private static final float DEFAULT_MIN_FACTOR = 0.85f;
 
   // Number of blocks to clear for each of the bucket size that is full
   private static final int DEFAULT_FREE_ENTIRE_BLOCK_FACTOR = 2;
@@ -217,15 +229,34 @@ public class BucketCache implements BlockCache, HeapSize {
   // Allocate or free space for the block
   private BucketAllocator bucketAllocator;
 
+  /** Acceptable size of cache (no evictions if size < acceptable) */
+  private float acceptableFactor;
+
+  /** Minimum threshold of cache (when evicting, evict until size < min) */
+  private float minFactor;
+
+  /** Free this floating point factor of extra blocks when evicting. For example free the number of blocks requested * (1 + extraFreeFactor) */
+  private float extraFreeFactor;
+
+  /** Single access bucket size */
+  private float singleFactor;
+
+  /** Multiple access bucket size */
+  private float multiFactor;
+
+  /** In-memory bucket size */
+  private float memoryFactor;
+
   public BucketCache(String ioEngineName, long capacity, int blockSize, int[] bucketSizes,
       int writerThreadNum, int writerQLen, String persistencePath) throws FileNotFoundException,
       IOException {
     this(ioEngineName, capacity, blockSize, bucketSizes, writerThreadNum, writerQLen,
-      persistencePath, DEFAULT_ERROR_TOLERATION_DURATION);
+      persistencePath, DEFAULT_ERROR_TOLERATION_DURATION, HBaseConfiguration.create());
   }
 
   public BucketCache(String ioEngineName, long capacity, int blockSize, int[] bucketSizes,
-      int writerThreadNum, int writerQLen, String persistencePath, int ioErrorsTolerationDuration)
+                     int writerThreadNum, int writerQLen, String persistencePath, int ioErrorsTolerationDuration,
+                     Configuration conf)
       throws FileNotFoundException, IOException {
     this.ioEngine = getIOEngineFromName(ioEngineName, capacity, persistencePath);
     this.writerThreads = new WriterThread[writerThreadNum];
@@ -235,6 +266,19 @@ public class BucketCache implements BlockCache, HeapSize {
       throw new IllegalArgumentException("Cache capacity is too large, only support 32TB now");
     }
 
+    this.acceptableFactor = conf.getFloat(ACCEPT_FACTOR_CONFIG_NAME, DEFAULT_ACCEPT_FACTOR);
+    this.minFactor = conf.getFloat(MIN_FACTOR_CONFIG_NAME, DEFAULT_MIN_FACTOR);
+    this.extraFreeFactor = conf.getFloat(EXTRA_FREE_FACTOR_CONFIG_NAME, DEFAULT_EXTRA_FREE_FACTOR);
+    this.singleFactor = conf.getFloat(SINGLE_FACTOR_CONFIG_NAME, DEFAULT_SINGLE_FACTOR);
+    this.multiFactor = conf.getFloat(MULTI_FACTOR_CONFIG_NAME, DEFAULT_MULTI_FACTOR);
+    this.memoryFactor = conf.getFloat(MEMORY_FACTOR_CONFIG_NAME, DEFAULT_MEMORY_FACTOR);
+
+    sanityCheckConfigs();
+
+    LOG.info("Instantiating BucketCache with acceptableFactor: " + acceptableFactor + ", minFactor: " + minFactor +
+        ", extraFreeFactor: " + extraFreeFactor + ", singleFactor: " + singleFactor + ", multiFactor: " + multiFactor +
+        ", memoryFactor: " + memoryFactor);
+
     this.cacheCapacity = capacity;
     this.persistencePath = persistencePath;
     this.blockSize = blockSize;
@@ -281,6 +325,18 @@ public class BucketCache implements BlockCache, HeapSize {
       persistencePath + ", bucketAllocator=" + this.bucketAllocator.getClass().getName());
   }
 
+  private void sanityCheckConfigs() {
+    Preconditions.checkArgument(acceptableFactor <= 1 && acceptableFactor >= 0, ACCEPT_FACTOR_CONFIG_NAME + " must be between 0.0 and 1.0");
+    Preconditions.checkArgument(minFactor <= 1 && minFactor >= 0, MIN_FACTOR_CONFIG_NAME + " must be between 0.0 and 1.0");
+    Preconditions.checkArgument(minFactor <= acceptableFactor, MIN_FACTOR_CONFIG_NAME + " must be <= " + ACCEPT_FACTOR_CONFIG_NAME);
+    Preconditions.checkArgument(extraFreeFactor >= 0, EXTRA_FREE_FACTOR_CONFIG_NAME + " must be greater than 0.0");
+    Preconditions.checkArgument(singleFactor <= 1 && singleFactor >= 0, SINGLE_FACTOR_CONFIG_NAME + " must be between 0.0 and 1.0");
+    Preconditions.checkArgument(multiFactor <= 1 && multiFactor >= 0, MULTI_FACTOR_CONFIG_NAME + " must be between 0.0 and 1.0");
+    Preconditions.checkArgument(memoryFactor <= 1 && memoryFactor >= 0, MEMORY_FACTOR_CONFIG_NAME + " must be between 0.0 and 1.0");
+    Preconditions.checkArgument((singleFactor + multiFactor + memoryFactor) == 1, SINGLE_FACTOR_CONFIG_NAME + ", " +
+        MULTI_FACTOR_CONFIG_NAME + ", and " + MEMORY_FACTOR_CONFIG_NAME + " segments must add up to 1.0");
+  }
+
   /**
    * Called by the constructor to start the writer threads. Used by tests that need to override
    * starting the threads.
@@ -623,26 +679,16 @@ public class BucketCache implements BlockCache, HeapSize {
   }
 
   private long acceptableSize() {
-    return (long) Math.floor(bucketAllocator.getTotalSize() * DEFAULT_ACCEPT_FACTOR);
-  }
-
-  private long singleSize() {
-    return (long) Math.floor(bucketAllocator.getTotalSize()
-        * DEFAULT_SINGLE_FACTOR * DEFAULT_MIN_FACTOR);
-  }
-
-  private long multiSize() {
-    return (long) Math.floor(bucketAllocator.getTotalSize() * DEFAULT_MULTI_FACTOR
-        * DEFAULT_MIN_FACTOR);
+    return (long) Math.floor(bucketAllocator.getTotalSize() * acceptableFactor);
   }
 
-  private long memorySize() {
-    return (long) Math.floor(bucketAllocator.getTotalSize() * DEFAULT_MEMORY_FACTOR
-        * DEFAULT_MIN_FACTOR);
+  @VisibleForTesting
+  long getPartitionSize(float partitionFactor) {
+    return (long) Math.floor(bucketAllocator.getTotalSize() * partitionFactor * minFactor);
   }
 
   /**
-   * Return the count of bucketSizeinfos still needf ree space
+   * Return the count of bucketSizeinfos still need free space
    */
   private int bucketSizesAboveThresholdCount(float minFactor) {
     BucketAllocator.IndexStatistics[] stats = bucketAllocator.getIndexStatistics();
@@ -708,7 +754,7 @@ public class BucketCache implements BlockCache, HeapSize {
       long[] bytesToFreeForBucket = new long[stats.length];
       for (int i = 0; i < stats.length; i++) {
         bytesToFreeForBucket[i] = 0;
-        long freeGoal = (long) Math.floor(stats[i].totalCount() * (1 - DEFAULT_MIN_FACTOR));
+        long freeGoal = (long) Math.floor(stats[i].totalCount() * (1 - minFactor));
         freeGoal = Math.max(freeGoal, 1);
         if (stats[i].freeCount() < freeGoal) {
           bytesToFreeForBucket[i] = stats[i].itemSize() * (freeGoal - stats[i].freeCount());
@@ -735,15 +781,15 @@ public class BucketCache implements BlockCache, HeapSize {
       }
 
       long bytesToFreeWithExtra = (long) Math.floor(bytesToFreeWithoutExtra
-          * (1 + DEFAULT_EXTRA_FREE_FACTOR));
+          * (1 + extraFreeFactor));
 
       // Instantiate priority buckets
       BucketEntryGroup bucketSingle = new BucketEntryGroup(bytesToFreeWithExtra,
-          blockSize, singleSize());
+          blockSize, getPartitionSize(singleFactor));
       BucketEntryGroup bucketMulti = new BucketEntryGroup(bytesToFreeWithExtra,
-          blockSize, multiSize());
+          blockSize, getPartitionSize(multiFactor));
       BucketEntryGroup bucketMemory = new BucketEntryGroup(bytesToFreeWithExtra,
-          blockSize, memorySize());
+          blockSize, getPartitionSize(memoryFactor));
 
       // Scan entire map putting bucket entry into appropriate bucket entry
       // group
@@ -785,7 +831,7 @@ public class BucketCache implements BlockCache, HeapSize {
       }
 
       // Check and free if there are buckets that still need freeing of space
-      if (bucketSizesAboveThresholdCount(DEFAULT_MIN_FACTOR) > 0) {
+      if (bucketSizesAboveThresholdCount(minFactor) > 0) {
         bucketQueue.clear();
         remainingBuckets = 3;
 
@@ -1532,4 +1578,28 @@ public class BucketCache implements BlockCache, HeapSize {
     }
     return 0;
   }
+
+  float getAcceptableFactor() {
+    return acceptableFactor;
+  }
+
+  float getMinFactor() {
+    return minFactor;
+  }
+
+  float getExtraFreeFactor() {
+    return extraFreeFactor;
+  }
+
+  float getSingleFactor() {
+    return singleFactor;
+  }
+
+  float getMultiFactor() {
+    return multiFactor;
+  }
+
+  float getMemoryFactor() {
+    return memoryFactor;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/0e32869f/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCache.java
index b0db13a..8cd665e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCache.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCache.java
@@ -19,17 +19,25 @@
 package org.apache.hadoop.hbase.io.hfile.bucket;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.List;
+import java.util.Map;
 import java.util.Random;
+import java.util.Set;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
+import com.google.common.collect.ImmutableMap;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.io.hfile.BlockCacheKey;
 import org.apache.hadoop.hbase.io.hfile.CacheTestUtils;
@@ -272,8 +280,112 @@ public class TestBucketCache {
   @Test
   public void testBucketAllocatorLargeBuckets() throws BucketAllocatorException {
     long availableSpace = 20 * 1024L * 1024 * 1024;
-    int[] bucketSizes = new int[] { 1024, 1024 * 1024, 1024 * 1024 * 1024 };
+    int[] bucketSizes = new int[]{1024, 1024 * 1024, 1024 * 1024 * 1024};
     BucketAllocator allocator = new BucketAllocator(availableSpace, bucketSizes);
     assertTrue(allocator.getBuckets().length > 0);
   }
+
+  @Test
+  public void testGetPartitionSize() throws IOException {
+    //Test default values
+    validateGetPartitionSize(cache, BucketCache.DEFAULT_SINGLE_FACTOR, BucketCache.DEFAULT_MIN_FACTOR);
+
+    Configuration conf = HBaseConfiguration.create();
+    conf.setFloat(BucketCache.MIN_FACTOR_CONFIG_NAME, 0.5f);
+    conf.setFloat(BucketCache.SINGLE_FACTOR_CONFIG_NAME, 0.1f);
+    conf.setFloat(BucketCache.MULTI_FACTOR_CONFIG_NAME, 0.7f);
+    conf.setFloat(BucketCache.MEMORY_FACTOR_CONFIG_NAME, 0.2f);
+
+    BucketCache cache = new BucketCache(ioEngineName, capacitySize, constructedBlockSize,
+        constructedBlockSizes, writeThreads, writerQLen, persistencePath, 100, conf);
+
+    validateGetPartitionSize(cache, 0.1f, 0.5f);
+    validateGetPartitionSize(cache, 0.7f, 0.5f);
+    validateGetPartitionSize(cache, 0.2f, 0.5f);
+  }
+
+  @Test
+  public void testValidBucketCacheConfigs() throws IOException {
+    Configuration conf = HBaseConfiguration.create();
+    conf.setFloat(BucketCache.ACCEPT_FACTOR_CONFIG_NAME, 0.9f);
+    conf.setFloat(BucketCache.MIN_FACTOR_CONFIG_NAME, 0.5f);
+    conf.setFloat(BucketCache.EXTRA_FREE_FACTOR_CONFIG_NAME, 0.5f);
+    conf.setFloat(BucketCache.SINGLE_FACTOR_CONFIG_NAME, 0.1f);
+    conf.setFloat(BucketCache.MULTI_FACTOR_CONFIG_NAME, 0.7f);
+    conf.setFloat(BucketCache.MEMORY_FACTOR_CONFIG_NAME, 0.2f);
+
+    BucketCache cache = new BucketCache(ioEngineName, capacitySize, constructedBlockSize,
+        constructedBlockSizes, writeThreads, writerQLen, persistencePath, 100, conf);
+
+    assertEquals(BucketCache.ACCEPT_FACTOR_CONFIG_NAME + " failed to propagate.", cache.getAcceptableFactor(), 0.9f, 0);
+    assertEquals(BucketCache.MIN_FACTOR_CONFIG_NAME + " failed to propagate.", cache.getMinFactor(), 0.5f, 0);
+    assertEquals(BucketCache.EXTRA_FREE_FACTOR_CONFIG_NAME + " failed to propagate.", cache.getExtraFreeFactor(), 0.5f, 0);
+    assertEquals(BucketCache.SINGLE_FACTOR_CONFIG_NAME + " failed to propagate.", cache.getSingleFactor(), 0.1f, 0);
+    assertEquals(BucketCache.MULTI_FACTOR_CONFIG_NAME + " failed to propagate.", cache.getMultiFactor(), 0.7f, 0);
+    assertEquals(BucketCache.MEMORY_FACTOR_CONFIG_NAME + " failed to propagate.", cache.getMemoryFactor(), 0.2f, 0);
+  }
+
+  @Test
+  public void testInvalidAcceptFactorConfig() throws IOException {
+    float[] configValues = {-1f, 0.2f, 0.86f, 1.05f};
+    boolean[] expectedOutcomes = {false, false, true, false};
+    Map<String, float[]> configMappings = ImmutableMap.of(BucketCache.ACCEPT_FACTOR_CONFIG_NAME, configValues);
+    Configuration conf = HBaseConfiguration.create();
+    checkConfigValues(conf, configMappings, expectedOutcomes);
+  }
+
+  @Test
+  public void testInvalidMinFactorConfig() throws IOException {
+    float[] configValues = {-1f, 0f, 0.96f, 1.05f};
+    //throws due to <0, in expected range, minFactor > acceptableFactor, > 1.0
+    boolean[] expectedOutcomes = {false, true, false, false};
+    Map<String, float[]> configMappings = ImmutableMap.of(BucketCache.MIN_FACTOR_CONFIG_NAME, configValues);
+    Configuration conf = HBaseConfiguration.create();
+    checkConfigValues(conf, configMappings, expectedOutcomes);
+  }
+
+  @Test
+  public void testInvalidExtraFreeFactorConfig() throws IOException {
+    float[] configValues = {-1f, 0f, 0.2f, 1.05f};
+    //throws due to <0, in expected range, in expected range, config can be > 1.0
+    boolean[] expectedOutcomes = {false, true, true, true};
+    Map<String, float[]> configMappings = ImmutableMap.of(BucketCache.EXTRA_FREE_FACTOR_CONFIG_NAME, configValues);
+    Configuration conf = HBaseConfiguration.create();
+    checkConfigValues(conf, configMappings, expectedOutcomes);
+  }
+
+  @Test
+  public void testInvalidCacheSplitFactorConfig() throws IOException {
+    float[] singleFactorConfigValues = {0.2f, 0f, -0.2f, 1f};
+    float[] multiFactorConfigValues = {0.4f, 0f, 1f, .05f};
+    float[] memoryFactorConfigValues = {0.4f, 0f, 0.2f, .5f};
+    // All configs add up to 1.0 and are between 0 and 1.0, configs don't add to 1.0, configs can't be negative, configs don't add to 1.0
+    boolean[] expectedOutcomes = {true, false, false, false};
+    Map<String, float[]> configMappings = ImmutableMap.of(BucketCache.SINGLE_FACTOR_CONFIG_NAME,
+        singleFactorConfigValues, BucketCache.MULTI_FACTOR_CONFIG_NAME, multiFactorConfigValues,
+        BucketCache.MEMORY_FACTOR_CONFIG_NAME, memoryFactorConfigValues);
+    Configuration conf = HBaseConfiguration.create();
+    checkConfigValues(conf, configMappings, expectedOutcomes);
+  }
+
+  private void checkConfigValues(Configuration conf, Map<String, float[]> configMap, boolean[] expectSuccess) throws IOException {
+    Set<String> configNames = configMap.keySet();
+    for (int i = 0; i < expectSuccess.length; i++) {
+      try {
+        for (String configName : configNames) {
+          conf.setFloat(configName, configMap.get(configName)[i]);
+        }
+        BucketCache cache = new BucketCache(ioEngineName, capacitySize, constructedBlockSize,
+            constructedBlockSizes, writeThreads, writerQLen, persistencePath, 100, conf);
+        assertTrue("Created BucketCache and expected it to succeed: " + expectSuccess[i] + ", but it actually was: " + !expectSuccess[i], expectSuccess[i]);
+      } catch (IllegalArgumentException e) {
+        assertFalse("Created BucketCache and expected it to succeed: " + expectSuccess[i] + ", but it actually was: " + !expectSuccess[i], expectSuccess[i]);
+      }
+    }
+  }
+
+  private void validateGetPartitionSize(BucketCache bucketCache, float partitionFactor, float minFactor) {
+    long expectedOutput = (long) Math.floor(bucketCache.getAllocator().getTotalSize() * partitionFactor * minFactor);
+    assertEquals(expectedOutput, bucketCache.getPartitionSize(partitionFactor));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/0e32869f/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketWriterThread.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketWriterThread.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketWriterThread.java
index cfba69a..4f6ffd2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketWriterThread.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketWriterThread.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hbase.io.hfile.bucket;
 
 import org.apache.hadoop.hbase.testclassification.IOTests;
+import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.io.hfile.BlockCacheKey;
 import org.apache.hadoop.hbase.io.hfile.Cacheable;
@@ -57,7 +58,7 @@ public class TestBucketWriterThread {
       int writerThreadNum, int writerQLen, String persistencePath, int ioErrorsTolerationDuration)
       throws FileNotFoundException, IOException {
       super(ioEngineName, capacity, blockSize, bucketSizes, writerThreadNum, writerQLen,
-        persistencePath, ioErrorsTolerationDuration);
+        persistencePath, ioErrorsTolerationDuration, HBaseConfiguration.create());
     }
 
     @Override


[37/50] [abbrv] hbase git commit: HBASE-18424 Fix TestAsyncTableGetMultiThreaded

Posted by st...@apache.org.
HBASE-18424 Fix TestAsyncTableGetMultiThreaded

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/665fd0d0
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/665fd0d0
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/665fd0d0

Branch: refs/heads/HBASE-14070.HLC
Commit: 665fd0d07e34141c2765f02398eb1ad9e376f32f
Parents: 5280c10
Author: Vladimir Rodionov <vl...@gmail.com>
Authored: Wed Aug 16 11:29:34 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Wed Aug 16 11:29:34 2017 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/client/TestAsyncTableGetMultiThreaded.java  | 8 +++++---
 1 file changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/665fd0d0/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableGetMultiThreaded.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableGetMultiThreaded.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableGetMultiThreaded.java
index 2abc54d..225060b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableGetMultiThreaded.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableGetMultiThreaded.java
@@ -37,7 +37,11 @@ import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 
 import org.apache.commons.io.IOUtils;
-import org.apache.hadoop.hbase.*;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.MemoryCompactionPolicy;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.io.ByteBufferPool;
 import org.apache.hadoop.hbase.regionserver.CompactingMemStore;
 import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -47,14 +51,12 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Threads;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 /**
  * Will split the table, and move region randomly when testing.
  */
-@Ignore // Can't move hbase:meta off master server in AMv2. TODO.
 @Category({ LargeTests.class, ClientTests.class })
 public class TestAsyncTableGetMultiThreaded {
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();


[32/50] [abbrv] hbase git commit: HBASE-18599 Add missing @Deprecated annotations

Posted by st...@apache.org.
HBASE-18599 Add missing @Deprecated annotations

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/70c4f78c
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/70c4f78c
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/70c4f78c

Branch: refs/heads/HBASE-14070.HLC
Commit: 70c4f78ce03cf9e13d148e75445b19d43571a09a
Parents: effd109
Author: Lars Francke <la...@gmail.com>
Authored: Tue Aug 15 09:36:51 2017 +0200
Committer: Michael Stack <st...@apache.org>
Committed: Tue Aug 15 10:44:50 2017 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hbase/HColumnDescriptor.java  | 17 +++++++++--
 .../apache/hadoop/hbase/HTableDescriptor.java   |  8 +++--
 .../org/apache/hadoop/hbase/client/Admin.java   | 32 +++++++++++++++-----
 .../apache/hadoop/hbase/client/AsyncAdmin.java  |  5 ++-
 .../client/metrics/ServerSideScanMetrics.java   | 15 +++++++--
 .../hbase/coprocessor/RegionObserver.java       |  5 ++-
 6 files changed, 65 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/70c4f78c/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
index 5fe85cc..507bf49 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
@@ -100,14 +100,18 @@ public class HColumnDescriptor implements ColumnFamilyDescriptor, Comparable<HCo
   public static final String NEW_VERSION_BEHAVIOR = ColumnFamilyDescriptorBuilder.NEW_VERSION_BEHAVIOR;
   public static final boolean DEFAULT_NEW_VERSION_BEHAVIOR = ColumnFamilyDescriptorBuilder.DEFAULT_NEW_VERSION_BEHAVIOR;
   protected final ModifyableColumnFamilyDescriptor delegatee;
+
   /**
    * Construct a column descriptor specifying only the family name
    * The other attributes are defaulted.
    *
    * @param familyName Column family name. Must be 'printable' -- digit or
    * letter -- and may not contain a <code>:</code>
-   * @deprecated use {@link ColumnFamilyDescriptorBuilder#of(String)}
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             (<a href="https://issues.apache.org/jira/browse/HBASE-18433">HBASE-18433</a>).
+   *             Use {@link ColumnFamilyDescriptorBuilder#of(String)}.
    */
+  @Deprecated
   public HColumnDescriptor(final String familyName) {
     this(Bytes.toBytes(familyName));
   }
@@ -118,8 +122,11 @@ public class HColumnDescriptor implements ColumnFamilyDescriptor, Comparable<HCo
    *
    * @param familyName Column family name. Must be 'printable' -- digit or
    * letter -- and may not contain a <code>:</code>
-   * @deprecated use {@link ColumnFamilyDescriptorBuilder#of(byte[])}
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             (<a href="https://issues.apache.org/jira/browse/HBASE-18433">HBASE-18433</a>).
+   *             Use {@link ColumnFamilyDescriptorBuilder#of(byte[])}.
    */
+  @Deprecated
   public HColumnDescriptor(final byte [] familyName) {
     this(new ModifyableColumnFamilyDescriptor(familyName));
   }
@@ -128,9 +135,13 @@ public class HColumnDescriptor implements ColumnFamilyDescriptor, Comparable<HCo
    * Constructor.
    * Makes a deep copy of the supplied descriptor.
    * Can make a modifiable descriptor from an UnmodifyableHColumnDescriptor.
+   *
    * @param desc The descriptor.
-   * @deprecated use {@link ColumnFamilyDescriptorBuilder#copy(ColumnFamilyDescriptor)}
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             (<a href="https://issues.apache.org/jira/browse/HBASE-18433">HBASE-18433</a>).
+   *             Use {@link ColumnFamilyDescriptorBuilder#copy(ColumnFamilyDescriptor)}.
    */
+  @Deprecated
   public HColumnDescriptor(HColumnDescriptor desc) {
     this(desc, true);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/70c4f78c/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
index c09d434..a0f23c1 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
@@ -44,7 +44,7 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder.ModifyableCo
  * if the table is read only, the maximum size of the memstore,
  * when the region split should occur, coprocessors associated with it etc...
  * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
- *             use {@link TableDescriptorBuilder} to build {@link HTableDescriptor}.
+ *             Use {@link TableDescriptorBuilder} to build {@link HTableDescriptor}.
  */
 @Deprecated
 @InterfaceAudience.Public
@@ -602,9 +602,13 @@ public class HTableDescriptor implements TableDescriptor, Comparable<HTableDescr
    * HTableDescriptor contains mapping of family name to HColumnDescriptors.
    * This returns all the keys of the family map which represents the column
    * family names of the table.
+   *
    * @return Immutable sorted set of the keys of the families.
-   * @deprecated Use {@link #getColumnFamilyNames()}.
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             (<a href="https://issues.apache.org/jira/browse/HBASE-18008">HBASE-18008</a>).
+   *             Use {@link #getColumnFamilyNames()}.
    */
+  @Deprecated
   public Set<byte[]> getFamiliesKeys() {
     return delegatee.getColumnFamilyNames();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/70c4f78c/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
index d2acae3..8de9f89 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
@@ -833,8 +833,11 @@ public interface Admin extends Abortable, Closeable {
    * @param regionname region name to close
    * @param serverName Deprecated. Not used.
    * @throws IOException if a remote or network exception occurs
-   * @deprecated Since 2.0. Will be removed in 3.0. Use {@link #unassign(byte[], boolean)} instead.
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             (<a href="https://issues.apache.org/jira/browse/HBASE-18231">HBASE-18231</a>).
+   *             Use {@link #unassign(byte[], boolean)}.
    */
+  @Deprecated
   void closeRegion(final String regionname, final String serverName) throws IOException;
 
   /**
@@ -843,8 +846,11 @@ public interface Admin extends Abortable, Closeable {
    * @param regionname region name to close
    * @param serverName Deprecated. Not used.
    * @throws IOException if a remote or network exception occurs
-   * @deprecated Since 2.0. Will be removed in 3.0. Use {@link #unassign(byte[], boolean)} instead.
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             (<a href="https://issues.apache.org/jira/browse/HBASE-18231">HBASE-18231</a>).
+   *             Use {@link #unassign(byte[], boolean)}.
    */
+  @Deprecated
   void closeRegion(final byte[] regionname, final String serverName) throws IOException;
 
   /**
@@ -857,8 +863,11 @@ public interface Admin extends Abortable, Closeable {
    * @param serverName Deprecated. Not used.
    * @return Deprecated. Returns true always.
    * @throws IOException if a remote or network exception occurs
-   * @deprecated Since 2.0. Will be removed in 3.0. Use {@link #unassign(byte[], boolean)} instead.
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             (<a href="https://issues.apache.org/jira/browse/HBASE-18231">HBASE-18231</a>).
+   *             Use {@link #unassign(byte[], boolean)}.
    */
+  @Deprecated
   boolean closeRegionWithEncodedRegionName(final String encodedRegionName, final String serverName)
       throws IOException;
 
@@ -866,8 +875,11 @@ public interface Admin extends Abortable, Closeable {
    * Used {@link #unassign(byte[], boolean)} to unassign the region. For expert-admins.
    *
    * @param sn Deprecated. Not used.
-   * @deprecated Since 2.0. Will be removed in 3.0. Use {@link #unassign(byte[], boolean)} instead.
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             (<a href="https://issues.apache.org/jira/browse/HBASE-18231">HBASE-18231</a>).
+   *             Use {@link #unassign(byte[], boolean)}.
    */
+  @Deprecated
   void closeRegion(final ServerName sn, final HRegionInfo hri) throws IOException;
 
   /**
@@ -1175,9 +1187,11 @@ public interface Admin extends Abortable, Closeable {
    *
    * @param regionName region to split
    * @throws IOException if a remote or network exception occurs
-   * @deprecated Since 2.0. Will be removed in 3.0. Use
-   *     {@link #splitRegionAsync(byte[], byte[])} instead.
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             (<a href="https://issues.apache.org/jira/browse/HBASE-18229">HBASE-18229</a>).
+   *             Use {@link #splitRegionAsync(byte[], byte[])}.
    */
+  @Deprecated
   void splitRegion(final byte[] regionName) throws IOException;
 
   /**
@@ -1196,9 +1210,11 @@ public interface Admin extends Abortable, Closeable {
    * @param regionName region to split
    * @param splitPoint the explicit position to split on
    * @throws IOException if a remote or network exception occurs
-   * @deprecated Since 2.0. Will be removed in 3.0. Use
-   *     {@link #splitRegionAsync(byte[], byte[])} instead.
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             (<a href="https://issues.apache.org/jira/browse/HBASE-18229">HBASE-18229</a>).
+   *             Use {@link #splitRegionAsync(byte[], byte[])}.
    */
+  @Deprecated
   void splitRegion(final byte[] regionName, final byte[] splitPoint)
     throws IOException;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/70c4f78c/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
index f2f2bf1..4b33812 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
@@ -296,8 +296,11 @@ public interface AsyncAdmin {
    * @param regionName region name to close
    * @param serverName Deprecated. Not used anymore after deprecation.
    * @return Deprecated. Always returns true now.
-   * @deprecated Since 2.0. Will be removed in 3.0. Use {@link #unassign(byte[], boolean)} instead.
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             (<a href="https://issues.apache.org/jira/browse/HBASE-18231">HBASE-18231</a>).
+   *             Use {@link #unassign(byte[], boolean)}.
    */
+  @Deprecated
   CompletableFuture<Boolean> closeRegion(byte[] regionName, Optional<ServerName> serverName);
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/70c4f78c/hbase-client/src/main/java/org/apache/hadoop/hbase/client/metrics/ServerSideScanMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/metrics/ServerSideScanMetrics.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/metrics/ServerSideScanMetrics.java
index 2f73a0e..33d1278 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/metrics/ServerSideScanMetrics.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/metrics/ServerSideScanMetrics.java
@@ -49,9 +49,20 @@ public class ServerSideScanMetrics {
   public static final String COUNT_OF_ROWS_SCANNED_KEY_METRIC_NAME = "ROWS_SCANNED";
   public static final String COUNT_OF_ROWS_FILTERED_KEY_METRIC_NAME = "ROWS_FILTERED";
 
-  /** @deprecated Use {@link #COUNT_OF_ROWS_SCANNED_KEY_METRIC_NAME} instead */
+  /**
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             (<a href="https://issues.apache.org/jira/browse/HBASE-17886">HBASE-17886</a>).
+   *             Use {@link #COUNT_OF_ROWS_SCANNED_KEY_METRIC_NAME}.
+   */
+  @Deprecated
   public static final String COUNT_OF_ROWS_SCANNED_KEY = COUNT_OF_ROWS_SCANNED_KEY_METRIC_NAME;
-  /** @deprecated Use {@link #COUNT_OF_ROWS_FILTERED_KEY_METRIC_NAME} instead */
+
+  /**
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             (<a href="https://issues.apache.org/jira/browse/HBASE-17886">HBASE-17886</a>).
+   *             Use {@link #COUNT_OF_ROWS_FILTERED_KEY_METRIC_NAME}.
+   */
+  @Deprecated
   public static final String COUNT_OF_ROWS_FILTERED_KEY = COUNT_OF_ROWS_FILTERED_KEY_METRIC_NAME;
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/70c4f78c/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
index da78a09..4f997c2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
@@ -1437,8 +1437,11 @@ public interface RegionObserver extends Coprocessor {
    * @param stagingFamilyPaths pairs of { CF, HFile path } submitted for bulk load
    * @param hasLoaded whether the bulkLoad was successful
    * @return the new value of hasLoaded
-   * @deprecated Use {@link #postBulkLoadHFile(ObserverContext, List, Map, boolean)}
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             (<a href="https://issues.apache.org/jira/browse/HBASE-17123">HBASE-17123</a>).
+   *             Use {@link #postBulkLoadHFile(ObserverContext, List, Map, boolean)}.
    */
+  @Deprecated
   default boolean postBulkLoadHFile(final ObserverContext<RegionCoprocessorEnvironment> ctx,
     List<Pair<byte[], String>> stagingFamilyPaths, boolean hasLoaded) throws IOException {
     return hasLoaded;


[47/50] [abbrv] hbase git commit: HBASE-18125 shell disregards spaces at the end of a split key in a split file

Posted by st...@apache.org.
HBASE-18125 shell disregards spaces at the end of a split key in a split file

Signed-off-by: fchenxi <fc...@126.com>
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/a17ed035
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/a17ed035
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/a17ed035

Branch: refs/heads/HBASE-14070.HLC
Commit: a17ed0356f12c6f7a682557d92cc401b7a4297f1
Parents: 4c3a64d
Author: Chenxi Tong <fc...@126.com>
Authored: Wed Aug 9 12:00:53 2017 +0800
Committer: Michael Stack <st...@apache.org>
Committed: Wed Aug 16 15:14:12 2017 -0700

----------------------------------------------------------------------
 hbase-shell/src/main/ruby/hbase/admin.rb | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a17ed035/hbase-shell/src/main/ruby/hbase/admin.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb
index 4b0de5f..2aacd7f 100644
--- a/hbase-shell/src/main/ruby/hbase/admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/admin.rb
@@ -386,7 +386,7 @@ module Hbase
           end
           arg[SPLITS] = []
           File.foreach(splits_file) do |line|
-            arg[SPLITS].push(line.strip)
+            arg[SPLITS].push(line.chomp)
           end
           htd.setValue(SPLITS_FILE, arg[SPLITS_FILE])
         end


[38/50] [abbrv] hbase git commit: HBASE-18598 AsyncNonMetaRegionLocator use FIFO algorithm to get a candidate locate request

Posted by st...@apache.org.
HBASE-18598 AsyncNonMetaRegionLocator use FIFO algorithm to get a candidate locate request


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

Branch: refs/heads/HBASE-14070.HLC
Commit: 59ffb6119b2e4613bc8baec9a0738096184a3d92
Parents: 665fd0d
Author: Guanghao Zhang <zg...@apache.org>
Authored: Tue Aug 15 16:15:29 2017 +0800
Committer: Guanghao Zhang <zg...@apache.org>
Committed: Wed Aug 16 13:08:40 2017 +0800

----------------------------------------------------------------------
 .../hbase/client/AsyncNonMetaRegionLocator.java | 119 ++++++++++---------
 .../client/TestAsyncNonMetaRegionLocator.java   |   1 +
 2 files changed, 63 insertions(+), 57 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/59ffb611/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
index 31f369c..ab1f0db 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
@@ -29,18 +29,18 @@ import static org.apache.hadoop.hbase.util.CollectionUtils.computeIfAbsent;
 
 import java.io.IOException;
 import java.util.Arrays;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
+import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ConcurrentNavigableMap;
 import java.util.concurrent.ConcurrentSkipListMap;
-import java.util.concurrent.ThreadLocalRandom;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -107,7 +107,7 @@ class AsyncNonMetaRegionLocator {
     public final Set<LocateRequest> pendingRequests = new HashSet<>();
 
     public final Map<LocateRequest, CompletableFuture<HRegionLocation>> allRequests =
-        new HashMap<>();
+        new LinkedHashMap<>();
 
     public boolean hasQuota(int max) {
       return pendingRequests.size() < max;
@@ -120,6 +120,49 @@ class AsyncNonMetaRegionLocator {
     public void send(LocateRequest req) {
       pendingRequests.add(req);
     }
+
+    public Optional<LocateRequest> getCandidate() {
+      return allRequests.keySet().stream().filter(r -> !isPending(r)).findFirst();
+    }
+
+    public void clearCompletedRequests(Optional<HRegionLocation> location) {
+      for (Iterator<Map.Entry<LocateRequest, CompletableFuture<HRegionLocation>>> iter = allRequests
+          .entrySet().iterator(); iter.hasNext();) {
+        Map.Entry<LocateRequest, CompletableFuture<HRegionLocation>> entry = iter.next();
+        if (tryComplete(entry.getKey(), entry.getValue(), location)) {
+          iter.remove();
+        }
+      }
+    }
+
+    private boolean tryComplete(LocateRequest req, CompletableFuture<HRegionLocation> future,
+        Optional<HRegionLocation> location) {
+      if (future.isDone()) {
+        return true;
+      }
+      if (!location.isPresent()) {
+        return false;
+      }
+      HRegionLocation loc = location.get();
+      boolean completed;
+      if (req.locateType.equals(RegionLocateType.BEFORE)) {
+        // for locating the row before current row, the common case is to find the previous region in
+        // reverse scan, so we check the endKey first. In general, the condition should be startKey <
+        // req.row and endKey >= req.row. Here we split it to endKey == req.row || (endKey > req.row
+        // && startKey < req.row). The two conditions are equal since startKey < endKey.
+        int c = Bytes.compareTo(loc.getRegionInfo().getEndKey(), req.row);
+        completed =
+            c == 0 || (c > 0 && Bytes.compareTo(loc.getRegionInfo().getStartKey(), req.row) < 0);
+      } else {
+        completed = loc.getRegionInfo().containsRow(req.row);
+      }
+      if (completed) {
+        future.complete(loc);
+        return true;
+      } else {
+        return false;
+      }
+    }
   }
 
   AsyncNonMetaRegionLocator(AsyncConnectionImpl conn) {
@@ -186,48 +229,27 @@ class AsyncNonMetaRegionLocator {
     }
   }
 
-  private boolean tryComplete(LocateRequest req, CompletableFuture<HRegionLocation> future,
-      HRegionLocation loc) {
-    if (future.isDone()) {
-      return true;
-    }
-    boolean completed;
-    if (req.locateType.equals(RegionLocateType.BEFORE)) {
-      // for locating the row before current row, the common case is to find the previous region in
-      // reverse scan, so we check the endKey first. In general, the condition should be startKey <
-      // req.row and endKey >= req.row. Here we split it to endKey == req.row || (endKey > req.row
-      // && startKey < req.row). The two conditions are equal since startKey < endKey.
-      int c = Bytes.compareTo(loc.getRegionInfo().getEndKey(), req.row);
-      completed =
-          c == 0 || (c > 0 && Bytes.compareTo(loc.getRegionInfo().getStartKey(), req.row) < 0);
-    } else {
-      completed = loc.getRegionInfo().containsRow(req.row);
-    }
-    if (completed) {
-      future.complete(loc);
-      return true;
-    } else {
-      return false;
-    }
-  }
-
   private void complete(TableName tableName, LocateRequest req, HRegionLocation loc,
       Throwable error) {
     if (error != null) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Failed to locate region in '" + tableName + "', row='" +
-            Bytes.toStringBinary(req.row) + "', locateType=" + req.locateType,
-          error);
-      }
+      LOG.warn(
+        "Failed to locate region in '" + tableName + "', row='" + Bytes.toStringBinary(req.row)
+            + "', locateType=" + req.locateType, error);
     }
-    LocateRequest toSend = null;
+    Optional<LocateRequest> toSend = Optional.empty();
     TableCache tableCache = getTableCache(tableName);
     if (loc != null) {
       if (!addToCache(tableCache, loc)) {
         // someone is ahead of us.
         synchronized (tableCache) {
           tableCache.pendingRequests.remove(req);
+          tableCache.clearCompletedRequests(Optional.empty());
+          // Remove a complete locate request in a synchronized block, so the table cache must have
+          // quota to send a candidate request.
+          toSend = tableCache.getCandidate();
+          toSend.ifPresent(r -> tableCache.send(r));
         }
+        toSend.ifPresent(r -> locateInMeta(tableName, r));
         return;
       }
     }
@@ -239,30 +261,13 @@ class AsyncNonMetaRegionLocator {
           future.completeExceptionally(error);
         }
       }
-      if (loc != null) {
-        for (Iterator<Map.Entry<LocateRequest, CompletableFuture<HRegionLocation>>> iter =
-            tableCache.allRequests.entrySet().iterator(); iter.hasNext();) {
-          Map.Entry<LocateRequest, CompletableFuture<HRegionLocation>> entry = iter.next();
-          if (tryComplete(entry.getKey(), entry.getValue(), loc)) {
-            iter.remove();
-          }
-        }
-      }
-      if (!tableCache.allRequests.isEmpty() &&
-          tableCache.hasQuota(maxConcurrentLocateRequestPerTable)) {
-        LocateRequest[] candidates = tableCache.allRequests.keySet().stream()
-            .filter(r -> !tableCache.isPending(r)).toArray(LocateRequest[]::new);
-        if (candidates.length > 0) {
-          // TODO: use a better algorithm to send a request which is more likely to fetch a new
-          // location.
-          toSend = candidates[ThreadLocalRandom.current().nextInt(candidates.length)];
-          tableCache.send(toSend);
-        }
-      }
-    }
-    if (toSend != null) {
-      locateInMeta(tableName, toSend);
+      tableCache.clearCompletedRequests(Optional.ofNullable(loc));
+      // Remove a complete locate request in a synchronized block, so the table cache must have
+      // quota to send a candidate request.
+      toSend = tableCache.getCandidate();
+      toSend.ifPresent(r -> tableCache.send(r));
     }
+    toSend.ifPresent(r -> locateInMeta(tableName, r));
   }
 
   private void onScanComplete(TableName tableName, LocateRequest req, List<Result> results,

http://git-wip-us.apache.org/repos/asf/hbase/blob/59ffb611/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java
index 0bb192b..80ed02e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java
@@ -209,6 +209,7 @@ public class TestAsyncNonMetaRegionLocator {
         throw new RuntimeException(e);
       }
     }));
+
     LOCATOR.clearCache(TABLE_NAME);
     byte[][] endKeys = getEndKeys();
     IntStream.range(0, 2).forEach(


[29/50] [abbrv] hbase git commit: HBASE-18582 Correct the docs for Mutation#setCellVisibility

Posted by st...@apache.org.
HBASE-18582 Correct the docs for Mutation#setCellVisibility

Signed-off-by: Chia-Ping Tsai <ch...@gmail.com>


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

Branch: refs/heads/HBASE-14070.HLC
Commit: d4317c80e62e4eb0c2e997adf4438b927dfbcd96
Parents: 0e32869
Author: brandboat <br...@gmail.com>
Authored: Mon Aug 14 22:10:23 2017 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Tue Aug 15 21:37:55 2017 +0800

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hbase/client/Mutation.java      | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d4317c80/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
index 25b088d..24b4cb8 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
@@ -331,7 +331,6 @@ public abstract class Mutation extends OperationWithAttributes implements Row, C
 
   /**
    * Sets the visibility expression associated with cells in this Mutation.
-   * It is illegal to set <code>CellVisibility</code> on <code>Delete</code> mutation.
    * @param expression
    */
   public Mutation setCellVisibility(CellVisibility expression) {


[04/50] [abbrv] hbase git commit: HBASE-17125 Inconsistent result when use filter to read data

Posted by st...@apache.org.
HBASE-17125 Inconsistent result when use filter to read data


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

Branch: refs/heads/HBASE-14070.HLC
Commit: 4dd24c52b84c74a477e00ab6177d081c29462dd8
Parents: c37432f
Author: Guanghao Zhang <zg...@apache.org>
Authored: Thu Aug 10 21:03:50 2017 +0800
Committer: Guanghao Zhang <zg...@apache.org>
Committed: Fri Aug 11 10:58:00 2017 +0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/client/Get.java     |  34 +++-
 .../org/apache/hadoop/hbase/client/Query.java   |   6 +-
 .../org/apache/hadoop/hbase/client/Scan.java    |  29 ++-
 .../querymatcher/ScanQueryMatcher.java          |   7 +-
 .../querymatcher/ScanWildcardColumnTracker.java |   7 +-
 .../querymatcher/UserScanQueryMatcher.java      | 166 +++++++++++-----
 .../hadoop/hbase/client/TestFromClientSide.java | 195 +++++++++++++++++++
 .../hadoop/hbase/regionserver/TestHRegion.java  |  64 ++++++
 .../hbase/regionserver/TestMinVersions.java     |   8 +-
 .../hadoop/hbase/regionserver/TestStore.java    |  19 +-
 10 files changed, 463 insertions(+), 72 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/4dd24c52/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Get.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Get.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Get.java
index b774a9a..086a0b4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Get.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Get.java
@@ -267,10 +267,12 @@ public class Get extends Query
   /**
    * Get all available versions.
    * @return this for invocation chaining
+   * @deprecated It is easy to misunderstand with column family's max versions, so use
+   *             {@link #readAllVersions()} instead.
    */
+  @Deprecated
   public Get setMaxVersions() {
-    this.maxVersions = Integer.MAX_VALUE;
-    return this;
+    return readAllVersions();
   }
 
   /**
@@ -278,12 +280,34 @@ public class Get extends Query
    * @param maxVersions maximum versions for each column
    * @throws IOException if invalid number of versions
    * @return this for invocation chaining
+   * @deprecated It is easy to misunderstand with column family's max versions, so use
+   *             {@link #readVersions(int)} instead.
    */
+  @Deprecated
   public Get setMaxVersions(int maxVersions) throws IOException {
-    if(maxVersions <= 0) {
-      throw new IOException("maxVersions must be positive");
+    return readVersions(maxVersions);
+  }
+
+  /**
+   * Get all available versions.
+   * @return this for invocation chaining
+   */
+  public Get readAllVersions() {
+    this.maxVersions = Integer.MAX_VALUE;
+    return this;
+  }
+
+  /**
+   * Get up to the specified number of versions of each column.
+   * @param versions specified number of versions for each column
+   * @throws IOException if invalid number of versions
+   * @return this for invocation chaining
+   */
+  public Get readVersions(int versions) throws IOException {
+    if (versions <= 0) {
+      throw new IOException("versions must be positive");
     }
-    this.maxVersions = maxVersions;
+    this.maxVersions = versions;
     return this;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/4dd24c52/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Query.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Query.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Query.java
index 0bf54ae..cc9e9d4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Query.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Query.java
@@ -53,9 +53,9 @@ public abstract class Query extends OperationWithAttributes {
   }
 
   /**
-   * Apply the specified server-side filter when performing the Query.
-   * Only {@link Filter#filterKeyValue(org.apache.hadoop.hbase.Cell)} is called AFTER all tests
-   * for ttl, column match, deletes and max versions have been run.
+   * Apply the specified server-side filter when performing the Query. Only
+   * {@link Filter#filterKeyValue(org.apache.hadoop.hbase.Cell)} is called AFTER all tests for ttl,
+   * column match, deletes and column family's max versions have been run.
    * @param filter filter to run on the server
    * @return this for invocation chaining
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/4dd24c52/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 e84716f..5b75151 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
@@ -593,19 +593,42 @@ public class Scan extends Query {
   /**
    * Get all available versions.
    * @return this
+   * @deprecated It is easy to misunderstand with column family's max versions, so use
+   *             {@link #readAllVersions()} instead.
    */
+  @Deprecated
   public Scan setMaxVersions() {
-    this.maxVersions = Integer.MAX_VALUE;
-    return this;
+    return readAllVersions();
   }
 
   /**
    * Get up to the specified number of versions of each column.
    * @param maxVersions maximum versions for each column
    * @return this
+   * @deprecated It is easy to misunderstand with column family's max versions, so use
+   *             {@link #readVersions(int)} instead.
    */
+  @Deprecated
   public Scan setMaxVersions(int maxVersions) {
-    this.maxVersions = maxVersions;
+    return readVersions(maxVersions);
+  }
+
+  /**
+   * Get all available versions.
+   * @return this
+   */
+  public Scan readAllVersions() {
+    this.maxVersions = Integer.MAX_VALUE;
+    return this;
+  }
+
+  /**
+   * Get up to the specified number of versions of each column.
+   * @param versions specified number of versions for each column
+   * @return this
+   */
+  public Scan readVersions(int versions) {
+    this.maxVersions = versions;
     return this;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/4dd24c52/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/ScanQueryMatcher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/ScanQueryMatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/ScanQueryMatcher.java
index 8bdab08..524d3f7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/ScanQueryMatcher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/ScanQueryMatcher.java
@@ -355,13 +355,16 @@ public abstract class ScanQueryMatcher implements ShipperListener {
       NavigableSet<byte[]> columns, ScanInfo scanInfo, long oldestUnexpiredTS, Scan userScan)
       throws IOException {
     int resultMaxVersion = scanInfo.getMaxVersions();
+    int maxVersionToCheck = resultMaxVersion;
     if (userScan != null) {
       if (userScan.isRaw()) {
         resultMaxVersion = userScan.getMaxVersions();
       } else {
         resultMaxVersion = Math.min(userScan.getMaxVersions(), scanInfo.getMaxVersions());
       }
+      maxVersionToCheck = userScan.hasFilter() ? scanInfo.getMaxVersions() : resultMaxVersion;
     }
+
     DeleteTracker deleteTracker;
     if (scanInfo.isNewVersionBehavior() && (userScan == null || !userScan.isRaw())) {
       deleteTracker = new NewVersionBehaviorTracker(columns, scanInfo.getMinVersions(),
@@ -382,11 +385,11 @@ public abstract class ScanQueryMatcher implements ShipperListener {
     if (deleteTracker instanceof NewVersionBehaviorTracker) {
       columnTracker = (NewVersionBehaviorTracker) deleteTracker;
     } else if (columns == null || columns.size() == 0) {
-      columnTracker = new ScanWildcardColumnTracker(scanInfo.getMinVersions(), resultMaxVersion,
+      columnTracker = new ScanWildcardColumnTracker(scanInfo.getMinVersions(), maxVersionToCheck,
           oldestUnexpiredTS);
     } else {
       columnTracker = new ExplicitColumnTracker(columns, scanInfo.getMinVersions(),
-          resultMaxVersion, oldestUnexpiredTS);
+        maxVersionToCheck, oldestUnexpiredTS);
     }
     return new Pair<>(deleteTracker, columnTracker);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4dd24c52/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/ScanWildcardColumnTracker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/ScanWildcardColumnTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/ScanWildcardColumnTracker.java
index a73cc0b..9f0a461 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/ScanWildcardColumnTracker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/ScanWildcardColumnTracker.java
@@ -36,8 +36,9 @@ import org.apache.hadoop.hbase.util.Bytes;
 public class ScanWildcardColumnTracker implements ColumnTracker {
   private Cell columnCell = null;
   private int currentCount = 0;
-  private int maxVersions;
-  private int minVersions;
+  private final int maxVersions;
+  private final int minVersions;
+
   /*
    * Keeps track of the latest timestamp and type included for current column. Used to eliminate
    * duplicates.
@@ -74,7 +75,6 @@ public class ScanWildcardColumnTracker implements ColumnTracker {
   @Override
   public ScanQueryMatcher.MatchCode checkVersions(Cell cell, long timestamp, byte type,
       boolean ignoreCount) throws IOException {
-
     if (columnCell == null) {
       // first iteration.
       resetCell(cell);
@@ -143,7 +143,6 @@ public class ScanWildcardColumnTracker implements ColumnTracker {
     } else {
       return MatchCode.SEEK_NEXT_COL;
     }
-
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/4dd24c52/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/UserScanQueryMatcher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/UserScanQueryMatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/UserScanQueryMatcher.java
index 250a4a3..1debb5e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/UserScanQueryMatcher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/UserScanQueryMatcher.java
@@ -22,6 +22,7 @@ import java.util.NavigableSet;
 
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.filter.Filter;
@@ -52,6 +53,12 @@ public abstract class UserScanQueryMatcher extends ScanQueryMatcher {
 
   protected final TimeRange tr;
 
+  private final int versionsAfterFilter;
+
+  private int count = 0;
+
+  private Cell curColCell = null;
+
   private static Cell createStartKey(Scan scan, ScanInfo scanInfo) {
     if (scan.includeStartRow()) {
       return createStartKeyFromRow(scan.getStartRow(), scanInfo);
@@ -65,6 +72,13 @@ public abstract class UserScanQueryMatcher extends ScanQueryMatcher {
     super(createStartKey(scan, scanInfo), scanInfo, columns, oldestUnexpiredTS, now);
     this.hasNullColumn = hasNullColumn;
     this.filter = scan.getFilter();
+    if (this.filter != null) {
+      this.versionsAfterFilter =
+          scan.isRaw() ? scan.getMaxVersions() : Math.min(scan.getMaxVersions(),
+            scanInfo.getMaxVersions());
+    } else {
+      this.versionsAfterFilter = 0;
+    }
     this.stopRow = scan.getStopRow();
     TimeRange timeRange = scan.getColumnFamilyTimeRange().get(scanInfo.getFamily());
     if (timeRange == null) {
@@ -98,6 +112,14 @@ public abstract class UserScanQueryMatcher extends ScanQueryMatcher {
     }
   }
 
+  @Override
+  public void beforeShipped() throws IOException {
+    super.beforeShipped();
+    if (curColCell != null) {
+      this.curColCell = KeyValueUtil.toNewKeyCell(this.curColCell);
+    }
+  }
+
   protected final MatchCode matchColumn(Cell cell, long timestamp, byte typeByte)
       throws IOException {
     int tsCmp = tr.compare(timestamp);
@@ -108,57 +130,111 @@ public abstract class UserScanQueryMatcher extends ScanQueryMatcher {
       return columns.getNextRowOrNextColumn(cell);
     }
     // STEP 1: Check if the column is part of the requested columns
-    MatchCode colChecker = columns.checkColumn(cell, typeByte);
-    if (colChecker != MatchCode.INCLUDE) {
-      return colChecker;
+    MatchCode matchCode = columns.checkColumn(cell, typeByte);
+    if (matchCode != MatchCode.INCLUDE) {
+      return matchCode;
+    }
+    /*
+     * STEP 2: check the number of versions needed. This method call returns SKIP, SEEK_NEXT_COL,
+     * INCLUDE, INCLUDE_AND_SEEK_NEXT_COL, or INCLUDE_AND_SEEK_NEXT_ROW.
+     */
+    matchCode = columns.checkVersions(cell, timestamp, typeByte, false);
+    switch (matchCode) {
+      case SKIP:
+        return MatchCode.SKIP;
+      case SEEK_NEXT_COL:
+        return MatchCode.SEEK_NEXT_COL;
+      default:
+        // It means it is INCLUDE, INCLUDE_AND_SEEK_NEXT_COL or INCLUDE_AND_SEEK_NEXT_ROW.
+        assert matchCode == MatchCode.INCLUDE || matchCode == MatchCode.INCLUDE_AND_SEEK_NEXT_COL
+            || matchCode == MatchCode.INCLUDE_AND_SEEK_NEXT_ROW;
+        break;
     }
-    ReturnCode filterResponse = ReturnCode.SKIP;
-    // STEP 2: Yes, the column is part of the requested columns. Check if filter is present
-    if (filter != null) {
-      // STEP 3: Filter the key value and return if it filters out
-      filterResponse = filter.filterKeyValue(cell);
-      switch (filterResponse) {
-        case SKIP:
+
+    return filter == null ? matchCode : mergeFilterResponse(cell, matchCode,
+      filter.filterKeyValue(cell));
+  }
+
+  /*
+   * Call this when scan has filter. Decide the desired behavior by checkVersions's MatchCode
+   * and filterKeyValue's ReturnCode. Cell may be skipped by filter, so the column versions
+   * in result may be less than user need. It will check versions again after filter.
+   *
+   * ColumnChecker                FilterResponse               Desired behavior
+   * INCLUDE                      SKIP                         SKIP
+   * INCLUDE                      NEXT_COL                     SEEK_NEXT_COL or SEEK_NEXT_ROW
+   * INCLUDE                      NEXT_ROW                     SEEK_NEXT_ROW
+   * INCLUDE                      SEEK_NEXT_USING_HINT         SEEK_NEXT_USING_HINT
+   * INCLUDE                      INCLUDE                      INCLUDE
+   * INCLUDE                      INCLUDE_AND_NEXT_COL         INCLUDE_AND_SEEK_NEXT_COL
+   * INCLUDE                      INCLUDE_AND_SEEK_NEXT_ROW    INCLUDE_AND_SEEK_NEXT_ROW
+   * INCLUDE_AND_SEEK_NEXT_COL    SKIP                         SEEK_NEXT_COL
+   * INCLUDE_AND_SEEK_NEXT_COL    NEXT_COL                     SEEK_NEXT_COL or SEEK_NEXT_ROW
+   * INCLUDE_AND_SEEK_NEXT_COL    NEXT_ROW                     SEEK_NEXT_ROW
+   * INCLUDE_AND_SEEK_NEXT_COL    SEEK_NEXT_USING_HINT         SEEK_NEXT_USING_HINT
+   * INCLUDE_AND_SEEK_NEXT_COL    INCLUDE                      INCLUDE_AND_SEEK_NEXT_COL
+   * INCLUDE_AND_SEEK_NEXT_COL    INCLUDE_AND_NEXT_COL         INCLUDE_AND_SEEK_NEXT_COL
+   * INCLUDE_AND_SEEK_NEXT_COL    INCLUDE_AND_SEEK_NEXT_ROW    INCLUDE_AND_SEEK_NEXT_ROW
+   * INCLUDE_AND_SEEK_NEXT_ROW    SKIP                         SEEK_NEXT_ROW
+   * INCLUDE_AND_SEEK_NEXT_ROW    NEXT_COL                     SEEK_NEXT_ROW
+   * INCLUDE_AND_SEEK_NEXT_ROW    NEXT_ROW                     SEEK_NEXT_ROW
+   * INCLUDE_AND_SEEK_NEXT_ROW    SEEK_NEXT_USING_HINT         SEEK_NEXT_USING_HINT
+   * INCLUDE_AND_SEEK_NEXT_ROW    INCLUDE                      INCLUDE_AND_SEEK_NEXT_ROW
+   * INCLUDE_AND_SEEK_NEXT_ROW    INCLUDE_AND_NEXT_COL         INCLUDE_AND_SEEK_NEXT_ROW
+   * INCLUDE_AND_SEEK_NEXT_ROW    INCLUDE_AND_SEEK_NEXT_ROW    INCLUDE_AND_SEEK_NEXT_ROW
+   */
+  private final MatchCode mergeFilterResponse(Cell cell, MatchCode matchCode,
+      ReturnCode filterResponse) {
+    switch (filterResponse) {
+      case SKIP:
+        if (matchCode == MatchCode.INCLUDE) {
           return MatchCode.SKIP;
-        case NEXT_COL:
+        } else if (matchCode == MatchCode.INCLUDE_AND_SEEK_NEXT_COL) {
+          return MatchCode.SEEK_NEXT_COL;
+        } else if (matchCode == MatchCode.INCLUDE_AND_SEEK_NEXT_ROW) {
+          return MatchCode.SEEK_NEXT_ROW;
+        }
+        break;
+      case NEXT_COL:
+        if (matchCode == MatchCode.INCLUDE || matchCode == MatchCode.INCLUDE_AND_SEEK_NEXT_COL) {
           return columns.getNextRowOrNextColumn(cell);
-        case NEXT_ROW:
+        } else if (matchCode == MatchCode.INCLUDE_AND_SEEK_NEXT_ROW) {
           return MatchCode.SEEK_NEXT_ROW;
-        case SEEK_NEXT_USING_HINT:
-          return MatchCode.SEEK_NEXT_USING_HINT;
-        default:
-          // It means it is either include or include and seek next
-          break;
-      }
+        }
+        break;
+      case NEXT_ROW:
+        return MatchCode.SEEK_NEXT_ROW;
+      case SEEK_NEXT_USING_HINT:
+        return MatchCode.SEEK_NEXT_USING_HINT;
+      case INCLUDE:
+        break;
+      case INCLUDE_AND_NEXT_COL:
+        if (matchCode == MatchCode.INCLUDE) {
+          matchCode = MatchCode.INCLUDE_AND_SEEK_NEXT_COL;
+        }
+        break;
+      case INCLUDE_AND_SEEK_NEXT_ROW:
+        break;
+      default:
+        throw new RuntimeException("UNEXPECTED");
     }
-    /*
-     * STEP 4: Reaching this step means the column is part of the requested columns and either
-     * the filter is null or the filter has returned INCLUDE or INCLUDE_AND_NEXT_COL response.
-     * Now check the number of versions needed. This method call returns SKIP, INCLUDE,
-     * INCLUDE_AND_SEEK_NEXT_ROW, INCLUDE_AND_SEEK_NEXT_COL.
-     *
-     * FilterResponse            ColumnChecker               Desired behavior
-     * INCLUDE                   SKIP                        row has already been included, SKIP.
-     * INCLUDE                   INCLUDE                     INCLUDE
-     * INCLUDE                   INCLUDE_AND_SEEK_NEXT_COL   INCLUDE_AND_SEEK_NEXT_COL
-     * INCLUDE                   INCLUDE_AND_SEEK_NEXT_ROW   INCLUDE_AND_SEEK_NEXT_ROW
-     * INCLUDE_AND_SEEK_NEXT_COL SKIP                        row has already been included, SKIP.
-     * INCLUDE_AND_SEEK_NEXT_COL INCLUDE                     INCLUDE_AND_SEEK_NEXT_COL
-     * INCLUDE_AND_SEEK_NEXT_COL INCLUDE_AND_SEEK_NEXT_COL   INCLUDE_AND_SEEK_NEXT_COL
-     * INCLUDE_AND_SEEK_NEXT_COL INCLUDE_AND_SEEK_NEXT_ROW   INCLUDE_AND_SEEK_NEXT_ROW
-     *
-     * In all the above scenarios, we return the column checker return value except for
-     * FilterResponse (INCLUDE_AND_SEEK_NEXT_COL) and ColumnChecker(INCLUDE)
-     */
-    colChecker = columns.checkVersions(cell, timestamp, typeByte, false);
-    if (filterResponse == ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW) {
-      if (colChecker != MatchCode.SKIP) {
-        return MatchCode.INCLUDE_AND_SEEK_NEXT_ROW;
-      }
-      return MatchCode.SEEK_NEXT_ROW;
+
+    // It means it is INCLUDE, INCLUDE_AND_SEEK_NEXT_COL or INCLUDE_AND_SEEK_NEXT_ROW.
+    assert matchCode == MatchCode.INCLUDE || matchCode == MatchCode.INCLUDE_AND_SEEK_NEXT_COL
+        || matchCode == MatchCode.INCLUDE_AND_SEEK_NEXT_ROW;
+
+    if (matchCode == MatchCode.INCLUDE_AND_SEEK_NEXT_COL
+        || matchCode == MatchCode.INCLUDE_AND_SEEK_NEXT_ROW) {
+      return matchCode;
+    }
+
+    // Now we will check versions again.
+    if (curColCell == null || !CellUtil.matchingRowColumn(cell, curColCell)) {
+      count = 0;
+      curColCell = cell;
     }
-    return (filterResponse == ReturnCode.INCLUDE_AND_NEXT_COL && colChecker == MatchCode.INCLUDE)
-        ? MatchCode.INCLUDE_AND_SEEK_NEXT_COL : colChecker;
+    count += 1;
+    return count > versionsAfterFilter ? MatchCode.SEEK_NEXT_COL : MatchCode.INCLUDE;
   }
 
   protected abstract boolean isGet();

http://git-wip-us.apache.org/repos/asf/hbase/blob/4dd24c52/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
index a93fbb2..8a3841e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
@@ -88,6 +88,8 @@ import org.apache.hadoop.hbase.filter.QualifierFilter;
 import org.apache.hadoop.hbase.filter.RegexStringComparator;
 import org.apache.hadoop.hbase.filter.RowFilter;
 import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.filter.SubstringComparator;
+import org.apache.hadoop.hbase.filter.ValueFilter;
 import org.apache.hadoop.hbase.filter.WhileMatchFilter;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.io.hfile.BlockCache;
@@ -6424,4 +6426,197 @@ public class TestFromClientSide {
     }
   }
 
+  @Test
+  public void testDeleteSpecifiedVersionOfSpecifiedColumn() throws Exception {
+    Admin admin = TEST_UTIL.getAdmin();
+    final TableName tableName = TableName.valueOf(name.getMethodName());
+
+    byte[][] VALUES = makeN(VALUE, 5);
+    long[] ts = { 1000, 2000, 3000, 4000, 5000 };
+
+    Table ht = TEST_UTIL.createTable(tableName, FAMILY, 5);
+
+    Put put = new Put(ROW);
+    // Put version 1000,2000,3000,4000 of column FAMILY:QUALIFIER
+    for (int t = 0; t < 4; t++) {
+      put.addColumn(FAMILY, QUALIFIER, ts[t], VALUES[t]);
+    }
+    ht.put(put);
+
+    Delete delete = new Delete(ROW);
+    // Delete version 3000 of column FAMILY:QUALIFIER
+    delete.addColumn(FAMILY, QUALIFIER, ts[2]);
+    ht.delete(delete);
+
+    Get get = new Get(ROW);
+    get.addColumn(FAMILY, QUALIFIER);
+    get.setMaxVersions(Integer.MAX_VALUE);
+    Result result = ht.get(get);
+    // verify version 1000,2000,4000 remains for column FAMILY:QUALIFIER
+    assertNResult(result, ROW, FAMILY, QUALIFIER, new long[] { ts[0], ts[1], ts[3] }, new byte[][] {
+        VALUES[0], VALUES[1], VALUES[3] }, 0, 2);
+
+    delete = new Delete(ROW);
+    // Delete a version 5000 of column FAMILY:QUALIFIER which didn't exist
+    delete.addColumn(FAMILY, QUALIFIER, ts[4]);
+    ht.delete(delete);
+
+    get = new Get(ROW);
+    get.addColumn(FAMILY, QUALIFIER);
+    get.setMaxVersions(Integer.MAX_VALUE);
+    result = ht.get(get);
+    // verify version 1000,2000,4000 remains for column FAMILY:QUALIFIER
+    assertNResult(result, ROW, FAMILY, QUALIFIER, new long[] { ts[0], ts[1], ts[3] }, new byte[][] {
+        VALUES[0], VALUES[1], VALUES[3] }, 0, 2);
+
+    ht.close();
+    admin.close();
+  }
+
+  @Test
+  public void testDeleteLatestVersionOfSpecifiedColumn() throws Exception {
+    Admin admin = TEST_UTIL.getAdmin();
+    final TableName tableName = TableName.valueOf(name.getMethodName());
+
+    byte[][] VALUES = makeN(VALUE, 5);
+    long[] ts = { 1000, 2000, 3000, 4000, 5000 };
+
+    Table ht = TEST_UTIL.createTable(tableName, FAMILY, 5);
+
+    Put put = new Put(ROW);
+    // Put version 1000,2000,3000,4000 of column FAMILY:QUALIFIER
+    for (int t = 0; t < 4; t++) {
+      put.addColumn(FAMILY, QUALIFIER, ts[t], VALUES[t]);
+    }
+    ht.put(put);
+
+    Delete delete = new Delete(ROW);
+    // Delete latest version of column FAMILY:QUALIFIER
+    delete.addColumn(FAMILY, QUALIFIER);
+    ht.delete(delete);
+
+    Get get = new Get(ROW);
+    get.addColumn(FAMILY, QUALIFIER);
+    get.setMaxVersions(Integer.MAX_VALUE);
+    Result result = ht.get(get);
+    // verify version 1000,2000,3000 remains for column FAMILY:QUALIFIER
+    assertNResult(result, ROW, FAMILY, QUALIFIER, new long[] { ts[0], ts[1], ts[2] }, new byte[][] {
+        VALUES[0], VALUES[1], VALUES[2] }, 0, 2);
+
+    delete = new Delete(ROW);
+    // Delete two latest version of column FAMILY:QUALIFIER
+    delete.addColumn(FAMILY, QUALIFIER);
+    delete.addColumn(FAMILY, QUALIFIER);
+    ht.delete(delete);
+
+    get = new Get(ROW);
+    get.addColumn(FAMILY, QUALIFIER);
+    get.setMaxVersions(Integer.MAX_VALUE);
+    result = ht.get(get);
+    // verify version 1000 remains for column FAMILY:QUALIFIER
+    assertNResult(result, ROW, FAMILY, QUALIFIER, new long[] { ts[0] }, new byte[][] { VALUES[0] },
+      0, 0);
+
+    put = new Put(ROW);
+    // Put a version 5000 of column FAMILY:QUALIFIER
+    put.addColumn(FAMILY, QUALIFIER, ts[4], VALUES[4]);
+    ht.put(put);
+
+    get = new Get(ROW);
+    get.addColumn(FAMILY, QUALIFIER);
+    get.setMaxVersions(Integer.MAX_VALUE);
+    result = ht.get(get);
+    // verify version 1000,5000 remains for column FAMILY:QUALIFIER
+    assertNResult(result, ROW, FAMILY, QUALIFIER, new long[] { ts[0], ts[4] }, new byte[][] {
+        VALUES[0], VALUES[4] }, 0, 1);
+
+    ht.close();
+    admin.close();
+  }
+
+  /**
+   * Test for HBASE-17125
+   */
+  @Test
+  public void testReadWithFilter() throws Exception {
+    Admin admin = TEST_UTIL.getAdmin();
+    final TableName tableName = TableName.valueOf(name.getMethodName());
+    Table table = TEST_UTIL.createTable(tableName, FAMILY, 3);
+
+    byte[] VALUEA = Bytes.toBytes("value-a");
+    byte[] VALUEB = Bytes.toBytes("value-b");
+    long[] ts = { 1000, 2000, 3000, 4000 };
+
+    Put put = new Put(ROW);
+    // Put version 1000,2000,3000,4000 of column FAMILY:QUALIFIER
+    for (int t = 0; t <= 3; t++) {
+      if (t <= 1) {
+        put.addColumn(FAMILY, QUALIFIER, ts[t], VALUEA);
+      } else {
+        put.addColumn(FAMILY, QUALIFIER, ts[t], VALUEB);
+      }
+    }
+    table.put(put);
+
+    Scan scan =
+        new Scan().setFilter(new ValueFilter(CompareOp.EQUAL, new SubstringComparator("value-a")))
+            .setMaxVersions(3);
+    ResultScanner scanner = table.getScanner(scan);
+    Result result = scanner.next();
+    // ts[0] has gone from user view. Only read ts[2] which value is less or equal to 3
+    assertNResult(result, ROW, FAMILY, QUALIFIER, new long[] { ts[1] }, new byte[][] { VALUEA }, 0,
+      0);
+
+    Get get =
+        new Get(ROW)
+            .setFilter(new ValueFilter(CompareOp.EQUAL, new SubstringComparator("value-a")))
+            .setMaxVersions(3);
+    result = table.get(get);
+    // ts[0] has gone from user view. Only read ts[2] which value is less or equal to 3
+    assertNResult(result, ROW, FAMILY, QUALIFIER, new long[] { ts[1] }, new byte[][] { VALUEA }, 0,
+      0);
+
+    // Test with max versions 1, it should still read ts[1]
+    scan =
+        new Scan().setFilter(new ValueFilter(CompareOp.EQUAL, new SubstringComparator("value-a")))
+            .setMaxVersions(1);
+    scanner = table.getScanner(scan);
+    result = scanner.next();
+    // ts[0] has gone from user view. Only read ts[2] which value is less or equal to 3
+    assertNResult(result, ROW, FAMILY, QUALIFIER, new long[] { ts[1] }, new byte[][] { VALUEA }, 0,
+      0);
+
+    // Test with max versions 1, it should still read ts[1]
+    get =
+        new Get(ROW)
+            .setFilter(new ValueFilter(CompareOp.EQUAL, new SubstringComparator("value-a")))
+            .setMaxVersions(1);
+    result = table.get(get);
+    // ts[0] has gone from user view. Only read ts[2] which value is less or equal to 3
+    assertNResult(result, ROW, FAMILY, QUALIFIER, new long[] { ts[1] }, new byte[][] { VALUEA }, 0,
+      0);
+
+    // Test with max versions 5, it should still read ts[1]
+    scan =
+        new Scan().setFilter(new ValueFilter(CompareOp.EQUAL, new SubstringComparator("value-a")))
+            .setMaxVersions(5);
+    scanner = table.getScanner(scan);
+    result = scanner.next();
+    // ts[0] has gone from user view. Only read ts[2] which value is less or equal to 3
+    assertNResult(result, ROW, FAMILY, QUALIFIER, new long[] { ts[1] }, new byte[][] { VALUEA }, 0,
+      0);
+
+    // Test with max versions 5, it should still read ts[1]
+    get =
+        new Get(ROW)
+            .setFilter(new ValueFilter(CompareOp.EQUAL, new SubstringComparator("value-a")))
+            .setMaxVersions(5);
+    result = table.get(get);
+    // ts[0] has gone from user view. Only read ts[2] which value is less or equal to 3
+    assertNResult(result, ROW, FAMILY, QUALIFIER, new long[] { ts[1] }, new byte[][] { VALUEA }, 0,
+      0);
+
+    table.close();
+    admin.close();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4dd24c52/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 9db7c16..b8020d8 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
@@ -123,6 +123,8 @@ import org.apache.hadoop.hbase.filter.NullComparator;
 import org.apache.hadoop.hbase.filter.PrefixFilter;
 import org.apache.hadoop.hbase.filter.SingleColumnValueExcludeFilter;
 import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.filter.SubstringComparator;
+import org.apache.hadoop.hbase.filter.ValueFilter;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
@@ -2638,6 +2640,68 @@ public class TestHRegion {
     }
   }
 
+  @Test
+  public void testGetWithFilter() throws IOException, InterruptedException {
+    byte[] row1 = Bytes.toBytes("row1");
+    byte[] fam1 = Bytes.toBytes("fam1");
+    byte[] col1 = Bytes.toBytes("col1");
+    byte[] value1 = Bytes.toBytes("value1");
+    byte[] value2 = Bytes.toBytes("value2");
+
+    final int maxVersions = 3;
+    HColumnDescriptor hcd = new HColumnDescriptor(fam1);
+    hcd.setMaxVersions(maxVersions);
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("testFilterAndColumnTracker"));
+    htd.addFamily(hcd);
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
+    HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
+    Path logDir = TEST_UTIL.getDataTestDirOnTestFS(method + ".log");
+    final WAL wal = HBaseTestingUtility.createWal(TEST_UTIL.getConfiguration(), logDir, info);
+    this.region = TEST_UTIL.createLocalHRegion(info, htd, wal);
+
+    try {
+      // Put 4 version to memstore
+      long ts = 0;
+      Put put = new Put(row1, ts);
+      put.addColumn(fam1, col1, value1);
+      region.put(put);
+      put = new Put(row1, ts + 1);
+      put.addColumn(fam1, col1, Bytes.toBytes("filter1"));
+      region.put(put);
+      put = new Put(row1, ts + 2);
+      put.addColumn(fam1, col1, Bytes.toBytes("filter2"));
+      region.put(put);
+      put = new Put(row1, ts + 3);
+      put.addColumn(fam1, col1, value2);
+      region.put(put);
+
+      Get get = new Get(row1);
+      get.setMaxVersions();
+      Result res = region.get(get);
+      // Get 3 versions, the oldest version has gone from user view
+      assertEquals(maxVersions, res.size());
+
+      get.setFilter(new ValueFilter(CompareOp.EQUAL, new SubstringComparator("value")));
+      res = region.get(get);
+      // When use value filter, the oldest version should still gone from user view and it
+      // should only return one key vaule
+      assertEquals(1, res.size());
+      assertTrue(CellUtil.matchingValue(new KeyValue(row1, fam1, col1, value2), res.rawCells()[0]));
+      assertEquals(ts + 3, res.rawCells()[0].getTimestamp());
+
+      region.flush(true);
+      region.compact(true);
+      Thread.sleep(1000);
+      res = region.get(get);
+      // After flush and compact, the result should be consistent with previous result
+      assertEquals(1, res.size());
+      assertTrue(CellUtil.matchingValue(new KeyValue(row1, fam1, col1, value2), res.rawCells()[0]));
+    } finally {
+      HBaseTestingUtility.closeRegionAndWAL(this.region);
+      this.region = null;
+    }
+  }
+
   // ////////////////////////////////////////////////////////////////////////////
   // Scanner tests
   // ////////////////////////////////////////////////////////////////////////////

http://git-wip-us.apache.org/repos/asf/hbase/blob/4dd24c52/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMinVersions.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMinVersions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMinVersions.java
index 52b5a40..e8d60e6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMinVersions.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMinVersions.java
@@ -431,24 +431,27 @@ public class TestMinVersions {
       tss.add(ts-1);
       tss.add(ts-2);
 
+      // Sholud only get T2, versions is 2, so T1 is gone from user view.
       Get g = new Get(T1);
       g.addColumn(c1,c1);
       g.setFilter(new TimestampsFilter(tss));
       g.setMaxVersions();
       Result r = region.get(g);
-      checkResult(r, c1, T2,T1);
+      checkResult(r, c1, T2);
 
+      // Sholud only get T2, versions is 2, so T1 is gone from user view.
       g = new Get(T1);
       g.addColumn(c0,c0);
       g.setFilter(new TimestampsFilter(tss));
       g.setMaxVersions();
       r = region.get(g);
-      checkResult(r, c0, T2,T1);
+      checkResult(r, c0, T2);
 
       // now flush/compact
       region.flush(true);
       region.compact(true);
 
+      // After flush/compact, the result should be consistent with previous result
       g = new Get(T1);
       g.addColumn(c1,c1);
       g.setFilter(new TimestampsFilter(tss));
@@ -456,6 +459,7 @@ public class TestMinVersions {
       r = region.get(g);
       checkResult(r, c1, T2);
 
+      // After flush/compact, the result should be consistent with previous result
       g = new Get(T1);
       g.addColumn(c0,c0);
       g.setFilter(new TimestampsFilter(tss));

http://git-wip-us.apache.org/repos/asf/hbase/blob/4dd24c52/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
index 1e3c84c..537601d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
@@ -1067,10 +1067,11 @@ public class TestStore {
   @Test
   public void testFlushBeforeCompletingScanWoFilter() throws IOException, InterruptedException {
     final AtomicBoolean timeToGoNextRow = new AtomicBoolean(false);
+    final int expectedSize = 3;
     testFlushBeforeCompletingScan(new MyListHook() {
       @Override
       public void hook(int currentSize) {
-        if (currentSize == 2) {
+        if (currentSize == expectedSize - 1) {
           try {
             flushStore(store, id++);
             timeToGoNextRow.set(true);
@@ -1084,16 +1085,17 @@ public class TestStore {
       public Filter.ReturnCode filterKeyValue(Cell v) throws IOException {
         return ReturnCode.INCLUDE;
       }
-    });
+    }, expectedSize);
   }
 
   @Test
   public void testFlushBeforeCompletingScanWithFilter() throws IOException, InterruptedException {
     final AtomicBoolean timeToGoNextRow = new AtomicBoolean(false);
+    final int expectedSize = 2;
     testFlushBeforeCompletingScan(new MyListHook() {
       @Override
       public void hook(int currentSize) {
-        if (currentSize == 2) {
+        if (currentSize == expectedSize - 1) {
           try {
             flushStore(store, id++);
             timeToGoNextRow.set(true);
@@ -1112,16 +1114,17 @@ public class TestStore {
           return ReturnCode.INCLUDE;
         }
       }
-    });
+    }, expectedSize);
   }
 
   @Test
   public void testFlushBeforeCompletingScanWithFilterHint() throws IOException, InterruptedException {
     final AtomicBoolean timeToGetHint = new AtomicBoolean(false);
+    final int expectedSize = 2;
     testFlushBeforeCompletingScan(new MyListHook() {
       @Override
       public void hook(int currentSize) {
-        if (currentSize == 2) {
+        if (currentSize == expectedSize - 1) {
           try {
             flushStore(store, id++);
             timeToGetHint.set(true);
@@ -1144,10 +1147,10 @@ public class TestStore {
       public Cell getNextCellHint(Cell currentCell) throws IOException {
         return currentCell;
       }
-    });
+    }, expectedSize);
   }
 
-  private void testFlushBeforeCompletingScan(MyListHook hook, Filter filter)
+  private void testFlushBeforeCompletingScan(MyListHook hook, Filter filter, int expectedSize)
           throws IOException, InterruptedException {
     Configuration conf = HBaseConfiguration.create();
     HColumnDescriptor hcd = new HColumnDescriptor(family);
@@ -1188,7 +1191,7 @@ public class TestStore {
           scan, null, seqId + 3)){
       // r1
       scanner.next(myList);
-      assertEquals(3, myList.size());
+      assertEquals(expectedSize, myList.size());
       for (Cell c : myList) {
         byte[] actualValue = CellUtil.cloneValue(c);
         assertTrue("expected:" + Bytes.toStringBinary(value1)


[27/50] [abbrv] hbase git commit: HBASE-18303 Clean up @Parameter boilerplate

Posted by st...@apache.org.
HBASE-18303 Clean up @Parameter boilerplate


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

Branch: refs/heads/HBASE-14070.HLC
Commit: 0b26ccdaa1b8700e7958aeebbaf9cad81e737dd0
Parents: ea8fa59
Author: Mike Drob <md...@apache.org>
Authored: Fri Jun 30 12:13:56 2017 -0500
Committer: Mike Drob <md...@apache.org>
Committed: Mon Aug 14 14:23:24 2017 -0500

----------------------------------------------------------------------
 .../hadoop/hbase/filter/TestKeyOnlyFilter.java  | 15 ++++---------
 .../hadoop/hbase/HBaseCommonTestingUtility.java | 22 ++++++++++++++++++++
 .../apache/hadoop/hbase/types/TestStruct.java   | 18 +++++++---------
 .../hadoop/hbase/util/TestByteBufferUtils.java  |  8 ++-----
 .../hbase/codec/keyvalue/TestKeyValueTool.java  | 16 +++++++-------
 .../codec/prefixtree/row/TestRowEncoder.java    |  6 +-----
 .../hadoop/hbase/rest/TestMultiRowResource.java |  5 +----
 .../hadoop/hbase/HBaseTestingUtility.java       | 20 +-----------------
 .../encoding/TestSeekToBlockWithEncoders.java   |  4 ++--
 .../hadoop/hbase/io/hfile/TestCacheOnWrite.java |  3 ++-
 .../apache/hadoop/hbase/io/hfile/TestHFile.java |  3 ++-
 .../hbase/io/hfile/TestHFileBlockIndex.java     |  3 ++-
 .../hbase/io/hfile/TestHFileWriterV3.java       |  3 ++-
 .../hbase/util/TestCoprocessorScanPolicy.java   |  3 ++-
 14 files changed, 59 insertions(+), 70 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/0b26ccda/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestKeyOnlyFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestKeyOnlyFilter.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestKeyOnlyFilter.java
index 33e3cd9..f957b59 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestKeyOnlyFilter.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestKeyOnlyFilter.java
@@ -26,6 +26,7 @@ import java.util.List;
 import org.apache.hadoop.hbase.ByteBufferKeyValue;
 
 import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.KeyValueUtil;
@@ -44,20 +45,12 @@ import org.junit.runners.Parameterized.Parameters;
 @RunWith(Parameterized.class)
 public class TestKeyOnlyFilter {
 
-  private final boolean lenAsVal;
+  @Parameterized.Parameter
+  public boolean lenAsVal;
 
   @Parameters
   public static Collection<Object[]> parameters() {
-    List<Object[]> paramList = new ArrayList<>(2);
-    {
-      paramList.add(new Object[] { false });
-      paramList.add(new Object[] { true });
-    }
-    return paramList;
-  }
-
-  public TestKeyOnlyFilter(boolean lenAsVal) {
-    this.lenAsVal = lenAsVal;
+    return HBaseCommonTestingUtility.BOOLEAN_PARAMETERIZED;
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hbase/blob/0b26ccda/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseCommonTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseCommonTestingUtility.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseCommonTestingUtility.java
index e191046..1790f4a 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseCommonTestingUtility.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseCommonTestingUtility.java
@@ -20,6 +20,8 @@ package org.apache.hadoop.hbase;
 
 import java.io.File;
 import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
 import java.util.UUID;
 
 import org.apache.commons.io.FileUtils;
@@ -28,6 +30,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.io.compress.Compression;
 
 /**
  * Common helpers for testing HBase that do not depend on specific server/etc. things.
@@ -37,6 +40,25 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 public class HBaseCommonTestingUtility {
   protected static final Log LOG = LogFactory.getLog(HBaseCommonTestingUtility.class);
 
+  /** Compression algorithms to use in parameterized JUnit 4 tests */
+  public static final List<Object[]> COMPRESSION_ALGORITHMS_PARAMETERIZED =
+    Arrays.asList(new Object[][] {
+      { Compression.Algorithm.NONE },
+      { Compression.Algorithm.GZ }
+    });
+
+  /** This is for unit tests parameterized with a two booleans. */
+  public static final List<Object[]> BOOLEAN_PARAMETERIZED =
+      Arrays.asList(new Object[][] {
+          {false},
+          {true}
+      });
+
+  /** Compression algorithms to use in testing */
+  public static final Compression.Algorithm[] COMPRESSION_ALGORITHMS = {
+      Compression.Algorithm.NONE, Compression.Algorithm.GZ
+  };
+
   protected Configuration conf;
 
   public HBaseCommonTestingUtility() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/0b26ccda/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestStruct.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestStruct.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestStruct.java
index 8dc239b..4fcaaec 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestStruct.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestStruct.java
@@ -47,17 +47,15 @@ import org.junit.runners.Parameterized.Parameters;
 @Category({MiscTests.class, SmallTests.class})
 public class TestStruct {
 
-  private Struct generic;
+  @Parameterized.Parameter(value = 0)
+  public Struct generic;
+
   @SuppressWarnings("rawtypes")
-  private DataType specialized;
-  private Object[][] constructorArgs;
-
-  public TestStruct(Struct generic, @SuppressWarnings("rawtypes") DataType specialized,
-      Object[][] constructorArgs) {
-    this.generic = generic;
-    this.specialized = specialized;
-    this.constructorArgs = constructorArgs;
-  }
+  @Parameterized.Parameter(value = 1)
+  public DataType specialized;
+
+  @Parameterized.Parameter(value = 2)
+  public Object[][] constructorArgs;
 
   @Parameters
   public static Collection<Object[]> params() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/0b26ccda/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java
index ee03c7b..81923ca 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java
@@ -44,6 +44,7 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
 
+import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
@@ -69,12 +70,7 @@ public class TestByteBufferUtils {
 
   @Parameterized.Parameters
   public static Collection<Object[]> parameters() {
-    List<Object[]> paramList = new ArrayList<>(2);
-    {
-      paramList.add(new Object[] { false });
-      paramList.add(new Object[] { true });
-    }
-    return paramList;
+    return HBaseCommonTestingUtility.BOOLEAN_PARAMETERIZED;
   }
 
   private static void setUnsafe(String fieldName, boolean value) throws Exception {

http://git-wip-us.apache.org/repos/asf/hbase/blob/0b26ccda/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/keyvalue/TestKeyValueTool.java
----------------------------------------------------------------------
diff --git a/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/keyvalue/TestKeyValueTool.java b/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/keyvalue/TestKeyValueTool.java
index 9e27942..3e5f0f8 100644
--- a/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/keyvalue/TestKeyValueTool.java
+++ b/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/keyvalue/TestKeyValueTool.java
@@ -31,12 +31,15 @@ import org.apache.hadoop.hbase.codec.prefixtree.row.TestRowData;
 import org.apache.hadoop.hbase.codec.prefixtree.row.data.TestRowDataRandomKeyValuesWithTags;
 import org.apache.hadoop.hbase.codec.prefixtree.row.data.TestRowDataTrivialWithTags;
 import org.junit.Assert;
+import org.junit.Assume;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
+import static org.junit.Assume.assumeFalse;
+
 @Category({MiscTests.class, SmallTests.class})
 @RunWith(Parameterized.class)
 public class TestKeyValueTool {
@@ -46,17 +49,14 @@ public class TestKeyValueTool {
     return TestRowData.InMemory.getAllAsObjectArray();
   }
 
-  private TestRowData rows;
-
-  public TestKeyValueTool(TestRowData testRows) {
-    this.rows = testRows;
-  }
+  @Parameterized.Parameter
+  public TestRowData rows;
 
   @Test
   public void testRoundTripToBytes() {
-    if(rows instanceof TestRowDataTrivialWithTags || rows instanceof TestRowDataRandomKeyValuesWithTags) {
-      return;
-    }
+    assumeFalse(rows instanceof TestRowDataTrivialWithTags);
+    assumeFalse(rows instanceof TestRowDataRandomKeyValuesWithTags);
+
     List<KeyValue> kvs = rows.getInputs();
     ByteBuffer bb = KeyValueTestUtil.toByteBufferAndRewind(kvs, false);
     List<KeyValue> roundTrippedKvs = KeyValueTestUtil.rewindThenToList(bb, false, false);

http://git-wip-us.apache.org/repos/asf/hbase/blob/0b26ccda/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/TestRowEncoder.java
----------------------------------------------------------------------
diff --git a/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/TestRowEncoder.java b/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/TestRowEncoder.java
index 22119be..a8e5feb 100644
--- a/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/TestRowEncoder.java
+++ b/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/TestRowEncoder.java
@@ -55,11 +55,7 @@ public class TestRowEncoder {
 
   @Parameters
   public static Collection<Object[]> parameters() {
-    List<Object[]> parameters = Lists.newArrayList();
-    for (TestRowData testRows : TestRowData.InMemory.getAll()) {
-      parameters.add(new Object[] { testRows });
-    }
-    return parameters;
+    return TestRowData.InMemory.getAllAsObjectArray();
   }
 
   protected TestRowData rows;

http://git-wip-us.apache.org/repos/asf/hbase/blob/0b26ccda/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestMultiRowResource.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestMultiRowResource.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestMultiRowResource.java
index 2a0b460..537a293 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestMultiRowResource.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestMultiRowResource.java
@@ -82,10 +82,7 @@ public class TestMultiRowResource {
 
   @Parameterized.Parameters
   public static Collection<Object[]> data() {
-    List<Object[]> params = new ArrayList<>(2);
-    params.add(new Object[] {Boolean.TRUE});
-    params.add(new Object[] {Boolean.FALSE});
-    return params;
+    return HBaseCommonTestingUtility.BOOLEAN_PARAMETERIZED;
   }
 
   public TestMultiRowResource(Boolean csrf) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/0b26ccda/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index 8761ef6..44aeb88 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -216,26 +216,8 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
   /** Filesystem URI used for map-reduce mini-cluster setup */
   private static String FS_URI;
 
-  /** Compression algorithms to use in parameterized JUnit 4 tests */
-  public static final List<Object[]> COMPRESSION_ALGORITHMS_PARAMETERIZED =
-    Arrays.asList(new Object[][] {
-      { Compression.Algorithm.NONE },
-      { Compression.Algorithm.GZ }
-    });
-
-  /** This is for unit tests parameterized with a two booleans. */
-  public static final List<Object[]> BOOLEAN_PARAMETERIZED =
-      Arrays.asList(new Object[][] {
-          {false},
-          {true}
-      });
-
   /** This is for unit tests parameterized with a single boolean. */
   public static final List<Object[]> MEMSTORETS_TAGS_PARAMETRIZED = memStoreTSAndTagsCombination();
-  /** Compression algorithms to use in testing */
-  public static final Compression.Algorithm[] COMPRESSION_ALGORITHMS ={
-      Compression.Algorithm.NONE, Compression.Algorithm.GZ
-    };
 
   /**
    * Checks to see if a specific port is available.
@@ -278,7 +260,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
   private static List<Object[]> bloomAndCompressionCombinations() {
     List<Object[]> configurations = new ArrayList<>();
     for (Compression.Algorithm comprAlgo :
-         HBaseTestingUtility.COMPRESSION_ALGORITHMS) {
+         HBaseCommonTestingUtility.COMPRESSION_ALGORITHMS) {
       for (BloomType bloomType : BloomType.values()) {
         configurations.add(new Object[] { comprAlgo, bloomType });
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/0b26ccda/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestSeekToBlockWithEncoders.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestSeekToBlockWithEncoders.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestSeekToBlockWithEncoders.java
index cc70dc1..4a7f2eb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestSeekToBlockWithEncoders.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestSeekToBlockWithEncoders.java
@@ -27,7 +27,7 @@ import java.util.List;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.io.compress.Compression;
@@ -51,7 +51,7 @@ public class TestSeekToBlockWithEncoders {
 
   @Parameters
   public static Collection<Object[]> parameters() {
-    return HBaseTestingUtility.BOOLEAN_PARAMETERIZED;
+    return HBaseCommonTestingUtility.BOOLEAN_PARAMETERIZED;
   }
 
   public TestSeekToBlockWithEncoders(boolean useOffheapData) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/0b26ccda/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
index d4f8a59..950beec 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
@@ -179,7 +180,7 @@ public class TestCacheOnWrite {
     List<Object[]> params = new ArrayList<>();
     for (BlockCache blockCache : getBlockCaches()) {
       for (CacheOnWriteType cowType : CacheOnWriteType.values()) {
-        for (Compression.Algorithm compress : HBaseTestingUtility.COMPRESSION_ALGORITHMS) {
+        for (Compression.Algorithm compress : HBaseCommonTestingUtility.COMPRESSION_ALGORITHMS) {
           for (boolean cacheCompressedData : new boolean[] { false, true }) {
             params.add(new Object[] { cowType, compress, cacheCompressedData, blockCache });
           }

http://git-wip-us.apache.org/repos/asf/hbase/blob/0b26ccda/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
index d209430..62a7c48 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
@@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
@@ -440,7 +441,7 @@ public class TestHFile  {
   public void testNullMetaBlocks() throws Exception {
     if (cacheConf == null) cacheConf = new CacheConfig(conf);
     for (Compression.Algorithm compressAlgo :
-        HBaseTestingUtility.COMPRESSION_ALGORITHMS) {
+        HBaseCommonTestingUtility.COMPRESSION_ALGORITHMS) {
       Path mFile = new Path(ROOT_DIR, "nometa_" + compressAlgo + ".hfile");
       FSDataOutputStream fout = createFSOutput(mFile);
       HFileContext meta = new HFileContextBuilder().withCompression(compressAlgo)

http://git-wip-us.apache.org/repos/asf/hbase/blob/0b26ccda/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java
index bdd3de2..3d1af90 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
@@ -73,7 +74,7 @@ public class TestHFileBlockIndex {
 
   @Parameters
   public static Collection<Object[]> compressionAlgorithms() {
-    return HBaseTestingUtility.COMPRESSION_ALGORITHMS_PARAMETERIZED;
+    return HBaseCommonTestingUtility.COMPRESSION_ALGORITHMS_PARAMETERIZED;
   }
 
   public TestHFileBlockIndex(Compression.Algorithm compr) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/0b26ccda/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java
index 0bf20b3..400c72f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
@@ -81,7 +82,7 @@ public class TestHFileWriterV3 {
   }
   @Parameters
   public static Collection<Object[]> parameters() {
-    return HBaseTestingUtility.BOOLEAN_PARAMETERIZED;
+    return HBaseCommonTestingUtility.BOOLEAN_PARAMETERIZED;
   }
 
   @Before

http://git-wip-us.apache.org/repos/asf/hbase/blob/0b26ccda/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java
index 2a44243..993a1fd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java
@@ -31,6 +31,7 @@ import java.util.NavigableSet;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
@@ -89,7 +90,7 @@ public class TestCoprocessorScanPolicy {
 
   @Parameters
   public static Collection<Object[]> parameters() {
-    return HBaseTestingUtility.BOOLEAN_PARAMETERIZED;
+    return HBaseCommonTestingUtility.BOOLEAN_PARAMETERIZED;
   }
 
   public TestCoprocessorScanPolicy(boolean parallelSeekEnable) {


[02/50] [abbrv] hbase git commit: HBASE-18024 HRegion#initializeRegionInternals should not re-create .hregioninfo file when the region directory no longer exists

Posted by st...@apache.org.
HBASE-18024 HRegion#initializeRegionInternals should not re-create .hregioninfo file when the region directory no longer exists


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

Branch: refs/heads/HBASE-14070.HLC
Commit: efd211debd8a37f215b1a6fdb982aa3ca890bc40
Parents: 5507150
Author: Esteban Gutierrez <es...@apache.org>
Authored: Fri Jul 21 13:13:00 2017 -0500
Committer: Esteban Gutierrez <es...@apache.org>
Committed: Thu Aug 10 17:56:17 2017 -0500

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/HRegion.java      | 11 +++-
 .../hbase/regionserver/HRegionFileSystem.java   | 31 ++++++++++--
 .../hadoop/hbase/regionserver/TestHRegion.java  |  7 ++-
 .../hbase/regionserver/TestRegionOpen.java      | 53 +++++++++++++++++++-
 .../TestStoreFileRefresherChore.java            |  1 +
 .../TestWALMonotonicallyIncreasingSeqId.java    |  5 +-
 6 files changed, 97 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/efd211de/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 483cb36..3b24f3d 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
@@ -888,8 +888,15 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     }
 
     // Write HRI to a file in case we need to recover hbase:meta
-    status.setStatus("Writing region info on filesystem");
-    fs.checkRegionInfoOnFilesystem();
+    // Only the primary replica should write .regioninfo
+    if (this.getRegionInfo().getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID) {
+      status.setStatus("Writing region info on filesystem");
+      fs.checkRegionInfoOnFilesystem();
+    } else {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Skipping creation of .regioninfo file for " + this.getRegionInfo());
+      }
+    }
 
     // Initialize all the HStores
     status.setStatus("Initializing all the Stores");

http://git-wip-us.apache.org/repos/asf/hbase/blob/efd211de/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
index 1041260..9cb1316 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
@@ -893,9 +893,19 @@ public class HRegionFileSystem {
     // only should be sufficient. I don't want to read the file every time to check if it pb
     // serialized.
     byte[] content = getRegionInfoFileContent(regionInfoForFs);
+
+    // Verify if the region directory exists before opening a region. We need to do this since if
+    // the region directory doesn't exist we will re-create the region directory and a new HRI
+    // when HRegion.openHRegion() is called.
     try {
-      Path regionInfoFile = new Path(getRegionDir(), REGION_INFO_FILE);
+      FileStatus status = fs.getFileStatus(getRegionDir());
+    } catch (FileNotFoundException e) {
+      LOG.warn(getRegionDir() + " doesn't exist for region: " + regionInfoForFs.getEncodedName() +
+          " on table " + regionInfo.getTable());
+    }
 
+    try {
+      Path regionInfoFile = new Path(getRegionDir(), REGION_INFO_FILE);
       FileStatus status = fs.getFileStatus(regionInfoFile);
       if (status != null && status.getLen() == content.length) {
         // Then assume the content good and move on.
@@ -988,7 +998,13 @@ public class HRegionFileSystem {
     }
 
     // Write HRI to a file in case we need to recover hbase:meta
-    regionFs.writeRegionInfoOnFilesystem(false);
+    // Only primary replicas should write region info
+    if (regionInfo.getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID) {
+      regionFs.writeRegionInfoOnFilesystem(false);
+    } else {
+      if (LOG.isDebugEnabled())
+        LOG.debug("Skipping creation of .regioninfo file for " + regionInfo);
+    }
     return regionFs;
   }
 
@@ -1018,8 +1034,15 @@ public class HRegionFileSystem {
       regionFs.cleanupSplitsDir();
       regionFs.cleanupMergesDir();
 
-      // if it doesn't exists, Write HRI to a file, in case we need to recover hbase:meta
-      regionFs.checkRegionInfoOnFilesystem();
+      // If it doesn't exists, Write HRI to a file, in case we need to recover hbase:meta
+      // Only create HRI if we are the default replica
+      if (regionInfo.getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID) {
+        regionFs.checkRegionInfoOnFilesystem();
+      } else {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Skipping creation of .regioninfo file for " + regionInfo);
+        }
+      }
     }
 
     return regionFs;

http://git-wip-us.apache.org/repos/asf/hbase/blob/efd211de/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 2838949..9db7c16 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
@@ -5899,6 +5899,10 @@ public class TestHRegion {
 
   @Test
   public void testCloseRegionWrittenToWAL() throws Exception {
+
+    Path rootDir = new Path(dir + name.getMethodName());
+    FSUtils.setRootDir(TEST_UTIL.getConfiguration(), rootDir);
+
     final ServerName serverName = ServerName.valueOf("testCloseRegionWrittenToWAL", 100, 42);
     final RegionServerServices rss = spy(TEST_UTIL.createMockRegionServerService(serverName));
 
@@ -5916,7 +5920,8 @@ public class TestHRegion {
     when(rss.getWAL((HRegionInfo) any())).thenReturn(wal);
 
 
-    // open a region first so that it can be closed later
+    // create and then open a region first so that it can be closed later
+    region = HRegion.createHRegion(hri, rootDir, TEST_UTIL.getConfiguration(), htd, rss.getWAL(hri));
     region = HRegion.openHRegion(hri, htd, rss.getWAL(hri),
       TEST_UTIL.getConfiguration(), rss, null);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/efd211de/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionOpen.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionOpen.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionOpen.java
index f49bdb1..3a1769e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionOpen.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionOpen.java
@@ -19,26 +19,39 @@
 package org.apache.hadoop.hbase.regionserver;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
+
+import java.io.IOException;
+import java.util.List;
 import java.util.concurrent.ThreadPoolExecutor;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+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.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.executor.ExecutorType;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+import static org.junit.Assert.fail;
 
 @Category({MediumTests.class, RegionServerTests.class})
 public class TestRegionOpen {
@@ -47,7 +60,9 @@ public class TestRegionOpen {
   private static final int NB_SERVERS = 1;
 
   private static final HBaseTestingUtility HTU = new HBaseTestingUtility();
-  final TableName tableName = TableName.valueOf(TestRegionOpen.class.getSimpleName());
+
+  @Rule
+  public TestName name = new TestName();
 
   @BeforeClass
   public static void before() throws Exception {
@@ -65,6 +80,7 @@ public class TestRegionOpen {
 
   @Test(timeout = 60000)
   public void testPriorityRegionIsOpenedWithSeparateThreadPool() throws Exception {
+    final TableName tableName = TableName.valueOf(TestRegionOpen.class.getSimpleName());
     ThreadPoolExecutor exec = getRS().getExecutorService()
         .getExecutorThreadPool(ExecutorType.RS_OPEN_PRIORITY_REGION);
 
@@ -80,4 +96,39 @@ public class TestRegionOpen {
 
     assertEquals(1, exec.getCompletedTaskCount());
   }
+
+  @Test(timeout = 60000)
+  public void testNonExistentRegionReplica() throws Exception {
+    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final byte[] FAMILYNAME = Bytes.toBytes("fam");
+    FileSystem fs = HTU.getTestFileSystem();
+    Admin admin = HTU.getAdmin();
+    Configuration conf = HTU.getConfiguration();
+    Path rootDir = HTU.getDataTestDirOnTestFS();
+
+    HTableDescriptor htd = new HTableDescriptor(tableName);
+    htd.addFamily(new HColumnDescriptor(FAMILYNAME));
+    admin.createTable(htd);
+    HTU.waitUntilNoRegionsInTransition(60000);
+
+    // Create new HRI with non-default region replica id
+    HRegionInfo hri = new HRegionInfo(htd.getTableName(),  Bytes.toBytes("A"), Bytes.toBytes("B"), false,
+        System.currentTimeMillis(), 2);
+    HRegionFileSystem regionFs = HRegionFileSystem.createRegionOnFileSystem(conf, fs,
+        FSUtils.getTableDir(rootDir, hri.getTable()), hri);
+    Path regionDir = regionFs.getRegionDir();
+    try {
+      HRegionFileSystem.loadRegionInfoFileContent(fs, regionDir);
+    } catch (IOException e) {
+      LOG.info("Caught expected IOE due missing .regioninfo file, due: " + e.getMessage() + " skipping region open.");
+      // We should only have 1 region online
+      List<HRegionInfo> regions = admin.getTableRegions(tableName);
+      LOG.info("Regions: " + regions);
+      if (regions.size() != 1) {
+        fail("Table " + tableName + " should have only one region, but got more: " + regions);
+      }
+      return;
+    }
+    fail("Should have thrown IOE when attempting to open a non-existing region.");
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/efd211de/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java
index 99dd00d..5ac7efb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java
@@ -176,6 +176,7 @@ public class TestStoreFileRefresherChore {
     when(regionServer.getConfiguration()).thenReturn(TEST_UTIL.getConfiguration());
 
     HTableDescriptor htd = getTableDesc(TableName.valueOf(name.getMethodName()), families);
+    htd.setRegionReplication(2);
     Region primary = initHRegion(htd, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, 0);
     Region replica1 = initHRegion(htd, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, 1);
     regions.add(primary);

http://git-wip-us.apache.org/repos/asf/hbase/blob/efd211de/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java
index e63bad9..6294541 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java
@@ -99,9 +99,8 @@ public class TestWALMonotonicallyIncreasingSeqId {
     this.walConf = walConf;
     wals = new WALFactory(walConf, null, "log_" + replicaId);
     ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
-    HRegion region = new HRegion(fs, wals.getWAL(info.getEncodedNameAsBytes(),
-        info.getTable().getNamespace()), conf, htd, null);
-    region.initialize();
+    HRegion region = HRegion.createHRegion(info, TEST_UTIL.getDefaultRootDirPath(), conf, htd,
+        wals.getWAL(info.getEncodedNameAsBytes(), info.getTable().getNamespace()));
     return region;
   }
 


[25/50] [abbrv] hbase git commit: HBASE-18522 Add RowMutations support to Batch

Posted by st...@apache.org.
HBASE-18522 Add RowMutations support to Batch


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

Branch: refs/heads/HBASE-14070.HLC
Commit: 096dac2e83c675f212bad4f91888d8440ba152ca
Parents: bd40073
Author: Jerry He <je...@apache.org>
Authored: Mon Aug 14 10:39:46 2017 -0700
Committer: Jerry He <je...@apache.org>
Committed: Mon Aug 14 10:39:46 2017 -0700

----------------------------------------------------------------------
 .../hbase/client/MultiServerCallable.java       | 64 +++++++++++++++-----
 .../org/apache/hadoop/hbase/client/Table.java   |  4 +-
 .../hbase/shaded/protobuf/RequestConverter.java |  6 +-
 .../shaded/protobuf/ResponseConverter.java      | 37 ++++++++++-
 .../hbase/client/TestFromClientSide3.java       | 46 ++++++++++++++
 .../hadoop/hbase/client/TestMultiParallel.java  | 34 ++++++++++-
 6 files changed, 168 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/096dac2e/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 33c9a0b..7f6052e 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
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.client;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
@@ -93,30 +94,64 @@ class MultiServerCallable extends CancellableRegionServerCallable<MultiResponse>
     RegionAction.Builder regionActionBuilder = RegionAction.newBuilder();
     ClientProtos.Action.Builder actionBuilder = ClientProtos.Action.newBuilder();
     MutationProto.Builder mutationBuilder = MutationProto.newBuilder();
-    List<CellScannable> cells = null;
-    // The multi object is a list of Actions by region.  Iterate by region.
+
+    // Pre-size. Presume at least a KV per Action. There are likely more.
+    List<CellScannable> cells =
+        (this.cellBlock ? new ArrayList<CellScannable>(countOfActions) : null);
+
     long nonceGroup = multiAction.getNonceGroup();
     if (nonceGroup != HConstants.NO_NONCE) {
       multiRequestBuilder.setNonceGroup(nonceGroup);
     }
+    // Index to track RegionAction within the MultiRequest
+    int regionActionIndex = -1;
+    // Map from a created RegionAction to the original index for a RowMutations within
+    // its original list of actions
+    Map<Integer, Integer> rowMutationsIndexMap = new HashMap<>();
+    // The multi object is a list of Actions by region. Iterate by region.
     for (Map.Entry<byte[], List<Action>> e: this.multiAction.actions.entrySet()) {
       final byte [] regionName = e.getKey();
       final List<Action> actions = e.getValue();
       regionActionBuilder.clear();
       regionActionBuilder.setRegion(RequestConverter.buildRegionSpecifier(
           HBaseProtos.RegionSpecifier.RegionSpecifierType.REGION_NAME, regionName));
-      if (this.cellBlock) {
-        // Pre-size. Presume at least a KV per Action.  There are likely more.
-        if (cells == null) cells = new ArrayList<>(countOfActions);
-        // Send data in cellblocks. The call to buildNoDataMultiRequest will skip RowMutations.
-        // They have already been handled above. Guess at count of cells
-        regionActionBuilder = RequestConverter.buildNoDataRegionAction(regionName, actions, cells,
-          regionActionBuilder, actionBuilder, mutationBuilder);
-      } else {
-        regionActionBuilder = RequestConverter.buildRegionAction(regionName, actions,
-          regionActionBuilder, actionBuilder, mutationBuilder);
+
+      int rowMutations = 0;
+      for (Action action : actions) {
+        Row row = action.getAction();
+        // Row Mutations are a set of Puts and/or Deletes all to be applied atomically
+        // on the one row. We do separate RegionAction for each RowMutations.
+        // We maintain a map to keep track of this RegionAction and the original Action index.
+        if (row instanceof RowMutations) {
+          RowMutations rms = (RowMutations)row;
+          if (this.cellBlock) {
+            // Build a multi request absent its Cell payload. Send data in cellblocks.
+            regionActionBuilder = RequestConverter.buildNoDataRegionAction(regionName, rms, cells,
+              regionActionBuilder, actionBuilder, mutationBuilder);
+          } else {
+            regionActionBuilder = RequestConverter.buildRegionAction(regionName, rms);
+          }
+          regionActionBuilder.setAtomic(true);
+          multiRequestBuilder.addRegionAction(regionActionBuilder.build());
+          regionActionIndex++;
+          rowMutationsIndexMap.put(regionActionIndex, action.getOriginalIndex());
+          rowMutations++;
+        }
+      }
+
+      if (actions.size() > rowMutations) {
+        if (this.cellBlock) {
+          // Send data in cellblocks. The call to buildNoDataRegionAction will skip RowMutations.
+          // They have already been handled above. Guess at count of cells
+          regionActionBuilder = RequestConverter.buildNoDataRegionAction(regionName, actions, cells,
+            regionActionBuilder, actionBuilder, mutationBuilder);
+        } else {
+          regionActionBuilder = RequestConverter.buildRegionAction(regionName, actions,
+            regionActionBuilder, actionBuilder, mutationBuilder);
+        }
+        multiRequestBuilder.addRegionAction(regionActionBuilder.build());
+        regionActionIndex++;
       }
-      multiRequestBuilder.addRegionAction(regionActionBuilder.build());
     }
 
     if (cells != null) {
@@ -125,7 +160,8 @@ class MultiServerCallable extends CancellableRegionServerCallable<MultiResponse>
     ClientProtos.MultiRequest requestProto = multiRequestBuilder.build();
     ClientProtos.MultiResponse responseProto = getStub().multi(getRpcController(), requestProto);
     if (responseProto == null) return null; // Occurs on cancel
-    return ResponseConverter.getResults(requestProto, responseProto, getRpcControllerCellScanner());
+    return ResponseConverter.getResults(requestProto, rowMutationsIndexMap, responseProto,
+      getRpcControllerCellScanner());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/096dac2e/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java
index cfe435e..a215903 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java
@@ -111,12 +111,12 @@ public interface Table extends Closeable {
   boolean[] existsAll(List<Get> gets) throws IOException;
 
   /**
-   * Method that does a batch call on Deletes, Gets, Puts, Increments and Appends.
+   * Method that does a batch call on Deletes, Gets, Puts, Increments, Appends, RowMutations.
    * The ordering of execution of the actions is not defined. Meaning if you do a Put and a
    * Get in the same {@link #batch} call, you will not necessarily be
    * guaranteed that the Get returns what the Put had put.
    *
-   * @param actions list of Get, Put, Delete, Increment, Append objects
+   * @param actions list of Get, Put, Delete, Increment, Append, RowMutations.
    * @param results Empty Object[], same size as actions. Provides access to partial
    *                results, in case an exception is thrown. A null in the result array means that
    *                the call for that action failed, even after retries. The order of the objects

http://git-wip-us.apache.org/repos/asf/hbase/blob/096dac2e/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
index be46c19..08ed3dc 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
@@ -668,7 +668,8 @@ public final class RequestConverter {
               .setMethodName(exec.getMethod().getName())
               .setRequest(value)));
       } else if (row instanceof RowMutations) {
-        throw new UnsupportedOperationException("No RowMutations in multi calls; use mutateRow");
+        // Skip RowMutations, which has been separately converted to RegionAction
+        continue;
       } else {
         throw new DoNotRetryIOException("Multi doesn't support " + row.getClass().getName());
       }
@@ -756,7 +757,8 @@ public final class RequestConverter {
               .setMethodName(exec.getMethod().getName())
               .setRequest(value)));
       } else if (row instanceof RowMutations) {
-        throw new UnsupportedOperationException("No RowMutations in multi calls; use mutateRow");
+        // Skip RowMutations, which has been separately converted to RegionAction
+        continue;
       } else {
         throw new DoNotRetryIOException("Multi doesn't support " + row.getClass().getName());
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/096dac2e/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ResponseConverter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ResponseConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ResponseConverter.java
index c489628..98e6f69 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ResponseConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ResponseConverter.java
@@ -89,7 +89,8 @@ public final class ResponseConverter {
   /**
    * Get the results from a protocol buffer MultiResponse
    *
-   * @param request the protocol buffer MultiResponse to convert
+   * @param request the original protocol buffer MultiRequest
+   * @param response the protocol buffer MultiResponse to convert
    * @param cells Cells to go with the passed in <code>proto</code>.  Can be null.
    * @return the results that were in the MultiResponse (a Result or an Exception).
    * @throws IOException
@@ -97,6 +98,22 @@ public final class ResponseConverter {
   public static org.apache.hadoop.hbase.client.MultiResponse getResults(final MultiRequest request,
       final MultiResponse response, final CellScanner cells)
   throws IOException {
+    return getResults(request, null, response, cells);
+  }
+
+  /**
+   * Get the results from a protocol buffer MultiResponse
+   *
+   * @param request the original protocol buffer MultiRequest
+   * @param rowMutationsIndexMap Used to support RowMutations in batch
+   * @param response the protocol buffer MultiResponse to convert
+   * @param cells Cells to go with the passed in <code>proto</code>.  Can be null.
+   * @return the results that were in the MultiResponse (a Result or an Exception).
+   * @throws IOException
+   */
+  public static org.apache.hadoop.hbase.client.MultiResponse getResults(final MultiRequest request,
+      final Map<Integer, Integer> rowMutationsIndexMap, final MultiResponse response,
+      final CellScanner cells) throws IOException {
     int requestRegionActionCount = request.getRegionActionCount();
     int responseRegionActionResultCount = response.getRegionActionResultCount();
     if (requestRegionActionCount != responseRegionActionResultCount) {
@@ -130,8 +147,24 @@ public final class ResponseConverter {
             actionResult.getResultOrExceptionCount() + " for region " + actions.getRegion());
       }
 
+      Object responseValue;
+
+      // For RowMutations action, if there is an exception, the exception is set
+      // at the RegionActionResult level and the ResultOrException is null at the original index
+      Integer rowMutationsIndex =
+          (rowMutationsIndexMap == null ? null : rowMutationsIndexMap.get(i));
+      if (rowMutationsIndex != null) {
+        // This RegionAction is from a RowMutations in a batch.
+        // If there is an exception from the server, the exception is set at
+        // the RegionActionResult level, which has been handled above.
+        responseValue = response.getProcessed() ?
+            ProtobufUtil.EMPTY_RESULT_EXISTS_TRUE :
+            ProtobufUtil.EMPTY_RESULT_EXISTS_FALSE;
+        results.add(regionName, rowMutationsIndex, responseValue);
+        continue;
+      }
+
       for (ResultOrException roe : actionResult.getResultOrExceptionList()) {
-        Object responseValue;
         if (roe.hasException()) {
           responseValue = ProtobufUtil.toException(roe.getException());
         } else if (roe.hasResult()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/096dac2e/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
index 668bfbb..f20c050 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
@@ -310,6 +310,52 @@ public class TestFromClientSide3 {
   }
 
   @Test
+  public void testBatchWithRowMutation() throws Exception {
+    LOG.info("Starting testBatchWithRowMutation");
+    final TableName TABLENAME = TableName.valueOf("testBatchWithRowMutation");
+    try (Table t = TEST_UTIL.createTable(TABLENAME, FAMILY)) {
+      byte [][] QUALIFIERS = new byte [][] {
+        Bytes.toBytes("a"), Bytes.toBytes("b")
+      };
+      RowMutations arm = new RowMutations(ROW);
+      Put p = new Put(ROW);
+      p.addColumn(FAMILY, QUALIFIERS[0], VALUE);
+      arm.add(p);
+      Object[] batchResult = new Object[1];
+      t.batch(Arrays.asList(arm), batchResult);
+
+      Get g = new Get(ROW);
+      Result r = t.get(g);
+      assertEquals(0, Bytes.compareTo(VALUE, r.getValue(FAMILY, QUALIFIERS[0])));
+
+      arm = new RowMutations(ROW);
+      p = new Put(ROW);
+      p.addColumn(FAMILY, QUALIFIERS[1], VALUE);
+      arm.add(p);
+      Delete d = new Delete(ROW);
+      d.addColumns(FAMILY, QUALIFIERS[0]);
+      arm.add(d);
+      t.batch(Arrays.asList(arm), batchResult);
+      r = t.get(g);
+      assertEquals(0, Bytes.compareTo(VALUE, r.getValue(FAMILY, QUALIFIERS[1])));
+      assertNull(r.getValue(FAMILY, QUALIFIERS[0]));
+
+      // Test that we get the correct remote exception for RowMutations from batch()
+      try {
+        arm = new RowMutations(ROW);
+        p = new Put(ROW);
+        p.addColumn(new byte[]{'b', 'o', 'g', 'u', 's'}, QUALIFIERS[0], VALUE);
+        arm.add(p);
+        t.batch(Arrays.asList(arm), batchResult);
+        fail("Expected RetriesExhaustedWithDetailsException with NoSuchColumnFamilyException");
+      } catch(RetriesExhaustedWithDetailsException e) {
+        String msg = e.getMessage();
+        assertTrue(msg.contains("NoSuchColumnFamilyException"));
+      }
+    }
+  }
+
+  @Test
   public void testHTableExistsMethodSingleRegionSingleGet() throws Exception {
       // Test with a single region table.
       Table table = TEST_UTIL.createTable(

http://git-wip-us.apache.org/repos/asf/hbase/blob/096dac2e/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
index a3c9649..62b6ae5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
@@ -642,6 +642,23 @@ public class TestMultiParallel {
     put.addColumn(BYTES_FAMILY, qual2, val2);
     actions.add(put);
 
+    // 6 RowMutations
+    RowMutations rm = new RowMutations(KEYS[50]);
+    put = new Put(KEYS[50]);
+    put.addColumn(BYTES_FAMILY, qual2, val2);
+    rm.add(put);
+    byte[] qual3 = Bytes.toBytes("qual3");
+    byte[] val3 = Bytes.toBytes("putvalue3");
+    put = new Put(KEYS[50]);
+    put.addColumn(BYTES_FAMILY, qual3, val3);
+    rm.add(put);
+    actions.add(rm);
+
+    // 7 Add another Get to the mixed sequence after RowMutations
+    get = new Get(KEYS[10]);
+    get.addColumn(BYTES_FAMILY, QUALIFIER);
+    actions.add(get);
+
     results = new Object[actions.size()];
     table.batch(actions, results);
 
@@ -649,10 +666,11 @@ public class TestMultiParallel {
 
     validateResult(results[0]);
     validateResult(results[1]);
-    validateEmpty(results[2]);
     validateEmpty(results[3]);
     validateResult(results[4]);
     validateEmpty(results[5]);
+    validateEmpty(results[6]);
+    validateResult(results[7]);
 
     // validate last put, externally from the batch
     get = new Get(KEYS[40]);
@@ -660,6 +678,17 @@ public class TestMultiParallel {
     Result r = table.get(get);
     validateResult(r, qual2, val2);
 
+    // validate last RowMutations, externally from the batch
+    get = new Get(KEYS[50]);
+    get.addColumn(BYTES_FAMILY, qual2);
+    r = table.get(get);
+    validateResult(r, qual2, val2);
+
+    get = new Get(KEYS[50]);
+    get.addColumn(BYTES_FAMILY, qual3);
+    r = table.get(get);
+    validateResult(r, qual3, val3);
+
     table.close();
   }
 
@@ -736,8 +765,7 @@ public class TestMultiParallel {
   private void validateEmpty(Object r1) {
     Result result = (Result)r1;
     Assert.assertTrue(result != null);
-    Assert.assertTrue(result.getRow() == null);
-    Assert.assertEquals(0, result.rawCells().length);
+    Assert.assertTrue(result.isEmpty());
   }
 
   private void validateSizeAndEmpty(Object[] results, int expectedSize) {


[09/50] [abbrv] hbase git commit: HBASE-18558 clean up duplicate dependency management entries for hbase-shaded-miscellaneous

Posted by st...@apache.org.
HBASE-18558 clean up duplicate dependency management entries for hbase-shaded-miscellaneous


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

Branch: refs/heads/HBASE-14070.HLC
Commit: 043ec9b37e43328e8784f88e3d6867b007a31d1d
Parents: 6f44b24
Author: Michael Stack <st...@apache.org>
Authored: Fri Aug 11 07:24:17 2017 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Fri Aug 11 07:24:17 2017 -0700

----------------------------------------------------------------------
 pom.xml | 5 -----
 1 file changed, 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/043ec9b3/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 7648e8e..7925e4e 100755
--- a/pom.xml
+++ b/pom.xml
@@ -1718,11 +1718,6 @@
         <version>${metrics-core.version}</version>
       </dependency>
       <dependency>
-        <groupId>org.apache.hbase.thirdparty</groupId>
-        <artifactId>hbase-shaded-miscellaneous</artifactId>
-        <version>${hbase-thirdparty.version}</version>
-      </dependency>
-      <dependency>
         <groupId>commons-collections</groupId>
         <artifactId>commons-collections</artifactId>
         <version>${collections.version}</version>


[45/50] [abbrv] hbase git commit: HBASE-18271 Shade netty Purge mention of netty-all; ADDENDUM for sparktest

Posted by st...@apache.org.
HBASE-18271 Shade netty Purge mention of netty-all; ADDENDUM for sparktest


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

Branch: refs/heads/HBASE-14070.HLC
Commit: 5d2c3ddf5238fa0b48b8aa4fb99108eaf69f16aa
Parents: 4734467
Author: Michael Stack <st...@duboce.net>
Authored: Wed Aug 16 10:04:33 2017 -0700
Committer: Michael Stack <st...@duboce.net>
Committed: Wed Aug 16 10:04:56 2017 -0700

----------------------------------------------------------------------
 hbase-spark/pom.xml | 8 +++++---
 1 file changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/5d2c3ddf/hbase-spark/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-spark/pom.xml b/hbase-spark/pom.xml
index 8137b53..d294835 100644
--- a/hbase-spark/pom.xml
+++ b/hbase-spark/pom.xml
@@ -559,9 +559,11 @@
               <goal>test</goal>
             </goals>
             <configuration>
-              <argLine>
-                                -Xmx1536m -XX:ReservedCodeCacheSize=512m
-                            </argLine>
+              <systemProperties>
+                <org.apache.hadoop.hbase.shaded.io.netty.packagePrefix>org.apache.hadoop.hbase.shaded.</org.apache.hadoop.hbase.shaded.io.netty.packagePrefix>
+              </systemProperties>
+              <argLine> -Xmx1536m -XX:ReservedCodeCacheSize=512m
+              </argLine>
               <parallel>false</parallel>
             </configuration>
           </execution>


[12/50] [abbrv] hbase git commit: HBASE-18526 FIFOCompactionPolicy pre-check uses wrong scope (Vladimir Rodionov)

Posted by st...@apache.org.
HBASE-18526 FIFOCompactionPolicy pre-check uses wrong scope (Vladimir Rodionov)


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

Branch: refs/heads/HBASE-14070.HLC
Commit: aa8f67a148cbefbfc4bfdc25b2dc48c7ed947212
Parents: 1070888
Author: tedyu <yu...@gmail.com>
Authored: Fri Aug 11 16:41:40 2017 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Fri Aug 11 16:41:40 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/master/HMaster.java   | 18 +++++++++---------
 1 file changed, 9 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/aa8f67a1/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 96bf859..421ae8b 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
@@ -53,7 +53,6 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.ClusterStatus;
 import org.apache.hadoop.hbase.CoordinatedStateException;
 import org.apache.hadoop.hbase.CoordinatedStateManager;
@@ -79,6 +78,7 @@ import org.apache.hadoop.hbase.UnknownRegionException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.MasterSwitchType;
 import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.coprocessor.BypassCoprocessorException;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
@@ -164,6 +164,9 @@ import org.apache.hadoop.hbase.replication.master.TableCFsUpdater;
 import org.apache.hadoop.hbase.replication.regionserver.Replication;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.security.UserProvider;
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionServerInfo;
@@ -200,9 +203,6 @@ import org.eclipse.jetty.server.ServerConnector;
 import org.eclipse.jetty.servlet.ServletHolder;
 import org.eclipse.jetty.webapp.WebAppContext;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
 import com.google.protobuf.Descriptors;
 import com.google.protobuf.Service;
 
@@ -1937,14 +1937,14 @@ public class HMaster extends HRegionServer implements MasterServices {
       }
 
       // 3. blocking file count
-      String sbfc = htd.getConfigurationValue(HStore.BLOCKING_STOREFILES_KEY);
-      if (sbfc != null) {
-        blockingFileCount = Integer.parseInt(sbfc);
+      sv = hcd.getConfigurationValue(HStore.BLOCKING_STOREFILES_KEY);
+      if (sv != null) {
+        blockingFileCount = Integer.parseInt(sv);
       }
       if (blockingFileCount < 1000) {
         message =
-            "blocking file count '" + HStore.BLOCKING_STOREFILES_KEY + "' " + blockingFileCount
-                + " is below recommended minimum of 1000";
+            "Blocking file count '" + HStore.BLOCKING_STOREFILES_KEY + "' " + blockingFileCount
+                + " is below recommended minimum of 1000 for column family "+ hcd.getNameAsString();
         throw new IOException(message);
       }
     }


[39/50] [abbrv] hbase git commit: HBASE-18437 Revoke access permissions of a user from a table does not work as expected

Posted by st...@apache.org.
HBASE-18437 Revoke access permissions of a user from a table does not work as expected

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


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

Branch: refs/heads/HBASE-14070.HLC
Commit: b0878184a31804a4bf061df7581964157b4849d5
Parents: 59ffb611
Author: Ashish Singhi <as...@apache.org>
Authored: Fri Aug 11 12:48:32 2017 +0530
Committer: Andrew Purtell <ap...@apache.org>
Committed: Tue Aug 15 22:29:16 2017 -0700

----------------------------------------------------------------------
 .../hbase/security/access/Permission.java       |  6 ++
 .../security/access/AccessControlLists.java     | 37 +++++++-
 .../security/access/TestAccessController.java   | 96 ++++++++++++++------
 3 files changed, 106 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b0878184/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/Permission.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/Permission.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/Permission.java
index 8476f61..18096e1 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/Permission.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/Permission.java
@@ -110,6 +110,12 @@ public class Permission extends VersionedWritable {
     return false;
   }
 
+  public void setActions(Action[] assigned) {
+    if (assigned != null && assigned.length > 0) {
+      actions = Arrays.copyOf(assigned, assigned.length);
+    }
+  }
+
   @Override
   public boolean equals(Object obj) {
     if (!(obj instanceof Permission)) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/b0878184/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 12bdc22..38e292c 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
@@ -241,13 +241,40 @@ public class AccessControlLists {
    */
   static void removeUserPermission(Configuration conf, UserPermission userPerm, Table t)
       throws IOException {
-    Delete d = new Delete(userPermissionRowKey(userPerm));
-    byte[] key = userPermissionKey(userPerm);
-
+    if (null == userPerm.getActions()) {
+      removePermissionRecord(conf, userPerm, t);
+    } else {
+      // Get all the global user permissions from the acl table
+      List<UserPermission> permsList = getUserPermissions(conf, userPermissionRowKey(userPerm));
+      List<Permission.Action> remainingActions = new ArrayList<>();
+      List<Permission.Action> dropActions = Arrays.asList(userPerm.getActions());
+      for (UserPermission perm : permsList) {
+        // Find the user and remove only the requested permissions
+        if (Bytes.toString(perm.getUser()).equals(Bytes.toString(userPerm.getUser()))) {
+          for (Permission.Action oldAction : perm.getActions()) {
+            if (!dropActions.contains(oldAction)) {
+              remainingActions.add(oldAction);
+            }
+          }
+          if (!remainingActions.isEmpty()) {
+            perm.setActions(remainingActions.toArray(new Permission.Action[remainingActions.size()]));
+            addUserPermission(conf, perm, t);
+          } else {
+            removePermissionRecord(conf, userPerm, t);
+          }
+          break;
+        }
+      }
+    }
     if (LOG.isDebugEnabled()) {
-      LOG.debug("Removing permission "+ userPerm.toString());
+      LOG.debug("Removed permission "+ userPerm.toString());
     }
-    d.addColumns(ACL_LIST_FAMILY, key);
+  }
+
+  private static void removePermissionRecord(Configuration conf, UserPermission userPerm, Table t)
+      throws IOException {
+    Delete d = new Delete(userPermissionRowKey(userPerm));
+    d.addColumns(ACL_LIST_FAMILY, userPermissionKey(userPerm));
     try {
       t.delete(d);
     } finally {

http://git-wip-us.apache.org/repos/asf/hbase/blob/b0878184/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 c1fbb28..6583366 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
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hbase.security.access;
 
 import static org.apache.hadoop.hbase.AuthUtil.toGroupEntry;
+import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
@@ -2380,28 +2381,30 @@ public class TestAccessController extends SecureTestUtil {
     // Grant table READ permissions to testGlobalGrantRevoke.
     String userName = testGlobalGrantRevoke.getShortName();
     try {
-      grantGlobalUsingAccessControlClient(TEST_UTIL, systemUserConnection,
-          userName, Permission.Action.READ);
+      grantGlobalUsingAccessControlClient(TEST_UTIL, systemUserConnection, userName,
+        Permission.Action.READ);
     } catch (Throwable e) {
       LOG.error("error during call of AccessControlClient.grant. ", e);
     }
     try {
       // Now testGlobalGrantRevoke should be able to read also
       verifyAllowed(getAction, testGlobalGrantRevoke);
-
-      // Revoke table READ permission to testGlobalGrantRevoke.
-      try {
-        revokeGlobalUsingAccessControlClient(TEST_UTIL, systemUserConnection,
-          userName, Permission.Action.READ);
-      } catch (Throwable e) {
-        LOG.error("error during call of AccessControlClient.revoke ", e);
-      }
-
-      // Now testGlobalGrantRevoke shouldn't be able read
-      verifyDenied(getAction, testGlobalGrantRevoke);
-    } finally {
+    } catch (Exception e) {
       revokeGlobal(TEST_UTIL, userName, Permission.Action.READ);
+      throw e;
     }
+
+    // Revoke table READ permission to testGlobalGrantRevoke.
+    try {
+      revokeGlobalUsingAccessControlClient(TEST_UTIL, systemUserConnection, userName,
+        Permission.Action.READ);
+    } catch (Throwable e) {
+      LOG.error("error during call of AccessControlClient.revoke ", e);
+    }
+
+    // Now testGlobalGrantRevoke shouldn't be able read
+    verifyDenied(getAction, testGlobalGrantRevoke);
+
   }
 
   @Test(timeout = 180000)
@@ -2547,28 +2550,29 @@ public class TestAccessController extends SecureTestUtil {
     String namespace = TEST_TABLE.getNamespaceAsString();
     // Grant namespace READ to testNS, this should supersede any table permissions
     try {
-      grantOnNamespaceUsingAccessControlClient(TEST_UTIL, systemUserConnection, userName,
-        namespace, Permission.Action.READ);
+      grantOnNamespaceUsingAccessControlClient(TEST_UTIL, systemUserConnection, userName, namespace,
+        Permission.Action.READ);
     } catch (Throwable e) {
       LOG.error("error during call of AccessControlClient.grant. ", e);
     }
     try {
       // Now testNS should be able to read also
       verifyAllowed(getAction, testNS);
-
-      // Revoke namespace READ to testNS, this should supersede any table permissions
-      try {
-        revokeFromNamespaceUsingAccessControlClient(TEST_UTIL, systemUserConnection, userName,
-          namespace, Permission.Action.READ);
-      } catch (Throwable e) {
-        LOG.error("error during call of AccessControlClient.revoke ", e);
-      }
-
-      // Now testNS shouldn't be able read
-      verifyDenied(getAction, testNS);
-    } finally {
+    } catch (Exception e) {
       revokeFromNamespace(TEST_UTIL, userName, namespace, Permission.Action.READ);
+      throw e;
+    }
+
+    // Revoke namespace READ to testNS, this should supersede any table permissions
+    try {
+      revokeFromNamespaceUsingAccessControlClient(TEST_UTIL, systemUserConnection, userName,
+        namespace, Permission.Action.READ);
+    } catch (Throwable e) {
+      LOG.error("error during call of AccessControlClient.revoke ", e);
     }
+
+    // Now testNS shouldn't be able read
+    verifyDenied(getAction, testNS);
   }
 
 
@@ -3175,4 +3179,40 @@ public class TestAccessController extends SecureTestUtil {
     verifyAllowed(regionLockHeartbeatAction, SUPERUSER, USER_ADMIN, namespaceUser, tableACUser);
     verifyDenied(regionLockHeartbeatAction, globalRWXUser, tableRWXUser);
   }
+
+  @Test
+  public void testAccessControlRevokeOnlyFewPermission() throws Throwable {
+    TableName tname = TableName.valueOf("revoke");
+    try {
+      TEST_UTIL.createTable(tname, TEST_FAMILY);
+      User testUserPerms = User.createUserForTesting(conf, "revokePerms", new String[0]);
+      Permission.Action[] actions = { Action.READ, Action.WRITE };
+      AccessControlClient.grant(TEST_UTIL.getConnection(), tname, testUserPerms.getShortName(),
+        null, null, actions);
+
+      List<UserPermission> userPermissions = AccessControlClient
+          .getUserPermissions(TEST_UTIL.getConnection(), tname.getNameAsString());
+      assertEquals(2, userPermissions.size());
+
+      AccessControlClient.revoke(TEST_UTIL.getConnection(), tname, testUserPerms.getShortName(),
+        null, null, Action.WRITE);
+
+      userPermissions = AccessControlClient.getUserPermissions(TEST_UTIL.getConnection(),
+        tname.getNameAsString());
+      assertEquals(2, userPermissions.size());
+
+      Permission.Action[] expectedAction = { Action.READ };
+      boolean userFound = false;
+      for (UserPermission p : userPermissions) {
+        if (testUserPerms.getShortName().equals(Bytes.toString(p.getUser()))) {
+          assertArrayEquals(expectedAction, p.getActions());
+          userFound = true;
+          break;
+        }
+      }
+      assertTrue(userFound);
+    } finally {
+      TEST_UTIL.deleteTable(tname);
+    }
+  }
 }


[08/50] [abbrv] hbase git commit: HBASE-18551 [AMv2] UnassignProcedure and crashed regionservers

Posted by st...@apache.org.
HBASE-18551 [AMv2] UnassignProcedure and crashed regionservers

If an unassign is unable to communicate with its target server,
expire the server and then wait on a signal from ServerCrashProcedure
before proceeding. The unassign has lock on the region so no one else
can proceed till we complete. We prevent any subsequent assign from
running until logs have been split for crashed server.

In AssignProcedure, do not assign if table is DISABLING or DISABLED.

M hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
 Change remoteCallFailed so it returns boolean on whether implementor
wants to stay suspended or not.

M hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/UnassignProcedure.java
  Doc. Also, if we are unable to talk to remote server, expire it and
then wait on SCP to wake us up after it has processed logs for failed
server.


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

Branch: refs/heads/HBASE-14070.HLC
Commit: 6f44b24860192d81dbf88ffd834d4b998a6fe636
Parents: cabdbf1
Author: Michael Stack <st...@apache.org>
Authored: Thu Aug 10 14:22:56 2017 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Fri Aug 11 07:16:33 2017 -0700

----------------------------------------------------------------------
 .../hbase/procedure2/ProcedureExecutor.java     | 10 +--
 .../hadoop/hbase/master/MasterRpcServices.java  |  2 +-
 .../hbase/master/TableNamespaceManager.java     |  2 +-
 .../hadoop/hbase/master/TableStateManager.java  |  1 +
 .../master/assignment/AssignProcedure.java      | 13 +++-
 .../assignment/RegionTransitionProcedure.java   | 44 ++++++------
 .../master/assignment/UnassignProcedure.java    | 70 ++++++++++----------
 .../master/procedure/DisableTableProcedure.java |  4 +-
 .../master/procedure/RSProcedureDispatcher.java |  2 +-
 .../master/procedure/ServerCrashException.java  |  3 +-
 .../master/procedure/ServerCrashProcedure.java  |  3 +-
 .../TestSplitTransactionOnCluster.java          | 17 +++--
 12 files changed, 100 insertions(+), 71 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/6f44b248/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
index c110c2d..d0052f6 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
@@ -315,7 +315,7 @@ public class ProcedureExecutor<TEnvironment> {
       @Override
       public void setMaxProcId(long maxProcId) {
         assert lastProcId.get() < 0 : "expected only one call to setMaxProcId()";
-        LOG.debug("Load maxProcId=" + maxProcId);
+        LOG.debug("Load max pid=" + maxProcId);
         lastProcId.set(maxProcId);
       }
 
@@ -727,7 +727,7 @@ public class ProcedureExecutor<TEnvironment> {
            !(procedures.containsKey(oldProcId) || completed.containsKey(oldProcId)) &&
            nonceKeysToProcIdsMap.containsKey(nonceKey)) {
       if (traceEnabled) {
-        LOG.trace("Waiting for procId=" + oldProcId.longValue() + " to be submitted");
+        LOG.trace("Waiting for pid=" + oldProcId.longValue() + " to be submitted");
       }
       Threads.sleep(100);
     }
@@ -999,9 +999,9 @@ public class ProcedureExecutor<TEnvironment> {
   public void removeResult(final long procId) {
     CompletedProcedureRetainer retainer = completed.get(procId);
     if (retainer == null) {
-      assert !procedures.containsKey(procId) : "procId=" + procId + " is still running";
+      assert !procedures.containsKey(procId) : "pid=" + procId + " is still running";
       if (LOG.isDebugEnabled()) {
-        LOG.debug("procId=" + procId + " already removed by the cleaner.");
+        LOG.debug("pid=" + procId + " already removed by the cleaner.");
       }
       return;
     }
@@ -1349,7 +1349,7 @@ public class ProcedureExecutor<TEnvironment> {
       return LockState.LOCK_YIELD_WAIT;
     } catch (Throwable e) {
       // Catch NullPointerExceptions or similar errors...
-      LOG.fatal("CODE-BUG: Uncaught runtime exception fo " + proc, e);
+      LOG.fatal("CODE-BUG: Uncaught runtime exception for " + proc, e);
     }
 
     // allows to kill the executor before something is stored to the wal.

http://git-wip-us.apache.org/repos/asf/hbase/blob/6f44b248/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index 5a2cd17..995df9b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -1007,7 +1007,7 @@ public class MasterRpcServices extends RSRpcServices
   @Override
   public GetProcedureResultResponse getProcedureResult(RpcController controller,
       GetProcedureResultRequest request) throws ServiceException {
-    LOG.debug("Checking to see if procedure is done procId=" + request.getProcId());
+    LOG.debug("Checking to see if procedure is done pid=" + request.getProcId());
     try {
       master.checkInitialized();
       GetProcedureResultResponse.Builder builder = GetProcedureResultResponse.newBuilder();

http://git-wip-us.apache.org/repos/asf/hbase/blob/6f44b248/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java
index 3a11e23..d69b0c0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java
@@ -240,7 +240,7 @@ public class TableNamespaceManager {
       // Sleep some
       Threads.sleep(10);
     }
-    throw new TimeoutIOException("Procedure " + procId + " is still running");
+    throw new TimeoutIOException("Procedure pid=" + procId + " is still running");
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/6f44b248/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
index 3b13b87..18f6856 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.client.TableState;
 /**
  * This is a helper class used to manage table states.
  * States persisted in tableinfo and cached internally.
+ * TODO: Cache state. Cut down on meta looksups.
  */
 @InterfaceAudience.Private
 public class TableStateManager {

http://git-wip-us.apache.org/repos/asf/hbase/blob/6f44b248/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignProcedure.java
index 6338983..d78ba74 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignProcedure.java
@@ -27,10 +27,13 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.RetriesExhaustedException;
+import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
 import org.apache.hadoop.hbase.master.RegionState.State;
+import org.apache.hadoop.hbase.master.TableStateManager;
 import org.apache.hadoop.hbase.master.assignment.RegionStates.RegionStateNode;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.procedure.RSProcedureDispatcher.RegionOpenOperation;
@@ -150,6 +153,13 @@ public class AssignProcedure extends RegionTransitionProcedure {
       LOG.info("Assigned, not reassigning; " + this + "; " + regionNode.toShortString());
       return false;
     }
+    // Don't assign if table is in disabling of disabled state.
+    TableStateManager tsm = env.getMasterServices().getTableStateManager();
+    TableName tn = regionNode.getRegionInfo().getTable();
+    if (tsm.isTableState(tn, TableState.State.DISABLING, TableState.State.DISABLED)) {
+      LOG.info("Table " + tn + " state=" + tsm.getTableState(tn) + ", skipping " + this);
+      return false;
+    }
     // If the region is SPLIT, we can't assign it. But state might be CLOSED, rather than
     // SPLIT which is what a region gets set to when Unassigned as part of SPLIT. FIX.
     if (regionNode.isInState(State.SPLIT) ||
@@ -321,9 +331,10 @@ public class AssignProcedure extends RegionTransitionProcedure {
   }
 
   @Override
-  protected void remoteCallFailed(final MasterProcedureEnv env, final RegionStateNode regionNode,
+  protected boolean remoteCallFailed(final MasterProcedureEnv env, final RegionStateNode regionNode,
       final IOException exception) {
     handleFailure(env, regionNode);
+    return true;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/6f44b248/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
index dd8dedc..5fc171a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
@@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProto
  * The AssignmentManager will notify this procedure when the RS completes
  * the operation and reports the transitioned state
  * (see the Assign and Unassign class for more detail).
+ *
  * <p>Procedures move from the REGION_TRANSITION_QUEUE state when they are
  * first submitted, to the REGION_TRANSITION_DISPATCH state when the request
  * to remote server is sent and the Procedure is suspended waiting on external
@@ -67,20 +68,15 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProto
  * assignment to a different target server by setting {@link AssignProcedure#forceNewPlan}. When
  * the number of attempts reach hreshold configuration 'hbase.assignment.maximum.attempts',
  * the procedure is aborted. For {@link UnassignProcedure}, similar re-attempts are
- * intentionally not implemented. It is a 'one shot' procedure.
+ * intentionally not implemented. It is a 'one shot' procedure. See its class doc for how it
+ * handles failure.
  * </li>
  * </ul>
  *
  * <p>TODO: Considering it is a priority doing all we can to get make a region available as soon as possible,
  * re-attempting with any target makes sense if specified target fails in case of
- * {@link AssignProcedure}. For {@link UnassignProcedure}, if communication with RS fails,
- * similar re-attempt makes little sense (what should be different from previous attempt?). Also it
- * could be complex with current implementation of
- * {@link RegionTransitionProcedure#execute(MasterProcedureEnv)} and {@link UnassignProcedure}.
- * We have made a choice of keeping {@link UnassignProcedure} simple, where the procedure either
- * succeeds or fails depending on communication with RS. As parent will have broader context, parent
- * can better handle the failed instance of {@link UnassignProcedure}. Similar simplicity for
- * {@link AssignProcedure} is desired and should be explored/ discussed further.
+ * {@link AssignProcedure}. For {@link UnassignProcedure}, our concern is preventing data loss
+ * on failed unassign. See class doc for explanation.
  */
 @InterfaceAudience.Private
 public abstract class RegionTransitionProcedure
@@ -165,7 +161,13 @@ public abstract class RegionTransitionProcedure
       RegionStateNode regionNode, TransitionCode code, long seqId) throws UnexpectedStateException;
 
   public abstract RemoteOperation remoteCallBuild(MasterProcedureEnv env, ServerName serverName);
-  protected abstract void remoteCallFailed(MasterProcedureEnv env,
+
+  /**
+   * @return True if processing of fail is complete; the procedure will be woken from its suspend
+   * and we'll go back to running through procedure steps:
+   * otherwise if false we leave the procedure in suspended state.
+   */
+  protected abstract boolean remoteCallFailed(MasterProcedureEnv env,
       RegionStateNode regionNode, IOException exception);
 
   @Override
@@ -181,12 +183,15 @@ public abstract class RegionTransitionProcedure
     assert serverName.equals(regionNode.getRegionLocation());
     String msg = exception.getMessage() == null? exception.getClass().getSimpleName():
       exception.getMessage();
-    LOG.warn("Failed " + this + "; " + regionNode.toShortString() + "; exception=" + msg);
-    remoteCallFailed(env, regionNode, exception);
-    // NOTE: This call to wakeEvent puts this Procedure back on the scheduler.
-    // Thereafter, another Worker can be in here so DO NOT MESS WITH STATE beyond
-    // this method. Just get out of this current processing quickly.
-    env.getProcedureScheduler().wakeEvent(regionNode.getProcedureEvent());
+    LOG.warn("Remote call failed " + this + "; " + regionNode.toShortString() +
+      "; exception=" + msg);
+    if (remoteCallFailed(env, regionNode, exception)) {
+      // NOTE: This call to wakeEvent puts this Procedure back on the scheduler.
+      // Thereafter, another Worker can be in here so DO NOT MESS WITH STATE beyond
+      // this method. Just get out of this current processing quickly.
+      env.getProcedureScheduler().wakeEvent(regionNode.getProcedureEvent());
+    }
+    // else leave the procedure in suspended state; it is waiting on another call to this callback
   }
 
   /**
@@ -210,9 +215,10 @@ public abstract class RegionTransitionProcedure
     // from remote regionserver. Means Procedure associated ProcedureEvent is marked not 'ready'.
     env.getProcedureScheduler().suspendEvent(getRegionState(env).getProcedureEvent());
 
-    // Tricky because this can fail. If it fails need to backtrack on stuff like
-    // the 'suspend' done above -- tricky as the 'wake' requeues us -- and ditto
-    // up in the caller; it needs to undo state changes.
+    // Tricky because the below call to addOperationToNode can fail. If it fails, we need to
+    // backtrack on stuff like the 'suspend' done above -- tricky as the 'wake' requeues us -- and
+    // ditto up in the caller; it needs to undo state changes. Inside in remoteCallFailed, it does
+    // wake to undo the above suspend.
     if (!env.getRemoteDispatcher().addOperationToNode(targetServer, this)) {
       remoteCallFailed(env, targetServer,
           new FailedRemoteDispatchException(this + " to " + targetServer));

http://git-wip-us.apache.org/repos/asf/hbase/blob/6f44b248/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/UnassignProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/UnassignProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/UnassignProcedure.java
index c6b7e4b..c9f0fac 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/UnassignProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/UnassignProcedure.java
@@ -22,6 +22,7 @@ package org.apache.hadoop.hbase.master.assignment;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.net.ConnectException;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.commons.logging.Log;
@@ -49,18 +50,28 @@ import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
 
 
 /**
- * Procedure that describe the unassignment of a single region.
- * There can only be one RegionTransitionProcedure per region running at the time,
- * since each procedure takes a lock on the region.
+ * Procedure that describes the unassignment of a single region.
+ * There can only be one RegionTransitionProcedure -- i.e. an assign or an unassign -- per region
+ * running at a time, since each procedure takes a lock on the region.
  *
  * <p>The Unassign starts by placing a "close region" request in the Remote Dispatcher
- * queue, and the procedure will then go into a "waiting state".
+ * queue, and the procedure will then go into a "waiting state" (suspend).
  * The Remote Dispatcher will batch the various requests for that server and
  * they will be sent to the RS for execution.
  * The RS will complete the open operation by calling master.reportRegionStateTransition().
- * The AM will intercept the transition report, and notify the procedure.
- * The procedure will finish the unassign by publishing its new state on meta
- * or it will retry the unassign.
+ * The AM will intercept the transition report, and notify this procedure.
+ * The procedure will wakeup and finish the unassign by publishing its new state on meta.
+ * <p>If we are unable to contact the remote regionserver whether because of ConnectException
+ * or socket timeout, we will call expire on the server we were trying to contact. We will remain
+ * in suspended state waiting for a wake up from the ServerCrashProcedure that is processing the
+ * failed server. The basic idea is that if we notice a crashed server, then we have a
+ * responsibility; i.e. we should not let go of the region until we are sure the server that was
+ * hosting has had its crash processed. If we let go of the region before then, an assign might
+ * run before the logs have been split which would make for data loss.
+ *
+ * <p>TODO: Rather than this tricky coordination between SCP and this Procedure, instead, work on
+ * returning a SCP as our subprocedure; probably needs work on the framework to do this,
+ * especially if the SCP already created.
  */
 @InterfaceAudience.Private
 public class UnassignProcedure extends RegionTransitionProcedure {
@@ -75,8 +86,6 @@ public class UnassignProcedure extends RegionTransitionProcedure {
    */
   protected volatile ServerName destinationServer;
 
-  protected final AtomicBoolean serverCrashed = new AtomicBoolean(false);
-
   // TODO: should this be in a reassign procedure?
   //       ...and keep unassign for 'disable' case?
   private boolean force;
@@ -161,15 +170,6 @@ public class UnassignProcedure extends RegionTransitionProcedure {
       return false;
     }
 
-    // if the server is down, mark the operation as failed. region cannot be unassigned
-    // if server is down
-    if (serverCrashed.get() || !isServerOnline(env, regionNode)) {
-      LOG.warn("Server already down: " + this + "; " + regionNode.toShortString());
-      setFailure("source region server not online",
-          new ServerCrashException(getProcId(), regionNode.getRegionLocation()));
-      return false;
-    }
-
     // if we haven't started the operation yet, we can abort
     if (aborted.get() && regionNode.isInState(State.OPEN)) {
       setAbortFailure(getClass().getSimpleName(), "abort requested");
@@ -181,13 +181,10 @@ public class UnassignProcedure extends RegionTransitionProcedure {
 
     // Add the close region operation the the server dispatch queue.
     if (!addToRemoteDispatcher(env, regionNode.getRegionLocation())) {
-      // If addToRemoteDispatcher fails, it calls #remoteCallFailed which
-      // does all cleanup.
+      // If addToRemoteDispatcher fails, it calls the callback #remoteCallFailed.
     }
 
-    // We always return true, even if we fail dispatch because addToRemoteDispatcher
-    // failure processing sets state back to REGION_TRANSITION_QUEUE so we try again;
-    // i.e. return true to keep the Procedure running; it has been reset to startover.
+    // Return true to keep the procedure running.
     return true;
   }
 
@@ -218,13 +215,15 @@ public class UnassignProcedure extends RegionTransitionProcedure {
   }
 
   @Override
-  protected void remoteCallFailed(final MasterProcedureEnv env, final RegionStateNode regionNode,
+  protected boolean remoteCallFailed(final MasterProcedureEnv env, final RegionStateNode regionNode,
       final IOException exception) {
     // TODO: Is there on-going rpc to cleanup?
     if (exception instanceof ServerCrashException) {
       // This exception comes from ServerCrashProcedure after log splitting.
-      // It is ok to let this procedure go on to complete close now.
-      // This will release lock on this region so the subsequent assign can succeed.
+      // SCP found this region as a RIT. Its call into here says it is ok to let this procedure go
+      // on to a complete close now. This will release lock on this region so subsequent action on
+      // region can succeed; e.g. the assign that follows this unassign when a move (w/o wait on SCP
+      // the assign could run w/o logs being split so data loss).
       try {
         reportTransition(env, regionNode, TransitionCode.CLOSED, HConstants.NO_SEQNUM);
       } catch (UnexpectedStateException e) {
@@ -237,19 +236,22 @@ public class UnassignProcedure extends RegionTransitionProcedure {
       // TODO
       // RS is aborting, we cannot offline the region since the region may need to do WAL
       // recovery. Until we see the RS expiration, we should retry.
+      // TODO: This should be suspend like the below where we call expire on server?
       LOG.info("Ignoring; waiting on ServerCrashProcedure", exception);
-      // serverCrashed.set(true);
     } else if (exception instanceof NotServingRegionException) {
-      LOG.info("IS THIS OK? ANY LOGS TO REPLAY; ACTING AS THOUGH ALL GOOD " + regionNode, exception);
+      LOG.info("IS THIS OK? ANY LOGS TO REPLAY; ACTING AS THOUGH ALL GOOD " + regionNode,
+        exception);
       setTransitionState(RegionTransitionState.REGION_TRANSITION_FINISH);
     } else {
-      // TODO: kill the server in case we get an exception we are not able to handle
-      LOG.warn("Killing server; unexpected exception; " +
-          this + "; " + regionNode.toShortString() +
-        " exception=" + exception);
+      LOG.warn("Expiring server " + this + "; " + regionNode.toShortString() +
+        ", exception=" + exception);
       env.getMasterServices().getServerManager().expireServer(regionNode.getRegionLocation());
-      serverCrashed.set(true);
+      // Return false so this procedure stays in suspended state. It will be woken up by a
+      // ServerCrashProcedure when it notices this RIT.
+      // TODO: Add a SCP as a new subprocedure that we now come to depend on.
+      return false;
     }
+    return true;
   }
 
   @Override
@@ -267,4 +269,4 @@ public class UnassignProcedure extends RegionTransitionProcedure {
   protected ProcedureMetrics getProcedureMetrics(MasterProcedureEnv env) {
     return env.getAssignmentManager().getAssignmentManagerMetrics().getUnassignProcMetrics();
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/6f44b248/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java
index 409ca26..58c4bd0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java
@@ -118,7 +118,7 @@ public class DisableTableProcedure
         postDisable(env, state);
         return Flow.NO_MORE_STATE;
       default:
-        throw new UnsupportedOperationException("unhandled state=" + state);
+        throw new UnsupportedOperationException("Unhandled state=" + state);
       }
     } catch (IOException e) {
       if (isRollbackSupported(state)) {
@@ -147,7 +147,7 @@ public class DisableTableProcedure
     }
 
     // The delete doesn't have a rollback. The execution will succeed, at some point.
-    throw new UnsupportedOperationException("unhandled state=" + state);
+    throw new UnsupportedOperationException("Unhandled state=" + state);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/6f44b248/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
index 5d871ad..c4cca2b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
@@ -197,7 +197,7 @@ public class RSProcedureDispatcher
       }
 
       // trying to send the request elsewhere instead
-      LOG.warn(String.format("the request should be tried elsewhere instead; server=%s try=%d",
+      LOG.warn(String.format("Failed dispatch to server=%s try=%d",
                   serverName, numberOfAttemptsSoFar), e);
       return false;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6f44b248/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashException.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashException.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashException.java
index ca351f6..f85b51f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashException.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashException.java
@@ -23,7 +23,8 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 
 /**
  * Passed as Exception by {@link ServerCrashProcedure}
- * notifying on-going RIT that server has failed.
+ * notifying on-going RIT that server has failed. This exception is less an error-condition than
+ * it is a signal to waiting procedures that they can now proceed.
  */
 @InterfaceAudience.Private
 @SuppressWarnings("serial")

http://git-wip-us.apache.org/repos/asf/hbase/blob/6f44b248/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
index 4f3e5ce..4370a8c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
@@ -356,7 +356,8 @@ implements ServerProcedureInterface {
    * Handle any outstanding RIT that are up against this.serverName, the crashed server.
    * Notify them of crash. Remove assign entries from the passed in <code>regions</code>
    * otherwise we have two assigns going on and they will fight over who has lock.
-   * Notify Unassigns also.
+   * Notify Unassigns. If unable to unassign because server went away, unassigns block waiting
+   * on the below callback from a ServerCrashProcedure before proceeding.
    * @param env
    * @param regions Regions that were on crashed server
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/6f44b248/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
index da086ea..aa517b5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
@@ -403,12 +403,16 @@ public class TestSplitTransactionOnCluster {
       for (HRegion d: daughters) {
         LOG.info("Regions after crash: " + d);
       }
+      if (daughters.size() != regions.size()) {
+        LOG.info("Daughters=" + daughters.size() + ", regions=" + regions.size());
+      }
       assertEquals(daughters.size(), regions.size());
       for (HRegion r: regions) {
-        LOG.info("Regions post crash " + r);
+        LOG.info("Regions post crash " + r + ", contains=" + daughters.contains(r));
         assertTrue("Missing region post crash " + r, daughters.contains(r));
       }
     } finally {
+      LOG.info("EXITING");
       admin.setBalancerRunning(true, false);
       cluster.getMaster().setCatalogJanitorEnabled(true);
       t.close();
@@ -799,7 +803,7 @@ public class TestSplitTransactionOnCluster {
       throws IOException, InterruptedException {
     this.admin.splitRegion(hri.getRegionName());
     for (int i = 0; this.cluster.getRegions(hri.getTable()).size() <= regionCount && i < 60; i++) {
-      LOG.debug("Waiting on region to split");
+      LOG.debug("Waiting on region " + hri.getRegionNameAsString() + " to split");
       Thread.sleep(2000);
     }
 
@@ -827,10 +831,13 @@ public class TestSplitTransactionOnCluster {
     // the table region serving server.
     int metaServerIndex = cluster.getServerWithMeta();
     assertTrue(metaServerIndex == -1); // meta is on master now
+    // TODO: When we change master so it doesn't carry regions, be careful here.
     HRegionServer metaRegionServer = cluster.getMaster();
     int tableRegionIndex = cluster.getServerWith(hri.getRegionName());
     assertTrue(tableRegionIndex != -1);
     HRegionServer tableRegionServer = cluster.getRegionServer(tableRegionIndex);
+    LOG.info("MetaRegionServer=" + metaRegionServer.getServerName() +
+      ", other=" + tableRegionServer.getServerName());
     if (metaRegionServer.getServerName().equals(tableRegionServer.getServerName())) {
       HRegionServer hrs = getOtherRegionServer(cluster, metaRegionServer);
       assertNotNull(hrs);
@@ -848,8 +855,8 @@ public class TestSplitTransactionOnCluster {
         tableRegionIndex + " and metaServerIndex=" + metaServerIndex);
       Thread.sleep(100);
     }
-    assertTrue("Region not moved off hbase:meta server", tableRegionIndex != -1
-        && tableRegionIndex != metaServerIndex);
+    assertTrue("Region not moved off hbase:meta server, tableRegionIndex=" + tableRegionIndex,
+      tableRegionIndex != -1 && tableRegionIndex != metaServerIndex);
     // Verify for sure table region is not on same server as hbase:meta
     tableRegionIndex = cluster.getServerWith(hri.getRegionName());
     assertTrue(tableRegionIndex != -1);
@@ -899,7 +906,7 @@ public class TestSplitTransactionOnCluster {
 
   private void awaitDaughters(TableName tableName, int numDaughters) throws InterruptedException {
     // Wait till regions are back on line again.
-    for (int i=0; cluster.getRegions(tableName).size() < numDaughters && i<60; i++) {
+    for (int i = 0; cluster.getRegions(tableName).size() < numDaughters && i < 60; i++) {
       LOG.info("Waiting for repair to happen");
       Thread.sleep(1000);
     }


[23/50] [abbrv] hbase git commit: Revert "HBASE-18588 Verify we're using netty .so epolling on linux post HBASE-18271" References the wrong JIRA

Posted by st...@apache.org.
Revert "HBASE-18588 Verify we're using netty .so epolling on linux post HBASE-18271"
References the wrong JIRA

This reverts commit ddbaf56ca8c712dc44608d3323280f578c56aed2.


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

Branch: refs/heads/HBASE-14070.HLC
Commit: 424dff20607577901c06cb40b1293ea5051ec5c5
Parents: ddbaf56
Author: Michael Stack <st...@apache.org>
Authored: Mon Aug 14 09:12:51 2017 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Mon Aug 14 09:12:51 2017 -0700

----------------------------------------------------------------------
 hbase-protocol-shaded/pom.xml | 17 -----------------
 1 file changed, 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/424dff20/hbase-protocol-shaded/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/pom.xml b/hbase-protocol-shaded/pom.xml
index 4c72eca..b28c03e 100644
--- a/hbase-protocol-shaded/pom.xml
+++ b/hbase-protocol-shaded/pom.xml
@@ -192,23 +192,6 @@
                     <ignore/>
                   </action>
                 </pluginExecution>
-                <pluginExecution>
-                  <pluginExecutionFilter>
-                    <groupId>
-                      com.google.code.maven-replacer-plugin
-                    </groupId>
-                    <artifactId>replacer</artifactId>
-                    <versionRange>[1.5.3,)</versionRange>
-                    <goals>
-                      <goal>replace</goal>
-                    </goals>
-                  </pluginExecutionFilter>
-                  <action>
-                    <execute>
-                     <runOnIncremental>false</runOnIncremental>
-                    </execute>
-                  </action>
-                </pluginExecution>
               </pluginExecutions>
             </lifecycleMappingMetadata>
           </configuration>


[11/50] [abbrv] hbase git commit: HBASE-18551 [AMv2] UnassignProcedure and crashed regionservers; AMENDMENT -- disable TestAM#testSocketTimeout... mock is insufficent for new processing

Posted by st...@apache.org.
HBASE-18551 [AMv2] UnassignProcedure and crashed regionservers; AMENDMENT -- disable TestAM#testSocketTimeout... mock is insufficent for new processing


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

Branch: refs/heads/HBASE-14070.HLC
Commit: 1070888fff3a89d435018f11bfb2fd5609be8bab
Parents: 71a9a9a
Author: Michael Stack <st...@apache.org>
Authored: Fri Aug 11 14:20:06 2017 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Fri Aug 11 14:20:35 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/master/assignment/TestAssignmentManager.java | 7 +++----
 1 file changed, 3 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/1070888f/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java
index d18c12a..4d2a894 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java
@@ -243,7 +243,7 @@ public class TestAssignmentManager {
     }
   }
 
-  @Test
+  @Ignore @Test // Disabled for now. Since HBASE-18551, this mock is insufficient.
   public void testSocketTimeout() throws Exception {
     final TableName tableName = TableName.valueOf(this.name.getMethodName());
     final HRegionInfo hri = createRegionInfo(tableName, 1);
@@ -254,9 +254,8 @@ public class TestAssignmentManager {
     rsDispatcher.setMockRsExecutor(new SocketTimeoutRsExecutor(20, 3));
     waitOnFuture(submitProcedure(am.createAssignProcedure(hri, false)));
 
-    rsDispatcher.setMockRsExecutor(new SocketTimeoutRsExecutor(20, 3));
-
-    exception.expect(ServerCrashException.class);
+    rsDispatcher.setMockRsExecutor(new SocketTimeoutRsExecutor(20, 1));
+    // exception.expect(ServerCrashException.class);
     waitOnFuture(submitProcedure(am.createUnassignProcedure(hri, null, false)));
 
     assertEquals(assignSubmittedCount + 1, assignProcMetrics.getSubmittedCounter().getCount());


[35/50] [abbrv] hbase git commit: HBASE-18581 Removed dead code and some tidy up work in BaseLoadBalancer

Posted by st...@apache.org.
HBASE-18581 Removed dead code and some tidy up work in BaseLoadBalancer

  * calls to methods getLowestLocalityRegionServer() & getLeastLoadedTopServerForRegion() got removed in HBASE-18164
  * call to calculateRegionServerLocalities() got removed in HBASE-15486
  * Some other minor improvements

Change-Id: Ib149530d8d20c019b0891c026e23180e260f59db
Signed-off-by: Apekshit Sharma <ap...@apache.org>


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

Branch: refs/heads/HBASE-14070.HLC
Commit: 2b88edfd8d6c1cb512abf1d9f3316c50ed342cfc
Parents: 310934d
Author: Umesh Agashe <ua...@cloudera.com>
Authored: Fri Aug 11 11:18:13 2017 -0700
Committer: Apekshit Sharma <ap...@apache.org>
Committed: Tue Aug 15 14:55:52 2017 -0700

----------------------------------------------------------------------
 .../hbase/master/balancer/BaseLoadBalancer.java | 190 ++++---------------
 1 file changed, 32 insertions(+), 158 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2b88edfd/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
index 8f5b6f5..30f59a9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
@@ -1,4 +1,4 @@
- /**
+ /*
  * 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
@@ -34,6 +34,7 @@ import java.util.Random;
 import java.util.Set;
 import java.util.TreeMap;
 import java.util.function.Predicate;
+import java.util.stream.Collectors;
 
 import org.apache.commons.lang.NotImplementedException;
 import org.apache.commons.logging.Log;
@@ -360,10 +361,10 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
       }
 
       numMaxRegionsPerTable = new int[numTables];
-      for (int serverIndex = 0 ; serverIndex < numRegionsPerServerPerTable.length; serverIndex++) {
-        for (tableIndex = 0 ; tableIndex < numRegionsPerServerPerTable[serverIndex].length; tableIndex++) {
-          if (numRegionsPerServerPerTable[serverIndex][tableIndex] > numMaxRegionsPerTable[tableIndex]) {
-            numMaxRegionsPerTable[tableIndex] = numRegionsPerServerPerTable[serverIndex][tableIndex];
+      for (int[] aNumRegionsPerServerPerTable : numRegionsPerServerPerTable) {
+        for (tableIndex = 0; tableIndex < aNumRegionsPerServerPerTable.length; tableIndex++) {
+          if (aNumRegionsPerServerPerTable[tableIndex] > numMaxRegionsPerTable[tableIndex]) {
+            numMaxRegionsPerTable[tableIndex] = aNumRegionsPerServerPerTable[tableIndex];
           }
         }
       }
@@ -375,10 +376,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
         } else {
           hasRegionReplicas = true;
           HRegionInfo primaryInfo = RegionReplicaUtil.getRegionInfoForDefaultReplica(info);
-          regionIndexToPrimaryIndex[i] =
-              regionsToIndex.containsKey(primaryInfo) ?
-              regionsToIndex.get(primaryInfo):
-              -1;
+          regionIndexToPrimaryIndex[i] = regionsToIndex.getOrDefault(primaryInfo, -1);
         }
       }
 
@@ -608,7 +606,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
 
     /** An action to move or swap a region */
     public static class Action {
-      public static enum Type {
+      public enum Type {
         ASSIGN_REGION,
         MOVE_REGION,
         SWAP_REGIONS,
@@ -806,9 +804,9 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
           == numMaxRegionsPerTable[tableIndex]) {
         //recompute maxRegionsPerTable since the previous value was coming from the old server
         numMaxRegionsPerTable[tableIndex] = 0;
-        for (int serverIndex = 0 ; serverIndex < numRegionsPerServerPerTable.length; serverIndex++) {
-          if (numRegionsPerServerPerTable[serverIndex][tableIndex] > numMaxRegionsPerTable[tableIndex]) {
-            numMaxRegionsPerTable[tableIndex] = numRegionsPerServerPerTable[serverIndex][tableIndex];
+        for (int[] aNumRegionsPerServerPerTable : numRegionsPerServerPerTable) {
+          if (aNumRegionsPerServerPerTable[tableIndex] > numMaxRegionsPerTable[tableIndex]) {
+            numMaxRegionsPerTable[tableIndex] = aNumRegionsPerServerPerTable[tableIndex];
           }
         }
       }
@@ -912,49 +910,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
       return Arrays.binarySearch(arr, val) >= 0;
     }
 
-    private Comparator<Integer> numRegionsComparator = new Comparator<Integer>() {
-      @Override
-      public int compare(Integer integer, Integer integer2) {
-        return Integer.compare(getNumRegions(integer), getNumRegions(integer2));
-      }
-    };
-
-    void sortServersByLocality() {
-      Arrays.sort(serverIndicesSortedByLocality, localityComparator);
-    }
-
-    float getLocality(int server) {
-      return localityPerServer[server];
-    }
-
-    private Comparator<Integer> localityComparator = new Comparator<Integer>() {
-      @Override
-      public int compare(Integer integer, Integer integer2) {
-        return Float.compare(getLocality(integer), getLocality(integer2));
-      }
-    };
-
-    int getLowestLocalityRegionServer() {
-      if (regionFinder == null) {
-        return -1;
-      } else {
-        sortServersByLocality();
-        // We want to find server with non zero regions having lowest locality.
-        int i = 0;
-        int lowestLocalityServerIndex = serverIndicesSortedByLocality[i];
-        while (localityPerServer[lowestLocalityServerIndex] == 0
-            && (regionsPerServer[lowestLocalityServerIndex].length == 0)) {
-          i++;
-          lowestLocalityServerIndex = serverIndicesSortedByLocality[i];
-        }
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("Lowest locality region server with non zero regions is "
-            + servers[lowestLocalityServerIndex].getHostname() + " with locality "
-            + localityPerServer[lowestLocalityServerIndex]);
-        }
-        return lowestLocalityServerIndex;
-      }
-    }
+    private Comparator<Integer> numRegionsComparator = Comparator.comparingInt(this::getNumRegions);
 
     int getLowestLocalityRegionOnServer(int serverIndex) {
       if (regionFinder != null) {
@@ -1003,62 +959,6 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
       }
     }
 
-    /**
-     * Returns a least loaded server which has better locality for this region
-     * than the current server.
-     */
-    int getLeastLoadedTopServerForRegion(int region, int currentServer) {
-      if (regionFinder != null) {
-        List<ServerName> topLocalServers = regionFinder.getTopBlockLocations(regions[region],
-          servers[currentServer].getHostname());
-        int leastLoadedServerIndex = -1;
-        int load = Integer.MAX_VALUE;
-        for (ServerName sn : topLocalServers) {
-          if (!serversToIndex.containsKey(sn.getHostAndPort())) {
-            continue;
-          }
-          int index = serversToIndex.get(sn.getHostAndPort());
-          if (regionsPerServer[index] == null) {
-            continue;
-          }
-          int tempLoad = regionsPerServer[index].length;
-          if (tempLoad <= load) {
-            leastLoadedServerIndex = index;
-            load = tempLoad;
-          }
-        }
-        if (leastLoadedServerIndex != -1) {
-          if (LOG.isTraceEnabled()) {
-            LOG.trace("Pick the least loaded server " +
-                servers[leastLoadedServerIndex].getHostname() +
-                " with better locality for region " + regions[region].getShortNameToLog());
-          }
-        }
-        return leastLoadedServerIndex;
-      } else {
-        return -1;
-      }
-    }
-
-    void calculateRegionServerLocalities() {
-      if (regionFinder == null) {
-        LOG.warn("Region location finder found null, skipping locality calculations.");
-        return;
-      }
-      for (int i = 0; i < regionsPerServer.length; i++) {
-        HDFSBlocksDistribution distribution = new HDFSBlocksDistribution();
-        if (regionsPerServer[i].length > 0) {
-          for (int j = 0; j < regionsPerServer[i].length; j++) {
-            int regionIndex = regionsPerServer[i][j];
-            distribution.add(regionFinder.getBlockDistribution(regions[regionIndex]));
-          }
-        } else {
-          LOG.debug("Server " + servers[i].getHostname() + " had 0 regions.");
-        }
-        localityPerServer[i] = distribution.getBlockLocalityIndex(servers[i].getHostname());
-      }
-    }
-
     @VisibleForTesting
     protected void setNumRegions(int numRegions) {
       this.numRegions = numRegions;
@@ -1073,32 +973,19 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
         justification="Not important but should be fixed")
     @Override
     public String toString() {
-      String desc = "Cluster{" +
-          "servers=[";
-          for(ServerName sn:servers) {
-             desc += sn.getHostAndPort() + ", ";
-          }
-          desc +=
-          ", serverIndicesSortedByRegionCount="+
-          Arrays.toString(serverIndicesSortedByRegionCount) +
-          ", regionsPerServer=[";
+      StringBuilder desc = new StringBuilder("Cluster={servers=[");
+      for(ServerName sn:servers) {
+        desc.append(sn.getHostAndPort()).append(", ");
+      }
+      desc.append("], serverIndicesSortedByRegionCount=")
+          .append(Arrays.toString(serverIndicesSortedByRegionCount))
+          .append(", regionsPerServer=").append(Arrays.deepToString(regionsPerServer));
 
-          for (int[]r:regionsPerServer) {
-            desc += Arrays.toString(r);
-          }
-          desc += "]" +
-          ", numMaxRegionsPerTable=" +
-          Arrays.toString(numMaxRegionsPerTable) +
-          ", numRegions=" +
-          numRegions +
-          ", numServers=" +
-          numServers +
-          ", numTables=" +
-          numTables +
-          ", numMovedRegions=" +
-          numMovedRegions +
-          '}';
-      return desc;
+      desc.append(", numMaxRegionsPerTable=").append(Arrays.toString(numMaxRegionsPerTable))
+          .append(", numRegions=").append(numRegions).append(", numServers=").append(numServers)
+          .append(", numTables=").append(numTables).append(", numMovedRegions=")
+          .append(numMovedRegions).append('}');
+      return desc.toString();
     }
   }
 
@@ -1364,9 +1251,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
       List<HRegionInfo> masterRegions = assignments.get(masterServerName);
       if (!masterRegions.isEmpty()) {
         regions = new ArrayList<>(regions);
-        for (HRegionInfo region: masterRegions) {
-          regions.remove(region);
-        }
+        regions.removeAll(masterRegions);
       }
     }
     if (regions == null || regions.isEmpty()) {
@@ -1404,11 +1289,8 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
       for (int j = 0; j < numServers; j++) { // try all servers one by one
         ServerName serverName = servers.get((j + serverIdx) % numServers);
         if (!cluster.wouldLowerAvailability(region, serverName)) {
-          List<HRegionInfo> serverRegions = assignments.get(serverName);
-          if (serverRegions == null) {
-            serverRegions = new ArrayList<>();
-            assignments.put(serverName, serverRegions);
-          }
+          List<HRegionInfo> serverRegions =
+              assignments.computeIfAbsent(serverName, k -> new ArrayList<>());
           serverRegions.add(region);
           cluster.doAssignRegion(region, serverName);
           serverIdx = (j + serverIdx + 1) % numServers; //remain from next server
@@ -1425,11 +1307,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
     for (HRegionInfo region : lastFewRegions) {
       int i = RANDOM.nextInt(numServers);
       ServerName server = servers.get(i);
-      List<HRegionInfo> serverRegions = assignments.get(server);
-      if (serverRegions == null) {
-        serverRegions = new ArrayList<>();
-        assignments.put(server, serverRegions);
-      }
+      List<HRegionInfo> serverRegions = assignments.computeIfAbsent(server, k -> new ArrayList<>());
       serverRegions.add(region);
       cluster.doAssignRegion(region, server);
     }
@@ -1438,7 +1316,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
 
   protected Cluster createCluster(List<ServerName> servers,
       Collection<HRegionInfo> regions, boolean forceRefresh) {
-    if (forceRefresh == true) {
+    if (forceRefresh) {
       regionFinder.refreshAndWait(regions);
     }
     // Get the snapshot of the current assignments for the regions in question, and then create
@@ -1525,14 +1403,10 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
       // Guarantee not to put other regions on master
       servers.remove(masterServerName);
       List<HRegionInfo> masterRegions = assignments.get(masterServerName);
-      if (!masterRegions.isEmpty()) {
-        regions = new HashMap<>(regions);
-        for (HRegionInfo region: masterRegions) {
-          regions.remove(region);
-        }
-      }
+      regions = regions.entrySet().stream().filter(e -> !masterRegions.contains(e.getKey()))
+          .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
     }
-    if (regions == null || regions.isEmpty()) {
+    if (regions.isEmpty()) {
       return assignments;
     }
 


[03/50] [abbrv] hbase git commit: HBASE-18563 Fix RAT License complaint about website jenkins scripts

Posted by st...@apache.org.
HBASE-18563 Fix RAT License complaint about website jenkins scripts


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

Branch: refs/heads/HBASE-14070.HLC
Commit: c37432fefbf4d1ff5bf80f5227c986f3bde281a1
Parents: efd211d
Author: Esteban Gutierrez <es...@apache.org>
Authored: Thu Aug 10 20:08:03 2017 -0500
Committer: Esteban Gutierrez <es...@apache.org>
Committed: Thu Aug 10 20:08:03 2017 -0500

----------------------------------------------------------------------
 .../jenkins-scripts/check-website-links.sh      | 20 +++++++++++++++++++-
 .../jenkins-scripts/generate-hbase-website.sh   | 18 ++++++++++++++++++
 2 files changed, 37 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c37432fe/dev-support/jenkins-scripts/check-website-links.sh
----------------------------------------------------------------------
diff --git a/dev-support/jenkins-scripts/check-website-links.sh b/dev-support/jenkins-scripts/check-website-links.sh
index c23abbb..1f7cd1c 100755
--- a/dev-support/jenkins-scripts/check-website-links.sh
+++ b/dev-support/jenkins-scripts/check-website-links.sh
@@ -1,4 +1,22 @@
 #!/bin/bash
+#
+#/**
+# * 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.
+# */
 
 # This script is designed to run as a Jenkins job, such as at
 # https://builds.apache.org/view/All/job/HBase%20Website%20Link%20Checker/
@@ -44,4 +62,4 @@ if ! grep -q 'ERROR' link_report/index.html; then
   exit 1
 else
   echo "No errors found. Warnings might be present."
-fi
\ No newline at end of file
+fi

http://git-wip-us.apache.org/repos/asf/hbase/blob/c37432fe/dev-support/jenkins-scripts/generate-hbase-website.sh
----------------------------------------------------------------------
diff --git a/dev-support/jenkins-scripts/generate-hbase-website.sh b/dev-support/jenkins-scripts/generate-hbase-website.sh
index a3f7823..06d160a 100644
--- a/dev-support/jenkins-scripts/generate-hbase-website.sh
+++ b/dev-support/jenkins-scripts/generate-hbase-website.sh
@@ -1,4 +1,22 @@
 #!/bin/bash
+#
+#/**
+# * 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.
+# */
 
 # This script is meant to run as part of a Jenkins job such as
 # https://builds.apache.org/job/hbase_generate_website/


[13/50] [abbrv] hbase git commit: HBASE-18557: Change splitable to mergeable in MergeTableRegionsProcedure

Posted by st...@apache.org.
HBASE-18557: Change splitable to mergeable in MergeTableRegionsProcedure

Signed-off-by: Jerry He <je...@apache.org>


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

Branch: refs/heads/HBASE-14070.HLC
Commit: 95e883967cbb383b48d8fae548fb55b88c7f0529
Parents: aa8f67a
Author: Yi Liang <yl...@us.ibm.com>
Authored: Thu Aug 10 11:15:59 2017 -0700
Committer: Jerry He <je...@apache.org>
Committed: Fri Aug 11 22:45:22 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/95e88396/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
index 74d9b75..9aaf297 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
@@ -513,7 +513,7 @@ public class MergeTableRegionsProcedure
   throws IOException {
     GetRegionInfoResponse response =
       Util.getRegionInfoResponse(env, rs.getServerName(), rs.getRegion());
-    return response.hasSplittable() && response.getSplittable();
+    return response.hasMergeable() && response.getMergeable();
   }
 
   /**


[21/50] [abbrv] hbase git commit: HBASE-18528 DON'T allow user to modify the passed table/column descriptor

Posted by st...@apache.org.
HBASE-18528 DON'T allow user to modify the passed table/column descriptor


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

Branch: refs/heads/HBASE-14070.HLC
Commit: e2b797be390f05c55a490a64bc72e2d8c19fcbb7
Parents: c6bf4d5
Author: Chia-Ping Tsai <ch...@gmail.com>
Authored: Mon Aug 14 14:02:30 2017 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Mon Aug 14 14:02:30 2017 +0800

----------------------------------------------------------------------
 .../client/ImmutableHColumnDescriptor.java      |  5 +-
 .../hbase/client/ImmutableHTableDescriptor.java | 11 ++-
 .../hbase/coprocessor/MasterObserver.java       | 48 ++++++------
 .../hbase/master/MasterCoprocessorHost.java     | 80 +++++++++++---------
 4 files changed, 79 insertions(+), 65 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e2b797be/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ImmutableHColumnDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ImmutableHColumnDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ImmutableHColumnDescriptor.java
index c8d34ff..89ef851 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ImmutableHColumnDescriptor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ImmutableHColumnDescriptor.java
@@ -35,8 +35,9 @@ public class ImmutableHColumnDescriptor extends HColumnDescriptor {
     super(desc, false);
   }
 
-  ImmutableHColumnDescriptor(final ModifyableColumnFamilyDescriptor desc) {
-    super(desc);
+  public ImmutableHColumnDescriptor(final ColumnFamilyDescriptor desc) {
+    super(desc instanceof ModifyableColumnFamilyDescriptor ?
+      (ModifyableColumnFamilyDescriptor) desc : new ModifyableColumnFamilyDescriptor(desc));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/e2b797be/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ImmutableHTableDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ImmutableHTableDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ImmutableHTableDescriptor.java
index 4e9e9af..169f143 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ImmutableHTableDescriptor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ImmutableHTableDescriptor.java
@@ -28,19 +28,17 @@ import org.apache.hadoop.hbase.HTableDescriptor;
  * Read-only table descriptor.
  */
 @Deprecated // deprecated for hbase 2.0, remove for hbase 3.0. see HTableDescriptor.
-@InterfaceAudience.Public
+@InterfaceAudience.Private
 public class ImmutableHTableDescriptor extends HTableDescriptor {
 
   @Override
   protected HColumnDescriptor toHColumnDescriptor(ColumnFamilyDescriptor desc) {
     if (desc == null) {
       return null;
-    } else if (desc instanceof ModifyableColumnFamilyDescriptor) {
-      return new ImmutableHColumnDescriptor((ModifyableColumnFamilyDescriptor) desc);
     } else if (desc instanceof HColumnDescriptor) {
       return new ImmutableHColumnDescriptor((HColumnDescriptor) desc);
     } else {
-      return new ImmutableHColumnDescriptor(new ModifyableColumnFamilyDescriptor(desc));
+      return new ImmutableHColumnDescriptor(desc);
     }
   }
   /*
@@ -51,6 +49,11 @@ public class ImmutableHTableDescriptor extends HTableDescriptor {
     super(desc, false);
   }
 
+  public ImmutableHTableDescriptor(final TableDescriptor desc) {
+    super(desc instanceof ModifyableTableDescriptor ?
+      (ModifyableTableDescriptor) desc : new ModifyableTableDescriptor(desc.getTableName(), desc));
+  }
+
   @Override
   protected ModifyableTableDescriptor getDelegateeForModification() {
     throw new UnsupportedOperationException("HTableDescriptor is read-only");

http://git-wip-us.apache.org/repos/asf/hbase/blob/e2b797be/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
index f4f5db3..8e368ba 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
@@ -107,7 +107,7 @@ public interface MasterObserver extends Coprocessor {
    * table handler and it is async to the create RPC call.
    * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
    * @param ctx the environment to interact with the framework and master
-   * @param desc the HTableDescriptor for the table
+   * @param desc the read-only HTableDescriptor for the table
    * @param regions the initial regions created for the table
    * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
    *     (<a href="https://issues.apache.org/jira/browse/HBASE-15575">HBASE-15575</a>).
@@ -122,7 +122,7 @@ public interface MasterObserver extends Coprocessor {
    * of create table RPC call.  Called as part of create table handler and
    * it is async to the create RPC call.
    * @param ctx the environment to interact with the framework and master
-   * @param desc the HTableDescriptor for the table
+   * @param desc the read-only HTableDescriptor for the table
    * @param regions the initial regions created for the table
    * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
    *   (<a href="https://issues.apache.org/jira/browse/HBASE-15575">HBASE-15575</a>).
@@ -368,7 +368,7 @@ public interface MasterObserver extends Coprocessor {
    * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
-   * @param htd the HTableDescriptor
+   * @param htd the read-only HTableDescriptor
    * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
    *     (<a href="https://issues.apache.org/jira/browse/HBASE-15575">HBASE-15575</a>).
    *     Use {@link #preModifyTableAction(ObserverContext, TableName, TableDescriptor)}.
@@ -384,7 +384,7 @@ public interface MasterObserver extends Coprocessor {
    * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
-   * @param htd the HTableDescriptor
+   * @param htd the read-only HTableDescriptor
    * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
    *     (<a href="https://issues.apache.org/jira/browse/HBASE-13645">HBASE-13645</a>).
    *     Use {@link #postCompletedModifyTableAction(ObserverContext, TableName, TableDescriptor)}.
@@ -435,7 +435,7 @@ public interface MasterObserver extends Coprocessor {
    * add column RPC call.
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
-   * @param columnFamily the HColumnDescriptor
+   * @param columnFamily the read-only HColumnDescriptor
    * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
    *             (<a href="https://issues.apache.org/jira/browse/HBASE-13645">HBASE-13645</a>).
    *             Use {@link #preAddColumnFamily(ObserverContext, TableName, ColumnFamilyDescriptor)}.
@@ -464,7 +464,7 @@ public interface MasterObserver extends Coprocessor {
    * add column RPC call.
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
-   * @param columnFamily the HColumnDescriptor
+   * @param columnFamily the read-only HColumnDescriptor
    * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
    *             (<a href="https://issues.apache.org/jira/browse/HBASE-13645">HBASE-13645</a>).
    *             Use {@link #postAddColumnFamily(ObserverContext, TableName, ColumnFamilyDescriptor)}.
@@ -493,7 +493,7 @@ public interface MasterObserver extends Coprocessor {
    * add column handler.
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
-   * @param columnFamily the HColumnDescriptor
+   * @param columnFamily the read-only HColumnDescriptor
    * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
    *          (<a href="https://issues.apache.org/jira/browse/HBASE-13645">HBASE-13645</a>). Use
    *          {@link #preAddColumnFamilyAction(ObserverContext, TableName, ColumnFamilyDescriptor)}.
@@ -525,7 +525,7 @@ public interface MasterObserver extends Coprocessor {
    * add column handler.
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
-   * @param columnFamily the HColumnDescriptor
+   * @param columnFamily the read-only HColumnDescriptor
    * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
    *     (<a href="https://issues.apache.org/jira/browse/HBASE-13645">HBASE-13645</a>). Use
    *     {@link #postCompletedAddColumnFamilyAction(ObserverContext, TableName, ColumnFamilyDescriptor)}.
@@ -557,7 +557,7 @@ public interface MasterObserver extends Coprocessor {
    * modify column RPC call.
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
-   * @param columnFamily the HColumnDescriptor
+   * @param columnFamily the read-only HColumnDescriptor
    * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
    *             (<a href="https://issues.apache.org/jira/browse/HBASE-13645">HBASE-13645</a>).
    *             Use {@link #preModifyColumnFamily(ObserverContext, TableName, ColumnFamilyDescriptor)}.
@@ -586,7 +586,7 @@ public interface MasterObserver extends Coprocessor {
    * column RPC call.
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
-   * @param columnFamily the HColumnDescriptor
+   * @param columnFamily the read-only HColumnDescriptor
    * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
    *             (<a href="https://issues.apache.org/jira/browse/HBASE-13645">HBASE-13645</a>).
    *             Use {@link #postModifyColumnFamily(ObserverContext, TableName, ColumnFamilyDescriptor)}.
@@ -615,7 +615,7 @@ public interface MasterObserver extends Coprocessor {
    * modify column handler.
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
-   * @param columnFamily the HColumnDescriptor
+   * @param columnFamily the read-only HColumnDescriptor
    * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
    *     (<a href="https://issues.apache.org/jira/browse/HBASE-13645">HBASE-13645</a>).
    *     Use {@link #preModifyColumnFamilyAction(ObserverContext, TableName, ColumnFamilyDescriptor)}.
@@ -647,7 +647,7 @@ public interface MasterObserver extends Coprocessor {
    * column handler.
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
-   * @param columnFamily the HColumnDescriptor
+   * @param columnFamily the read-only HColumnDescriptor
    * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
    *   (<a href="https://issues.apache.org/jira/browse/HBASE-13645">HBASE-13645</a>). Use
    *   {@link #postCompletedModifyColumnFamilyAction(ObserverContext,TableName,ColumnFamilyDescriptor)}.
@@ -1284,10 +1284,10 @@ public interface MasterObserver extends Coprocessor {
    * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
    * @param ctx the environment to interact with the framework and master
    * @param snapshot the SnapshotDescriptor for the snapshot
-   * @param hTableDescriptor the TableDescriptor of the table to snapshot
+   * @param tableDescriptor the TableDescriptor of the table to snapshot
    */
   default void preSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final SnapshotDescription snapshot, final TableDescriptor hTableDescriptor)
+      final SnapshotDescription snapshot, final TableDescriptor tableDescriptor)
       throws IOException {}
 
   /**
@@ -1295,10 +1295,10 @@ public interface MasterObserver extends Coprocessor {
    * Called as part of snapshot RPC call.
    * @param ctx the environment to interact with the framework and master
    * @param snapshot the SnapshotDescriptor for the snapshot
-   * @param hTableDescriptor the TableDescriptor of the table to snapshot
+   * @param tableDescriptor the TableDescriptor of the table to snapshot
    */
   default void postSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final SnapshotDescription snapshot, final TableDescriptor hTableDescriptor)
+      final SnapshotDescription snapshot, final TableDescriptor tableDescriptor)
       throws IOException {}
 
   /**
@@ -1325,10 +1325,10 @@ public interface MasterObserver extends Coprocessor {
    * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
    * @param ctx the environment to interact with the framework and master
    * @param snapshot the SnapshotDescriptor for the snapshot
-   * @param hTableDescriptor the TableDescriptor of the table to create
+   * @param tableDescriptor the TableDescriptor of the table to create
    */
   default void preCloneSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final SnapshotDescription snapshot, final TableDescriptor hTableDescriptor)
+      final SnapshotDescription snapshot, final TableDescriptor tableDescriptor)
       throws IOException {}
 
   /**
@@ -1336,10 +1336,10 @@ public interface MasterObserver extends Coprocessor {
    * Called as part of restoreSnapshot RPC call.
    * @param ctx the environment to interact with the framework and master
    * @param snapshot the SnapshotDescriptor for the snapshot
-   * @param hTableDescriptor the v of the table to create
+   * @param tableDescriptor the v of the table to create
    */
   default void postCloneSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final SnapshotDescription snapshot, final TableDescriptor hTableDescriptor)
+      final SnapshotDescription snapshot, final TableDescriptor tableDescriptor)
       throws IOException {}
 
   /**
@@ -1348,10 +1348,10 @@ public interface MasterObserver extends Coprocessor {
    * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
    * @param ctx the environment to interact with the framework and master
    * @param snapshot the SnapshotDescriptor for the snapshot
-   * @param hTableDescriptor the TableDescriptor of the table to restore
+   * @param tableDescriptor the TableDescriptor of the table to restore
    */
   default void preRestoreSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final SnapshotDescription snapshot, final TableDescriptor hTableDescriptor)
+      final SnapshotDescription snapshot, final TableDescriptor tableDescriptor)
       throws IOException {}
 
   /**
@@ -1359,10 +1359,10 @@ public interface MasterObserver extends Coprocessor {
    * Called as part of restoreSnapshot RPC call.
    * @param ctx the environment to interact with the framework and master
    * @param snapshot the SnapshotDescriptor for the snapshot
-   * @param hTableDescriptor the TableDescriptor of the table to restore
+   * @param tableDescriptor the TableDescriptor of the table to restore
    */
   default void postRestoreSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final SnapshotDescription snapshot, final TableDescriptor hTableDescriptor)
+      final SnapshotDescription snapshot, final TableDescriptor tableDescriptor)
       throws IOException {}
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/e2b797be/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 04bdacf..004f91d 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
@@ -22,7 +22,6 @@ package org.apache.hadoop.hbase.master;
 import java.io.IOException;
 import java.util.List;
 import java.util.Set;
-import java.util.stream.Collectors;
 
 import org.apache.commons.lang.ClassUtils;
 import org.apache.commons.logging.Log;
@@ -38,6 +37,9 @@ import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.ImmutableHColumnDescriptor;
+import org.apache.hadoop.hbase.client.ImmutableHTableDescriptor;
 import org.apache.hadoop.hbase.client.MasterSwitchType;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.TableDescriptor;
@@ -243,7 +245,7 @@ public class MasterCoprocessorHost
 
   /* Implementation of hooks for invoking MasterObservers */
 
-  public void preCreateTable(final HTableDescriptor htd, final HRegionInfo[] regions)
+  public void preCreateTable(final TableDescriptor htd, final HRegionInfo[] regions)
       throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override
@@ -254,7 +256,7 @@ public class MasterCoprocessorHost
     });
   }
 
-  public void postCreateTable(final HTableDescriptor htd, final HRegionInfo[] regions)
+  public void postCreateTable(final TableDescriptor htd, final HRegionInfo[] regions)
       throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override
@@ -265,26 +267,26 @@ public class MasterCoprocessorHost
     });
   }
 
-  public void preCreateTableAction(final HTableDescriptor htd, final HRegionInfo[] regions,
+  public void preCreateTableAction(final TableDescriptor htd, final HRegionInfo[] regions,
                                    final User user)
       throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
           throws IOException {
-        oserver.preCreateTableHandler(ctx, htd, regions);
+        oserver.preCreateTableHandler(ctx, toImmutableHTableDescriptor(htd), regions);
         oserver.preCreateTableAction(ctx, htd, regions);
       }
     });
   }
 
   public void postCompletedCreateTableAction(
-      final HTableDescriptor htd, final HRegionInfo[] regions, final User user) throws IOException {
+      final TableDescriptor htd, final HRegionInfo[] regions, final User user) throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
           throws IOException {
-        oserver.postCreateTableHandler(ctx, htd, regions);
+        oserver.postCreateTableHandler(ctx, toImmutableHTableDescriptor(htd), regions);
         oserver.postCompletedCreateTableAction(ctx, htd, regions);
       }
     });
@@ -376,7 +378,7 @@ public class MasterCoprocessorHost
     });
   }
 
-  public void preModifyTable(final TableName tableName, final HTableDescriptor htd)
+  public void preModifyTable(final TableName tableName, final TableDescriptor htd)
       throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override
@@ -387,7 +389,7 @@ public class MasterCoprocessorHost
     });
   }
 
-  public void postModifyTable(final TableName tableName, final HTableDescriptor htd)
+  public void postModifyTable(final TableName tableName, final TableDescriptor htd)
       throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override
@@ -398,51 +400,51 @@ public class MasterCoprocessorHost
     });
   }
 
-  public void preModifyTableAction(final TableName tableName, final HTableDescriptor htd,
+  public void preModifyTableAction(final TableName tableName, final TableDescriptor htd,
                                    final User user)
       throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
           throws IOException {
-        oserver.preModifyTableHandler(ctx, tableName, htd);
+        oserver.preModifyTableHandler(ctx, tableName, toImmutableHTableDescriptor(htd));
         oserver.preModifyTableAction(ctx, tableName, htd);
       }
     });
   }
 
-  public void postCompletedModifyTableAction(final TableName tableName, final HTableDescriptor htd,
+  public void postCompletedModifyTableAction(final TableName tableName, final TableDescriptor htd,
                                              final User user)
       throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
           throws IOException {
-        oserver.postModifyTableHandler(ctx, tableName, htd);
+        oserver.postModifyTableHandler(ctx, tableName, toImmutableHTableDescriptor(htd));
         oserver.postCompletedModifyTableAction(ctx, tableName, htd);
       }
     });
   }
 
-  public boolean preAddColumn(final TableName tableName, final HColumnDescriptor columnFamily)
+  public boolean preAddColumn(final TableName tableName, final ColumnFamilyDescriptor columnFamily)
       throws IOException {
     return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
           throws IOException {
-        oserver.preAddColumn(ctx, tableName, columnFamily);
+        oserver.preAddColumn(ctx, tableName, toImmutableHColumnDescriptor(columnFamily));
         oserver.preAddColumnFamily(ctx, tableName, columnFamily);
       }
     });
   }
 
-  public void postAddColumn(final TableName tableName, final HColumnDescriptor columnFamily)
+  public void postAddColumn(final TableName tableName, final ColumnFamilyDescriptor columnFamily)
       throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
           throws IOException {
-        oserver.postAddColumn(ctx, tableName, columnFamily);
+        oserver.postAddColumn(ctx, tableName, toImmutableHColumnDescriptor(columnFamily));
         oserver.postAddColumnFamily(ctx, tableName, columnFamily);
       }
     });
@@ -450,14 +452,14 @@ public class MasterCoprocessorHost
 
   public boolean preAddColumnFamilyAction(
       final TableName tableName,
-      final HColumnDescriptor columnFamily,
+      final ColumnFamilyDescriptor columnFamily,
       final User user)
       throws IOException {
     return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
           throws IOException {
-        oserver.preAddColumnHandler(ctx, tableName, columnFamily);
+        oserver.preAddColumnHandler(ctx, tableName, toImmutableHColumnDescriptor(columnFamily));
         oserver.preAddColumnFamilyAction(ctx, tableName, columnFamily);
       }
     });
@@ -465,38 +467,38 @@ public class MasterCoprocessorHost
 
   public void postCompletedAddColumnFamilyAction(
       final TableName tableName,
-      final HColumnDescriptor columnFamily,
+      final ColumnFamilyDescriptor columnFamily,
       final User user)
       throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
           throws IOException {
-        oserver.postAddColumnHandler(ctx, tableName, columnFamily);
+        oserver.postAddColumnHandler(ctx, tableName, toImmutableHColumnDescriptor(columnFamily));
         oserver.postCompletedAddColumnFamilyAction(ctx, tableName, columnFamily);
       }
     });
   }
 
-  public boolean preModifyColumn(final TableName tableName, final HColumnDescriptor columnFamily)
+  public boolean preModifyColumn(final TableName tableName, final ColumnFamilyDescriptor columnFamily)
       throws IOException {
     return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
           throws IOException {
-        oserver.preModifyColumn(ctx, tableName, columnFamily);
+        oserver.preModifyColumn(ctx, tableName, toImmutableHColumnDescriptor(columnFamily));
         oserver.preModifyColumnFamily(ctx, tableName, columnFamily);
       }
     });
   }
 
-  public void postModifyColumn(final TableName tableName, final HColumnDescriptor columnFamily)
+  public void postModifyColumn(final TableName tableName, final ColumnFamilyDescriptor columnFamily)
       throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
           throws IOException {
-        oserver.postModifyColumn(ctx, tableName, columnFamily);
+        oserver.postModifyColumn(ctx, tableName, toImmutableHColumnDescriptor(columnFamily));
         oserver.postModifyColumnFamily(ctx, tableName, columnFamily);
       }
     });
@@ -504,13 +506,13 @@ public class MasterCoprocessorHost
 
   public boolean preModifyColumnFamilyAction(
       final TableName tableName,
-      final HColumnDescriptor columnFamily,
+      final ColumnFamilyDescriptor columnFamily,
       final User user) throws IOException {
     return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
           throws IOException {
-        oserver.preModifyColumnHandler(ctx, tableName, columnFamily);
+        oserver.preModifyColumnHandler(ctx, tableName, toImmutableHColumnDescriptor(columnFamily));
         oserver.preModifyColumnFamilyAction(ctx, tableName, columnFamily);
       }
     });
@@ -518,13 +520,13 @@ public class MasterCoprocessorHost
 
   public void postCompletedModifyColumnFamilyAction(
       final TableName tableName,
-      final HColumnDescriptor columnFamily,
+      final ColumnFamilyDescriptor columnFamily,
       final User user) throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
           throws IOException {
-        oserver.postModifyColumnHandler(ctx, tableName, columnFamily);
+        oserver.postModifyColumnHandler(ctx, tableName, toImmutableHColumnDescriptor(columnFamily));
         oserver.postCompletedModifyColumnFamilyAction(ctx, tableName, columnFamily);
       }
     });
@@ -1155,7 +1157,7 @@ public class MasterCoprocessorHost
   }
 
   public void preSnapshot(final SnapshotDescription snapshot,
-      final HTableDescriptor hTableDescriptor) throws IOException {
+      final TableDescriptor hTableDescriptor) throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
@@ -1166,7 +1168,7 @@ public class MasterCoprocessorHost
   }
 
   public void postSnapshot(final SnapshotDescription snapshot,
-      final HTableDescriptor hTableDescriptor) throws IOException {
+      final TableDescriptor hTableDescriptor) throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
@@ -1197,7 +1199,7 @@ public class MasterCoprocessorHost
   }
 
   public void preCloneSnapshot(final SnapshotDescription snapshot,
-      final HTableDescriptor hTableDescriptor) throws IOException {
+      final TableDescriptor hTableDescriptor) throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
@@ -1208,7 +1210,7 @@ public class MasterCoprocessorHost
   }
 
   public void postCloneSnapshot(final SnapshotDescription snapshot,
-      final HTableDescriptor hTableDescriptor) throws IOException {
+      final TableDescriptor hTableDescriptor) throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
@@ -1219,7 +1221,7 @@ public class MasterCoprocessorHost
   }
 
   public void preRestoreSnapshot(final SnapshotDescription snapshot,
-      final HTableDescriptor hTableDescriptor) throws IOException {
+      final TableDescriptor hTableDescriptor) throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
@@ -1230,7 +1232,7 @@ public class MasterCoprocessorHost
   }
 
   public void postRestoreSnapshot(final SnapshotDescription snapshot,
-      final HTableDescriptor hTableDescriptor) throws IOException {
+      final TableDescriptor hTableDescriptor) throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
@@ -1875,4 +1877,12 @@ public class MasterCoprocessorHost
       }
     });
   }
+
+  private static ImmutableHTableDescriptor toImmutableHTableDescriptor(TableDescriptor desc) {
+    return new ImmutableHTableDescriptor(desc);
+  }
+
+  private static ImmutableHColumnDescriptor toImmutableHColumnDescriptor(ColumnFamilyDescriptor desc) {
+    return new ImmutableHColumnDescriptor(desc);
+  }
 }


[40/50] [abbrv] hbase git commit: HBASE-18251 Remove unnecessary traversing to the first and last keys in the CellSet (Toshihoro Suzuki)

Posted by st...@apache.org.
HBASE-18251 Remove unnecessary traversing to the first and last keys in
the CellSet (Toshihoro Suzuki)


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

Branch: refs/heads/HBASE-14070.HLC
Commit: 9da4e6906e9d7f62b8a8fe5dc996b066dac4066e
Parents: b087818
Author: Ramkrishna <ra...@intel.com>
Authored: Wed Aug 16 11:05:43 2017 +0530
Committer: Ramkrishna <ra...@intel.com>
Committed: Wed Aug 16 11:06:31 2017 +0530

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/CellFlatMap.java  | 63 +++++++++++++++++---
 .../hadoop/hbase/regionserver/CellSet.java      |  7 +--
 2 files changed, 57 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/9da4e690/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellFlatMap.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellFlatMap.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellFlatMap.java
index c83a382..aff6018 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellFlatMap.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellFlatMap.java
@@ -282,37 +282,85 @@ public abstract class CellFlatMap implements NavigableMap<Cell,Cell> {
   }
 
   // -------------------------------- Entry's getters --------------------------------
-  // all interfaces returning Entries are unsupported because we are dealing only with the keys
+
+  private static class CellFlatMapEntry implements Entry<Cell, Cell> {
+    private final Cell cell;
+
+    public CellFlatMapEntry (Cell cell) {
+      this.cell = cell;
+    }
+
+    @Override
+    public Cell getKey() {
+      return cell;
+    }
+
+    @Override
+    public Cell getValue() {
+      return cell;
+    }
+
+    @Override
+    public Cell setValue(Cell value) {
+      throw new UnsupportedOperationException();
+    }
+  }
+
   @Override
   public Entry<Cell, Cell> lowerEntry(Cell k) {
-    throw new UnsupportedOperationException();
+    Cell cell = lowerKey(k);
+    if (cell == null) {
+      return null;
+    }
+    return new CellFlatMapEntry(cell);
   }
 
   @Override
   public Entry<Cell, Cell> higherEntry(Cell k) {
-    throw new UnsupportedOperationException();
+    Cell cell = higherKey(k);
+    if (cell == null) {
+      return null;
+    }
+    return new CellFlatMapEntry(cell);
   }
 
   @Override
   public Entry<Cell, Cell> ceilingEntry(Cell k) {
-    throw new UnsupportedOperationException();
+    Cell cell = ceilingKey(k);
+    if (cell == null) {
+      return null;
+    }
+    return new CellFlatMapEntry(cell);
   }
 
   @Override
   public Entry<Cell, Cell> floorEntry(Cell k) {
-    throw new UnsupportedOperationException();
+    Cell cell = floorKey(k);
+    if (cell == null) {
+      return null;
+    }
+    return new CellFlatMapEntry(cell);
   }
 
   @Override
   public Entry<Cell, Cell> firstEntry() {
-    throw new UnsupportedOperationException();
+    Cell cell = firstKey();
+    if (cell == null) {
+      return null;
+    }
+    return new CellFlatMapEntry(cell);
   }
 
   @Override
   public Entry<Cell, Cell> lastEntry() {
-    throw new UnsupportedOperationException();
+    Cell cell = lastKey();
+    if (cell == null) {
+      return null;
+    }
+    return new CellFlatMapEntry(cell);
   }
 
+  // The following 2 methods (pollFirstEntry, pollLastEntry) are unsupported because these are updating methods.
   @Override
   public Entry<Cell, Cell> pollFirstEntry() {
     throw new UnsupportedOperationException();
@@ -323,7 +371,6 @@ public abstract class CellFlatMap implements NavigableMap<Cell,Cell> {
     throw new UnsupportedOperationException();
   }
 
-
   // -------------------------------- Updates --------------------------------
   // All updating methods below are unsupported.
   // Assuming an array of Cells will be allocated externally,

http://git-wip-us.apache.org/repos/asf/hbase/blob/9da4e690/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellSet.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellSet.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellSet.java
index 48262a9..6da57d3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellSet.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellSet.java
@@ -126,15 +126,12 @@ public class CellSet implements NavigableSet<Cell>  {
     throw new UnsupportedOperationException("Not implemented");
   }
 
-  // TODO: why do we have a double traversing through map? Recall we have Cell to Cell mapping...
-  // First for first/last key, which actually returns Cell and then get for the same Cell?
-  // TODO: Consider just return the first/lastKey(), should be twice more effective...
   public Cell first() {
-    return this.delegatee.get(this.delegatee.firstKey());
+    return this.delegatee.firstEntry().getValue();
   }
 
   public Cell last() {
-    return this.delegatee.get(this.delegatee.lastKey());
+    return this.delegatee.lastEntry().getValue();
   }
 
   public boolean add(Cell e) {


[10/50] [abbrv] hbase git commit: HBASE-18025 CatalogJanitor should collect outdated RegionStates from the AM

Posted by st...@apache.org.
HBASE-18025 CatalogJanitor should collect outdated RegionStates from the AM


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

Branch: refs/heads/HBASE-14070.HLC
Commit: 71a9a9a9440c9f2e2e9dd301dd372197e38e70c5
Parents: 043ec9b
Author: Esteban Gutierrez <es...@apache.org>
Authored: Fri Jul 21 14:13:13 2017 -0500
Committer: Esteban Gutierrez <es...@apache.org>
Committed: Fri Aug 11 13:36:38 2017 -0500

----------------------------------------------------------------------
 .../hadoop/hbase/master/CatalogJanitor.java     |  13 +-
 .../hadoop/hbase/master/ServerManager.java      |   7 +
 .../hbase/master/assignment/RegionStates.java   |   6 +
 .../TestCatalogJanitorInMemoryStates.java       | 185 +++++++++++++++++++
 4 files changed, 209 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/71a9a9a9/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
index ba92c76..8daa7db 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
@@ -221,6 +221,11 @@ public class CatalogJanitor extends ScheduledChore {
       ProcedureExecutor<MasterProcedureEnv> pe = this.services.getMasterProcedureExecutor();
       pe.submitProcedure(new GCMergedRegionsProcedure(pe.getEnvironment(),
           mergedRegion, regionA, regionB));
+      // Remove from in-memory states
+      this.services.getAssignmentManager().getRegionStates().deleteRegion(regionA);
+      this.services.getAssignmentManager().getRegionStates().deleteRegion(regionB);
+      this.services.getServerManager().removeRegion(regionA);
+      this.services.getServerManager().removeRegion(regionB);
       return true;
     }
     return false;
@@ -234,6 +239,7 @@ public class CatalogJanitor extends ScheduledChore {
    */
   int scan() throws IOException {
     int result = 0;
+
     try {
       if (!alreadyRunning.compareAndSet(false, true)) {
         LOG.debug("CatalogJanitor already running");
@@ -281,8 +287,8 @@ public class CatalogJanitor extends ScheduledChore {
         }
 
         if (!parentNotCleaned.contains(e.getKey().getEncodedName()) &&
-              cleanParent(e.getKey(), e.getValue())) {
-            result++;
+            cleanParent(e.getKey(), e.getValue())) {
+          result++;
         } else {
           // We could not clean the parent, so it's daughters should not be
           // cleaned either (HBASE-6160)
@@ -355,6 +361,9 @@ public class CatalogJanitor extends ScheduledChore {
         " -- no longer hold references");
       ProcedureExecutor<MasterProcedureEnv> pe = this.services.getMasterProcedureExecutor();
       pe.submitProcedure(new GCRegionProcedure(pe.getEnvironment(), parent));
+      // Remove from in-memory states
+      this.services.getAssignmentManager().getRegionStates().deleteRegion(parent);
+      this.services.getServerManager().removeRegion(parent);
       return true;
     }
     return false;

http://git-wip-us.apache.org/repos/asf/hbase/blob/71a9a9a9/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 c9c792a..f0e9b88 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
@@ -1028,6 +1028,13 @@ public class ServerManager {
     flushedSequenceIdByRegion.remove(encodedName);
   }
 
+  @VisibleForTesting
+  public boolean isRegionInServerManagerStates(final HRegionInfo hri) {
+    final byte[] encodedName = hri.getEncodedNameAsBytes();
+    return (storeFlushedSequenceIdsByRegion.containsKey(encodedName)
+        || flushedSequenceIdByRegion.containsKey(encodedName));
+  }
+
   /**
    * Called by delete table and similar to notify the ServerManager that a region was removed.
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/71a9a9a9/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
index df55c94..1169dda 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
@@ -432,6 +432,12 @@ public class RegionStates {
     serverMap.clear();
   }
 
+  @VisibleForTesting
+  public boolean isRegionInRegionStates(final HRegionInfo hri) {
+    return (regionsMap.containsKey(hri.getRegionName()) || regionInTransition.containsKey(hri)
+        || regionOffline.containsKey(hri));
+  }
+
   // ==========================================================================
   //  RegionStateNode helpers
   // ==========================================================================

http://git-wip-us.apache.org/repos/asf/hbase/blob/71a9a9a9/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitorInMemoryStates.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitorInMemoryStates.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitorInMemoryStates.java
new file mode 100644
index 0000000..38abe57
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitorInMemoryStates.java
@@ -0,0 +1,185 @@
+/**
+ *
+ * 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 org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.*;
+import org.apache.hadoop.hbase.client.*;
+import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.PairOfSameType;
+import org.apache.hadoop.hbase.util.Threads;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.junit.rules.TestName;
+import org.junit.rules.TestRule;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertNotNull;
+
+@Category({MasterTests.class, MediumTests.class})
+public class TestCatalogJanitorInMemoryStates {
+  private static final Log LOG = LogFactory.getLog(TestCatalogJanitorInMemoryStates.class);
+  @Rule public final TestRule timeout = CategoryBasedTimeout.builder().
+     withTimeout(this.getClass()).withLookingForStuckThread(true).build();
+  @Rule public final TestName name = new TestName();
+  protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static byte [] ROW = Bytes.toBytes("testRow");
+  private static byte [] FAMILY = Bytes.toBytes("testFamily");
+  private static byte [] QUALIFIER = Bytes.toBytes("testQualifier");
+  private static byte [] VALUE = Bytes.toBytes("testValue");
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    Configuration conf = TEST_UTIL.getConfiguration();
+    TEST_UTIL.startMiniCluster(1);
+  }
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  /**
+   * Test clearing a split parent from memory.
+   */
+  @Test(timeout = 180000)
+  public void testInMemoryParentCleanup() throws IOException, InterruptedException {
+    final AssignmentManager am = TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager();
+    final ServerManager sm = TEST_UTIL.getHBaseCluster().getMaster().getServerManager();
+    final CatalogJanitor janitor = TEST_UTIL.getHBaseCluster().getMaster().getCatalogJanitor();
+
+    Admin admin = TEST_UTIL.getAdmin();
+    admin.enableCatalogJanitor(false);
+
+    final TableName tableName = TableName.valueOf(name.getMethodName());
+    Table t = TEST_UTIL.createTable(tableName, FAMILY);
+    int rowCount = TEST_UTIL.loadTable(t, FAMILY, false);
+
+    RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
+    List<HRegionLocation> allRegionLocations = locator.getAllRegionLocations();
+
+    // We need to create a valid split with daughter regions
+    HRegionLocation parent = allRegionLocations.get(0);
+    List<HRegionLocation> daughters = splitRegion(parent.getRegionInfo());
+    LOG.info("Parent region: " + parent);
+    LOG.info("Daughter regions: " + daughters);
+    assertNotNull("Should have found daughter regions for " + parent, daughters);
+
+    assertTrue("Parent region should exist in RegionStates",
+        am.getRegionStates().isRegionInRegionStates(parent.getRegionInfo()));
+    assertTrue("Parent region should exist in ServerManager",
+        sm.isRegionInServerManagerStates(parent.getRegionInfo()));
+
+    // clean the parent
+    Result r = MetaMockingUtil.getMetaTableRowResult(parent.getRegionInfo(), null,
+        daughters.get(0).getRegionInfo(), daughters.get(1).getRegionInfo());
+    janitor.cleanParent(parent.getRegionInfo(), r);
+    assertFalse("Parent region should have been removed from RegionStates",
+        am.getRegionStates().isRegionInRegionStates(parent.getRegionInfo()));
+    assertFalse("Parent region should have been removed from ServerManager",
+        sm.isRegionInServerManagerStates(parent.getRegionInfo()));
+
+  }
+
+  /*
+ * Splits a region
+ * @param t Region to split.
+ * @return List of region locations
+ * @throws IOException, InterruptedException
+ */
+  private List<HRegionLocation> splitRegion(final HRegionInfo r)
+      throws IOException, InterruptedException {
+    List<HRegionLocation> locations = new ArrayList<>();
+    // Split this table in two.
+    Admin admin = TEST_UTIL.getAdmin();
+    Connection connection = TEST_UTIL.getConnection();
+    admin.splitRegion(r.getEncodedNameAsBytes());
+    admin.close();
+    PairOfSameType<HRegionInfo> regions = waitOnDaughters(r);
+    if (regions != null) {
+      try (RegionLocator rl = connection.getRegionLocator(r.getTable())) {
+        locations.add(rl.getRegionLocation(regions.getFirst().getEncodedNameAsBytes()));
+        locations.add(rl.getRegionLocation(regions.getSecond().getEncodedNameAsBytes()));
+      }
+      return locations;
+    }
+    return locations;
+  }
+
+  /*
+   * Wait on region split. May return because we waited long enough on the split
+   * and it didn't happen.  Caller should check.
+   * @param r
+   * @return Daughter regions; caller needs to check table actually split.
+   */
+  private PairOfSameType<HRegionInfo> waitOnDaughters(final HRegionInfo r)
+      throws IOException {
+    long start = System.currentTimeMillis();
+    PairOfSameType<HRegionInfo> pair = null;
+    try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
+         Table metaTable = conn.getTable(TableName.META_TABLE_NAME)) {
+      Result result = null;
+      HRegionInfo region = null;
+      while ((System.currentTimeMillis() - start) < 60000) {
+        result = metaTable.get(new Get(r.getRegionName()));
+        if (result == null) {
+          break;
+        }
+        region = MetaTableAccessor.getHRegionInfo(result);
+        if (region.isSplitParent()) {
+          LOG.debug(region.toString() + " IS a parent!");
+          pair = MetaTableAccessor.getDaughterRegions(result);
+          break;
+        }
+        Threads.sleep(100);
+      }
+
+      if (pair.getFirst() == null || pair.getSecond() == null) {
+        throw new IOException("Failed to get daughters, for parent region: " + r);
+      }
+      return pair;
+    }
+  }
+}
\ No newline at end of file


[42/50] [abbrv] hbase git commit: HBASE-18553 Expose scan cursor for asynchronous scanner

Posted by st...@apache.org.
HBASE-18553 Expose scan cursor for asynchronous scanner


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

Branch: refs/heads/HBASE-14070.HLC
Commit: 4c74a73d57e09fd2c0ecde862a196c28dc6cd219
Parents: 2a9cdd5
Author: zhangduo <zh...@apache.org>
Authored: Tue Aug 15 17:15:06 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Wed Aug 16 21:04:57 2017 +0800

----------------------------------------------------------------------
 .../AsyncScanSingleRegionRpcRetryingCaller.java |  35 ++++-
 .../hbase/client/AsyncTableResultScanner.java   |  20 ++-
 .../hbase/client/RawScanResultConsumer.java     |  11 +-
 .../client/AbstractTestResultScannerCursor.java |  89 +++++++++++
 .../client/TestAsyncResultScannerCursor.java    |  49 ++++++
 .../hbase/client/TestRawAsyncScanCursor.java    | 157 +++++++++++++------
 .../hbase/client/TestResultScannerCursor.java   |  34 ++++
 .../hadoop/hbase/client/TestScanCursor.java     |  90 -----------
 8 files changed, 330 insertions(+), 155 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/4c74a73d/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncScanSingleRegionRpcRetryingCaller.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncScanSingleRegionRpcRetryingCaller.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncScanSingleRegionRpcRetryingCaller.java
index 02a4357..d16cb8b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncScanSingleRegionRpcRetryingCaller.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncScanSingleRegionRpcRetryingCaller.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.shaded.io.netty.util.Timeout;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
@@ -157,10 +158,9 @@ class AsyncScanSingleRegionRpcRetryingCaller {
 
     private ScanResumerImpl resumer;
 
-    public ScanControllerImpl(ScanResponse resp) {
-      callerThread = Thread.currentThread();
-      cursor = resp.hasCursor() ? Optional.of(ProtobufUtil.toCursor(resp.getCursor()))
-          : Optional.empty();
+    public ScanControllerImpl(Optional<Cursor> cursor) {
+      this.callerThread = Thread.currentThread();
+      this.cursor = cursor;
     }
 
     private void preCheck() {
@@ -476,10 +476,11 @@ class AsyncScanSingleRegionRpcRetryingCaller {
     }
     updateServerSideMetrics(scanMetrics, resp);
     boolean isHeartbeatMessage = resp.hasHeartbeatMessage() && resp.getHeartbeatMessage();
+    Result[] rawResults;
     Result[] results;
     int numberOfCompleteRowsBefore = resultCache.numberOfCompleteRows();
     try {
-      Result[] rawResults = ResponseConverter.getResults(controller.cellScanner(), resp);
+      rawResults = ResponseConverter.getResults(controller.cellScanner(), resp);
       updateResultsMetrics(scanMetrics, rawResults, isHeartbeatMessage);
       results = resultCache.addAndGet(
         Optional.ofNullable(rawResults).orElse(ScanResultCache.EMPTY_RESULT_ARRAY),
@@ -493,12 +494,30 @@ class AsyncScanSingleRegionRpcRetryingCaller {
       return;
     }
 
-    ScanControllerImpl scanController = new ScanControllerImpl(resp);
+    ScanControllerImpl scanController;
     if (results.length > 0) {
+      scanController = new ScanControllerImpl(
+          resp.hasCursor() ? Optional.of(ProtobufUtil.toCursor(resp.getCursor()))
+              : Optional.empty());
       updateNextStartRowWhenError(results[results.length - 1]);
       consumer.onNext(results, scanController);
-    } else if (resp.hasHeartbeatMessage() && resp.getHeartbeatMessage()) {
-      consumer.onHeartbeat(scanController);
+    } else {
+      Optional<Cursor> cursor = Optional.empty();
+      if (resp.hasCursor()) {
+        cursor = Optional.of(ProtobufUtil.toCursor(resp.getCursor()));
+      } else if (scan.isNeedCursorResult() && rawResults.length > 0) {
+        // It is size limit exceed and we need to return the last Result's row.
+        // When user setBatch and the scanner is reopened, the server may return Results that
+        // user has seen and the last Result can not be seen because the number is not enough.
+        // So the row keys of results may not be same, we must use the last one.
+        cursor = Optional.of(new Cursor(rawResults[rawResults.length - 1].getRow()));
+      }
+      scanController = new ScanControllerImpl(cursor);
+      if (isHeartbeatMessage || cursor.isPresent()) {
+        // only call onHeartbeat if server tells us explicitly this is a heartbeat message, or we
+        // want to pass a cursor to upper layer.
+        consumer.onHeartbeat(scanController);
+      }
     }
     ScanControllerState state = scanController.destroy();
     if (state == ScanControllerState.TERMINATED) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/4c74a73d/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableResultScanner.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableResultScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableResultScanner.java
index 28a5568..3050084 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableResultScanner.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableResultScanner.java
@@ -19,9 +19,6 @@ package org.apache.hadoop.hbase.client;
 
 import static org.apache.hadoop.hbase.client.ConnectionUtils.calcEstimatedSize;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.hbase.shaded.com.google.common.base.Throwables;
-
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.util.ArrayDeque;
@@ -31,6 +28,8 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Throwables;
 
 /**
  * The {@link ResultScanner} implementation for {@link AsyncTable}. It will fetch data automatically
@@ -46,6 +45,8 @@ class AsyncTableResultScanner implements ResultScanner, RawScanResultConsumer {
 
   private final long maxCacheSize;
 
+  private final Scan scan;
+
   private final Queue<Result> queue = new ArrayDeque<>();
 
   private ScanMetrics scanMetrics;
@@ -61,6 +62,7 @@ class AsyncTableResultScanner implements ResultScanner, RawScanResultConsumer {
   public AsyncTableResultScanner(RawAsyncTable table, Scan scan, long maxCacheSize) {
     this.rawTable = table;
     this.maxCacheSize = maxCacheSize;
+    this.scan = scan;
     table.scan(scan, this);
   }
 
@@ -98,6 +100,10 @@ class AsyncTableResultScanner implements ResultScanner, RawScanResultConsumer {
   public synchronized void onHeartbeat(ScanController controller) {
     if (closed) {
       controller.terminate();
+      return;
+    }
+    if (scan.isNeedCursorResult()) {
+      controller.cursor().ifPresent(c -> queue.add(Result.createCursorResult(c)));
     }
   }
 
@@ -143,9 +149,11 @@ class AsyncTableResultScanner implements ResultScanner, RawScanResultConsumer {
       }
     }
     Result result = queue.poll();
-    cacheSize -= calcEstimatedSize(result);
-    if (resumer != null && cacheSize <= maxCacheSize / 2) {
-      resumePrefetch();
+    if (!result.isCursor()) {
+      cacheSize -= calcEstimatedSize(result);
+      if (resumer != null && cacheSize <= maxCacheSize / 2) {
+        resumePrefetch();
+      }
     }
     return result;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4c74a73d/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawScanResultConsumer.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawScanResultConsumer.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawScanResultConsumer.java
index 54d4887..4fedb0f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawScanResultConsumer.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawScanResultConsumer.java
@@ -95,8 +95,15 @@ public interface RawScanResultConsumer {
   void onNext(Result[] results, ScanController controller);
 
   /**
-   * Indicate that there is an heartbeat message but we have not cumulated enough cells to call
-   * onNext.
+   * Indicate that there is a heartbeat message but we have not cumulated enough cells to call
+   * {@link #onNext(Result[], ScanController)}.
+   * <p>
+   * Note that this method will always be called when RS returns something to us but we do not have
+   * enough cells to call {@link #onNext(Result[], ScanController)}. Sometimes it may not be a
+   * 'heartbeat' message for RS, for example, we have a large row with many cells and size limit is
+   * exceeded before sending all the cells for this row. For RS it does send some data to us and the
+   * time limit has not been reached, but we can not return the data to client so here we call this
+   * method to tell client we have already received something.
    * <p>
    * This method give you a chance to terminate a slow scan operation.
    * @param controller used to suspend or terminate the scan. Notice that the {@code controller}

http://git-wip-us.apache.org/repos/asf/hbase/blob/4c74a73d/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestResultScannerCursor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestResultScannerCursor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestResultScannerCursor.java
new file mode 100644
index 0000000..3df7a7b
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestResultScannerCursor.java
@@ -0,0 +1,89 @@
+/**
+ * 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 static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+
+import org.junit.Test;
+
+public abstract class AbstractTestResultScannerCursor extends AbstractTestScanCursor {
+
+  protected abstract ResultScanner getScanner(Scan scan) throws Exception;
+
+  @Test
+  public void testHeartbeatWithSparseFilter() throws Exception {
+    try (ResultScanner scanner = getScanner(createScanWithSparseFilter())) {
+      int num = 0;
+      Result r;
+      while ((r = scanner.next()) != null) {
+        if (num < (NUM_ROWS - 1) * NUM_FAMILIES * NUM_QUALIFIERS) {
+          assertTrue(r.isCursor());
+          assertArrayEquals(ROWS[num / NUM_FAMILIES / NUM_QUALIFIERS],
+            r.getCursor().getRow());
+        } else {
+          assertFalse(r.isCursor());
+          assertArrayEquals(ROWS[num / NUM_FAMILIES / NUM_QUALIFIERS], r.getRow());
+        }
+        num++;
+      }
+    }
+  }
+
+  @Test
+  public void testHeartbeatWithSparseFilterReversed() throws Exception {
+    try (ResultScanner scanner = getScanner(createReversedScanWithSparseFilter())) {
+      int num = 0;
+      Result r;
+      while ((r = scanner.next()) != null) {
+        if (num < (NUM_ROWS - 1) * NUM_FAMILIES * NUM_QUALIFIERS) {
+          assertTrue(r.isCursor());
+          assertArrayEquals(ROWS[NUM_ROWS - 1 - num / NUM_FAMILIES / NUM_QUALIFIERS],
+            r.getCursor().getRow());
+        } else {
+          assertFalse(r.isCursor());
+          assertArrayEquals(ROWS[0], r.getRow());
+        }
+        num++;
+      }
+    }
+  }
+
+  @Test
+  public void testSizeLimit() throws IOException {
+    try (ResultScanner scanner =
+        TEST_UTIL.getConnection().getTable(TABLE_NAME).getScanner(createScanWithSizeLimit())) {
+      int num = 0;
+      Result r;
+      while ((r = scanner.next()) != null) {
+        if (num % (NUM_FAMILIES * NUM_QUALIFIERS) != (NUM_FAMILIES * NUM_QUALIFIERS) - 1) {
+          assertTrue(r.isCursor());
+          assertArrayEquals(ROWS[num / NUM_FAMILIES / NUM_QUALIFIERS],
+            r.getCursor().getRow());
+        } else {
+          assertFalse(r.isCursor());
+          assertArrayEquals(ROWS[num / NUM_FAMILIES / NUM_QUALIFIERS], r.getRow());
+        }
+        num++;
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/4c74a73d/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncResultScannerCursor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncResultScannerCursor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncResultScannerCursor.java
new file mode 100644
index 0000000..5aebb4a
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncResultScannerCursor.java
@@ -0,0 +1,49 @@
+/**
+ * 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 java.util.concurrent.ForkJoinPool;
+
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.junit.BeforeClass;
+import org.junit.experimental.categories.Category;
+
+@Category({ MediumTests.class, ClientTests.class })
+public class TestAsyncResultScannerCursor extends AbstractTestResultScannerCursor {
+
+  private static AsyncConnection CONN;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    AbstractTestScanCursor.setUpBeforeClass();
+    CONN = ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get();
+  }
+
+  public static void tearDownAfterClass() throws Exception {
+    if (CONN != null) {
+      CONN.close();
+    }
+    AbstractTestScanCursor.tearDownAfterClass();
+  }
+
+  @Override
+  protected ResultScanner getScanner(Scan scan) throws Exception {
+    return CONN.getTable(TABLE_NAME, ForkJoinPool.commonPool()).getScanner(scan);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/4c74a73d/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRawAsyncScanCursor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRawAsyncScanCursor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRawAsyncScanCursor.java
index 9caf942..4bca451 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRawAsyncScanCursor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRawAsyncScanCursor.java
@@ -27,70 +27,83 @@ import java.util.concurrent.ExecutionException;
 
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 @Category({ MediumTests.class, ClientTests.class })
 public class TestRawAsyncScanCursor extends AbstractTestScanCursor {
 
+  private static AsyncConnection CONN;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    AbstractTestScanCursor.setUpBeforeClass();
+    CONN = ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get();
+  }
+
+  public static void tearDownAfterClass() throws Exception {
+    if (CONN != null) {
+      CONN.close();
+    }
+    AbstractTestScanCursor.tearDownAfterClass();
+  }
+
   private void doTest(boolean reversed)
       throws InterruptedException, ExecutionException, IOException {
     CompletableFuture<Void> future = new CompletableFuture<>();
-    try (AsyncConnection conn =
-        ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get()) {
-      RawAsyncTable table = conn.getRawTable(TABLE_NAME);
-      table.scan(reversed ? createReversedScanWithSparseFilter() : createScanWithSparseFilter(),
-        new RawScanResultConsumer() {
-
-          private int count;
-
-          @Override
-          public void onHeartbeat(ScanController controller) {
-            int row = count / NUM_FAMILIES / NUM_QUALIFIERS;
-            if (reversed) {
-              row = NUM_ROWS - 1 - row;
-            }
-            try {
-              assertArrayEquals(ROWS[row], controller.cursor().get().getRow());
-              count++;
-            } catch (Throwable e) {
-              future.completeExceptionally(e);
-              throw e;
-            }
+    RawAsyncTable table = CONN.getRawTable(TABLE_NAME);
+    table.scan(reversed ? createReversedScanWithSparseFilter() : createScanWithSparseFilter(),
+      new RawScanResultConsumer() {
+
+        private int count;
+
+        @Override
+        public void onHeartbeat(ScanController controller) {
+          int row = count / NUM_FAMILIES / NUM_QUALIFIERS;
+          if (reversed) {
+            row = NUM_ROWS - 1 - row;
+          }
+          try {
+            assertArrayEquals(ROWS[row], controller.cursor().get().getRow());
+            count++;
+          } catch (Throwable e) {
+            future.completeExceptionally(e);
+            throw e;
           }
+        }
 
-          @Override
-          public void onNext(Result[] results, ScanController controller) {
-            try {
-              assertEquals(1, results.length);
-              assertEquals(NUM_ROWS - 1, count / NUM_FAMILIES / NUM_QUALIFIERS);
-              // we will always provide a scan cursor if time limit is reached.
-              if (count == NUM_ROWS * NUM_FAMILIES * NUM_QUALIFIERS - 1) {
-                assertFalse(controller.cursor().isPresent());
-              } else {
-                assertArrayEquals(ROWS[reversed ? 0 : NUM_ROWS - 1],
-                  controller.cursor().get().getRow());
-              }
-              assertArrayEquals(ROWS[reversed ? 0 : NUM_ROWS - 1], results[0].getRow());
-              count++;
-            } catch (Throwable e) {
-              future.completeExceptionally(e);
-              throw e;
+        @Override
+        public void onNext(Result[] results, ScanController controller) {
+          try {
+            assertEquals(1, results.length);
+            assertEquals(NUM_ROWS - 1, count / NUM_FAMILIES / NUM_QUALIFIERS);
+            // we will always provide a scan cursor if time limit is reached.
+            if (count == NUM_ROWS * NUM_FAMILIES * NUM_QUALIFIERS - 1) {
+              assertFalse(controller.cursor().isPresent());
+            } else {
+              assertArrayEquals(ROWS[reversed ? 0 : NUM_ROWS - 1],
+                controller.cursor().get().getRow());
             }
+            assertArrayEquals(ROWS[reversed ? 0 : NUM_ROWS - 1], results[0].getRow());
+            count++;
+          } catch (Throwable e) {
+            future.completeExceptionally(e);
+            throw e;
           }
+        }
 
-          @Override
-          public void onError(Throwable error) {
-            future.completeExceptionally(error);
-          }
+        @Override
+        public void onError(Throwable error) {
+          future.completeExceptionally(error);
+        }
 
-          @Override
-          public void onComplete() {
-            future.complete(null);
-          }
-        });
-      future.get();
-    }
+        @Override
+        public void onComplete() {
+          future.complete(null);
+        }
+      });
+    future.get();
   }
 
   @Test
@@ -104,4 +117,50 @@ public class TestRawAsyncScanCursor extends AbstractTestScanCursor {
       throws IOException, InterruptedException, ExecutionException {
     doTest(true);
   }
+
+  @Test
+  public void testSizeLimit() throws InterruptedException, ExecutionException {
+    CompletableFuture<Void> future = new CompletableFuture<>();
+    RawAsyncTable table = CONN.getRawTable(TABLE_NAME);
+    table.scan(createScanWithSizeLimit(), new RawScanResultConsumer() {
+
+      private int count;
+
+      @Override
+      public void onHeartbeat(ScanController controller) {
+        try {
+          assertArrayEquals(ROWS[count / NUM_FAMILIES / NUM_QUALIFIERS],
+            controller.cursor().get().getRow());
+          count++;
+        } catch (Throwable e) {
+          future.completeExceptionally(e);
+          throw e;
+        }
+      }
+
+      @Override
+      public void onNext(Result[] results, ScanController controller) {
+        try {
+          assertFalse(controller.cursor().isPresent());
+          assertEquals(1, results.length);
+          assertArrayEquals(ROWS[count / NUM_FAMILIES / NUM_QUALIFIERS], results[0].getRow());
+          count++;
+        } catch (Throwable e) {
+          future.completeExceptionally(e);
+          throw e;
+        }
+      }
+
+      @Override
+      public void onError(Throwable error) {
+        future.completeExceptionally(error);
+      }
+
+      @Override
+      public void onComplete() {
+        future.complete(null);
+      }
+    });
+    future.get();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4c74a73d/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResultScannerCursor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResultScannerCursor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResultScannerCursor.java
new file mode 100644
index 0000000..3b2ef2c
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResultScannerCursor.java
@@ -0,0 +1,34 @@
+/**
+ * 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 java.io.IOException;
+
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.junit.experimental.categories.Category;
+
+@Category({ MediumTests.class, ClientTests.class })
+public class TestResultScannerCursor extends AbstractTestResultScannerCursor {
+
+  @Override
+  protected ResultScanner getScanner(Scan scan) throws IOException {
+    return TEST_UTIL.getConnection().getTable(TABLE_NAME).getScanner(scan);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/4c74a73d/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScanCursor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScanCursor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScanCursor.java
deleted file mode 100644
index f7798f0..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScanCursor.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/**
- * 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 java.io.IOException;
-
-import org.apache.hadoop.hbase.testclassification.ClientTests;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category({ MediumTests.class, ClientTests.class })
-public class TestScanCursor extends AbstractTestScanCursor {
-
-  @Test
-  public void testHeartbeatWithSparseFilter() throws Exception {
-    try (ResultScanner scanner =
-        TEST_UTIL.getConnection().getTable(TABLE_NAME).getScanner(createScanWithSparseFilter())) {
-      int num = 0;
-      Result r;
-      while ((r = scanner.next()) != null) {
-        if (num < (NUM_ROWS - 1) * NUM_FAMILIES * NUM_QUALIFIERS) {
-          Assert.assertTrue(r.isCursor());
-          Assert.assertArrayEquals(ROWS[num / NUM_FAMILIES / NUM_QUALIFIERS],
-            r.getCursor().getRow());
-        } else {
-          Assert.assertFalse(r.isCursor());
-          Assert.assertArrayEquals(ROWS[num / NUM_FAMILIES / NUM_QUALIFIERS], r.getRow());
-        }
-        num++;
-      }
-    }
-  }
-
-  @Test
-  public void testHeartbeatWithSparseFilterReversed() throws Exception {
-    try (ResultScanner scanner = TEST_UTIL.getConnection().getTable(TABLE_NAME)
-        .getScanner(createReversedScanWithSparseFilter())) {
-      int num = 0;
-      Result r;
-      while ((r = scanner.next()) != null) {
-        if (num < (NUM_ROWS - 1) * NUM_FAMILIES * NUM_QUALIFIERS) {
-          Assert.assertTrue(r.isCursor());
-          Assert.assertArrayEquals(ROWS[NUM_ROWS - 1 - num / NUM_FAMILIES / NUM_QUALIFIERS],
-            r.getCursor().getRow());
-        } else {
-          Assert.assertFalse(r.isCursor());
-          Assert.assertArrayEquals(ROWS[0], r.getRow());
-        }
-        num++;
-      }
-    }
-  }
-
-  @Test
-  public void testSizeLimit() throws IOException {
-    try (ResultScanner scanner =
-        TEST_UTIL.getConnection().getTable(TABLE_NAME).getScanner(createScanWithSizeLimit())) {
-      int num = 0;
-      Result r;
-      while ((r = scanner.next()) != null) {
-        if (num % (NUM_FAMILIES * NUM_QUALIFIERS) != (NUM_FAMILIES * NUM_QUALIFIERS) - 1) {
-          Assert.assertTrue(r.isCursor());
-          Assert.assertArrayEquals(ROWS[num / NUM_FAMILIES / NUM_QUALIFIERS],
-            r.getCursor().getRow());
-        } else {
-          Assert.assertFalse(r.isCursor());
-          Assert.assertArrayEquals(ROWS[num / NUM_FAMILIES / NUM_QUALIFIERS], r.getRow());
-        }
-        num++;
-      }
-    }
-  }
-}


[31/50] [abbrv] hbase git commit: HBASE-17064 Add TaskMonitor#getTasks() variant which accepts type selection

Posted by st...@apache.org.
HBASE-17064 Add TaskMonitor#getTasks() variant which accepts type selection

Signed-off-by: tedyu <yu...@gmail.com>


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

Branch: refs/heads/HBASE-14070.HLC
Commit: effd1093b559aeba2bf66a4cf81cd4a0013de184
Parents: d37266f
Author: Reid Chan <re...@outlook.com>
Authored: Tue Aug 15 15:50:22 2017 +0800
Committer: tedyu <yu...@gmail.com>
Committed: Tue Aug 15 09:45:19 2017 -0700

----------------------------------------------------------------------
 .../hbase/tmpl/common/TaskMonitorTmpl.jamon     | 21 +----
 .../hadoop/hbase/monitoring/TaskMonitor.java    | 97 +++++++++++++++++---
 .../hbase/monitoring/TestTaskMonitor.java       | 48 ++++++++++
 3 files changed, 133 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/effd1093/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/common/TaskMonitorTmpl.jamon
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/common/TaskMonitorTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/common/TaskMonitorTmpl.jamon
index b4a5fea..986bc3a 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/common/TaskMonitorTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/common/TaskMonitorTmpl.jamon
@@ -27,27 +27,8 @@ String filter = "general";
 String format = "html";
 </%args>
 <%java>
-List<? extends MonitoredTask> tasks = taskMonitor.getTasks();
-Iterator<? extends MonitoredTask> iter = tasks.iterator();
 // apply requested filter
-while (iter.hasNext()) {
-  MonitoredTask t = iter.next();
-  if (filter.equals("general")) {
-    if (t instanceof MonitoredRPCHandler)
-      iter.remove();
-  } else if (filter.equals("handler")) {
-    if (!(t instanceof MonitoredRPCHandler))
-      iter.remove();
-  } else if (filter.equals("rpc")) {
-    if (!(t instanceof MonitoredRPCHandler) || 
-        !((MonitoredRPCHandler) t).isRPCRunning())
-      iter.remove();
-  } else if (filter.equals("operation")) {
-    if (!(t instanceof MonitoredRPCHandler) || 
-        !((MonitoredRPCHandler) t).isOperationRunning())
-      iter.remove();
-  }
-}
+List<? extends MonitoredTask> tasks = taskMonitor.getTasks(filter);
 long now = System.currentTimeMillis();
 Collections.reverse(tasks);
 boolean first = true;

http://git-wip-us.apache.org/repos/asf/hbase/blob/effd1093/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/TaskMonitor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/TaskMonitor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/TaskMonitor.java
index 780916f..ad9bd02 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/TaskMonitor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/TaskMonitor.java
@@ -157,22 +157,52 @@ public class TaskMonitor {
    * MonitoredTasks handled by this TaskMonitor.
    * @return A complete list of MonitoredTasks.
    */
-  public synchronized List<MonitoredTask> getTasks() {
+  public List<MonitoredTask> getTasks() {
+    return getTasks(null);
+  }
+
+  /**
+   * Produces a list containing copies of the current state of all non-expired 
+   * MonitoredTasks handled by this TaskMonitor.
+   * @param filter type of wanted tasks
+   * @return A filtered list of MonitoredTasks.
+   */
+  public synchronized List<MonitoredTask> getTasks(String filter) {
     purgeExpiredTasks();
-    ArrayList<MonitoredTask> ret = Lists.newArrayListWithCapacity(tasks.size() + rpcTasks.size());
-    for (Iterator<TaskAndWeakRefPair> it = tasks.iterator();
-         it.hasNext();) {
-      TaskAndWeakRefPair pair = it.next();
-      MonitoredTask t = pair.get();
-      ret.add(t.clone());
+    TaskFilter taskFilter = createTaskFilter(filter);
+    ArrayList<MonitoredTask> results =
+        Lists.newArrayListWithCapacity(tasks.size() + rpcTasks.size());
+    processTasks(tasks, taskFilter, results);
+    processTasks(rpcTasks, taskFilter, results);
+    return results;
+  }
+
+  /**
+   * Create a task filter according to a given filter type.
+   * @param filter type of monitored task
+   * @return a task filter
+   */
+  private static TaskFilter createTaskFilter(String filter) {
+    switch (TaskFilter.TaskType.getTaskType(filter)) {
+      case GENERAL: return task -> task instanceof MonitoredRPCHandler;
+      case HANDLER: return task -> !(task instanceof MonitoredRPCHandler);
+      case RPC: return task -> !(task instanceof MonitoredRPCHandler) ||
+                               !((MonitoredRPCHandler) task).isRPCRunning();
+      case OPERATION: return task -> !(task instanceof MonitoredRPCHandler) ||
+                                     !((MonitoredRPCHandler) task).isOperationRunning();
+      default: return task -> false;
     }
-    for (Iterator<TaskAndWeakRefPair> it = rpcTasks.iterator();
-         it.hasNext();) {
-      TaskAndWeakRefPair pair = it.next();
-      MonitoredTask t = pair.get();
-      ret.add(t.clone());
+  }
+
+  private static void processTasks(Iterable<TaskAndWeakRefPair> tasks,
+                                   TaskFilter filter,
+                                   List<MonitoredTask> results) {
+    for (TaskAndWeakRefPair task : tasks) {
+      MonitoredTask t = task.get();
+      if (!filter.filter(t)) {
+        results.add(t.clone());
+      }
     }
-    return ret;
   }
 
   private boolean canPurge(MonitoredTask stat) {
@@ -280,4 +310,45 @@ public class TaskMonitor {
       }
     }
   }
+
+  private interface TaskFilter {
+    enum TaskType {
+      GENERAL("general"),
+      HANDLER("handler"),
+      RPC("rpc"),
+      OPERATION("operation"),
+      ALL("all");
+
+      private String type;
+
+      private TaskType(String type) {
+        this.type = type.toLowerCase();
+      }
+
+      static TaskType getTaskType(String type) {
+        if (type == null || type.isEmpty()) {
+          return ALL;
+        }
+        type = type.toLowerCase();
+        for (TaskType taskType : values()) {
+          if (taskType.toString().equals(type)) {
+            return taskType;
+          }
+        }
+        return ALL;
+      }
+
+      @Override
+      public String toString() {
+        return type;
+      }
+    }
+
+    /**
+     * Filter out unwanted task.
+     * @param task monitored task
+     * @return false if a task is accepted, true if it is filtered
+     */
+    boolean filter(MonitoredTask t);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/effd1093/hbase-server/src/test/java/org/apache/hadoop/hbase/monitoring/TestTaskMonitor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/monitoring/TestTaskMonitor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/monitoring/TestTaskMonitor.java
index 718339a..7abcde8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/monitoring/TestTaskMonitor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/monitoring/TestTaskMonitor.java
@@ -20,9 +20,15 @@ package org.apache.hadoop.hbase.monitoring;
 
 import static org.junit.Assert.*;
 
+import java.util.ArrayList;
+import java.util.List;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Query;
+import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@@ -143,5 +149,47 @@ public class TestTaskMonitor {
     tm.shutdown();
   }
 
+  @Test
+  public void testGetTasksWithFilter() throws Exception {
+    TaskMonitor tm = new TaskMonitor(new Configuration());
+    assertTrue("Task monitor should start empty", tm.getTasks().isEmpty());
+    // Create 5 general tasks
+    tm.createStatus("General task1");
+    tm.createStatus("General task2");
+    tm.createStatus("General task3");
+    tm.createStatus("General task4");
+    tm.createStatus("General task5");
+    // Create 5 rpc tasks, and mark 1 completed
+    int length = 5;
+    ArrayList<MonitoredRPCHandler> rpcHandlers = new ArrayList<>(length);
+    for (int i = 0; i < length; i++) {
+      MonitoredRPCHandler rpcHandler = tm.createRPCStatus("Rpc task" + i);
+      rpcHandlers.add(rpcHandler);
+    }
+    // Create rpc opertions
+    byte[] row = new byte[] { 0x01 };
+    Mutation m = new Put(row);
+    Query q = new Scan();
+    String notOperation = "for test";
+    rpcHandlers.get(0).setRPC("operations", new Object[]{ m, q }, 3000);
+    rpcHandlers.get(1).setRPC("operations", new Object[]{ m, q }, 3000);
+    rpcHandlers.get(2).setRPC("operations", new Object[]{ m, q }, 3000);
+    rpcHandlers.get(3).setRPC("operations", new Object[]{ notOperation }, 3000);
+    rpcHandlers.get(4).setRPC("operations", new Object[]{ m, q }, 3000);
+    MonitoredRPCHandler completed = rpcHandlers.get(4);
+    completed.markComplete("Completed!");
+    // Test get tasks with filter
+    List<MonitoredTask> generalTasks = tm.getTasks("general");
+    assertEquals(5, generalTasks.size());
+    List<MonitoredTask> handlerTasks = tm.getTasks("handler");
+    assertEquals(5, handlerTasks.size());
+    List<MonitoredTask> rpcTasks = tm.getTasks("rpc");
+    // The last rpc handler is stopped
+    assertEquals(4, rpcTasks.size());
+    List<MonitoredTask> operationTasks = tm.getTasks("operation");
+    // Handler 3 doesn't handle Operation.
+    assertEquals(3, operationTasks.size());
+    tm.shutdown();
+  }
 }
 


[05/50] [abbrv] hbase git commit: HBASE-18387: [Thrift] Make principal configurable in DemoClient.java

Posted by st...@apache.org.
HBASE-18387: [Thrift] Make principal configurable in DemoClient.java

Added optional (fourth) parameter "server-principal"
The solution is backward compatible, in case not given, uses "hbase" as default value
If the third parameter is skipped the fourth cannot be set.

Signed-off-by: Josh Elser <el...@apache.org>


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

Branch: refs/heads/HBASE-14070.HLC
Commit: 8da77b414657f0ee3b093913de5f92eba17ecd2a
Parents: 4dd24c5
Author: Tamas Penzes <ta...@cloudera.com>
Authored: Tue Aug 8 13:45:09 2017 +0200
Committer: Josh Elser <el...@apache.org>
Committed: Thu Aug 10 23:47:07 2017 -0400

----------------------------------------------------------------------
 hbase-examples/README.txt                            |  3 ++-
 .../org/apache/hadoop/hbase/thrift/DemoClient.java   | 15 ++++++++++++---
 2 files changed, 14 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/8da77b41/hbase-examples/README.txt
----------------------------------------------------------------------
diff --git a/hbase-examples/README.txt b/hbase-examples/README.txt
index 22d1103..bf28180 100644
--- a/hbase-examples/README.txt
+++ b/hbase-examples/README.txt
@@ -28,7 +28,8 @@ Example code.
       2. If HBase server is not secure, or authentication is not enabled for the Thrift server, execute:
       {java -cp hbase-examples-[VERSION].jar:${HBASE_EXAMPLE_CLASSPATH} org.apache.hadoop.hbase.thrift.DemoClient <host> <port>}
       3. If HBase server is secure, and authentication is enabled for the Thrift server, run kinit at first, then execute:
-      {java -cp hbase-examples-[VERSION].jar:${HBASE_EXAMPLE_CLASSPATH} org.apache.hadoop.hbase.thrift.DemoClient <host> <port> true}
+      {java -cp hbase-examples-[VERSION].jar:${HBASE_EXAMPLE_CLASSPATH} org.apache.hadoop.hbase.thrift.DemoClient <host> <port> true <server-principal>}
+      <server-principal> should only be specified when the client connects to a secure cluster. It's default value is "hbase".
       4. Here is a lazy example that just pulls in all hbase dependency jars and that goes against default location on localhost.
       It should work with a standalone hbase instance started by doing ./bin/start-hbase.sh:
       {java -cp ./hbase-examples/target/hbase-examples-2.0.0-SNAPSHOT.jar:`./bin/hbase classpath` org.apache.hadoop.hbase.thrift.DemoClient localhost 9090}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8da77b41/hbase-examples/src/main/java/org/apache/hadoop/hbase/thrift/DemoClient.java
----------------------------------------------------------------------
diff --git a/hbase-examples/src/main/java/org/apache/hadoop/hbase/thrift/DemoClient.java b/hbase-examples/src/main/java/org/apache/hadoop/hbase/thrift/DemoClient.java
index cb0cfbb..706f82f 100644
--- a/hbase-examples/src/main/java/org/apache/hadoop/hbase/thrift/DemoClient.java
+++ b/hbase-examples/src/main/java/org/apache/hadoop/hbase/thrift/DemoClient.java
@@ -60,13 +60,14 @@ public class DemoClient {
     CharsetDecoder decoder = null;
 
     private static boolean secure = false;
+    private static String serverPrincipal = "hbase";
 
     public static void main(String[] args) throws Exception {
 
-        if (args.length < 2 || args.length > 3) {
+        if (args.length < 2 || args.length > 4 || (args.length > 2 && !isBoolean(args[2]))) {
 
             System.out.println("Invalid arguments!");
-            System.out.println("Usage: DemoClient host port [secure=false]");
+            System.out.println("Usage: DemoClient host port [secure=false [server-principal=hbase] ]");
 
             System.exit(-1);
         }
@@ -77,6 +78,10 @@ public class DemoClient {
           secure = Boolean.parseBoolean(args[2]);
         }
 
+        if (args.length == 4) {
+          serverPrincipal = args[3];
+        }
+
         final DemoClient client = new DemoClient();
         Subject.doAs(getSubject(),
           new PrivilegedExceptionAction<Void>() {
@@ -88,6 +93,10 @@ public class DemoClient {
           });
     }
 
+    private static boolean isBoolean(String s){
+      return Boolean.TRUE.toString().equalsIgnoreCase(s) || Boolean.FALSE.toString().equalsIgnoreCase(s);
+    }
+
     DemoClient() {
         decoder = Charset.forName("UTF-8").newDecoder();
     }
@@ -123,7 +132,7 @@ public class DemoClient {
            * The HBase cluster must be secure, allow proxy user.
            */
           transport = new TSaslClientTransport("GSSAPI", null,
-            "hbase", // Thrift server user name, should be an authorized proxy user.
+            serverPrincipal, // Thrift server user name, should be an authorized proxy user.
             host, // Thrift server domain
             saslProperties, null, transport);
         }


[30/50] [abbrv] hbase git commit: HBASE-2631 Decide between InMB and MB as suffix for field names in ClusterStatus objects

Posted by st...@apache.org.
HBASE-2631 Decide between InMB and MB as suffix for field names in ClusterStatus objects

Signed-off-by: Chia-Ping Tsai <ch...@gmail.com>


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

Branch: refs/heads/HBASE-14070.HLC
Commit: d37266f63cf90068415a8cef05b1c63dccc0a9d9
Parents: d4317c8
Author: Deon Huang <yj...@gmail.com>
Authored: Sun Aug 13 21:52:03 2017 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Tue Aug 15 21:42:31 2017 +0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/ServerLoad.java     | 27 ++++++++++++++++++++
 .../tmpl/master/RegionServerListTmpl.jamon      |  4 +--
 .../org/apache/hadoop/hbase/TestServerLoad.java |  6 ++---
 3 files changed, 32 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d37266f6/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java
index 8547dfb..8d4c7d3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java
@@ -131,18 +131,45 @@ public class ServerLoad {
     return storeUncompressedSizeMB;
   }
 
+  /**
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   * Use {@link #getStorefileSizeMB()} instead.
+   */
+  @Deprecated
   public int getStorefileSizeInMB() {
     return storefileSizeMB;
   }
 
+  public int getStorefileSizeMB() {
+    return storefileSizeMB;
+  }
+
+  /**
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   * Use {@link #getMemstoreSizeMB()} instead.
+   */
+  @Deprecated
   public int getMemstoreSizeInMB() {
     return memstoreSizeMB;
   }
 
+  public int getMemstoreSizeMB() {
+    return memstoreSizeMB;
+  }
+
+  /**
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   * Use {@link #getStorefileIndexSizeMB()} instead.
+   */
+  @Deprecated
   public int getStorefileIndexSizeInMB() {
     return storefileIndexSizeMB;
   }
 
+  public int getStorefileIndexSizeMB() {
+    return storefileIndexSizeMB;
+  }
+
   public long getReadRequestsCount() {
     return readRequestsCount;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/d37266f6/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/RegionServerListTmpl.jamon
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/RegionServerListTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/RegionServerListTmpl.jamon
index a62d5eb..5dd10e8 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/RegionServerListTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/RegionServerListTmpl.jamon
@@ -153,7 +153,7 @@ for (ServerName serverName: serverNames) {
       * TraditionalBinaryPrefix.MEGA.value, "B", 1) %></td>
     <td><% TraditionalBinaryPrefix.long2String(sl.getMaxHeapMB()
       * TraditionalBinaryPrefix.MEGA.value, "B", 1) %></td>
-    <td><% TraditionalBinaryPrefix.long2String(sl.getMemstoreSizeInMB()
+    <td><% TraditionalBinaryPrefix.long2String(sl.getMemstoreSizeMB()
       * TraditionalBinaryPrefix.MEGA.value, "B", 1) %></td>
 
 </tr>
@@ -232,7 +232,7 @@ if (sl != null) {
 <td><% sl.getStorefiles() %></td>
 <td><% TraditionalBinaryPrefix.long2String(
   sl.getStoreUncompressedSizeMB() * TraditionalBinaryPrefix.MEGA.value, "B", 1) %></td>
-<td><% TraditionalBinaryPrefix.long2String(sl.getStorefileSizeInMB()
+<td><% TraditionalBinaryPrefix.long2String(sl.getStorefileSizeMB()
   * TraditionalBinaryPrefix.MEGA.value, "B", 1) %></td>
 <td><% TraditionalBinaryPrefix.long2String(sl.getTotalStaticIndexSizeKB()
   * TraditionalBinaryPrefix.KILO.value, "B", 1) %></td>

http://git-wip-us.apache.org/repos/asf/hbase/blob/d37266f6/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerLoad.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerLoad.java
index cbd76ce..2d248b0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerLoad.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerLoad.java
@@ -41,8 +41,8 @@ public class TestServerLoad {
     assertEquals(114, sl.getStorefiles());
     assertEquals(129, sl.getStoreUncompressedSizeMB());
     assertEquals(504, sl.getRootIndexSizeKB());
-    assertEquals(820, sl.getStorefileSizeInMB());
-    assertEquals(82, sl.getStorefileIndexSizeInMB());
+    assertEquals(820, sl.getStorefileSizeMB());
+    assertEquals(82, sl.getStorefileIndexSizeMB());
     assertEquals(((long)Integer.MAX_VALUE)*2, sl.getReadRequestsCount());
     assertEquals(300, sl.getFilteredReadRequestsCount());
     
@@ -68,7 +68,7 @@ public class TestServerLoad {
 	  assertEquals(totalCount, sl.getReadRequestsCount());
 	  assertEquals(totalCount, sl.getWriteRequestsCount());
   }
-  
+
   private ClusterStatusProtos.ServerLoad createServerLoadProto() {
     HBaseProtos.RegionSpecifier rSpecOne =
         HBaseProtos.RegionSpecifier.newBuilder()


[15/50] [abbrv] hbase git commit: HBASE-15511 ClusterStatus should be able to return responses by scope

Posted by st...@apache.org.
HBASE-15511 ClusterStatus should be able to return responses by scope

Signed-off-by: Chia-Ping Tsai <ch...@gmail.com>


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

Branch: refs/heads/HBASE-14070.HLC
Commit: 923195c39e872ebca92b5cc5f148e4c1d39718ea
Parents: 173dce7
Author: Reid Chan <re...@outlook.com>
Authored: Mon Aug 14 01:01:30 2017 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Mon Aug 14 01:02:39 2017 +0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/ClusterStatus.java  | 297 ++++++++++++++++++-
 .../org/apache/hadoop/hbase/client/Admin.java   |   8 +
 .../apache/hadoop/hbase/client/AsyncAdmin.java  |   6 +
 .../hadoop/hbase/client/AsyncHBaseAdmin.java    |   8 +-
 .../apache/hadoop/hbase/client/HBaseAdmin.java  |  12 +-
 .../hadoop/hbase/client/RawAsyncHBaseAdmin.java |   9 +-
 .../hbase/shaded/protobuf/ProtobufUtil.java     |  81 ++++-
 .../hbase/shaded/protobuf/RequestConverter.java |  14 +-
 .../src/main/protobuf/ClusterStatus.proto       |  12 +
 .../src/main/protobuf/Master.proto              |   1 +
 .../src/main/protobuf/ClusterStatus.proto       |  12 +
 .../hbase/master/ClusterStatusPublisher.java    |  18 +-
 .../org/apache/hadoop/hbase/master/HMaster.java |  62 ++--
 .../hadoop/hbase/master/MasterRpcServices.java  |   3 +-
 .../hbase/client/TestClientClusterStatus.java   | 221 ++++++++++++++
 15 files changed, 710 insertions(+), 54 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/923195c3/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java
index 95d77a2..0dc4984 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java
@@ -47,6 +47,28 @@ import org.apache.hadoop.io.VersionedWritable;
  * <li>Regions in transition at master</li>
  * <li>The unique cluster ID</li>
  * </ul>
+ * <tt>{@link Options}</tt> provides a way to filter out infos which unwanted.
+ * The following codes will retrieve all the cluster information.
+ * <pre>
+ * {@code
+ * // Original version still works
+ * Admin admin = connection.getAdmin();
+ * ClusterStatus status = admin.getClusterStatus();
+ * // or below, a new version which has the same effects
+ * ClusterStatus status = admin.getClusterStatus(Options.defaultOptions());
+ * }
+ * </pre>
+ * If information about dead servers and master coprocessors are unwanted,
+ * then codes in the following way:
+ * <pre>
+ * {@code
+ * Admin admin = connection.getAdmin();
+ * ClusterStatus status = admin.getClusterStatus(
+ *                                Options.defaultOptions()
+ *                                       .excludeDeadServers()
+ *                                       .excludeMasterCoprocessors());
+ * }
+ * </pre>
  */
 @InterfaceAudience.Public
 public class ClusterStatus extends VersionedWritable {
@@ -72,6 +94,12 @@ public class ClusterStatus extends VersionedWritable {
   private String[] masterCoprocessors;
   private Boolean balancerOn;
 
+  /**
+   * Use {@link ClusterStatus.Builder} to construct a ClusterStatus instead.
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             (<a href="https://issues.apache.org/jira/browse/HBASE-15511">HBASE-15511</a>).
+   */
+  @Deprecated
   public ClusterStatus(final String hbaseVersion, final String clusterid,
       final Map<ServerName, ServerLoad> servers,
       final Collection<ServerName> deadServers,
@@ -80,8 +108,8 @@ public class ClusterStatus extends VersionedWritable {
       final List<RegionState> rit,
       final String[] masterCoprocessors,
       final Boolean balancerOn) {
+    // TODO: make this constructor private
     this.hbaseVersion = hbaseVersion;
-
     this.liveServers = servers;
     this.deadServers = deadServers;
     this.master = master;
@@ -133,7 +161,8 @@ public class ClusterStatus extends VersionedWritable {
    */
   public double getAverageLoad() {
     int load = getRegionsCount();
-    return (double)load / (double)getServersSize();
+    int serverSize = getServersSize();
+    return serverSize != 0 ? (double)load / (double)serverSize : 0.0;
   }
 
   /**
@@ -333,4 +362,268 @@ public class ClusterStatus extends VersionedWritable {
     }
     return sb.toString();
   }
+
+  public static Builder newBuilder() {
+    return new Builder();
+  }
+
+  /**
+   * Builder for construct a ClusterStatus.
+   */
+  public static class Builder {
+    private String hbaseVersion = null;
+    private Map<ServerName, ServerLoad> liveServers = null;
+    private Collection<ServerName> deadServers = null;
+    private ServerName master = null;
+    private Collection<ServerName> backupMasters = null;
+    private List<RegionState> intransition = null;
+    private String clusterId = null;
+    private String[] masterCoprocessors = null;
+    private Boolean balancerOn = null;
+
+    private Builder() {}
+
+    public Builder setHBaseVersion(String hbaseVersion) {
+      this.hbaseVersion = hbaseVersion;
+      return this;
+    }
+
+    public Builder setLiveServers(Map<ServerName, ServerLoad> liveServers) {
+      this.liveServers = liveServers;
+      return this;
+    }
+
+    public Builder setDeadServers(Collection<ServerName> deadServers) {
+      this.deadServers = deadServers;
+      return this;
+    }
+
+    public Builder setMaster(ServerName master) {
+      this.master = master;
+      return this;
+    }
+
+    public Builder setBackupMasters(Collection<ServerName> backupMasters) {
+      this.backupMasters = backupMasters;
+      return this;
+    }
+
+    public Builder setRegionState(List<RegionState> intransition) {
+      this.intransition = intransition;
+      return this;
+    }
+
+    public Builder setClusterId(String clusterId) {
+      this.clusterId = clusterId;
+      return this;
+    }
+
+    public Builder setMasterCoprocessors(String[] masterCoprocessors) {
+      this.masterCoprocessors = masterCoprocessors;
+      return this;
+    }
+
+    public Builder setBalancerOn(Boolean balancerOn) {
+      this.balancerOn = balancerOn;
+      return this;
+    }
+
+    public ClusterStatus build() {
+      return new ClusterStatus(hbaseVersion, clusterId, liveServers,
+          deadServers, master, backupMasters, intransition, masterCoprocessors,
+          balancerOn);
+    }
+  }
+
+  /**
+   * Options provides a way to filter out unwanted information.
+   * For compatibility, default options includes all the information about a ClusterStatus.
+   * To filter out unwanted information, use the specific excludeXXX() method.
+   */
+  public static class Options {
+    private boolean includeHBaseVersion = true;
+    private boolean includeLiveServers = true;
+    private boolean includeDeadServers = true;
+    private boolean includeMaster = true;
+    private boolean includeBackupMasters = true;
+    private boolean includeRegionState = true;
+    private boolean includeClusterId = true;
+    private boolean includeMasterCoprocessors = true;
+    private boolean includeBalancerOn = true;
+
+    private Options() {}
+
+    /**
+     * Include all information about a ClusterStatus.
+     */
+    public static Options getDefaultOptions() {
+      return new Options();
+    }
+
+    /**
+     * Filter out hbase verision.
+     */
+    public Options excludeHBaseVersion() {
+      includeHBaseVersion = false;
+      return this;
+    }
+
+    /**
+     * Filter out live servers.
+     */
+    public Options excludeLiveServers() {
+      includeLiveServers = false;
+      return this;
+    }
+
+    /**
+     * Filter out dead servers info.
+     */
+    public Options excludeDeadServers() {
+      includeDeadServers = false;
+      return this;
+    }
+
+    /**
+     * Filter out master info.
+     */
+    public Options excludeMaster() {
+      includeMaster = false;
+      return this;
+    }
+
+    /**
+     * Filter out backup masters info.
+     */
+    public Options excludeBackupMasters() {
+      includeBackupMasters = false;
+      return this;
+    }
+
+    /**
+     * Filter out region state.
+     */
+    public Options excludeRegionState() {
+      includeRegionState = false;
+      return this;
+    }
+
+    /**
+     * Filter out cluster id.
+     */
+    public Options excludeClusterId() {
+      includeClusterId = false;
+      return this;
+    }
+
+    /**
+     * Filter out master's coprocessors info.
+     */
+    public Options excludeMasterCoprocessors() {
+      includeMasterCoprocessors = false;
+      return this;
+    }
+
+    /**
+     * Filter out balancer on info.
+     */
+    public Options excludeBalancerOn() {
+      includeBalancerOn = false;
+      return this;
+    }
+
+    /**
+     * Include hbase version info.
+     */
+    public boolean includeHBaseVersion() {
+      return includeHBaseVersion;
+    }
+
+    /**
+     * Include live servers info.
+     */
+    public boolean includeLiveServers() {
+      return includeLiveServers;
+    }
+
+    /**
+     * Include dead servers info.
+     */
+    public boolean includeDeadServers() {
+      return includeDeadServers;
+    }
+
+    /**
+     * Include master info.
+     */
+    public boolean includeMaster() {
+      return includeMaster;
+    }
+
+    /**
+     * Include backup masters info.
+     */
+    public boolean includeBackupMasters() {
+      return includeBackupMasters;
+    }
+
+    /**
+     * Include region states info.
+     */
+    public boolean includeRegionState() {
+      return includeRegionState;
+    }
+
+    /**
+     * Include cluster id info.
+     */
+    public boolean includeClusterId() {
+      return includeClusterId;
+    }
+
+    /**
+     * Include master's coprocessors.
+     */
+    public boolean includeMasterCoprocessors() {
+      return includeMasterCoprocessors;
+    }
+
+    /**
+     * Include balancer on info.
+     */
+    public boolean includeBalancerOn() {
+      return includeBalancerOn;
+    }
+
+    /**
+     * For an options reusable convenience, reset options to default.
+     */
+    public Options reset() {
+      includeHBaseVersion = true;
+      includeLiveServers = true;
+      includeDeadServers = true;
+      includeMaster = true;
+      includeBackupMasters = true;
+      includeRegionState = true;
+      includeClusterId = true;
+      includeMasterCoprocessors = true;
+      includeBalancerOn = true;
+      return this;
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder builder = new StringBuilder("ClusterStatus info: [");
+      builder.append("include hbase version: " + includeHBaseVersion + ", ");
+      builder.append("include cluster id: " + includeClusterId + ", ");
+      builder.append("include master info: " + includeMaster + ", ");
+      builder.append("include backup masters info: " + includeBackupMasters + ", ");
+      builder.append("include live servers info: " + includeLiveServers + ", ");
+      builder.append("include dead servers info: " + includeDeadServers + ", ");
+      builder.append("include masters coprocessors: " + includeMasterCoprocessors + ", ");
+      builder.append("include region state: " + includeRegionState + ", ");
+      builder.append("include balancer on: " + includeBalancerOn + "]");
+      return builder.toString();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/923195c3/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
index 4f5c128..d2acae3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
@@ -31,6 +31,7 @@ import java.util.regex.Pattern;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.ClusterStatus.Options;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
@@ -1305,6 +1306,13 @@ public interface Admin extends Abortable, Closeable {
   ClusterStatus getClusterStatus() throws IOException;
 
   /**
+   * Get cluster status with options to filter out unwanted status.
+   * @return cluster status
+   * @throws IOException if a remote or network exception occurs
+   */
+  ClusterStatus getClusterStatus(Options options) throws IOException;
+
+  /**
    * Get {@link RegionLoad} of all regions hosted on a regionserver.
    *
    * @param sn region server from which regionload is required.

http://git-wip-us.apache.org/repos/asf/hbase/blob/923195c3/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
index 7e4412d..f2f2bf1 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
@@ -27,6 +27,7 @@ import java.util.function.Function;
 import java.util.regex.Pattern;
 
 import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.ClusterStatus.Options;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.RegionLoad;
@@ -833,6 +834,11 @@ public interface AsyncAdmin {
   CompletableFuture<ClusterStatus> getClusterStatus();
 
   /**
+   * @return cluster status wrapped by {@link CompletableFuture}
+   */
+  CompletableFuture<ClusterStatus> getClusterStatus(Options options);
+
+  /**
    * @return current master server name wrapped by {@link CompletableFuture}
    */
   default CompletableFuture<ServerName> getMaster() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/923195c3/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
index 9ba3b73..fbd8690 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
@@ -36,6 +36,7 @@ import io.netty.util.TimerTask;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.ClusterStatus.Options;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.RegionLoad;
@@ -493,7 +494,12 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
 
   @Override
   public CompletableFuture<ClusterStatus> getClusterStatus() {
-    return wrap(rawAdmin.getClusterStatus());
+    return getClusterStatus(Options.getDefaultOptions());
+  }
+
+  @Override
+  public CompletableFuture<ClusterStatus> getClusterStatus(Options options) {
+    return wrap(rawAdmin.getClusterStatus(options));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/923195c3/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index 66dbac5..11f3273 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -45,6 +45,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.ClusterStatus.Options;
 import org.apache.hadoop.hbase.CompoundConfiguration;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseConfiguration;
@@ -2074,13 +2075,18 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public ClusterStatus getClusterStatus() throws IOException {
+    return getClusterStatus(Options.getDefaultOptions());
+  }
+
+  @Override
+  public ClusterStatus getClusterStatus(Options options) throws IOException {
     return executeCallable(new MasterCallable<ClusterStatus>(getConnection(),
         this.rpcControllerFactory) {
       @Override
       protected ClusterStatus rpcCall() throws Exception {
-        GetClusterStatusRequest req = RequestConverter.buildGetClusterStatusRequest();
-        return ProtobufUtil.convert(master.getClusterStatus(getRpcController(), req).
-            getClusterStatus());
+        GetClusterStatusRequest req = RequestConverter.buildGetClusterStatusRequest(options);
+        return ProtobufUtil.convert(
+          master.getClusterStatus(getRpcController(), req).getClusterStatus());
       }
     });
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/923195c3/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
index 285286a..03f64fc 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
@@ -55,7 +55,6 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.MetaTableAccessor.QueryType;
-import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.RegionLoad;
 import org.apache.hadoop.hbase.RegionLocations;
@@ -69,6 +68,7 @@ import org.apache.hadoop.hbase.TableNotDisabledException;
 import org.apache.hadoop.hbase.TableNotEnabledException;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.UnknownRegionException;
+import org.apache.hadoop.hbase.ClusterStatus.Options;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.AsyncRpcRetryingCallerFactory.AdminRequestCallerBuilder;
 import org.apache.hadoop.hbase.client.AsyncRpcRetryingCallerFactory.MasterRequestCallerBuilder;
@@ -2421,12 +2421,17 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
 
   @Override
   public CompletableFuture<ClusterStatus> getClusterStatus() {
+    return getClusterStatus(Options.getDefaultOptions());
+  }
+
+  @Override
+  public CompletableFuture<ClusterStatus> getClusterStatus(Options options) {
     return this
         .<ClusterStatus> newMasterCaller()
         .action(
           (controller, stub) -> this
               .<GetClusterStatusRequest, GetClusterStatusResponse, ClusterStatus> call(controller,
-                stub, RequestConverter.buildGetClusterStatusRequest(),
+                stub, RequestConverter.buildGetClusterStatusRequest(options),
                 (s, c, req, done) -> s.getClusterStatus(c, req, done),
                 resp -> ProtobufUtil.convert(resp.getClusterStatus()))).call();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/923195c3/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
index 65e95b6..a527883 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.shaded.protobuf;
 
+import java.awt.image.BandCombineOp;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
@@ -3069,6 +3070,7 @@ public final class ProtobufUtil {
    * @return the converted ClusterStatus
    */
   public static ClusterStatus convert(ClusterStatusProtos.ClusterStatus proto) {
+    ClusterStatus.Builder builder = ClusterStatus.newBuilder();
 
     Map<ServerName, ServerLoad> servers = null;
     servers = new HashMap<>(proto.getLiveServersList().size());
@@ -3103,10 +3105,74 @@ public final class ProtobufUtil {
       masterCoprocessors[i] = proto.getMasterCoprocessors(i).getName();
     }
 
-    return new ClusterStatus(proto.getHbaseVersion().getVersion(),
-      ClusterId.convert(proto.getClusterId()).toString(),servers,deadServers,
-      ProtobufUtil.toServerName(proto.getMaster()),backupMasters,rit,masterCoprocessors,
-      proto.getBalancerOn());
+    String clusterId = null;
+    if (proto.hasClusterId()) {
+      clusterId = ClusterId.convert(proto.getClusterId()).toString();
+    }
+
+    String hbaseVersion = null;
+    if (proto.hasHbaseVersion()) {
+      hbaseVersion = proto.getHbaseVersion().getVersion();
+    }
+
+    ServerName master = null;
+    if (proto.hasMaster()) {
+      master = ProtobufUtil.toServerName(proto.getMaster());
+    }
+
+    Boolean balancerOn = null;
+    if (proto.hasBalancerOn()) {
+      balancerOn = proto.getBalancerOn();
+    }
+    builder.setHBaseVersion(hbaseVersion)
+           .setClusterId(clusterId)
+           .setLiveServers(servers)
+           .setDeadServers(deadServers)
+           .setMaster(master)
+           .setBackupMasters(backupMasters)
+           .setRegionState(rit)
+           .setMasterCoprocessors(masterCoprocessors)
+           .setBalancerOn(balancerOn);
+    return builder.build();
+  }
+
+  /**
+   * Convert proto ClusterStatus.Options to ClusterStatusProtos.Options
+   * @param opt
+   * @return proto ClusterStatus.Options
+   */
+  public static ClusterStatus.Options toOptions (ClusterStatusProtos.Options opt) {
+    ClusterStatus.Options option = ClusterStatus.Options.getDefaultOptions();
+    if (!opt.getIncludeHbaseVersion()) option.excludeHBaseVersion();
+    if (!opt.getIncludeLiveServers()) option.excludeLiveServers();
+    if (!opt.getIncludeDeadServers()) option.excludeDeadServers();
+    if (!opt.getIncludeRegionsState()) option.excludeRegionState();
+    if (!opt.getIncludeClusterId()) option.excludeClusterId();
+    if (!opt.getIncludeMasterCoprocessors()) option.excludeMasterCoprocessors();
+    if (!opt.getIncludeMaster()) option.excludeMaster();
+    if (!opt.getIncludeBackupMasters()) option.excludeBackupMasters();
+    if (!opt.getIncludeBalancerOn()) option.excludeBalancerOn();
+    return option;
+  }
+
+  /**
+   * Convert ClusterStatus.Options to proto ClusterStatusProtos.Options
+   * @param opt
+   * @return ClusterStatus.Options
+   */
+  public static ClusterStatusProtos.Options toOptions(ClusterStatus.Options opt) {
+    ClusterStatusProtos.Options.Builder option =
+        ClusterStatusProtos.Options.newBuilder();
+    option.setIncludeHbaseVersion(opt.includeHBaseVersion())
+          .setIncludeLiveServers(opt.includeLiveServers())
+          .setIncludeDeadServers(opt.includeDeadServers())
+          .setIncludeRegionsState(opt.includeRegionState())
+          .setIncludeClusterId(opt.includeClusterId())
+          .setIncludeMasterCoprocessors(opt.includeMasterCoprocessors())
+          .setIncludeMaster(opt.includeMaster())
+          .setIncludeBackupMasters(opt.includeBackupMasters())
+          .setIncludeBalancerOn(opt.includeBalancerOn());
+    return option.build();
   }
 
   /**
@@ -3117,8 +3183,11 @@ public final class ProtobufUtil {
   public static ClusterStatusProtos.ClusterStatus convert(ClusterStatus status) {
     ClusterStatusProtos.ClusterStatus.Builder builder =
         ClusterStatusProtos.ClusterStatus.newBuilder();
-    builder
-        .setHbaseVersion(HBaseVersionFileContent.newBuilder().setVersion(status.getHBaseVersion()));
+    if (status.getHBaseVersion() != null) {
+       builder.setHbaseVersion(
+         HBaseVersionFileContent.newBuilder()
+                                .setVersion(status.getHBaseVersion()));
+    }
 
     if (status.getServers() != null) {
       for (ServerName serverName : status.getServers()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/923195c3/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
index e84a85f..be46c19 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
@@ -26,10 +26,10 @@ import java.util.Set;
 import java.util.regex.Pattern;
 
 import org.apache.hadoop.hbase.CellScannable;
+import org.apache.hadoop.hbase.ClusterStatus.Options;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
@@ -1508,18 +1508,14 @@ public final class RequestConverter {
   }
 
   /**
-   * @see {@link #buildGetClusterStatusRequest}
-   */
-  private static final GetClusterStatusRequest GET_CLUSTER_STATUS_REQUEST =
-      GetClusterStatusRequest.newBuilder().build();
-
-  /**
    * Creates a protocol buffer GetClusterStatusRequest
    *
    * @return A GetClusterStatusRequest
    */
-  public static GetClusterStatusRequest buildGetClusterStatusRequest() {
-    return GET_CLUSTER_STATUS_REQUEST;
+  public static GetClusterStatusRequest buildGetClusterStatusRequest(Options opt) {
+    return GetClusterStatusRequest.newBuilder()
+                                  .setClusterOptions(ProtobufUtil.toOptions(opt))
+                                  .build();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/923195c3/hbase-protocol-shaded/src/main/protobuf/ClusterStatus.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/ClusterStatus.proto b/hbase-protocol-shaded/src/main/protobuf/ClusterStatus.proto
index d1525f7..d647bb1 100644
--- a/hbase-protocol-shaded/src/main/protobuf/ClusterStatus.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/ClusterStatus.proto
@@ -225,3 +225,15 @@ message ClusterStatus {
   repeated ServerName backup_masters = 8;
   optional bool balancer_on = 9;
 }
+
+message Options {
+  optional bool include_hbase_version = 1 [default = true];
+  optional bool include_live_servers = 2 [default = true];
+  optional bool include_dead_servers = 3 [default = true];
+  optional bool include_regions_state = 4 [default = true];
+  optional bool include_cluster_id = 5 [default = true];
+  optional bool include_master_coprocessors = 6 [default = true];
+  optional bool include_master = 7 [default = true];
+  optional bool include_backup_masters = 8 [default = true];
+  optional bool include_balancer_on = 9 [default = true];
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/923195c3/hbase-protocol-shaded/src/main/protobuf/Master.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Master.proto b/hbase-protocol-shaded/src/main/protobuf/Master.proto
index 8d7cad9..33f9bf3 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Master.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Master.proto
@@ -485,6 +485,7 @@ message GetTableStateResponse {
 }
 
 message GetClusterStatusRequest {
+  required Options cluster_options = 1;
 }
 
 message GetClusterStatusResponse {

http://git-wip-us.apache.org/repos/asf/hbase/blob/923195c3/hbase-protocol/src/main/protobuf/ClusterStatus.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/ClusterStatus.proto b/hbase-protocol/src/main/protobuf/ClusterStatus.proto
index 54bc0c3..4ae3230 100644
--- a/hbase-protocol/src/main/protobuf/ClusterStatus.proto
+++ b/hbase-protocol/src/main/protobuf/ClusterStatus.proto
@@ -225,3 +225,15 @@ message ClusterStatus {
   repeated ServerName backup_masters = 8;
   optional bool balancer_on = 9;
 }
+
+message Options {
+  optional bool include_hbase_version = 1 [default = true];
+  optional bool include_live_servers = 2 [default = true];
+  optional bool include_dead_servers = 3 [default = true];
+  optional bool include_regions_state = 4 [default = true];
+  optional bool include_cluster_id = 5 [default = true];
+  optional bool include_master_coprocessors = 6 [default = true];
+  optional bool include_master = 7 [default = true];
+  optional bool include_backup_masters = 8 [default = true];
+  optional bool include_balancer_on = 9 [default = true];
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/923195c3/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterStatusPublisher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterStatusPublisher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterStatusPublisher.java
index ea5516d..2c903fb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterStatusPublisher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterStatusPublisher.java
@@ -161,18 +161,12 @@ public class ClusterStatusPublisher extends ScheduledChore {
     // We're reusing an existing protobuf message, but we don't send everything.
     // This could be extended in the future, for example if we want to send stuff like the
     //  hbase:meta server name.
-    ClusterStatus cs = new ClusterStatus(VersionInfo.getVersion(),
-        master.getMasterFileSystem().getClusterId().toString(),
-        null,
-        sns,
-        master.getServerName(),
-        null,
-        null,
-        null,
-        null);
-
-
-    publisher.publish(cs);
+    ClusterStatus.Builder csBuilder = ClusterStatus.newBuilder();
+    csBuilder.setHBaseVersion(VersionInfo.getVersion())
+             .setClusterId(master.getMasterFileSystem().getClusterId().toString())
+             .setMaster(master.getServerName())
+             .setDeadServers(sns);
+    publisher.publish(csBuilder.build());
   }
 
   protected void cleanup() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/923195c3/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 421ae8b..faf0893 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
@@ -54,6 +54,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.ClusterStatus.Options;
 import org.apache.hadoop.hbase.CoordinatedStateException;
 import org.apache.hadoop.hbase.CoordinatedStateManager;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
@@ -2431,6 +2432,48 @@ public class HMaster extends HRegionServer implements MasterServices {
    * @return cluster status
    */
   public ClusterStatus getClusterStatus() throws InterruptedIOException {
+    return getClusterStatus(Options.getDefaultOptions());
+  }
+
+  /**
+   * @return cluster status
+   */
+  public ClusterStatus getClusterStatus(Options options) throws InterruptedIOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Retrieving cluster status info. " + options);
+    }
+    ClusterStatus.Builder builder = ClusterStatus.newBuilder();
+    if (options.includeHBaseVersion()) {
+      builder.setHBaseVersion(VersionInfo.getVersion());
+    }
+    if (options.includeClusterId()) {
+      builder.setClusterId(getClusterId());
+    }
+    if (options.includeLiveServers() && serverManager != null) {
+      builder.setLiveServers(serverManager.getOnlineServers());
+    }
+    if (options.includeDeadServers() && serverManager != null) {
+      builder.setDeadServers(serverManager.getDeadServers().copyServerNames());
+    }
+    if (options.includeMaster()) {
+      builder.setMaster(getServerName());
+    }
+    if (options.includeBackupMasters()) {
+      builder.setBackupMasters(getBackupMasters());
+    }
+    if (options.includeRegionState() && assignmentManager != null) {
+      builder.setRegionState(assignmentManager.getRegionStates().getRegionsStateInTransition());
+    }
+    if (options.includeMasterCoprocessors() && cpHost != null) {
+      builder.setMasterCoprocessors(getMasterCoprocessors());
+    }
+    if (options.includeBalancerOn() && loadBalancerTracker != null) {
+      builder.setBalancerOn(loadBalancerTracker.isBalancerOn());
+    }
+    return builder.build();
+  }
+
+  private List<ServerName> getBackupMasters() throws InterruptedIOException {
     // Build Set of backup masters from ZK nodes
     List<String> backupMasterStrings;
     try {
@@ -2474,24 +2517,7 @@ public class HMaster extends HRegionServer implements MasterServices {
           return s1.getServerName().compareTo(s2.getServerName());
         }});
     }
-
-    String clusterId = fileSystemManager != null ?
-      fileSystemManager.getClusterId().toString() : null;
-    List<RegionState> regionsInTransition = assignmentManager != null ?
-      assignmentManager.getRegionStates().getRegionsStateInTransition() : null;
-
-    String[] coprocessors = cpHost != null ? getMasterCoprocessors() : null;
-    boolean balancerOn = loadBalancerTracker != null ?
-      loadBalancerTracker.isBalancerOn() : false;
-    Map<ServerName, ServerLoad> onlineServers = null;
-    Set<ServerName> deadServers = null;
-    if (serverManager != null) {
-      deadServers = serverManager.getDeadServers().copyServerNames();
-      onlineServers = serverManager.getOnlineServers();
-    }
-    return new ClusterStatus(VersionInfo.getVersion(), clusterId,
-      onlineServers, deadServers, serverName, backupMasters,
-      regionsInTransition, coprocessors, balancerOn);
+    return backupMasters;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/923195c3/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index 995df9b..6e9b1e2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -762,7 +762,8 @@ public class MasterRpcServices extends RSRpcServices
     GetClusterStatusResponse.Builder response = GetClusterStatusResponse.newBuilder();
     try {
       master.checkInitialized();
-      response.setClusterStatus(ProtobufUtil.convert(master.getClusterStatus()));
+      response.setClusterStatus(ProtobufUtil.convert(
+        master.getClusterStatus(ProtobufUtil.toOptions(req.getClusterOptions()))));
     } catch (IOException e) {
       throw new ServiceException(e);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/923195c3/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientClusterStatus.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientClusterStatus.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientClusterStatus.java
new file mode 100644
index 0000000..2cf7bc5
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientClusterStatus.java
@@ -0,0 +1,221 @@
+/**
+ * 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 java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.ClusterStatus.Options;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
+import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test the ClusterStatus.
+ */
+@Category(SmallTests.class)
+public class TestClientClusterStatus {
+  private static HBaseTestingUtility UTIL;
+  private static Admin ADMIN;
+  private final static int SLAVES = 5;
+  private final static int MASTERS = 3;
+  private static MiniHBaseCluster CLUSTER;
+  private static HRegionServer DEAD;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    Configuration conf = HBaseConfiguration.create();
+    UTIL = new HBaseTestingUtility(conf);
+    UTIL.startMiniCluster(MASTERS, SLAVES);
+    CLUSTER = UTIL.getHBaseCluster();
+    CLUSTER.waitForActiveAndReadyMaster();
+    ADMIN = UTIL.getAdmin();
+    // Kill one region server
+    List<RegionServerThread> rsts = CLUSTER.getLiveRegionServerThreads();
+    RegionServerThread rst = rsts.get(rsts.size() - 1);
+    DEAD = rst.getRegionServer();
+    DEAD.stop("Test dead servers status");
+    while (!DEAD.isStopped()) {
+      Thread.sleep(500);
+    }
+  }
+
+  @Test
+  public void testDefaults() throws Exception {
+    ClusterStatus origin = ADMIN.getClusterStatus();
+    ClusterStatus defaults = ADMIN.getClusterStatus(Options.getDefaultOptions());
+    Assert.assertEquals(origin.getHBaseVersion(), defaults.getHBaseVersion());
+    Assert.assertEquals(origin.getClusterId(), defaults.getClusterId());
+    Assert.assertTrue(origin.getAverageLoad() == defaults.getAverageLoad());
+    Assert.assertTrue(origin.getBackupMastersSize() == defaults.getBackupMastersSize());
+    Assert.assertTrue(origin.getDeadServersSize() == defaults.getDeadServersSize());
+    Assert.assertTrue(origin.getRegionsCount() == defaults.getRegionsCount());
+    Assert.assertTrue(origin.getServersSize() == defaults.getServersSize());
+  }
+
+  @Test
+  public void testExclude() throws Exception {
+    ClusterStatus.Options options = Options.getDefaultOptions();
+    // Only retrieve master's coprocessors which are null in this test env.
+    options.excludeHBaseVersion()
+           .excludeBackupMasters()
+           .excludeBalancerOn()
+           .excludeClusterId()
+           .excludeLiveServers()
+           .excludeDeadServers()
+           .excludeMaster()
+           .excludeRegionState();
+    ClusterStatus status = ADMIN.getClusterStatus(options);
+    // Other cluster status info should be either null or empty.
+    Assert.assertTrue(status.getMasterCoprocessors().length == 0);
+    Assert.assertNull(status.getHBaseVersion());
+    Assert.assertTrue(status.getBackupMasters().isEmpty());
+    Assert.assertNull(status.getBalancerOn());
+    Assert.assertNull(status.getClusterId());
+    Assert.assertTrue(status.getServers().isEmpty());
+    Assert.assertTrue(status.getDeadServerNames().isEmpty());
+    Assert.assertNull(status.getMaster());
+    Assert.assertTrue(status.getBackupMasters().isEmpty());
+  }
+
+  @Test
+  public void testAsyncClient() throws Exception {
+    AsyncRegistry registry = AsyncRegistryFactory.getRegistry(UTIL.getConfiguration());
+    AsyncConnectionImpl asyncConnect = new AsyncConnectionImpl(UTIL.getConfiguration(), registry,
+      registry.getClusterId().get(), User.getCurrent());
+    AsyncAdmin asyncAdmin = asyncConnect.getAdmin();
+    CompletableFuture<ClusterStatus> originFuture =
+        asyncAdmin.getClusterStatus();
+    CompletableFuture<ClusterStatus> defaultsFuture =
+        asyncAdmin.getClusterStatus(Options.getDefaultOptions());
+    ClusterStatus origin = originFuture.get();
+    ClusterStatus defaults = defaultsFuture.get();
+    Assert.assertEquals(origin.getHBaseVersion(), defaults.getHBaseVersion());
+    Assert.assertEquals(origin.getClusterId(), defaults.getClusterId());
+    Assert.assertTrue(origin.getAverageLoad() == defaults.getAverageLoad());
+    Assert.assertTrue(origin.getBackupMastersSize() == defaults.getBackupMastersSize());
+    Assert.assertTrue(origin.getDeadServersSize() == defaults.getDeadServersSize());
+    Assert.assertTrue(origin.getRegionsCount() == defaults.getRegionsCount());
+    Assert.assertTrue(origin.getServersSize() == defaults.getServersSize());
+    if (asyncConnect != null) {
+      asyncConnect.close();
+    }
+  }
+
+  @Test
+  public void testLiveAndDeadServersStatus() throws Exception {
+    List<RegionServerThread> regionserverThreads = CLUSTER.getLiveRegionServerThreads();
+    int numRs = 0;
+    int len = regionserverThreads.size();
+    for (int i = 0; i < len; i++) {
+      if (regionserverThreads.get(i).isAlive()) {
+        numRs++;
+      }
+    }
+    // Retrieve live servers and dead servers info.
+    ClusterStatus.Options options = Options.getDefaultOptions();
+    options.excludeHBaseVersion()
+           .excludeBackupMasters()
+           .excludeBalancerOn()
+           .excludeClusterId()
+           .excludeMaster()
+           .excludeMasterCoprocessors()
+           .excludeRegionState();
+    ClusterStatus status = ADMIN.getClusterStatus(options);
+    Assert.assertNotNull(status);
+    Assert.assertNotNull(status.getServers());
+    // exclude a dead region server
+    Assert.assertEquals(SLAVES - 1, numRs);
+    // live servers = primary master + nums of regionservers
+    Assert.assertEquals(status.getServers().size() - 1, numRs);
+    Assert.assertTrue(status.getRegionsCount() > 0);
+    Assert.assertNotNull(status.getDeadServerNames());
+    Assert.assertEquals(1, status.getDeadServersSize());
+    ServerName deadServerName = status.getDeadServerNames().iterator().next();
+    Assert.assertEquals(DEAD.getServerName(), deadServerName);
+  }
+
+  @Test
+  public void testMasterAndBackupMastersStatus() throws Exception {
+    // get all the master threads
+    List<MasterThread> masterThreads = CLUSTER.getMasterThreads();
+    int numActive = 0;
+    int activeIndex = 0;
+    ServerName activeName = null;
+    HMaster active = null;
+    for (int i = 0; i < masterThreads.size(); i++) {
+      if (masterThreads.get(i).getMaster().isActiveMaster()) {
+        numActive++;
+        activeIndex = i;
+        active = masterThreads.get(activeIndex).getMaster();
+        activeName = active.getServerName();
+      }
+    }
+    Assert.assertNotNull(active);
+    Assert.assertEquals(1, numActive);
+    Assert.assertEquals(MASTERS, masterThreads.size());
+    // Retrieve master and backup masters infos only.
+    ClusterStatus.Options options = Options.getDefaultOptions();
+    options.excludeHBaseVersion()
+           .excludeBalancerOn()
+           .excludeClusterId()
+           .excludeLiveServers()
+           .excludeDeadServers()
+           .excludeMasterCoprocessors()
+           .excludeRegionState();
+    ClusterStatus status = ADMIN.getClusterStatus(options);
+    Assert.assertTrue(status.getMaster().equals(activeName));
+    Assert.assertEquals(MASTERS - 1, status.getBackupMastersSize());
+  }
+
+  @Test
+  public void testOtherStatusInfos() throws Exception {
+    ClusterStatus.Options options = Options.getDefaultOptions();
+    options.excludeMaster()
+           .excludeBackupMasters()
+           .excludeRegionState()
+           .excludeLiveServers()
+           .excludeBackupMasters();
+    ClusterStatus status = ADMIN.getClusterStatus(options);
+    Assert.assertTrue(status.getMasterCoprocessors().length == 0);
+    Assert.assertNotNull(status.getHBaseVersion());
+    Assert.assertNotNull(status.getClusterId());
+    Assert.assertTrue(status.getAverageLoad() == 0.0);
+    Assert.assertNotNull(status.getBalancerOn() && !status.getBalancerOn());
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    if (ADMIN != null) ADMIN.close();
+    UTIL.shutdownMiniCluster();
+  }
+}


[19/50] [abbrv] hbase git commit: HBASE-14135 Merge backup images (Vladimir Rodionov)

Posted by st...@apache.org.
HBASE-14135 Merge backup images (Vladimir Rodionov)


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

Branch: refs/heads/HBASE-14070.HLC
Commit: 05e6e5695089640006d06c2f74126b50a73363b7
Parents: c6ac04a
Author: Josh Elser <el...@apache.org>
Authored: Sun Aug 13 20:55:58 2017 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Sun Aug 13 20:55:58 2017 -0400

----------------------------------------------------------------------
 .../apache/hadoop/hbase/backup/BackupAdmin.java |  20 +-
 .../hadoop/hbase/backup/BackupDriver.java       |   2 +
 .../apache/hadoop/hbase/backup/BackupInfo.java  |   5 +
 .../hadoop/hbase/backup/BackupMergeJob.java     |  40 +++
 .../hbase/backup/BackupRestoreFactory.java      |  20 +-
 .../hadoop/hbase/backup/HBackupFileSystem.java  |  57 ++--
 .../hbase/backup/impl/BackupAdminImpl.java      | 213 +++++++++---
 .../hbase/backup/impl/BackupCommands.java       | 163 ++++++---
 .../hadoop/hbase/backup/impl/BackupManager.java |  21 +-
 .../hbase/backup/impl/BackupManifest.java       |  24 +-
 .../hbase/backup/impl/BackupSystemTable.java    | 314 ++++++++++-------
 .../hbase/backup/impl/RestoreTablesClient.java  |  32 +-
 .../backup/mapreduce/HFileSplitterJob.java      | 181 ----------
 .../mapreduce/MapReduceBackupMergeJob.java      | 321 ++++++++++++++++++
 .../mapreduce/MapReduceHFileSplitterJob.java    | 181 ++++++++++
 .../backup/mapreduce/MapReduceRestoreJob.java   |  84 ++---
 .../hadoop/hbase/backup/util/BackupUtils.java   |  93 +++--
 .../TestIncrementalBackupMergeWithFailures.java | 336 +++++++++++++++++++
 .../backup/TestRepairAfterFailedDelete.java     |   2 +-
 19 files changed, 1574 insertions(+), 535 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/05e6e569/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupAdmin.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupAdmin.java
index 6f642a4..9dc6382 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupAdmin.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupAdmin.java
@@ -38,8 +38,8 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 public interface BackupAdmin extends Closeable {
 
   /**
-   * Backup given list of tables fully. This is a synchronous operation.
-   * It returns backup id on success or throw exception on failure.
+   * Backup given list of tables fully. This is a synchronous operation. It returns backup id on
+   * success or throw exception on failure.
    * @param userRequest BackupRequest instance
    * @return the backup Id
    */
@@ -61,16 +61,24 @@ public interface BackupAdmin extends Closeable {
    */
   BackupInfo getBackupInfo(String backupId) throws IOException;
 
-
   /**
    * Delete backup image command
-   * @param backupIds backup id list
+   * @param backupIds array of backup ids
    * @return total number of deleted sessions
    * @throws IOException exception
    */
   int deleteBackups(String[] backupIds) throws IOException;
 
   /**
+   * Merge backup images command
+   * @param backupIds array of backup ids of images to be merged
+   *        The resulting backup image will have the same backup id as the most
+   *        recent image from a list of images to be merged
+   * @throws IOException exception
+   */
+  void mergeBackups(String[] backupIds) throws IOException;
+
+  /**
    * Show backup history command
    * @param n last n backup sessions
    * @return list of backup info objects
@@ -113,7 +121,7 @@ public interface BackupAdmin extends Closeable {
   /**
    * Add tables to backup set command
    * @param name name of backup set.
-   * @param tables list of tables to be added to this set.
+   * @param tables array of tables to be added to this set.
    * @throws IOException exception
    */
   void addToBackupSet(String name, TableName[] tables) throws IOException;
@@ -121,7 +129,7 @@ public interface BackupAdmin extends Closeable {
   /**
    * Remove tables from backup set
    * @param name name of backup set.
-   * @param tables list of tables to be removed from this set.
+   * @param tables array of tables to be removed from this set.
    * @throws IOException exception
    */
   void removeFromBackupSet(String name, TableName[] tables) throws IOException;

http://git-wip-us.apache.org/repos/asf/hbase/blob/05e6e569/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java
index e2cdb2f..9dd8531 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java
@@ -111,6 +111,8 @@ public class BackupDriver extends AbstractHBaseTool {
       type = BackupCommand.SET;
     } else if (BackupCommand.REPAIR.name().equalsIgnoreCase(cmd)) {
       type = BackupCommand.REPAIR;
+    } else if (BackupCommand.MERGE.name().equalsIgnoreCase(cmd)) {
+      type = BackupCommand.MERGE;
     } else {
       System.out.println("Unsupported command for backup: " + cmd);
       printToolUsage();

http://git-wip-us.apache.org/repos/asf/hbase/blob/05e6e569/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java
index f6a1fe4..1765bf3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java
@@ -433,6 +433,11 @@ public class BackupInfo implements Comparable<BackupInfo> {
     }
   }
 
+  @Override
+  public String toString() {
+    return backupId;
+  }
+
   public byte[] toByteArray() throws IOException {
     return toProtosBackupInfo().toByteArray();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/05e6e569/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupMergeJob.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupMergeJob.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupMergeJob.java
new file mode 100644
index 0000000..136782f
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupMergeJob.java
@@ -0,0 +1,40 @@
+/**
+ * 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.backup;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * Backup merge operation job interface. Concrete implementation is provided by backup provider, see
+ * {@link BackupRestoreFactory}
+ */
+
+@InterfaceAudience.Private
+public interface BackupMergeJob extends Configurable {
+
+  /**
+   * Run backup merge operation
+   * @param backupIds backup image ids
+   * @throws IOException
+   */
+  void run(String[] backupIds) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/05e6e569/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.java
index 6d8967a..d72c884 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.backup;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.backup.mapreduce.MapReduceBackupCopyJob;
+import org.apache.hadoop.hbase.backup.mapreduce.MapReduceBackupMergeJob;
 import org.apache.hadoop.hbase.backup.mapreduce.MapReduceRestoreJob;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.util.ReflectionUtils;
@@ -32,6 +33,7 @@ public final class BackupRestoreFactory {
 
   public final static String HBASE_INCR_RESTORE_IMPL_CLASS = "hbase.incremental.restore.class";
   public final static String HBASE_BACKUP_COPY_IMPL_CLASS = "hbase.backup.copy.class";
+  public final static String HBASE_BACKUP_MERGE_IMPL_CLASS = "hbase.backup.merge.class";
 
   private BackupRestoreFactory() {
     throw new AssertionError("Instantiating utility class...");
@@ -40,7 +42,7 @@ public final class BackupRestoreFactory {
   /**
    * Gets backup restore job
    * @param conf configuration
-   * @return backup restore task instance
+   * @return backup restore job instance
    */
   public static RestoreJob getRestoreJob(Configuration conf) {
     Class<? extends RestoreJob> cls =
@@ -53,7 +55,7 @@ public final class BackupRestoreFactory {
   /**
    * Gets backup copy job
    * @param conf configuration
-   * @return backup copy task
+   * @return backup copy job instance
    */
   public static BackupCopyJob getBackupCopyJob(Configuration conf) {
     Class<? extends BackupCopyJob> cls =
@@ -63,4 +65,18 @@ public final class BackupRestoreFactory {
     service.setConf(conf);
     return service;
   }
+
+  /**
+   * Gets backup merge job
+   * @param conf configuration
+   * @return backup merge job instance
+   */
+  public static BackupMergeJob getBackupMergeJob(Configuration conf) {
+    Class<? extends BackupMergeJob> cls =
+        conf.getClass(HBASE_BACKUP_MERGE_IMPL_CLASS, MapReduceBackupMergeJob.class,
+          BackupMergeJob.class);
+    BackupMergeJob service = ReflectionUtils.newInstance(cls, conf);
+    service.setConf(conf);
+    return service;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/05e6e569/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java
index 46044db..1c43e88 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java
@@ -49,8 +49,8 @@ public class HBackupFileSystem {
   /**
    * Given the backup root dir, backup id and the table name, return the backup image location,
    * which is also where the backup manifest file is. return value look like:
-   * "hdfs://backup.hbase.org:9000/user/biadmin/backup/backup_1396650096738/default/t1_dn/",
-   * where "hdfs://backup.hbase.org:9000/user/biadmin/backup" is a backup root directory
+   * "hdfs://backup.hbase.org:9000/user/biadmin/backup/backup_1396650096738/default/t1_dn/", where
+   * "hdfs://backup.hbase.org:9000/user/biadmin/backup" is a backup root directory
    * @param backupRootDir backup root directory
    * @param backupId backup id
    * @param tableName table name
@@ -63,18 +63,26 @@ public class HBackupFileSystem {
         + Path.SEPARATOR;
   }
 
+  public static String getTableBackupDataDir(String backupRootDir, String backupId,
+      TableName tableName) {
+    return getTableBackupDir(backupRootDir, backupId, tableName) + Path.SEPARATOR + "data";
+  }
+
+  public static Path getBackupPath(String backupRootDir, String backupId) {
+    return new Path(backupRootDir + Path.SEPARATOR + backupId);
+  }
+
   /**
    * Given the backup root dir, backup id and the table name, return the backup image location,
    * which is also where the backup manifest file is. return value look like:
-   * "hdfs://backup.hbase.org:9000/user/biadmin/backup/backup_1396650096738/default/t1_dn/",
-   * where "hdfs://backup.hbase.org:9000/user/biadmin/backup" is a backup root directory
+   * "hdfs://backup.hbase.org:9000/user/biadmin/backup/backup_1396650096738/default/t1_dn/", where
+   * "hdfs://backup.hbase.org:9000/user/biadmin/backup" is a backup root directory
    * @param backupRootPath backup root path
    * @param tableName table name
    * @param backupId backup Id
    * @return backupPath for the particular table
    */
-  public static Path getTableBackupPath(TableName tableName,
-      Path backupRootPath, String backupId) {
+  public static Path getTableBackupPath(TableName tableName, Path backupRootPath, String backupId) {
     return new Path(getTableBackupDir(backupRootPath.toString(), backupId, tableName));
   }
 
@@ -94,33 +102,30 @@ public class HBackupFileSystem {
     return new Path(getLogBackupDir(backupRootDir, backupId));
   }
 
-  private static Path getManifestPath(TableName tableName, Configuration conf, Path backupRootPath,
-      String backupId) throws IOException {
-    Path manifestPath =
-        new Path(getTableBackupPath(tableName, backupRootPath, backupId),
-            BackupManifest.MANIFEST_FILE_NAME);
+  // TODO we do not keep WAL files anymore
+  // Move manifest file to other place
+  private static Path getManifestPath(Configuration conf, Path backupRootPath, String backupId)
+      throws IOException {
+    Path manifestPath = null;
 
     FileSystem fs = backupRootPath.getFileSystem(conf);
+    manifestPath =
+        new Path(getBackupPath(backupRootPath.toString(), backupId) + Path.SEPARATOR
+            + BackupManifest.MANIFEST_FILE_NAME);
     if (!fs.exists(manifestPath)) {
-      // check log dir for incremental backup case
-      manifestPath =
-          new Path(getLogBackupDir(backupRootPath.toString(), backupId) + Path.SEPARATOR
-              + BackupManifest.MANIFEST_FILE_NAME);
-      if (!fs.exists(manifestPath)) {
-        String errorMsg =
-            "Could not find backup manifest " + BackupManifest.MANIFEST_FILE_NAME + " for "
-                + backupId + ". File " + manifestPath + " does not exists. Did " + backupId
-                + " correspond to previously taken backup ?";
-        throw new IOException(errorMsg);
-      }
+      String errorMsg =
+          "Could not find backup manifest " + BackupManifest.MANIFEST_FILE_NAME + " for "
+              + backupId + ". File " + manifestPath + " does not exists. Did " + backupId
+              + " correspond to previously taken backup ?";
+      throw new IOException(errorMsg);
     }
     return manifestPath;
   }
 
-  public static BackupManifest getManifest(TableName tableName, Configuration conf,
-      Path backupRootPath, String backupId) throws IOException {
+  public static BackupManifest
+      getManifest(Configuration conf, Path backupRootPath, String backupId) throws IOException {
     BackupManifest manifest =
-        new BackupManifest(conf, getManifestPath(tableName, conf, backupRootPath, backupId));
+        new BackupManifest(conf, getManifestPath(conf, backupRootPath, backupId));
     return manifest;
   }
 
@@ -134,7 +139,7 @@ public class HBackupFileSystem {
       TableName[] tableArray, Configuration conf, Path backupRootPath, String backupId)
       throws IOException {
     for (TableName tableName : tableArray) {
-      BackupManifest manifest = getManifest(tableName, conf, backupRootPath, backupId);
+      BackupManifest manifest = getManifest(conf, backupRootPath, backupId);
       backupManifestMap.put(tableName, manifest);
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/05e6e569/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java
index 6e35d92..99fb06c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.backup.impl;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -36,8 +37,10 @@ import org.apache.hadoop.hbase.backup.BackupAdmin;
 import org.apache.hadoop.hbase.backup.BackupClientFactory;
 import org.apache.hadoop.hbase.backup.BackupInfo;
 import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
+import org.apache.hadoop.hbase.backup.BackupMergeJob;
 import org.apache.hadoop.hbase.backup.BackupRequest;
 import org.apache.hadoop.hbase.backup.BackupRestoreConstants;
+import org.apache.hadoop.hbase.backup.BackupRestoreFactory;
 import org.apache.hadoop.hbase.backup.BackupType;
 import org.apache.hadoop.hbase.backup.HBackupFileSystem;
 import org.apache.hadoop.hbase.backup.RestoreRequest;
@@ -46,9 +49,8 @@ import org.apache.hadoop.hbase.backup.util.BackupUtils;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 
 @InterfaceAudience.Private
 public class BackupAdminImpl implements BackupAdmin {
@@ -65,12 +67,8 @@ public class BackupAdminImpl implements BackupAdmin {
 
   @Override
   public void close() throws IOException {
-    if (conn != null) {
-      conn.close();
-    }
   }
 
-
   @Override
   public BackupInfo getBackupInfo(String backupId) throws IOException {
     BackupInfo backupInfo = null;
@@ -105,12 +103,12 @@ public class BackupAdminImpl implements BackupAdmin {
       // is running by using startBackupSession API
       // If there is an active session in progress, exception will be thrown
       try {
-        sysTable.startBackupSession();
+        sysTable.startBackupExclusiveOperation();
         deleteSessionStarted = true;
       } catch (IOException e) {
         LOG.warn("You can not run delete command while active backup session is in progress. \n"
             + "If there is no active backup session running, run backup repair utility to restore \n"
-            +"backup system integrity.");
+            + "backup system integrity.");
         return -1;
       }
 
@@ -126,7 +124,7 @@ public class BackupAdminImpl implements BackupAdmin {
       sysTable.startDeleteOperation(backupIds);
       // Step 4: Snapshot backup system table
       if (!BackupSystemTable.snapshotExists(conn)) {
-          BackupSystemTable.snapshot(conn);
+        BackupSystemTable.snapshot(conn);
       } else {
         LOG.warn("Backup system table snapshot exists");
       }
@@ -154,13 +152,13 @@ public class BackupAdminImpl implements BackupAdmin {
         // Fail delete operation
         // Step 1
         if (snapshotDone) {
-          if(BackupSystemTable.snapshotExists(conn)) {
+          if (BackupSystemTable.snapshotExists(conn)) {
             BackupSystemTable.restoreFromSnapshot(conn);
             // delete snapshot
             BackupSystemTable.deleteSnapshot(conn);
             // We still have record with unfinished delete operation
-            LOG.error("Delete operation failed, please run backup repair utility to restore "+
-                       "backup system integrity", e);
+            LOG.error("Delete operation failed, please run backup repair utility to restore "
+                + "backup system integrity", e);
             throw e;
           } else {
             LOG.warn("Delete operation succeeded, there were some errors: ", e);
@@ -169,7 +167,7 @@ public class BackupAdminImpl implements BackupAdmin {
 
       } finally {
         if (deleteSessionStarted) {
-          sysTable.finishBackupSession();
+          sysTable.finishBackupExclusiveOperation();
         }
       }
     }
@@ -206,17 +204,17 @@ public class BackupAdminImpl implements BackupAdmin {
   /**
    * Delete single backup and all related backups <br>
    * Algorithm:<br>
-   *  Backup type: FULL or INCREMENTAL <br>
-   *  Is this last backup session for table T: YES or NO <br>
-   *  For every table T from table list 'tables':<br>
-   *  if(FULL, YES) deletes only physical data (PD) <br>
-   *  if(FULL, NO), deletes PD, scans all newer backups and removes T from backupInfo,<br>
-   *  until we either reach the most recent backup for T in the system or FULL backup<br>
-   *  which includes T<br>
-   *  if(INCREMENTAL, YES) deletes only physical data (PD)
-   *  if(INCREMENTAL, NO) deletes physical data and for table T scans all backup images between last<br>
-   *  FULL backup, which is older than the backup being deleted and the next FULL backup (if exists) <br>
-   *  or last one for a particular table T and removes T from list of backup tables.
+   * Backup type: FULL or INCREMENTAL <br>
+   * Is this last backup session for table T: YES or NO <br>
+   * For every table T from table list 'tables':<br>
+   * if(FULL, YES) deletes only physical data (PD) <br>
+   * if(FULL, NO), deletes PD, scans all newer backups and removes T from backupInfo,<br>
+   * until we either reach the most recent backup for T in the system or FULL backup<br>
+   * which includes T<br>
+   * if(INCREMENTAL, YES) deletes only physical data (PD) if(INCREMENTAL, NO) deletes physical data
+   * and for table T scans all backup images between last<br>
+   * FULL backup, which is older than the backup being deleted and the next FULL backup (if exists) <br>
+   * or last one for a particular table T and removes T from list of backup tables.
    * @param backupId backup id
    * @param sysTable backup system table
    * @return total number of deleted backup images
@@ -285,8 +283,9 @@ public class BackupAdminImpl implements BackupAdmin {
     return totalDeleted;
   }
 
-  private void removeTableFromBackupImage(BackupInfo info, TableName tn,
-      BackupSystemTable sysTable) throws IOException {
+  private void
+      removeTableFromBackupImage(BackupInfo info, TableName tn, BackupSystemTable sysTable)
+          throws IOException {
     List<TableName> tables = info.getTableNames();
     LOG.debug("Remove " + tn + " from " + info.getBackupId() + " tables="
         + info.getTableListAsString());
@@ -485,7 +484,7 @@ public class BackupAdminImpl implements BackupAdmin {
 
   private String[] toStringArray(TableName[] list) {
     String[] arr = new String[list.length];
-    for(int i=0; i < list.length; i++) {
+    for (int i = 0; i < list.length; i++) {
       arr[i] = list[i].toString();
     }
     return arr;
@@ -521,7 +520,7 @@ public class BackupAdminImpl implements BackupAdmin {
     String targetRootDir = request.getTargetRootDir();
     List<TableName> tableList = request.getTableList();
 
-    String backupId =BackupRestoreConstants.BACKUPID_PREFIX + EnvironmentEdgeManager.currentTime();
+    String backupId = BackupRestoreConstants.BACKUPID_PREFIX + EnvironmentEdgeManager.currentTime();
     if (type == BackupType.INCREMENTAL) {
       Set<TableName> incrTableSet = null;
       try (BackupSystemTable table = new BackupSystemTable(conn)) {
@@ -529,19 +528,20 @@ public class BackupAdminImpl implements BackupAdmin {
       }
 
       if (incrTableSet.isEmpty()) {
-        String msg = "Incremental backup table set contains no tables. "
-            + "You need to run full backup first " +
-            (tableList != null ? "on "+StringUtils.join(tableList, ","): "");
+        String msg =
+            "Incremental backup table set contains no tables. "
+                + "You need to run full backup first "
+                + (tableList != null ? "on " + StringUtils.join(tableList, ",") : "");
 
         throw new IOException(msg);
       }
-      if(tableList != null) {
+      if (tableList != null) {
         tableList.removeAll(incrTableSet);
         if (!tableList.isEmpty()) {
           String extraTables = StringUtils.join(tableList, ",");
-          String msg = "Some tables (" + extraTables + ") haven't gone through full backup. "+
-            "Perform full backup on " + extraTables + " first, "
-            + "then retry the command";
+          String msg =
+              "Some tables (" + extraTables + ") haven't gone through full backup. "
+                  + "Perform full backup on " + extraTables + " first, " + "then retry the command";
           throw new IOException(msg);
         }
       }
@@ -584,14 +584,13 @@ public class BackupAdminImpl implements BackupAdmin {
 
     // update table list
     BackupRequest.Builder builder = new BackupRequest.Builder();
-    request = builder.withBackupType(request.getBackupType()).
-                      withTableList(tableList).
-                      withTargetRootDir(request.getTargetRootDir()).
-                      withBackupSetName(request.getBackupSetName()).
-                      withTotalTasks(request.getTotalTasks()).
-                      withBandwidthPerTasks((int)request.getBandwidth()).build();
-
-    TableBackupClient client =null;
+    request =
+        builder.withBackupType(request.getBackupType()).withTableList(tableList)
+            .withTargetRootDir(request.getTargetRootDir())
+            .withBackupSetName(request.getBackupSetName()).withTotalTasks(request.getTotalTasks())
+            .withBandwidthPerTasks((int) request.getBandwidth()).build();
+
+    TableBackupClient client = null;
     try {
       client = BackupClientFactory.create(conn, backupId, request);
     } catch (IOException e) {
@@ -613,4 +612,132 @@ public class BackupAdminImpl implements BackupAdmin {
     return tableList;
   }
 
+  @Override
+  public void mergeBackups(String[] backupIds) throws IOException {
+    try (final BackupSystemTable sysTable = new BackupSystemTable(conn);) {
+      checkIfValidForMerge(backupIds, sysTable);
+      BackupMergeJob job = BackupRestoreFactory.getBackupMergeJob(conn.getConfiguration());
+      job.run(backupIds);
+    }
+  }
+
+  /**
+   * Verifies that backup images are valid for merge.
+   *
+   * <ul>
+   * <li>All backups MUST be in the same destination
+   * <li>No FULL backups are allowed - only INCREMENTAL
+   * <li>All backups must be in COMPLETE state
+   * <li>No holes in backup list are allowed
+   * </ul>
+   * <p>
+   * @param backupIds list of backup ids
+   * @param table backup system table
+   * @throws IOException
+   */
+  private void checkIfValidForMerge(String[] backupIds, BackupSystemTable table) throws IOException {
+    String backupRoot = null;
+
+    final Set<TableName> allTables = new HashSet<TableName>();
+    final Set<String> allBackups = new HashSet<String>();
+    long minTime = Long.MAX_VALUE, maxTime = Long.MIN_VALUE;
+    for (String backupId : backupIds) {
+      BackupInfo bInfo = table.readBackupInfo(backupId);
+      if (bInfo == null) {
+        String msg = "Backup session " + backupId + " not found";
+        throw new IOException(msg);
+      }
+      if (backupRoot == null) {
+        backupRoot = bInfo.getBackupRootDir();
+      } else if (!bInfo.getBackupRootDir().equals(backupRoot)) {
+        throw new IOException("Found different backup destinations in a list of a backup sessions \n"
+            + "1. " + backupRoot + "\n" + "2. " + bInfo.getBackupRootDir());
+      }
+      if (bInfo.getType() == BackupType.FULL) {
+        throw new IOException("FULL backup image can not be merged for: \n" + bInfo);
+      }
+
+      if (bInfo.getState() != BackupState.COMPLETE) {
+        throw new IOException("Backup image " + backupId
+            + " can not be merged becuase of its state: " + bInfo.getState());
+      }
+      allBackups.add(backupId);
+      allTables.addAll(bInfo.getTableNames());
+      long time = bInfo.getStartTs();
+      if (time < minTime) {
+        minTime = time;
+      }
+      if (time > maxTime) {
+        maxTime = time;
+      }
+    }
+
+
+    final long startRangeTime  = minTime;
+    final long endRangeTime = maxTime;
+    final String backupDest = backupRoot;
+    // Check we have no 'holes' in backup id list
+    // Filter 1 : backupRoot
+    // Filter 2 : time range filter
+    // Filter 3 : table filter
+
+    BackupInfo.Filter destinationFilter = new  BackupInfo.Filter() {
+
+      @Override
+      public boolean apply(BackupInfo info) {
+        return info.getBackupRootDir().equals(backupDest);
+      }
+    };
+
+    BackupInfo.Filter timeRangeFilter = new  BackupInfo.Filter() {
+
+      @Override
+      public boolean apply(BackupInfo info) {
+        long time = info.getStartTs();
+        return time >= startRangeTime && time <= endRangeTime ;
+      }
+    };
+
+    BackupInfo.Filter tableFilter = new  BackupInfo.Filter() {
+
+      @Override
+      public boolean apply(BackupInfo info) {
+        List<TableName> tables = info.getTableNames();
+        return !Collections.disjoint(allTables, tables);
+      }
+    };
+
+    BackupInfo.Filter typeFilter = new  BackupInfo.Filter() {
+
+      @Override
+      public boolean apply(BackupInfo info) {
+        return info.getType() == BackupType.INCREMENTAL;
+      }
+    };
+
+    BackupInfo.Filter stateFilter = new  BackupInfo.Filter() {
+      @Override
+      public boolean apply(BackupInfo info) {
+        return info.getState() == BackupState.COMPLETE;
+      }
+    };
+
+    List<BackupInfo> allInfos =
+        table.getBackupHistory( -1, destinationFilter,
+          timeRangeFilter, tableFilter, typeFilter, stateFilter);
+    if (allInfos.size() != allBackups.size()) {
+      // Yes we have at least one  hole in backup image sequence
+      List<String> missingIds = new ArrayList<String>();
+      for(BackupInfo info: allInfos) {
+        if(allBackups.contains(info.getBackupId())) {
+          continue;
+        }
+        missingIds.add(info.getBackupId());
+      }
+      String errMsg =
+          "Sequence of backup ids has 'holes'. The following backup images must be added:" +
+           org.apache.hadoop.util.StringUtils.join(",", missingIds);
+      throw new IOException(errMsg);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/05e6e569/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java
index aa15fba..650ba2e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java
@@ -59,16 +59,15 @@ import org.apache.hadoop.hbase.backup.util.BackupUtils;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 
 /**
  * General backup commands, options and usage messages
  */
 
 @InterfaceAudience.Private
-public final class BackupCommands  {
+public final class BackupCommands {
 
   public final static String INCORRECT_USAGE = "Incorrect usage";
 
@@ -79,7 +78,8 @@ public final class BackupCommands  {
       + "  history    show history of all successful backups\n"
       + "  progress   show the progress of the latest backup request\n"
       + "  set        backup set management\n"
-      + "  repair     repair backup system table"
+      + "  repair     repair backup system table\n"
+      + "  merge      merge backup images\n"
       + "Run \'hbase backup COMMAND -h\' to see help message for each command\n";
 
   public static final String CREATE_CMD_USAGE =
@@ -109,17 +109,20 @@ public final class BackupCommands  {
 
   public static final String SET_CMD_USAGE = "Usage: hbase backup set COMMAND [name] [tables]\n"
       + "  name            Backup set name\n"
-      + "  tables          Comma separated list of tables.\n"
-      + "COMMAND is one of:\n" + "  add             add tables to a set, create a set if needed\n"
+      + "  tables          Comma separated list of tables.\n" + "COMMAND is one of:\n"
+      + "  add             add tables to a set, create a set if needed\n"
       + "  remove          remove tables from a set\n"
       + "  list            list all backup sets in the system\n"
       + "  describe        describe set\n" + "  delete          delete backup set\n";
+  public static final String MERGE_CMD_USAGE = "Usage: hbase backup merge [backup_ids]\n"
+      + "  backup_ids      Comma separated list of backup image ids.\n";
 
   public static final String USAGE_FOOTER = "";
 
   public static abstract class Command extends Configured {
     CommandLine cmdline;
     Connection conn;
+
     Command(Configuration conf) {
       if (conf == null) {
         conf = HBaseConfiguration.create();
@@ -140,7 +143,7 @@ public final class BackupCommands  {
         try (BackupSystemTable table = new BackupSystemTable(conn);) {
           List<BackupInfo> sessions = table.getBackupInfos(BackupState.RUNNING);
 
-          if(sessions.size() > 0) {
+          if (sessions.size() > 0) {
             System.err.println("Found backup session in a RUNNING state: ");
             System.err.println(sessions.get(0));
             System.err.println("This may indicate that a previous session has failed abnormally.");
@@ -154,11 +157,19 @@ public final class BackupCommands  {
         try (BackupSystemTable table = new BackupSystemTable(conn);) {
           String[] ids = table.getListOfBackupIdsFromDeleteOperation();
 
-          if(ids !=null && ids.length > 0) {
-            System.err.println("Found failed backup delete coommand. ");
+          if (ids != null && ids.length > 0) {
+            System.err.println("Found failed backup DELETE coommand. ");
             System.err.println("Backup system recovery is required.");
-            throw new IOException("Failed backup delete found, aborted command execution");
+            throw new IOException("Failed backup DELETE found, aborted command execution");
           }
+
+          ids = table.getListOfBackupIdsFromMergeOperation();
+          if (ids != null && ids.length > 0) {
+            System.err.println("Found failed backup MERGE coommand. ");
+            System.err.println("Backup system recovery is required.");
+            throw new IOException("Failed backup MERGE found, aborted command execution");
+          }
+
         }
       }
     }
@@ -178,10 +189,10 @@ public final class BackupCommands  {
     protected boolean requiresNoActiveSession() {
       return false;
     }
+
     /**
-     * Command requires consistent state of a backup system
-     * Backup system may become inconsistent because of an abnormal
-     * termination of a backup session or delete command
+     * Command requires consistent state of a backup system Backup system may become inconsistent
+     * because of an abnormal termination of a backup session or delete command
      * @return true, if yes
      */
     protected boolean requiresConsistentState() {
@@ -220,6 +231,9 @@ public final class BackupCommands  {
     case REPAIR:
       cmd = new RepairCommand(conf, cmdline);
       break;
+    case MERGE:
+      cmd = new MergeCommand(conf, cmdline);
+      break;
     case HELP:
     default:
       cmd = new HelpCommand(conf, cmdline);
@@ -257,7 +271,7 @@ public final class BackupCommands  {
         throw new IOException(INCORRECT_USAGE);
       }
       String[] args = cmdline.getArgs();
-      if (args.length !=3) {
+      if (args.length != 3) {
         printUsage();
         throw new IOException(INCORRECT_USAGE);
       }
@@ -274,7 +288,6 @@ public final class BackupCommands  {
         throw new IOException(INCORRECT_USAGE);
       }
 
-
       String tables = null;
 
       // Check if we have both: backup set and list of tables
@@ -310,14 +323,14 @@ public final class BackupCommands  {
 
       try (BackupAdminImpl admin = new BackupAdminImpl(conn);) {
 
-       BackupRequest.Builder builder = new BackupRequest.Builder();
-       BackupRequest request = builder.withBackupType(BackupType.valueOf(args[1].toUpperCase()))
-            .withTableList(tables != null ?
-                          Lists.newArrayList(BackupUtils.parseTableNames(tables)) : null)
-            .withTargetRootDir(args[2])
-            .withTotalTasks(workers)
-            .withBandwidthPerTasks(bandwidth)
-            .withBackupSetName(setName).build();
+        BackupRequest.Builder builder = new BackupRequest.Builder();
+        BackupRequest request =
+            builder
+                .withBackupType(BackupType.valueOf(args[1].toUpperCase()))
+                .withTableList(
+                  tables != null ? Lists.newArrayList(BackupUtils.parseTableNames(tables)) : null)
+                .withTargetRootDir(args[2]).withTotalTasks(workers)
+                .withBandwidthPerTasks(bandwidth).withBackupSetName(setName).build();
         String backupId = admin.backupTables(request);
         System.out.println("Backup session " + backupId + " finished. Status: SUCCESS");
       } catch (IOException e) {
@@ -544,7 +557,8 @@ public final class BackupCommands  {
         int deleted = admin.deleteBackups(backupIds);
         System.out.println("Deleted " + deleted + " backups. Total requested: " + args.length);
       } catch (IOException e) {
-        System.err.println("Delete command FAILED. Please run backup repair tool to restore backup system integrity");
+        System.err
+            .println("Delete command FAILED. Please run backup repair tool to restore backup system integrity");
         throw e;
       }
 
@@ -584,8 +598,9 @@ public final class BackupCommands  {
         if (list.size() == 0) {
           // No failed sessions found
           System.out.println("REPAIR status: no failed sessions found."
-          +" Checking failed delete backup operation ...");
+              + " Checking failed delete backup operation ...");
           repairFailedBackupDeletionIfAny(conn, sysTable);
+          repairFailedBackupMergeIfAny(conn, sysTable);
           return;
         }
         backupInfo = list.get(0);
@@ -606,32 +621,55 @@ public final class BackupCommands  {
         // If backup session is updated to FAILED state - means we
         // processed recovery already.
         sysTable.updateBackupInfo(backupInfo);
-        sysTable.finishBackupSession();
-        System.out.println("REPAIR status: finished repair failed session:\n "+ backupInfo);
+        sysTable.finishBackupExclusiveOperation();
+        System.out.println("REPAIR status: finished repair failed session:\n " + backupInfo);
 
       }
     }
 
     private void repairFailedBackupDeletionIfAny(Connection conn, BackupSystemTable sysTable)
-        throws IOException
-    {
+        throws IOException {
       String[] backupIds = sysTable.getListOfBackupIdsFromDeleteOperation();
-      if (backupIds == null ||backupIds.length == 0) {
-        System.out.println("No failed backup delete operation found");
+      if (backupIds == null || backupIds.length == 0) {
+        System.out.println("No failed backup DELETE operation found");
         // Delete backup table snapshot if exists
         BackupSystemTable.deleteSnapshot(conn);
         return;
       }
-      System.out.println("Found failed delete operation for: " + StringUtils.join(backupIds));
-      System.out.println("Running delete again ...");
+      System.out.println("Found failed DELETE operation for: " + StringUtils.join(backupIds));
+      System.out.println("Running DELETE again ...");
       // Restore table from snapshot
       BackupSystemTable.restoreFromSnapshot(conn);
       // Finish previous failed session
-      sysTable.finishBackupSession();
-      try(BackupAdmin admin = new BackupAdminImpl(conn);) {
+      sysTable.finishBackupExclusiveOperation();
+      try (BackupAdmin admin = new BackupAdminImpl(conn);) {
         admin.deleteBackups(backupIds);
       }
-      System.out.println("Delete operation finished OK: "+ StringUtils.join(backupIds));
+      System.out.println("DELETE operation finished OK: " + StringUtils.join(backupIds));
+
+    }
+
+    private void repairFailedBackupMergeIfAny(Connection conn, BackupSystemTable sysTable)
+        throws IOException {
+      String[] backupIds = sysTable.getListOfBackupIdsFromMergeOperation();
+      if (backupIds == null || backupIds.length == 0) {
+        System.out.println("No failed backup MERGE operation found");
+        // Delete backup table snapshot if exists
+        BackupSystemTable.deleteSnapshot(conn);
+        return;
+      }
+      System.out.println("Found failed MERGE operation for: " + StringUtils.join(backupIds));
+      System.out.println("Running MERGE again ...");
+      // Restore table from snapshot
+      BackupSystemTable.restoreFromSnapshot(conn);
+      // Unlock backupo system
+      sysTable.finishBackupExclusiveOperation();
+      // Finish previous failed session
+      sysTable.finishMergeOperation();
+      try (BackupAdmin admin = new BackupAdminImpl(conn);) {
+        admin.mergeBackups(backupIds);
+      }
+      System.out.println("MERGE operation finished OK: " + StringUtils.join(backupIds));
 
     }
 
@@ -641,6 +679,56 @@ public final class BackupCommands  {
     }
   }
 
+  private static class MergeCommand extends Command {
+
+    MergeCommand(Configuration conf, CommandLine cmdline) {
+      super(conf);
+      this.cmdline = cmdline;
+    }
+
+    @Override
+    protected boolean requiresNoActiveSession() {
+      return true;
+    }
+
+    @Override
+    protected boolean requiresConsistentState() {
+      return true;
+    }
+
+    @Override
+    public void execute() throws IOException {
+      super.execute();
+
+      String[] args = cmdline == null ? null : cmdline.getArgs();
+      if (args == null || (args.length != 2)) {
+        System.err.println("ERROR: wrong number of arguments: "
+            + (args == null ? null : args.length));
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+
+      String[] backupIds = args[1].split(",");
+      if (backupIds.length < 2) {
+        String msg = "ERROR: can not merge a single backup image. "+
+            "Number of images must be greater than 1.";
+        System.err.println(msg);
+        throw new IOException(msg);
+
+      }
+      Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create();
+      try (final Connection conn = ConnectionFactory.createConnection(conf);
+          final BackupAdminImpl admin = new BackupAdminImpl(conn);) {
+        admin.mergeBackups(backupIds);
+      }
+    }
+
+    @Override
+    protected void printUsage() {
+      System.out.println(MERGE_CMD_USAGE);
+    }
+  }
+
   // TODO Cancel command
 
   private static class CancelCommand extends Command {
@@ -672,7 +760,6 @@ public final class BackupCommands  {
     @Override
     public void execute() throws IOException {
 
-
       int n = parseHistoryLength();
       final TableName tableName = getTableName();
       final String setName = getTableSetName();
@@ -883,7 +970,7 @@ public final class BackupCommands  {
 
     private TableName[] toTableNames(String[] tables) {
       TableName[] arr = new TableName[tables.length];
-      for (int i=0; i < tables.length; i++) {
+      for (int i = 0; i < tables.length; i++) {
         arr[i] = TableName.valueOf(tables[i]);
       }
       return arr;

http://git-wip-us.apache.org/repos/asf/hbase/blob/05e6e569/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java
index bf80506..8fe5eaf 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java
@@ -115,8 +115,8 @@ public class BackupManager implements Closeable {
     }
 
     if (LOG.isDebugEnabled()) {
-      LOG.debug("Added log cleaner: " + cleanerClass +"\n" +
-                "Added master procedure manager: " + masterProcedureClass);
+      LOG.debug("Added log cleaner: " + cleanerClass + "\n" + "Added master procedure manager: "
+          + masterProcedureClass);
     }
 
   }
@@ -185,9 +185,8 @@ public class BackupManager implements Closeable {
    * @return BackupInfo
    * @throws BackupException exception
    */
-  public BackupInfo createBackupInfo(String backupId, BackupType type,
-      List<TableName> tableList, String targetRootDir, int workers, long bandwidth)
-      throws BackupException {
+  public BackupInfo createBackupInfo(String backupId, BackupType type, List<TableName> tableList,
+      String targetRootDir, int workers, long bandwidth) throws BackupException {
     if (targetRootDir == null) {
       throw new BackupException("Wrong backup request parameter: target backup root directory");
     }
@@ -313,7 +312,7 @@ public class BackupManager implements Closeable {
           }
         } else {
           Path logBackupPath =
-              HBackupFileSystem.getLogBackupPath(backup.getBackupRootDir(), backup.getBackupId());
+              HBackupFileSystem.getBackupPath(backup.getBackupRootDir(), backup.getBackupId());
           LOG.debug("Current backup has an incremental backup ancestor, "
               + "touching its image manifest in " + logBackupPath.toString()
               + " to construct the dependency.");
@@ -371,7 +370,7 @@ public class BackupManager implements Closeable {
    * @throws IOException if active session already exists
    */
   public void startBackupSession() throws IOException {
-    systemTable.startBackupSession();
+    systemTable.startBackupExclusiveOperation();
   }
 
   /**
@@ -379,10 +378,9 @@ public class BackupManager implements Closeable {
    * @throws IOException if no active session
    */
   public void finishBackupSession() throws IOException {
-    systemTable.finishBackupSession();
+    systemTable.finishBackupExclusiveOperation();
   }
 
-
   /**
    * Read the last backup start code (timestamp) of last successful backup. Will return null if
    * there is no startcode stored in backup system table or the value is of length 0. These two
@@ -413,7 +411,7 @@ public class BackupManager implements Closeable {
   }
 
   public Pair<Map<TableName, Map<String, Map<String, List<Pair<String, Boolean>>>>>, List<byte[]>>
-  readBulkloadRows(List<TableName> tableList) throws IOException {
+      readBulkloadRows(List<TableName> tableList) throws IOException {
     return systemTable.readBulkloadRows(tableList);
   }
 
@@ -448,8 +446,7 @@ public class BackupManager implements Closeable {
    */
   public void writeRegionServerLogTimestamp(Set<TableName> tables,
       HashMap<String, Long> newTimestamps) throws IOException {
-    systemTable.writeRegionServerLogTimestamp(tables, newTimestamps,
-      backupInfo.getBackupRootDir());
+    systemTable.writeRegionServerLogTimestamp(tables, newTimestamps, backupInfo.getBackupRootDir());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/05e6e569/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java
index b8adac9..7e3201e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.backup.BackupInfo;
 import org.apache.hadoop.hbase.backup.BackupType;
+import org.apache.hadoop.hbase.backup.HBackupFileSystem;
 import org.apache.hadoop.hbase.backup.util.BackupUtils;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
@@ -61,9 +62,8 @@ public class BackupManifest {
   public static final String MANIFEST_FILE_NAME = ".backup.manifest";
 
   /**
-   *  Backup image, the dependency graph is made up by series of backup images
-   *  BackupImage contains all the relevant information to restore the backup and
-   *  is used during restore operation
+   * Backup image, the dependency graph is made up by series of backup images BackupImage contains
+   * all the relevant information to restore the backup and is used during restore operation
    */
 
   public static class BackupImage implements Comparable<BackupImage> {
@@ -294,6 +294,16 @@ public class BackupManifest {
       return this.ancestors;
     }
 
+    public void removeAncestors(List<String> backupIds) {
+      List<BackupImage> toRemove = new ArrayList<BackupImage>();
+      for (BackupImage im : this.ancestors) {
+        if (backupIds.contains(im.getBackupId())) {
+          toRemove.add(im);
+        }
+      }
+      this.ancestors.removeAll(toRemove);
+    }
+
     private void addAncestor(BackupImage backupImage) {
       this.getAncestors().add(backupImage);
     }
@@ -464,18 +474,16 @@ public class BackupManifest {
   }
 
   /**
-   * Persist the manifest file.
+   * TODO: fix it. Persist the manifest file.
    * @throws IOException IOException when storing the manifest file.
    */
 
   public void store(Configuration conf) throws BackupException {
     byte[] data = backupImage.toProto().toByteArray();
     // write the file, overwrite if already exist
-    String logBackupDir =
-        BackupUtils.getLogBackupDir(backupImage.getRootDir(), backupImage.getBackupId());
     Path manifestFilePath =
-        new Path(new Path((tableBackupDir != null ? tableBackupDir : logBackupDir)),
-            MANIFEST_FILE_NAME);
+        new Path(HBackupFileSystem.getBackupPath(backupImage.getRootDir(),
+          backupImage.getBackupId()), MANIFEST_FILE_NAME);
     try (FSDataOutputStream out =
         manifestFilePath.getFileSystem(conf).create(manifestFilePath, true);) {
       out.write(data);

http://git-wip-us.apache.org/repos/asf/hbase/blob/05e6e569/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java
index e5a3daa..4dab046 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java
@@ -82,6 +82,7 @@ import org.apache.hadoop.hbase.util.Pair;
  * value = backupId and full WAL file name</li>
  * </ul></p>
  */
+
 @InterfaceAudience.Private
 public final class BackupSystemTable implements Closeable {
   private static final Log LOG = LogFactory.getLog(BackupSystemTable.class);
@@ -118,7 +119,7 @@ public final class BackupSystemTable implements Closeable {
 
   private TableName tableName;
   /**
-   *  Stores backup sessions (contexts)
+   * Stores backup sessions (contexts)
    */
   final static byte[] SESSIONS_FAMILY = "session".getBytes();
   /**
@@ -127,11 +128,10 @@ public final class BackupSystemTable implements Closeable {
   final static byte[] META_FAMILY = "meta".getBytes();
   final static byte[] BULK_LOAD_FAMILY = "bulk".getBytes();
   /**
-   *  Connection to HBase cluster, shared among all instances
+   * Connection to HBase cluster, shared among all instances
    */
   private final Connection connection;
 
-
   private final static String BACKUP_INFO_PREFIX = "session:";
   private final static String START_CODE_ROW = "startcode:";
   private final static byte[] ACTIVE_SESSION_ROW = "activesession:".getBytes();
@@ -147,6 +147,7 @@ public final class BackupSystemTable implements Closeable {
   private final static String BULK_LOAD_PREFIX = "bulk:";
   private final static byte[] BULK_LOAD_PREFIX_BYTES = BULK_LOAD_PREFIX.getBytes();
   private final static byte[] DELETE_OP_ROW = "delete_op_row".getBytes();
+  private final static byte[] MERGE_OP_ROW = "merge_op_row".getBytes();
 
   final static byte[] TBL_COL = Bytes.toBytes("tbl");
   final static byte[] FAM_COL = Bytes.toBytes("fam");
@@ -160,7 +161,7 @@ public final class BackupSystemTable implements Closeable {
   private final static String SET_KEY_PREFIX = "backupset:";
 
   // separator between BULK_LOAD_PREFIX and ordinals
- protected final static String BLK_LD_DELIM = ":";
+  protected final static String BLK_LD_DELIM = ":";
   private final static byte[] EMPTY_VALUE = new byte[] {};
 
   // Safe delimiter in a string
@@ -187,19 +188,19 @@ public final class BackupSystemTable implements Closeable {
   }
 
   private void verifyNamespaceExists(Admin admin) throws IOException {
-      String namespaceName  = tableName.getNamespaceAsString();
-      NamespaceDescriptor ns = NamespaceDescriptor.create(namespaceName).build();
-      NamespaceDescriptor[] list = admin.listNamespaceDescriptors();
-      boolean exists = false;
-      for( NamespaceDescriptor nsd: list) {
-        if (nsd.getName().equals(ns.getName())) {
-          exists = true;
-          break;
-        }
-      }
-      if (!exists) {
-        admin.createNamespace(ns);
+    String namespaceName = tableName.getNamespaceAsString();
+    NamespaceDescriptor ns = NamespaceDescriptor.create(namespaceName).build();
+    NamespaceDescriptor[] list = admin.listNamespaceDescriptors();
+    boolean exists = false;
+    for (NamespaceDescriptor nsd : list) {
+      if (nsd.getName().equals(ns.getName())) {
+        exists = true;
+        break;
       }
+    }
+    if (!exists) {
+      admin.createNamespace(ns);
+    }
   }
 
   private void waitForSystemTable(Admin admin) throws IOException {
@@ -211,15 +212,13 @@ public final class BackupSystemTable implements Closeable {
       } catch (InterruptedException e) {
       }
       if (EnvironmentEdgeManager.currentTime() - startTime > TIMEOUT) {
-        throw new IOException("Failed to create backup system table after "+ TIMEOUT+"ms");
+        throw new IOException("Failed to create backup system table after " + TIMEOUT + "ms");
       }
     }
     LOG.debug("Backup table exists and available");
 
   }
 
-
-
   @Override
   public void close() {
     // do nothing
@@ -257,7 +256,7 @@ public final class BackupSystemTable implements Closeable {
         byte[] row = CellUtil.cloneRow(res.listCells().get(0));
         for (Cell cell : res.listCells()) {
           if (CellComparator.compareQualifiers(cell, BackupSystemTable.PATH_COL, 0,
-              BackupSystemTable.PATH_COL.length) == 0) {
+            BackupSystemTable.PATH_COL.length) == 0) {
             map.put(row, Bytes.toString(CellUtil.cloneValue(cell)));
           }
         }
@@ -286,13 +285,13 @@ public final class BackupSystemTable implements Closeable {
         String path = null;
         for (Cell cell : res.listCells()) {
           if (CellComparator.compareQualifiers(cell, BackupSystemTable.TBL_COL, 0,
-              BackupSystemTable.TBL_COL.length) == 0) {
+            BackupSystemTable.TBL_COL.length) == 0) {
             tbl = TableName.valueOf(CellUtil.cloneValue(cell));
           } else if (CellComparator.compareQualifiers(cell, BackupSystemTable.FAM_COL, 0,
-              BackupSystemTable.FAM_COL.length) == 0) {
+            BackupSystemTable.FAM_COL.length) == 0) {
             fam = CellUtil.cloneValue(cell);
           } else if (CellComparator.compareQualifiers(cell, BackupSystemTable.PATH_COL, 0,
-              BackupSystemTable.PATH_COL.length) == 0) {
+            BackupSystemTable.PATH_COL.length) == 0) {
             path = Bytes.toString(CellUtil.cloneValue(cell));
           }
         }
@@ -313,9 +312,10 @@ public final class BackupSystemTable implements Closeable {
         }
         files.add(new Path(path));
         if (LOG.isDebugEnabled()) {
-          LOG.debug("found bulk loaded file : " + tbl + " " +  Bytes.toString(fam) + " " + path);
+          LOG.debug("found bulk loaded file : " + tbl + " " + Bytes.toString(fam) + " " + path);
         }
-      };
+      }
+      ;
       return mapForSrc;
     }
   }
@@ -359,16 +359,16 @@ public final class BackupSystemTable implements Closeable {
   public void writePathsPostBulkLoad(TableName tabName, byte[] region,
       Map<byte[], List<Path>> finalPaths) throws IOException {
     if (LOG.isDebugEnabled()) {
-      LOG.debug("write bulk load descriptor to backup " + tabName + " with " +
-          finalPaths.size() + " entries");
+      LOG.debug("write bulk load descriptor to backup " + tabName + " with " + finalPaths.size()
+          + " entries");
     }
     try (Table table = connection.getTable(tableName)) {
-      List<Put> puts = BackupSystemTable.createPutForCommittedBulkload(tabName, region,
-          finalPaths);
+      List<Put> puts = BackupSystemTable.createPutForCommittedBulkload(tabName, region, finalPaths);
       table.put(puts);
       LOG.debug("written " + puts.size() + " rows for bulk load of " + tabName);
     }
   }
+
   /*
    * For preCommitStoreFile() hook
    * @param tabName table name
@@ -376,15 +376,15 @@ public final class BackupSystemTable implements Closeable {
    * @param family column family
    * @param pairs list of paths for hfiles
    */
-  public void writeFilesForBulkLoadPreCommit(TableName tabName, byte[] region,
-      final byte[] family, final List<Pair<Path, Path>> pairs) throws IOException {
+  public void writeFilesForBulkLoadPreCommit(TableName tabName, byte[] region, final byte[] family,
+      final List<Pair<Path, Path>> pairs) throws IOException {
     if (LOG.isDebugEnabled()) {
-      LOG.debug("write bulk load descriptor to backup " + tabName + " with " +
-          pairs.size() + " entries");
+      LOG.debug("write bulk load descriptor to backup " + tabName + " with " + pairs.size()
+          + " entries");
     }
     try (Table table = connection.getTable(tableName)) {
-      List<Put> puts = BackupSystemTable.createPutForPreparedBulkload(tabName, region,
-          family, pairs);
+      List<Put> puts =
+          BackupSystemTable.createPutForPreparedBulkload(tabName, region, family, pairs);
       table.put(puts);
       LOG.debug("written " + puts.size() + " rows for bulk load of " + tabName);
     }
@@ -411,11 +411,11 @@ public final class BackupSystemTable implements Closeable {
   /*
    * Reads the rows from backup table recording bulk loaded hfiles
    * @param tableList list of table names
-   * @return The keys of the Map are table, region and column family.
-   *  Value of the map reflects whether the hfile was recorded by preCommitStoreFile hook (true)
+   * @return The keys of the Map are table, region and column family. Value of the map reflects
+   * whether the hfile was recorded by preCommitStoreFile hook (true)
    */
   public Pair<Map<TableName, Map<String, Map<String, List<Pair<String, Boolean>>>>>, List<byte[]>>
-  readBulkloadRows(List<TableName> tableList) throws IOException {
+      readBulkloadRows(List<TableName> tableList) throws IOException {
     Map<TableName, Map<String, Map<String, List<Pair<String, Boolean>>>>> map = new HashMap<>();
     List<byte[]> rows = new ArrayList<>();
     for (TableName tTable : tableList) {
@@ -437,13 +437,13 @@ public final class BackupSystemTable implements Closeable {
             String rowStr = Bytes.toString(row);
             region = BackupSystemTable.getRegionNameFromOrigBulkLoadRow(rowStr);
             if (CellComparator.compareQualifiers(cell, BackupSystemTable.FAM_COL, 0,
-                BackupSystemTable.FAM_COL.length) == 0) {
+              BackupSystemTable.FAM_COL.length) == 0) {
               fam = Bytes.toString(CellUtil.cloneValue(cell));
             } else if (CellComparator.compareQualifiers(cell, BackupSystemTable.PATH_COL, 0,
-                BackupSystemTable.PATH_COL.length) == 0) {
+              BackupSystemTable.PATH_COL.length) == 0) {
               path = Bytes.toString(CellUtil.cloneValue(cell));
             } else if (CellComparator.compareQualifiers(cell, BackupSystemTable.STATE_COL, 0,
-                BackupSystemTable.STATE_COL.length) == 0) {
+              BackupSystemTable.STATE_COL.length) == 0) {
               byte[] state = CellUtil.cloneValue(cell);
               if (Bytes.equals(BackupSystemTable.BL_PREPARE, state)) {
                 raw = true;
@@ -484,12 +484,13 @@ public final class BackupSystemTable implements Closeable {
         Map<byte[], List<Path>> map = maps[idx];
         TableName tn = sTableList.get(idx);
         if (map == null) continue;
-        for (Map.Entry<byte[], List<Path>> entry: map.entrySet()) {
+        for (Map.Entry<byte[], List<Path>> entry : map.entrySet()) {
           byte[] fam = entry.getKey();
           List<Path> paths = entry.getValue();
           for (Path p : paths) {
-            Put put = BackupSystemTable.createPutForBulkLoadedFile(tn, fam, p.toString(),
-                backupId, ts, cnt++);
+            Put put =
+                BackupSystemTable.createPutForBulkLoadedFile(tn, fam, p.toString(), backupId, ts,
+                  cnt++);
             puts.add(put);
           }
         }
@@ -564,18 +565,23 @@ public final class BackupSystemTable implements Closeable {
     }
   }
 
-  public void startBackupSession() throws IOException {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Start new backup session");
+  /**
+   * Exclusive operations are:
+   * create, delete, merge
+   * @throws IOException
+   */
+  public void startBackupExclusiveOperation() throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Start new backup exclusive operation");
     }
     try (Table table = connection.getTable(tableName)) {
       Put put = createPutForStartBackupSession();
-      //First try to put if row does not exist
+      // First try to put if row does not exist
       if (!table.checkAndPut(ACTIVE_SESSION_ROW, SESSIONS_FAMILY, ACTIVE_SESSION_COL, null, put)) {
         // Row exists, try to put if value == ACTIVE_SESSION_NO
         if (!table.checkAndPut(ACTIVE_SESSION_ROW, SESSIONS_FAMILY, ACTIVE_SESSION_COL,
           ACTIVE_SESSION_NO, put)) {
-          throw new IOException("There is an active backup session");
+          throw new IOException("There is an active backup exclusive operation");
         }
       }
     }
@@ -587,17 +593,15 @@ public final class BackupSystemTable implements Closeable {
     return put;
   }
 
-  public void finishBackupSession() throws IOException
-  {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Stop backup session");
+  public void finishBackupExclusiveOperation() throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Finish backup exclusive operation");
     }
     try (Table table = connection.getTable(tableName)) {
       Put put = createPutForStopBackupSession();
-      if(!table.checkAndPut(ACTIVE_SESSION_ROW, SESSIONS_FAMILY, ACTIVE_SESSION_COL,
-        ACTIVE_SESSION_YES, put))
-      {
-        throw new IOException("There is no active backup session");
+      if (!table.checkAndPut(ACTIVE_SESSION_ROW, SESSIONS_FAMILY, ACTIVE_SESSION_COL,
+        ACTIVE_SESSION_YES, put)) {
+        throw new IOException("There is no active backup exclusive operation");
       }
     }
   }
@@ -630,8 +634,7 @@ public final class BackupSystemTable implements Closeable {
         res.advance();
         Cell cell = res.current();
         byte[] row = CellUtil.cloneRow(cell);
-        String server =
-            getServerNameForReadRegionServerLastLogRollResult(row);
+        String server = getServerNameForReadRegionServerLastLogRollResult(row);
         byte[] data = CellUtil.cloneValue(cell);
         rsTimestampMap.put(server, Bytes.toLong(data));
       }
@@ -652,8 +655,7 @@ public final class BackupSystemTable implements Closeable {
       LOG.trace("write region server last roll log result to backup system table");
     }
     try (Table table = connection.getTable(tableName)) {
-      Put put =
-          createPutForRegionServerLastLogRollResult(server, ts, backupRoot);
+      Put put = createPutForRegionServerLastLogRollResult(server, ts, backupRoot);
       table.put(put);
     }
   }
@@ -685,14 +687,15 @@ public final class BackupSystemTable implements Closeable {
 
   /**
    * Get first n backup history records
-   * @param n number of records
+   * @param n number of records, if n== -1 - max number
+   *        is ignored
    * @return list of records
    * @throws IOException
    */
   public List<BackupInfo> getHistory(int n) throws IOException {
 
     List<BackupInfo> history = getBackupHistory();
-    if (history.size() <= n) return history;
+    if (n == -1 || history.size() <= n) return history;
     List<BackupInfo> list = new ArrayList<BackupInfo>();
     for (int i = 0; i < n; i++) {
       list.add(history.get(i));
@@ -703,7 +706,8 @@ public final class BackupSystemTable implements Closeable {
 
   /**
    * Get backup history records filtered by list of filters.
-   * @param n max number of records
+   * @param n max number of records, if n == -1 , then max number
+   *        is ignored
    * @param filters list of filters
    * @return backup records
    * @throws IOException
@@ -714,7 +718,7 @@ public final class BackupSystemTable implements Closeable {
     List<BackupInfo> history = getBackupHistory();
     List<BackupInfo> result = new ArrayList<BackupInfo>();
     for (BackupInfo bi : history) {
-      if (result.size() == n) break;
+      if (n >= 0 && result.size() == n) break;
       boolean passed = true;
       for (int i = 0; i < filters.length; i++) {
         if (!filters[i].apply(bi)) {
@@ -852,9 +856,7 @@ public final class BackupSystemTable implements Closeable {
     List<Put> puts = new ArrayList<Put>();
     for (TableName table : tables) {
       byte[] smapData = toTableServerTimestampProto(table, newTimestamps).toByteArray();
-      Put put =
-          createPutForWriteRegionServerLogTimestamp(table, smapData,
-            backupRoot);
+      Put put = createPutForWriteRegionServerLogTimestamp(table, smapData, backupRoot);
       puts.add(put);
     }
     try (Table table = connection.getTable(tableName)) {
@@ -1018,8 +1020,7 @@ public final class BackupSystemTable implements Closeable {
       }
     }
     try (Table table = connection.getTable(tableName)) {
-      List<Put> puts =
-          createPutsForAddWALFiles(files, backupId, backupRoot);
+      List<Put> puts = createPutsForAddWALFiles(files, backupId, backupRoot);
       table.put(puts);
     }
   }
@@ -1087,6 +1088,7 @@ public final class BackupSystemTable implements Closeable {
    * @param file name of a file to check
    * @return true, if deletable, false otherwise.
    * @throws IOException exception
+   * TODO: multiple backup destination support
    */
   public boolean isWALFileDeletable(String file) throws IOException {
     if (LOG.isTraceEnabled()) {
@@ -1271,12 +1273,12 @@ public final class BackupSystemTable implements Closeable {
       if (disjoint.length > 0 && disjoint.length != tables.length) {
         Put put = createPutForBackupSet(name, disjoint);
         table.put(put);
-      } else if(disjoint.length == tables.length) {
+      } else if (disjoint.length == tables.length) {
         LOG.warn("Backup set '" + name + "' does not contain tables ["
             + StringUtils.join(toRemove, " ") + "]");
       } else { // disjoint.length == 0 and tables.length >0
-        // Delete  backup set
-        LOG.info("Backup set '"+name+"' is empty. Deleting.");
+        // Delete backup set
+        LOG.info("Backup set '" + name + "' is empty. Deleting.");
         deleteBackupSet(name);
       }
     } finally {
@@ -1356,7 +1358,7 @@ public final class BackupSystemTable implements Closeable {
   }
 
   public static String getSnapshotName(Configuration conf) {
-    return "snapshot_"+getTableNameAsString(conf).replace(":", "_");
+    return "snapshot_" + getTableNameAsString(conf).replace(":", "_");
   }
 
   /**
@@ -1589,17 +1591,16 @@ public final class BackupSystemTable implements Closeable {
       for (Path path : entry.getValue()) {
         String file = path.toString();
         int lastSlash = file.lastIndexOf("/");
-        String filename = file.substring(lastSlash+1);
-        Put put = new Put(rowkey(BULK_LOAD_PREFIX, table.toString(), BLK_LD_DELIM,
-            Bytes.toString(region), BLK_LD_DELIM, filename));
+        String filename = file.substring(lastSlash + 1);
+        Put put =
+            new Put(rowkey(BULK_LOAD_PREFIX, table.toString(), BLK_LD_DELIM,
+              Bytes.toString(region), BLK_LD_DELIM, filename));
         put.addColumn(BackupSystemTable.META_FAMILY, TBL_COL, table.getName());
         put.addColumn(BackupSystemTable.META_FAMILY, FAM_COL, entry.getKey());
-        put.addColumn(BackupSystemTable.META_FAMILY, PATH_COL,
-            file.getBytes());
+        put.addColumn(BackupSystemTable.META_FAMILY, PATH_COL, file.getBytes());
         put.addColumn(BackupSystemTable.META_FAMILY, STATE_COL, BL_COMMIT);
         puts.add(put);
-        LOG.debug("writing done bulk path " + file + " for " + table + " " +
-            Bytes.toString(region));
+        LOG.debug("writing done bulk path " + file + " for " + table + " " + Bytes.toString(region));
       }
     }
     return puts;
@@ -1607,19 +1608,16 @@ public final class BackupSystemTable implements Closeable {
 
   public static void snapshot(Connection conn) throws IOException {
 
-    try (Admin admin = conn.getAdmin();){
+    try (Admin admin = conn.getAdmin();) {
       Configuration conf = conn.getConfiguration();
-      admin.snapshot(BackupSystemTable.getSnapshotName(conf),
-        BackupSystemTable.getTableName(conf));
+      admin.snapshot(BackupSystemTable.getSnapshotName(conf), BackupSystemTable.getTableName(conf));
     }
   }
 
-  public static void restoreFromSnapshot(Connection conn)
-      throws IOException {
+  public static void restoreFromSnapshot(Connection conn) throws IOException {
 
     Configuration conf = conn.getConfiguration();
-    LOG.debug("Restoring " + BackupSystemTable.getTableNameAsString(conf) +
-        " from snapshot");
+    LOG.debug("Restoring " + BackupSystemTable.getTableNameAsString(conf) + " from snapshot");
     try (Admin admin = conn.getAdmin();) {
       String snapshotName = BackupSystemTable.getSnapshotName(conf);
       if (snapshotExists(admin, snapshotName)) {
@@ -1631,8 +1629,8 @@ public final class BackupSystemTable implements Closeable {
         // Snapshot does not exists, i.e completeBackup failed after
         // deleting backup system table snapshot
         // In this case we log WARN and proceed
-        LOG.warn("Could not restore backup system table. Snapshot " + snapshotName+
-          " does not exists.");
+        LOG.warn("Could not restore backup system table. Snapshot " + snapshotName
+            + " does not exists.");
       }
     }
   }
@@ -1640,7 +1638,7 @@ public final class BackupSystemTable implements Closeable {
   protected static boolean snapshotExists(Admin admin, String snapshotName) throws IOException {
 
     List<SnapshotDescription> list = admin.listSnapshots();
-    for (SnapshotDescription desc: list) {
+    for (SnapshotDescription desc : list) {
       if (desc.getName().equals(snapshotName)) {
         return true;
       }
@@ -1648,26 +1646,25 @@ public final class BackupSystemTable implements Closeable {
     return false;
   }
 
-  public static boolean snapshotExists (Connection conn) throws IOException {
+  public static boolean snapshotExists(Connection conn) throws IOException {
     return snapshotExists(conn.getAdmin(), getSnapshotName(conn.getConfiguration()));
   }
 
-  public static void deleteSnapshot(Connection conn)
-      throws IOException {
+  public static void deleteSnapshot(Connection conn) throws IOException {
 
     Configuration conf = conn.getConfiguration();
-    LOG.debug("Deleting " + BackupSystemTable.getSnapshotName(conf) +
-        " from the system");
+    LOG.debug("Deleting " + BackupSystemTable.getSnapshotName(conf) + " from the system");
     try (Admin admin = conn.getAdmin();) {
       String snapshotName = BackupSystemTable.getSnapshotName(conf);
       if (snapshotExists(admin, snapshotName)) {
         admin.deleteSnapshot(snapshotName);
         LOG.debug("Done deleting backup system table snapshot");
       } else {
-        LOG.error("Snapshot "+snapshotName+" does not exists");
+        LOG.error("Snapshot " + snapshotName + " does not exists");
       }
     }
   }
+
   /*
    * Creates Put's for bulk load resulting from running LoadIncrementalHFiles
    */
@@ -1678,17 +1675,16 @@ public final class BackupSystemTable implements Closeable {
       Path path = pair.getSecond();
       String file = path.toString();
       int lastSlash = file.lastIndexOf("/");
-      String filename = file.substring(lastSlash+1);
-      Put put = new Put(rowkey(BULK_LOAD_PREFIX, table.toString(), BLK_LD_DELIM,
-          Bytes.toString(region), BLK_LD_DELIM, filename));
+      String filename = file.substring(lastSlash + 1);
+      Put put =
+          new Put(rowkey(BULK_LOAD_PREFIX, table.toString(), BLK_LD_DELIM, Bytes.toString(region),
+            BLK_LD_DELIM, filename));
       put.addColumn(BackupSystemTable.META_FAMILY, TBL_COL, table.getName());
       put.addColumn(BackupSystemTable.META_FAMILY, FAM_COL, family);
-      put.addColumn(BackupSystemTable.META_FAMILY, PATH_COL,
-          file.getBytes());
+      put.addColumn(BackupSystemTable.META_FAMILY, PATH_COL, file.getBytes());
       put.addColumn(BackupSystemTable.META_FAMILY, STATE_COL, BL_PREPARE);
       puts.add(put);
-      LOG.debug("writing raw bulk path " + file + " for " + table + " " +
-          Bytes.toString(region));
+      LOG.debug("writing raw bulk path " + file + " for " + table + " " + Bytes.toString(region));
     }
     return puts;
   }
@@ -1725,7 +1721,6 @@ public final class BackupSystemTable implements Closeable {
     return get;
   }
 
-
   public void startDeleteOperation(String[] backupIdList) throws IOException {
     if (LOG.isTraceEnabled()) {
       LOG.trace("Start delete operation for backups: " + StringUtils.join(backupIdList));
@@ -1765,6 +1760,96 @@ public final class BackupSystemTable implements Closeable {
     }
   }
 
+  private Put createPutForMergeOperation(String[] backupIdList) {
+
+    byte[] value = Bytes.toBytes(StringUtils.join(backupIdList, ","));
+    Put put = new Put(MERGE_OP_ROW);
+    put.addColumn(META_FAMILY, FAM_COL, value);
+    return put;
+  }
+
+  public boolean isMergeInProgress() throws IOException {
+    Get get = new Get(MERGE_OP_ROW);
+    try (Table table = connection.getTable(tableName)) {
+      Result res = table.get(get);
+      if (res.isEmpty()) {
+        return false;
+      }
+      return true;
+    }
+  }
+
+  private Put createPutForUpdateTablesForMerge(List<TableName> tables) {
+
+    byte[] value = Bytes.toBytes(StringUtils.join(tables, ","));
+    Put put = new Put(MERGE_OP_ROW);
+    put.addColumn(META_FAMILY, PATH_COL, value);
+    return put;
+  }
+
+  private Delete createDeleteForBackupMergeOperation() {
+
+    Delete delete = new Delete(MERGE_OP_ROW);
+    delete.addFamily(META_FAMILY);
+    return delete;
+  }
+
+  private Get createGetForMergeOperation() {
+
+    Get get = new Get(MERGE_OP_ROW);
+    get.addFamily(META_FAMILY);
+    return get;
+  }
+
+  public void startMergeOperation(String[] backupIdList) throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Start merge operation for backups: " + StringUtils.join(backupIdList));
+    }
+    Put put = createPutForMergeOperation(backupIdList);
+    try (Table table = connection.getTable(tableName)) {
+      table.put(put);
+    }
+  }
+
+  public void updateProcessedTablesForMerge(List<TableName> tables) throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Update tables for merge : " + StringUtils.join(tables, ","));
+    }
+    Put put = createPutForUpdateTablesForMerge(tables);
+    try (Table table = connection.getTable(tableName)) {
+      table.put(put);
+    }
+  }
+
+  public void finishMergeOperation() throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Finsih merge operation for backup ids ");
+    }
+    Delete delete = createDeleteForBackupMergeOperation();
+    try (Table table = connection.getTable(tableName)) {
+      table.delete(delete);
+    }
+  }
+
+  public String[] getListOfBackupIdsFromMergeOperation() throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Get backup ids for merge operation");
+    }
+    Get get = createGetForMergeOperation();
+    try (Table table = connection.getTable(tableName)) {
+      Result res = table.get(get);
+      if (res.isEmpty()) {
+        return null;
+      }
+      Cell cell = res.listCells().get(0);
+      byte[] val = CellUtil.cloneValue(cell);
+      if (val.length == 0) {
+        return null;
+      }
+      return new String(val).split(",");
+    }
+  }
+
   static Scan createScanForOrigBulkLoadedFiles(TableName table) throws IOException {
     Scan scan = new Scan();
     byte[] startRow = rowkey(BULK_LOAD_PREFIX, table.toString(), BLK_LD_DELIM);
@@ -1776,10 +1861,12 @@ public final class BackupSystemTable implements Closeable {
     scan.setMaxVersions(1);
     return scan;
   }
+
   static String getTableNameFromOrigBulkLoadRow(String rowStr) {
     String[] parts = rowStr.split(BLK_LD_DELIM);
     return parts[1];
   }
+
   static String getRegionNameFromOrigBulkLoadRow(String rowStr) {
     // format is bulk : namespace : table : region : file
     String[] parts = rowStr.split(BLK_LD_DELIM);
@@ -1791,6 +1878,7 @@ public final class BackupSystemTable implements Closeable {
     LOG.debug("bulk row string " + rowStr + " region " + parts[idx]);
     return parts[idx];
   }
+
   /*
    * Used to query bulk loaded hfiles which have been copied by incremental backup
    * @param backupId the backup Id. It can be null when querying for all tables
@@ -1798,13 +1886,14 @@ public final class BackupSystemTable implements Closeable {
    */
   static Scan createScanForBulkLoadedFiles(String backupId) throws IOException {
     Scan scan = new Scan();
-    byte[] startRow = backupId == null ? BULK_LOAD_PREFIX_BYTES :
-      rowkey(BULK_LOAD_PREFIX, backupId+BLK_LD_DELIM);
+    byte[] startRow =
+        backupId == null ? BULK_LOAD_PREFIX_BYTES : rowkey(BULK_LOAD_PREFIX, backupId
+            + BLK_LD_DELIM);
     byte[] stopRow = Arrays.copyOf(startRow, startRow.length);
     stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1);
     scan.setStartRow(startRow);
     scan.setStopRow(stopRow);
-    //scan.setTimeRange(lower, Long.MAX_VALUE);
+    // scan.setTimeRange(lower, Long.MAX_VALUE);
     scan.addFamily(BackupSystemTable.META_FAMILY);
     scan.setMaxVersions(1);
     return scan;
@@ -1812,12 +1901,13 @@ public final class BackupSystemTable implements Closeable {
 
   static Put createPutForBulkLoadedFile(TableName tn, byte[] fam, String p, String backupId,
       long ts, int idx) {
-    Put put = new Put(rowkey(BULK_LOAD_PREFIX, backupId+BLK_LD_DELIM+ts+BLK_LD_DELIM+idx));
+    Put put = new Put(rowkey(BULK_LOAD_PREFIX, backupId + BLK_LD_DELIM + ts + BLK_LD_DELIM + idx));
     put.addColumn(BackupSystemTable.META_FAMILY, TBL_COL, tn.getName());
     put.addColumn(BackupSystemTable.META_FAMILY, FAM_COL, fam);
     put.addColumn(BackupSystemTable.META_FAMILY, PATH_COL, p.getBytes());
     return put;
   }
+
   /**
    * Creates put list for list of WAL files
    * @param files list of WAL file paths
@@ -1825,8 +1915,9 @@ public final class BackupSystemTable implements Closeable {
    * @return put list
    * @throws IOException exception
    */
-  private List<Put> createPutsForAddWALFiles(List<String> files, String backupId,
-      String backupRoot) throws IOException {
+  private List<Put>
+      createPutsForAddWALFiles(List<String> files, String backupId, String backupRoot)
+          throws IOException {
 
     List<Put> puts = new ArrayList<Put>();
     for (String file : files) {
@@ -1957,5 +2048,4 @@ public final class BackupSystemTable implements Closeable {
     return sb.toString().getBytes();
   }
 
-
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/05e6e569/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java
index 381e9b1..ea7a7b8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java
@@ -39,7 +39,7 @@ import org.apache.hadoop.hbase.backup.BackupType;
 import org.apache.hadoop.hbase.backup.HBackupFileSystem;
 import org.apache.hadoop.hbase.backup.RestoreRequest;
 import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage;
-import org.apache.hadoop.hbase.backup.mapreduce.MapReduceRestoreJob;
+import org.apache.hadoop.hbase.backup.util.BackupUtils;
 import org.apache.hadoop.hbase.backup.util.RestoreTool;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Admin;
@@ -58,7 +58,6 @@ public class RestoreTablesClient {
   private Configuration conf;
   private Connection conn;
   private String backupId;
-  private String fullBackupId;
   private TableName[] sTableArray;
   private TableName[] tTableArray;
   private String targetRootDir;
@@ -107,8 +106,7 @@ public class RestoreTablesClient {
 
     if (existTableList.size() > 0) {
       if (!isOverwrite) {
-        LOG.error("Existing table ("
-            + existTableList
+        LOG.error("Existing table (" + existTableList
             + ") found in the restore target, please add "
             + "\"-overwrite\" option in the command if you mean"
             + " to restore to these existing tables");
@@ -148,9 +146,8 @@ public class RestoreTablesClient {
     Path tableBackupPath = HBackupFileSystem.getTableBackupPath(sTable, backupRoot, backupId);
     String lastIncrBackupId = images.length == 1 ? null : images[images.length - 1].getBackupId();
     // We need hFS only for full restore (see the code)
-    BackupManifest manifest = HBackupFileSystem.getManifest(sTable, conf, backupRoot, backupId);
+    BackupManifest manifest = HBackupFileSystem.getManifest(conf, backupRoot, backupId);
     if (manifest.getType() == BackupType.FULL) {
-      fullBackupId = manifest.getBackupImage().getBackupId();
       LOG.info("Restoring '" + sTable + "' to '" + tTable + "' from full" + " backup image "
           + tableBackupPath.toString());
       restoreTool.fullRestoreTable(conn, tableBackupPath, sTable, tTable, truncateIfExists,
@@ -169,8 +166,8 @@ public class RestoreTablesClient {
     // full backup path comes first
     for (int i = 1; i < images.length; i++) {
       BackupImage im = images[i];
-      String fileBackupDir = HBackupFileSystem.getTableBackupDir(im.getRootDir(),
-                  im.getBackupId(), sTable)+ Path.SEPARATOR+"data";
+      String fileBackupDir =
+          HBackupFileSystem.getTableBackupDataDir(im.getRootDir(), im.getBackupId(), sTable);
       dirList.add(new Path(fileBackupDir));
     }
 
@@ -196,8 +193,10 @@ public class RestoreTablesClient {
     TreeSet<BackupImage> restoreImageSet = new TreeSet<BackupImage>();
     boolean truncateIfExists = isOverwrite;
     Set<String> backupIdSet = new HashSet<>();
+
     for (int i = 0; i < sTableArray.length; i++) {
       TableName table = sTableArray[i];
+
       BackupManifest manifest = backupManifestMap.get(table);
       // Get the image list of this backup for restore in time order from old
       // to new.
@@ -213,11 +212,8 @@ public class RestoreTablesClient {
       if (restoreImageSet != null && !restoreImageSet.isEmpty()) {
         LOG.info("Restore includes the following image(s):");
         for (BackupImage image : restoreImageSet) {
-          LOG.info("Backup: "
-              + image.getBackupId()
-              + " "
-              + HBackupFileSystem.getTableBackupDir(image.getRootDir(), image.getBackupId(),
-                  table));
+          LOG.info("Backup: " + image.getBackupId() + " "
+              + HBackupFileSystem.getTableBackupDir(image.getRootDir(), image.getBackupId(), table));
           if (image.getType() == BackupType.INCREMENTAL) {
             backupIdSet.add(image.getBackupId());
             LOG.debug("adding " + image.getBackupId() + " for bulk load");
@@ -232,13 +228,13 @@ public class RestoreTablesClient {
         Map<byte[], List<Path>>[] mapForSrc = table.readBulkLoadedFiles(id, sTableList);
         Map<LoadQueueItem, ByteBuffer> loaderResult;
         conf.setBoolean(LoadIncrementalHFiles.ALWAYS_COPY_FILES, true);
-        LoadIncrementalHFiles loader = MapReduceRestoreJob.createLoader(conf);
+        LoadIncrementalHFiles loader = BackupUtils.createLoader(conf);
         for (int i = 0; i < sTableList.size(); i++) {
           if (mapForSrc[i] != null && !mapForSrc[i].isEmpty()) {
             loaderResult = loader.run(null, mapForSrc[i], tTableArray[i]);
             LOG.debug("bulk loading " + sTableList.get(i) + " to " + tTableArray[i]);
             if (loaderResult.isEmpty()) {
-              String msg = "Couldn't bulk load for " + sTableList.get(i) + " to " +tTableArray[i];
+              String msg = "Couldn't bulk load for " + sTableList.get(i) + " to " + tTableArray[i];
               LOG.error(msg);
               throw new IOException(msg);
             }
@@ -253,7 +249,7 @@ public class RestoreTablesClient {
     if (backupId == null) {
       return 0;
     }
-    return Long.parseLong(backupId.substring(backupId.lastIndexOf("_")+1));
+    return Long.parseLong(backupId.substring(backupId.lastIndexOf("_") + 1));
   }
 
   static boolean withinRange(long a, long lower, long upper) {
@@ -268,15 +264,15 @@ public class RestoreTablesClient {
     // case VALIDATION:
     // check the target tables
     checkTargetTables(tTableArray, isOverwrite);
+
     // case RESTORE_IMAGES:
     HashMap<TableName, BackupManifest> backupManifestMap = new HashMap<>();
     // check and load backup image manifest for the tables
     Path rootPath = new Path(targetRootDir);
     HBackupFileSystem.checkImageManifestExist(backupManifestMap, sTableArray, conf, rootPath,
       backupId);
+
     restore(backupManifestMap, sTableArray, tTableArray, isOverwrite);
   }
 
-
-
 }


[36/50] [abbrv] hbase git commit: HBASE-18587 Fix flaky TestFileIOEngine

Posted by st...@apache.org.
HBASE-18587 Fix flaky TestFileIOEngine

This short circuits reads and writes with 0 length and also removes flakiness in TestFileIOEngine

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/5280c100
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/5280c100
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/5280c100

Branch: refs/heads/HBASE-14070.HLC
Commit: 5280c100ff93f65cd568ce830e088cc12a2f5585
Parents: 2b88edf
Author: Zach York <zy...@amazon.com>
Authored: Thu Aug 10 16:55:28 2017 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Tue Aug 15 14:57:10 2017 -0700

----------------------------------------------------------------------
 .../hbase/io/hfile/bucket/FileIOEngine.java     |  23 ++--
 .../hbase/io/hfile/bucket/TestFileIOEngine.java | 123 +++++++++++--------
 2 files changed, 88 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/5280c100/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileIOEngine.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileIOEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileIOEngine.java
index a847bfe..ab77696 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileIOEngine.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileIOEngine.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.io.hfile.CacheableDeserializer;
 import org.apache.hadoop.hbase.io.hfile.Cacheable.MemoryType;
 import org.apache.hadoop.hbase.nio.ByteBuff;
 import org.apache.hadoop.hbase.nio.SingleByteBuff;
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
 import org.apache.hadoop.util.StringUtils;
 
 /**
@@ -122,15 +123,18 @@ public class FileIOEngine implements IOEngine {
   @Override
   public Cacheable read(long offset, int length, CacheableDeserializer<Cacheable> deserializer)
       throws IOException {
+    Preconditions.checkArgument(length >= 0, "Length of read can not be less than 0.");
     ByteBuffer dstBuffer = ByteBuffer.allocate(length);
-    accessFile(readAccessor, dstBuffer, offset);
-    // The buffer created out of the fileChannel is formed by copying the data from the file
-    // Hence in this case there is no shared memory that we point to. Even if the BucketCache evicts
-    // this buffer from the file the data is already copied and there is no need to ensure that
-    // the results are not corrupted before consuming them.
-    if (dstBuffer.limit() != length) {
-      throw new RuntimeException("Only " + dstBuffer.limit() + " bytes read, " + length
-          + " expected");
+    if (length != 0) {
+      accessFile(readAccessor, dstBuffer, offset);
+      // The buffer created out of the fileChannel is formed by copying the data from the file
+      // Hence in this case there is no shared memory that we point to. Even if the BucketCache evicts
+      // this buffer from the file the data is already copied and there is no need to ensure that
+      // the results are not corrupted before consuming them.
+      if (dstBuffer.limit() != length) {
+        throw new RuntimeException("Only " + dstBuffer.limit() + " bytes read, " + length
+            + " expected");
+      }
     }
     return deserializer.deserialize(new SingleByteBuff(dstBuffer), true, MemoryType.EXCLUSIVE);
   }
@@ -143,6 +147,9 @@ public class FileIOEngine implements IOEngine {
    */
   @Override
   public void write(ByteBuffer srcBuffer, long offset) throws IOException {
+    if (!srcBuffer.hasRemaining()) {
+      return;
+    }
     accessFile(writeAccessor, srcBuffer, offset);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/5280c100/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestFileIOEngine.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestFileIOEngine.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestFileIOEngine.java
index d13022d..4451c0c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestFileIOEngine.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestFileIOEngine.java
@@ -18,6 +18,7 @@
  */
 package org.apache.hadoop.hbase.io.hfile.bucket;
 
+import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertTrue;
 
 import java.io.File;
@@ -30,6 +31,8 @@ import org.apache.hadoop.hbase.io.hfile.bucket.TestByteBufferIOEngine.BufferGrab
 import org.apache.hadoop.hbase.nio.ByteBuff;
 import org.apache.hadoop.hbase.testclassification.IOTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -38,62 +41,82 @@ import org.junit.experimental.categories.Category;
  */
 @Category({IOTests.class, SmallTests.class})
 public class TestFileIOEngine {
-  @Test
-  public void testFileIOEngine() throws IOException {
-    long totalCapacity = 6 * 1024 * 1024; // 6 MB
-    String[] filePaths = { "testFileIOEngine1", "testFileIOEngine2",
-        "testFileIOEngine3" };
-    long sizePerFile = totalCapacity / filePaths.length; // 2 MB per File
-    List<Long> boundaryStartPositions = new ArrayList<Long>();
+
+  private static final long TOTAL_CAPACITY = 6 * 1024 * 1024; // 6 MB
+  private static final String[] FILE_PATHS = {"testFileIOEngine1", "testFileIOEngine2",
+      "testFileIOEngine3"};
+  private static final long SIZE_PER_FILE = TOTAL_CAPACITY / FILE_PATHS.length; // 2 MB per File
+  private final static List<Long> boundaryStartPositions = new ArrayList<Long>();
+  private final static List<Long> boundaryStopPositions = new ArrayList<Long>();
+
+  private FileIOEngine fileIOEngine;
+
+  static {
     boundaryStartPositions.add(0L);
-    for (int i = 1; i < filePaths.length; i++) {
-      boundaryStartPositions.add(sizePerFile * i - 1);
-      boundaryStartPositions.add(sizePerFile * i);
-      boundaryStartPositions.add(sizePerFile * i + 1);
+    for (int i = 1; i < FILE_PATHS.length; i++) {
+      boundaryStartPositions.add(SIZE_PER_FILE * i - 1);
+      boundaryStartPositions.add(SIZE_PER_FILE * i);
+      boundaryStartPositions.add(SIZE_PER_FILE * i + 1);
     }
-    List<Long> boundaryStopPositions = new ArrayList<Long>();
-    for (int i = 1; i < filePaths.length; i++) {
-      boundaryStopPositions.add(sizePerFile * i - 1);
-      boundaryStopPositions.add(sizePerFile * i);
-      boundaryStopPositions.add(sizePerFile * i + 1);
+    for (int i = 1; i < FILE_PATHS.length; i++) {
+      boundaryStopPositions.add(SIZE_PER_FILE * i - 1);
+      boundaryStopPositions.add(SIZE_PER_FILE * i);
+      boundaryStopPositions.add(SIZE_PER_FILE * i + 1);
+    }
+    boundaryStopPositions.add(SIZE_PER_FILE * FILE_PATHS.length - 1);
+  }
+
+  @Before
+  public void setUp() throws IOException {
+    fileIOEngine = new FileIOEngine(TOTAL_CAPACITY, false, FILE_PATHS);
+  }
+
+  @After
+  public void cleanUp() {
+    fileIOEngine.shutdown();
+    for (String filePath : FILE_PATHS) {
+      File file = new File(filePath);
+      if (file.exists()) {
+        file.delete();
+      }
     }
-    boundaryStopPositions.add(sizePerFile * filePaths.length - 1);
-    FileIOEngine fileIOEngine = new FileIOEngine(totalCapacity, false, filePaths);
-    try {
-      for (int i = 0; i < 500; i++) {
-        int len = (int) Math.floor(Math.random() * 100);
-        long offset = (long) Math.floor(Math.random() * totalCapacity % (totalCapacity - len));
-        if (i < boundaryStartPositions.size()) {
-          // make the boundary start positon
-          offset = boundaryStartPositions.get(i);
-        } else if ((i - boundaryStartPositions.size()) < boundaryStopPositions.size()) {
-          // make the boundary stop positon
-          offset = boundaryStopPositions.get(i - boundaryStartPositions.size()) - len + 1;
-        } else if (i % 2 == 0) {
-          // make the cross-files block writing/reading
-          offset = Math.max(1, i % filePaths.length) * sizePerFile - len / 2;
-        }
-        byte[] data1 = new byte[len];
-        for (int j = 0; j < data1.length; ++j) {
-          data1[j] = (byte) (Math.random() * 255);
-        }
-        fileIOEngine.write(ByteBuffer.wrap(data1), offset);
-        BufferGrabbingDeserializer deserializer = new BufferGrabbingDeserializer();
-        fileIOEngine.read(offset, len, deserializer);
-        ByteBuff data2 = deserializer.getDeserializedByteBuff();
-        for (int j = 0; j < data1.length; ++j) {
-          assertTrue(data1[j] == data2.get(j));
-        }
+  }
+
+  @Test
+  public void testFileIOEngine() throws IOException {
+    for (int i = 0; i < 500; i++) {
+      int len = (int) Math.floor(Math.random() * 100) + 1;
+      long offset = (long) Math.floor(Math.random() * TOTAL_CAPACITY % (TOTAL_CAPACITY - len));
+      if (i < boundaryStartPositions.size()) {
+        // make the boundary start positon
+        offset = boundaryStartPositions.get(i);
+      } else if ((i - boundaryStartPositions.size()) < boundaryStopPositions.size()) {
+        // make the boundary stop positon
+        offset = boundaryStopPositions.get(i - boundaryStartPositions.size()) - len + 1;
+      } else if (i % 2 == 0) {
+        // make the cross-files block writing/reading
+        offset = Math.max(1, i % FILE_PATHS.length) * SIZE_PER_FILE - len / 2;
       }
-    } finally {
-      fileIOEngine.shutdown();
-      for (String filePath : filePaths) {
-        File file = new File(filePath);
-        if (file.exists()) {
-          file.delete();
-        }
+      byte[] data1 = new byte[len];
+      for (int j = 0; j < data1.length; ++j) {
+        data1[j] = (byte) (Math.random() * 255);
       }
+      fileIOEngine.write(ByteBuffer.wrap(data1), offset);
+      BufferGrabbingDeserializer deserializer = new BufferGrabbingDeserializer();
+      fileIOEngine.read(offset, len, deserializer);
+      ByteBuff data2 = deserializer.getDeserializedByteBuff();
+      assertArrayEquals(data1, data2.array());
     }
+  }
+
+  @Test
+  public void testFileIOEngineHandlesZeroLengthInput() throws IOException {
+    byte[] data1 = new byte[0];
 
+    fileIOEngine.write(ByteBuffer.wrap(data1), 0);
+    BufferGrabbingDeserializer deserializer = new BufferGrabbingDeserializer();
+    fileIOEngine.read(0, 0, deserializer);
+    ByteBuff data2 = deserializer.getDeserializedByteBuff();
+    assertArrayEquals(data1, data2.array());
   }
 }


[34/50] [abbrv] hbase git commit: HBASE-18544 Move the HRegion#addRegionToMETA to TestDefaultMemStore

Posted by st...@apache.org.
HBASE-18544 Move the HRegion#addRegionToMETA to TestDefaultMemStore

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/310934d0
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/310934d0
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/310934d0

Branch: refs/heads/HBASE-14070.HLC
Commit: 310934d0604605fe361e836fe4277c48b5c493fa
Parents: 63e313b
Author: Chun-Hao Tang <ta...@gmail.com>
Authored: Wed Aug 16 00:43:02 2017 +0800
Committer: Michael Stack <st...@apache.org>
Committed: Tue Aug 15 14:52:33 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/HRegion.java      | 31 ++------------------
 .../hbase/regionserver/TestDefaultMemStore.java | 28 +++++++++++++++++-
 2 files changed, 29 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/310934d0/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 3b24f3d..b9cafd9 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
@@ -3928,7 +3928,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    * We throw RegionTooBusyException if above memstore limit
    * and expect client to retry using some kind of backoff
   */
-  private void checkResources() throws RegionTooBusyException {
+  void checkResources() throws RegionTooBusyException {
     // If catalog region, do not impose resource constraints or block updates.
     if (this.getRegionInfo().isMetaRegion()) return;
 
@@ -3974,7 +3974,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    * @param edits Cell updates by column
    * @throws IOException
    */
-  private void put(final byte [] row, byte [] family, List<Cell> edits)
+  void put(final byte [] row, byte [] family, List<Cell> edits)
   throws IOException {
     NavigableMap<byte[], List<Cell>> familyMap;
     familyMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
@@ -6878,33 +6878,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   }
 
   /**
-   * Inserts a new region's meta information into the passed
-   * <code>meta</code> region. Used by the HMaster bootstrap code adding
-   * new table to hbase:meta table.
-   *
-   * @param meta hbase:meta HRegion to be updated
-   * @param r HRegion to add to <code>meta</code>
-   *
-   * @throws IOException
-   */
-  // TODO remove since only test and merge use this
-  public static void addRegionToMETA(final HRegion meta, final HRegion r) throws IOException {
-    meta.checkResources();
-    // The row key is the region name
-    byte[] row = r.getRegionInfo().getRegionName();
-    final long now = EnvironmentEdgeManager.currentTime();
-    final List<Cell> cells = new ArrayList<>(2);
-    cells.add(new KeyValue(row, HConstants.CATALOG_FAMILY,
-      HConstants.REGIONINFO_QUALIFIER, now,
-      r.getRegionInfo().toByteArray()));
-    // Set into the root table the version of the meta table.
-    cells.add(new KeyValue(row, HConstants.CATALOG_FAMILY,
-      HConstants.META_VERSION_QUALIFIER, now,
-      Bytes.toBytes(HConstants.META_VERSION)));
-    meta.put(row, HConstants.CATALOG_FAMILY, cells);
-  }
-
-  /**
    * Computes the Path of the HRegion
    *
    * @param tabledir qualified path for table

http://git-wip-us.apache.org/repos/asf/hbase/blob/310934d0/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
index 0b1638b..7b10846 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
@@ -975,7 +975,7 @@ public class TestDefaultMemStore {
     HRegion r =
         HRegion.createHRegion(hri, testDir, conf, desc,
             wFactory.getWAL(hri.getEncodedNameAsBytes(), hri.getTable().getNamespace()));
-    HRegion.addRegionToMETA(meta, r);
+    addRegionToMETA(meta, r);
     edge.setCurrentTimeMillis(1234 + 100);
     StringBuffer sb = new StringBuffer();
     assertTrue(meta.shouldFlush(sb) == false);
@@ -983,6 +983,32 @@ public class TestDefaultMemStore {
     assertTrue(meta.shouldFlush(sb) == true);
   }
 
+  /**
+   * Inserts a new region's meta information into the passed
+   * <code>meta</code> region. Used by the HMaster bootstrap code adding
+   * new table to hbase:meta table.
+   *
+   * @param meta hbase:meta HRegion to be updated
+   * @param r HRegion to add to <code>meta</code>
+   *
+   * @throws IOException
+   */
+  public static void addRegionToMETA(final HRegion meta, final HRegion r) throws IOException {
+    meta.checkResources();
+    // The row key is the region name
+    byte[] row = r.getRegionInfo().getRegionName();
+    final long now = EnvironmentEdgeManager.currentTime();
+    final List<Cell> cells = new ArrayList<>(2);
+    cells.add(new KeyValue(row, HConstants.CATALOG_FAMILY,
+      HConstants.REGIONINFO_QUALIFIER, now,
+      r.getRegionInfo().toByteArray()));
+    // Set into the root table the version of the meta table.
+    cells.add(new KeyValue(row, HConstants.CATALOG_FAMILY,
+      HConstants.META_VERSION_QUALIFIER, now,
+      Bytes.toBytes(HConstants.META_VERSION)));
+    meta.put(row, HConstants.CATALOG_FAMILY, cells);
+  }
+
   private class EnvironmentEdgeForMemstoreTest implements EnvironmentEdge {
     long t = 1234;
     @Override


[07/50] [abbrv] hbase git commit: HBASE-18500 Performance issue: Don't use BufferedMutator for HTable's put method

Posted by st...@apache.org.
HBASE-18500 Performance issue: Don't use BufferedMutator for HTable's put method


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

Branch: refs/heads/HBASE-14070.HLC
Commit: cabdbf181a0b44bd1f9c32aa67b8a2ee3b863758
Parents: 679f34e
Author: Guanghao Zhang <zg...@apache.org>
Authored: Wed Aug 2 13:52:16 2017 +0800
Committer: Guanghao Zhang <zg...@apache.org>
Committed: Fri Aug 11 16:33:35 2017 +0800

----------------------------------------------------------------------
 .../hbase/client/BufferedMutatorImpl.java       |  19 ---
 .../org/apache/hadoop/hbase/client/HTable.java  | 126 ++++---------------
 .../org/apache/hadoop/hbase/client/Table.java   |  33 +----
 .../hadoop/hbase/client/TableBuilder.java       |   6 -
 .../hadoop/hbase/client/TableBuilderBase.java   |   9 --
 .../hadoop/hbase/client/TestAsyncProcess.java   |  71 +----------
 .../hadoop/hbase/rest/client/RemoteHTable.java  |  10 --
 .../hadoop/hbase/client/HTableWrapper.java      |  10 --
 .../security/access/AccessControlLists.java     |  16 ++-
 .../hadoop/hbase/PerformanceEvaluation.java     |  13 +-
 .../hadoop/hbase/client/TestClientPushback.java |   6 +-
 .../hadoop/hbase/client/TestFromClientSide.java |   5 +-
 .../org/apache/hadoop/hbase/client/TestHCM.java |   3 +-
 .../hadoop/hbase/client/TestMultiParallel.java  |   1 -
 .../hbase/client/TestServerBusyException.java   |   8 +-
 .../hadoop/hbase/constraint/TestConstraint.java |   9 +-
 .../hbase/coprocessor/TestHTableWrapper.java    |   8 --
 .../hbase/regionserver/RegionAsTable.java       |  10 --
 .../replication/TestMasterReplication.java      |   1 -
 .../replication/TestMultiSlaveReplication.java  |   3 -
 .../hbase/replication/TestReplicationBase.java  |   1 -
 .../replication/TestReplicationSmallTests.java  |   1 -
 .../replication/TestReplicationSyncUpTool.java  |   4 -
 .../hbase/security/access/SecureTestUtil.java   |   2 +-
 .../security/access/TestNamespaceCommands.java  |   1 +
 25 files changed, 69 insertions(+), 307 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/cabdbf18/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 0ddc159..b7d3104 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
@@ -305,25 +305,6 @@ public class BufferedMutatorImpl implements BufferedMutator {
       }
     };
   }
-  /**
-   * This is used for legacy purposes in {@link HTable#setWriteBufferSize(long)} only. This ought
-   * not be called for production uses.
-   * If the new buffer size is smaller than the stored data, the {@link BufferedMutatorImpl#flush()}
-   * will be called.
-   * @param writeBufferSize The max size of internal buffer where data is stored.
-   * @throws org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException
-   * if an I/O error occurs and there are too many retries.
-   * @throws java.io.InterruptedIOException if the I/O task is interrupted.
-   * @deprecated Going away when we drop public support for {@link HTable}.
-   */
-  @Deprecated
-  public void setWriteBufferSize(long writeBufferSize) throws RetriesExhaustedWithDetailsException,
-      InterruptedIOException {
-    this.writeBufferSize = writeBufferSize;
-    if (currentWriteBufferSize.get() > writeBufferSize) {
-      flush();
-    }
-  }
 
   /**
    * {@inheritDoc}

http://git-wip-us.apache.org/repos/asf/hbase/blob/cabdbf18/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 c0d321b..2920281 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
@@ -107,9 +107,6 @@ public class HTable implements Table {
   private final TableName tableName;
   private final Configuration configuration;
   private final ConnectionConfiguration connConfiguration;
-  @VisibleForTesting
-  volatile BufferedMutatorImpl mutator;
-  private final Object mutatorLock = new Object();
   private boolean closed = false;
   private final int scannerCaching;
   private final long scannerMaxResultSize;
@@ -120,7 +117,6 @@ public class HTable implements Table {
   private int writeRpcTimeout; // timeout for each write rpc request
   private final boolean cleanupPoolOnClose; // shutdown the pool in close()
   private final HRegionLocator locator;
-  private final long writeBufferSize;
 
   /** The Async process for batch */
   @VisibleForTesting
@@ -194,7 +190,6 @@ public class HTable implements Table {
     this.rpcTimeout = builder.rpcTimeout;
     this.readRpcTimeout = builder.readRpcTimeout;
     this.writeRpcTimeout = builder.writeRpcTimeout;
-    this.writeBufferSize = builder.writeBufferSize;
     this.scannerCaching = connConfiguration.getScannerCaching();
     this.scannerMaxResultSize = connConfiguration.getScannerMaxResultSize();
 
@@ -204,31 +199,6 @@ public class HTable implements Table {
   }
 
   /**
-   * For internal testing. Uses Connection provided in {@code params}.
-   * @throws IOException
-   */
-  @VisibleForTesting
-  protected HTable(ClusterConnection conn, BufferedMutatorImpl mutator) throws IOException {
-    connection = conn;
-    this.tableName = mutator.getName();
-    this.configuration = connection.getConfiguration();
-    connConfiguration = connection.getConnectionConfiguration();
-    cleanupPoolOnClose = false;
-    this.mutator = mutator;
-    this.operationTimeout = connConfiguration.getOperationTimeout();
-    this.rpcTimeout = connConfiguration.getRpcTimeout();
-    this.readRpcTimeout = connConfiguration.getReadRpcTimeout();
-    this.writeRpcTimeout = connConfiguration.getWriteRpcTimeout();
-    this.scannerCaching = connConfiguration.getScannerCaching();
-    this.scannerMaxResultSize = connConfiguration.getScannerMaxResultSize();
-    this.writeBufferSize = connConfiguration.getWriteBufferSize();
-    this.rpcControllerFactory = null;
-    this.rpcCallerFactory = null;
-    this.pool = mutator.getPool();
-    this.locator = null;
-  }
-
-  /**
    * @return maxKeyValueSize from configuration.
    */
   public static int getMaxKeyValueSize(Configuration conf) {
@@ -603,8 +573,21 @@ public class HTable implements Table {
    */
   @Override
   public void put(final Put put) throws IOException {
-    getBufferedMutator().mutate(put);
-    flushCommits();
+    validatePut(put);
+    ClientServiceCallable<Void> callable =
+        new ClientServiceCallable<Void>(this.connection, getName(), put.getRow(),
+            this.rpcControllerFactory.newController(), put.getPriority()) {
+          @Override
+          protected Void rpcCall() throws Exception {
+            MutateRequest request =
+                RequestConverter.buildMutateRequest(getLocation().getRegionInfo().getRegionName(),
+                  put);
+            doMutate(request);
+            return null;
+          }
+        };
+    rpcCallerFactory.<Void> newCaller(this.writeRpcTimeout).callWithRetries(callable,
+      this.operationTimeout);
   }
 
   /**
@@ -613,8 +596,15 @@ public class HTable implements Table {
    */
   @Override
   public void put(final List<Put> puts) throws IOException {
-    getBufferedMutator().mutate(puts);
-    flushCommits();
+    for (Put put : puts) {
+      validatePut(put);
+    }
+    Object[] results = new Object[puts.size()];
+    try {
+      batch(puts, results, writeRpcTimeout);
+    } catch (InterruptedException e) {
+      throw (InterruptedIOException) new InterruptedIOException().initCause(e);
+    }
   }
 
   /**
@@ -948,17 +938,6 @@ public class HTable implements Table {
   }
 
   /**
-   * @throws IOException
-   */
-  void flushCommits() throws IOException {
-    if (mutator == null) {
-      // nothing to flush if there's no mutator; don't bother creating one.
-      return;
-    }
-    getBufferedMutator().flush();
-  }
-
-  /**
    * Process a mixed batch of Get, Put and Delete actions. All actions for a
    * RegionServer are forwarded in one RPC call. Queries are executed in parallel.
    *
@@ -980,11 +959,6 @@ public class HTable implements Table {
     if (this.closed) {
       return;
     }
-    flushCommits();
-    if (mutator != null) {
-      mutator.close();
-      mutator = null;
-    }
     if (cleanupPoolOnClose) {
       this.pool.shutdown();
       try {
@@ -1023,37 +997,6 @@ public class HTable implements Table {
   }
 
   /**
-   * Returns the maximum size in bytes of the write buffer for this HTable.
-   * <p>
-   * The default value comes from the configuration parameter
-   * {@code hbase.client.write.buffer}.
-   * @return The size of the write buffer in bytes.
-   */
-  @Override
-  public long getWriteBufferSize() {
-    if (mutator == null) {
-      return connConfiguration.getWriteBufferSize();
-    } else {
-      return mutator.getWriteBufferSize();
-    }
-  }
-
-  /**
-   * Sets the size of the buffer in bytes.
-   * <p>
-   * If the new size is less than the current amount of data in the
-   * write buffer, the buffer gets flushed.
-   * @param writeBufferSize The new write buffer size, in bytes.
-   * @throws IOException if a remote or network exception occurs.
-   */
-  @Override
-  @Deprecated
-  public void setWriteBufferSize(long writeBufferSize) throws IOException {
-    getBufferedMutator();
-    mutator.setWriteBufferSize(writeBufferSize);
-  }
-
-  /**
    * The pool is used for mutli requests for this HTable
    * @return the pool used for mutli
    */
@@ -1154,9 +1097,6 @@ public class HTable implements Table {
   @Deprecated
   public void setOperationTimeout(int operationTimeout) {
     this.operationTimeout = operationTimeout;
-    if (mutator != null) {
-      mutator.setOperationTimeout(operationTimeout);
-    }
   }
 
   @Override
@@ -1186,9 +1126,6 @@ public class HTable implements Table {
   @Deprecated
   public void setWriteRpcTimeout(int writeRpcTimeout) {
     this.writeRpcTimeout = writeRpcTimeout;
-    if (mutator != null) {
-      mutator.setRpcTimeout(writeRpcTimeout);
-    }
   }
 
   @Override
@@ -1318,19 +1255,4 @@ public class HTable implements Table {
   public RegionLocator getRegionLocator() {
     return this.locator;
   }
-
-  @VisibleForTesting
-  BufferedMutator getBufferedMutator() throws IOException {
-    if (mutator == null) {
-      synchronized (mutatorLock) {
-        if (mutator == null) {
-          this.mutator = (BufferedMutatorImpl) connection.getBufferedMutator(
-            new BufferedMutatorParams(tableName).pool(pool).writeBufferSize(writeBufferSize)
-                .maxKeyValueSize(connConfiguration.getMaxKeyValueSize())
-                .opertationTimeout(operationTimeout).rpcTimeout(writeRpcTimeout));
-        }
-      }
-    }
-    return mutator;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/cabdbf18/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java
index c76c2f5..cfe435e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java
@@ -209,13 +209,8 @@ public interface Table extends Closeable {
   /**
    * Puts some data in the table, in batch.
    * <p>
-   * This can be used for group commit, or for submitting user defined
-   * batches.  The writeBuffer will be periodically inspected while the List
-   * is processed, so depending on the List size the writeBuffer may flush
-   * not at all, or more than once.
-   * @param puts The list of mutations to apply. The batch put is done by
-   * aggregating the iteration of the Puts over the write buffer
-   * at the client-side for a single RPC call.
+   * This can be used for group commit, or for submitting user defined batches.
+   * @param puts The list of mutations to apply.
    * @throws IOException if a remote or network exception occurs.
    * @since 0.20.0
    */
@@ -483,30 +478,6 @@ public interface Table extends Closeable {
     final Batch.Callback<R> callback) throws ServiceException, Throwable;
 
   /**
-   * Returns the maximum size in bytes of the write buffer for this HTable.
-   * <p>
-   * The default value comes from the configuration parameter
-   * {@code hbase.client.write.buffer}.
-   * @return The size of the write buffer in bytes.
-    * @deprecated as of 1.0.1 (should not have been in 1.0.0). Replaced by {@link BufferedMutator#getWriteBufferSize()}
-   */
-  @Deprecated
-  long getWriteBufferSize();
-
-  /**
-   * Sets the size of the buffer in bytes.
-   * <p>
-   * If the new size is less than the current amount of data in the
-   * write buffer, the buffer gets flushed.
-   * @param writeBufferSize The new write buffer size, in bytes.
-   * @throws IOException if a remote or network exception occurs.
-   * @deprecated as of 1.0.1 (should not have been in 1.0.0). Replaced by {@link BufferedMutator} and
-   * {@link BufferedMutatorParams#writeBufferSize(long)}
-   */
-  @Deprecated
-  void setWriteBufferSize(long writeBufferSize) throws IOException;
-
-  /**
    * Creates an instance of the given {@link com.google.protobuf.Service} subclass for each table
    * region spanning the range from the {@code startKey} row to {@code endKey} row (inclusive), all
    * the invocations to the same region server will be batched into one call. The coprocessor

http://git-wip-us.apache.org/repos/asf/hbase/blob/cabdbf18/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableBuilder.java
index 3eedb10..67b8bd2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableBuilder.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableBuilder.java
@@ -57,12 +57,6 @@ public interface TableBuilder {
   TableBuilder setWriteRpcTimeout(int timeout);
 
   /**
-   * Set the write buffer size which by default is specified by the
-   * {@code hbase.client.write.buffer} setting.
-   */
-  TableBuilder setWriteBufferSize(long writeBufferSize);
-
-  /**
    * Create the {@link Table} instance.
    */
   Table build();

http://git-wip-us.apache.org/repos/asf/hbase/blob/cabdbf18/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableBuilderBase.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableBuilderBase.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableBuilderBase.java
index adf1abb..0af4776 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableBuilderBase.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableBuilderBase.java
@@ -36,8 +36,6 @@ abstract class TableBuilderBase implements TableBuilder {
 
   protected int writeRpcTimeout;
 
-  protected long writeBufferSize;
-
   TableBuilderBase(TableName tableName, ConnectionConfiguration connConf) {
     if (tableName == null) {
       throw new IllegalArgumentException("Given table name is null");
@@ -48,7 +46,6 @@ abstract class TableBuilderBase implements TableBuilder {
     this.rpcTimeout = connConf.getRpcTimeout();
     this.readRpcTimeout = connConf.getReadRpcTimeout();
     this.writeRpcTimeout = connConf.getWriteRpcTimeout();
-    this.writeBufferSize = connConf.getWriteBufferSize();
   }
 
   @Override
@@ -74,10 +71,4 @@ abstract class TableBuilderBase implements TableBuilder {
     this.writeRpcTimeout = timeout;
     return this;
   }
-
-  @Override
-  public TableBuilder setWriteBufferSize(long writeBufferSize) {
-    this.writeBufferSize = writeBufferSize;
-    return this;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/cabdbf18/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 6c5c1e4..471ed96 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
@@ -79,7 +79,6 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-
 @Category({ClientTests.class, MediumTests.class})
 public class TestAsyncProcess {
   @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()).
@@ -258,8 +257,6 @@ public class TestAsyncProcess {
 
   }
 
-
-
   static class MyAsyncRequestFutureImpl<Res> extends AsyncRequestFutureImpl<Res> {
     private final Map<ServerName, List<Long>> heapSizesByServer = new HashMap<>();
     public MyAsyncRequestFutureImpl(AsyncProcessTask task, List<Action> actions,
@@ -650,10 +647,9 @@ public class TestAsyncProcess {
 
     MyAsyncProcess ap = new MyAsyncProcess(conn, CONF, true);
     BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE);
-    BufferedMutatorImpl mutator = new BufferedMutatorImpl(conn, bufferParam, ap);
-    try (HTable ht = new HTable(conn, mutator)) {
-      Assert.assertEquals(0L, ht.mutator.getCurrentWriteBufferSize());
-      ht.put(puts);
+    try (BufferedMutatorImpl mutator = new BufferedMutatorImpl(conn, bufferParam, ap);) {
+      mutator.mutate(puts);
+      mutator.flush();
       List<AsyncRequestFuture> reqs = ap.allReqs;
 
       int actualSnReqCount = 0;
@@ -1095,54 +1091,6 @@ public class TestAsyncProcess {
     assertFalse(ap.service.isShutdown());
   }
 
-  private void doHTableFailedPut(boolean bufferOn) throws Exception {
-    ClusterConnection conn = createHConnection();
-    MyAsyncProcess ap = new MyAsyncProcess(conn, CONF, true);
-    BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE);
-    if (bufferOn) {
-      bufferParam.writeBufferSize(1024L * 1024L);
-    } else {
-      bufferParam.writeBufferSize(0L);
-    }
-    BufferedMutatorImpl mutator = new BufferedMutatorImpl(conn, bufferParam, ap);
-    HTable ht = new HTable(conn, mutator);
-
-    Put put = createPut(1, false);
-
-    Assert.assertEquals(0L, ht.mutator.getCurrentWriteBufferSize());
-    try {
-      ht.put(put);
-      if (bufferOn) {
-        ht.flushCommits();
-      }
-      Assert.fail();
-    } catch (RetriesExhaustedException expected) {
-    }
-    Assert.assertEquals(0L, ht.mutator.getCurrentWriteBufferSize());
-    // The table should have sent one request, maybe after multiple attempts
-    AsyncRequestFuture ars = null;
-    for (AsyncRequestFuture someReqs : ap.allReqs) {
-      if (someReqs.getResults().length == 0) continue;
-      Assert.assertTrue(ars == null);
-      ars = someReqs;
-    }
-    Assert.assertTrue(ars != null);
-    verifyResult(ars, false);
-
-    // This should not raise any exception, puts have been 'received' before by the catch.
-    ht.close();
-  }
-
-  @Test
-  public void testHTableFailedPutWithBuffer() throws Exception {
-    doHTableFailedPut(true);
-  }
-
-  @Test
-  public void testHTableFailedPutWithoutBuffer() throws Exception {
-    doHTableFailedPut(false);
-  }
-
   @Test
   public void testHTableFailedPutAndNewPut() throws Exception {
     ClusterConnection conn = createHConnection();
@@ -1193,10 +1141,7 @@ public class TestAsyncProcess {
   @Test
   public void testBatch() throws IOException, InterruptedException {
     ClusterConnection conn = new MyConnectionImpl(CONF);
-    MyAsyncProcess ap = new MyAsyncProcess(conn, CONF, true);
-    BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE);
-    BufferedMutatorImpl mutator = new BufferedMutatorImpl(conn, bufferParam, ap);
-    HTable ht = new HTable(conn, mutator);
+    HTable ht = (HTable) conn.getTable(DUMMY_TABLE);
     ht.multiAp = new MyAsyncProcess(conn, CONF, false);
 
     List<Put> puts = new ArrayList<>(7);
@@ -1258,9 +1203,7 @@ public class TestAsyncProcess {
     ClusterConnection conn = createHConnection();
     Mockito.when(conn.getConfiguration()).thenReturn(copyConf);
     MyAsyncProcess ap = new MyAsyncProcess(conn, copyConf, true);
-    BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE);
-    BufferedMutatorImpl mutator = new BufferedMutatorImpl(conn, bufferParam, ap);
-    try (HTable ht = new HTable(conn, mutator)) {
+    try (HTable ht = (HTable) conn.getTable(DUMMY_TABLE)) {
       ht.multiAp = ap;
       List<Get> gets = new LinkedList<>();
       gets.add(new Get(DUMMY_BYTES_1));
@@ -1350,9 +1293,7 @@ public class TestAsyncProcess {
 
     MyConnectionImpl2 con = new MyConnectionImpl2(hrls);
     MyAsyncProcess ap = new MyAsyncProcess(con, CONF, con.nbThreads);
-    BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE);
-    BufferedMutatorImpl mutator = new BufferedMutatorImpl(con , bufferParam, ap);
-    HTable ht = new HTable(con, mutator);
+    HTable ht = (HTable) con.getTable(DUMMY_TABLE, ap.service);
     ht.multiAp = ap;
     ht.batch(gets, null);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/cabdbf18/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java
index 63dfcaa..aeee96e 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java
@@ -822,16 +822,6 @@ public class RemoteHTable implements Table {
   }
 
   @Override
-  public long getWriteBufferSize() {
-    throw new UnsupportedOperationException("getWriteBufferSize not implemented");
-  }
-
-  @Override
-  public void setWriteBufferSize(long writeBufferSize) throws IOException {
-    throw new IOException("setWriteBufferSize not supported");
-  }
-
-  @Override
   public <R extends Message> Map<byte[], R> batchCoprocessorService(
       Descriptors.MethodDescriptor method, Message request,
       byte[] startKey, byte[] endKey, R responsePrototype) throws ServiceException, Throwable {

http://git-wip-us.apache.org/repos/asf/hbase/blob/cabdbf18/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableWrapper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableWrapper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableWrapper.java
index 8824872..14e4271 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableWrapper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableWrapper.java
@@ -271,16 +271,6 @@ public final class HTableWrapper implements Table {
   }
 
   @Override
-  public long getWriteBufferSize() {
-     return table.getWriteBufferSize();
-  }
-
-  @Override
-  public void setWriteBufferSize(long writeBufferSize) throws IOException {
-    table.setWriteBufferSize(writeBufferSize);
-  }
-
-  @Override
   public <R extends Message> Map<byte[], R> batchCoprocessorService(
       MethodDescriptor methodDescriptor, Message request, byte[] startKey, byte[] endKey,
       R responsePrototype) throws ServiceException, Throwable {

http://git-wip-us.apache.org/repos/asf/hbase/blob/cabdbf18/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 dfadbb9..12bdc22 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
@@ -28,6 +28,7 @@ import java.io.DataInputStream;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -200,7 +201,20 @@ public class AccessControlLists {
           );
     }
     try {
-      t.put(p);
+      /**
+       * TODO: Use Table.put(Put) instead. This Table.put() happens within the RS. We are already in
+       * AccessController. Means already there was an RPC happened to server (Actual grant call from
+       * client side). At RpcServer we have a ThreadLocal where we keep the CallContext and inside
+       * that the current RPC called user info is set. The table on which put was called is created
+       * via the RegionCP env and that uses a special Connection. The normal RPC channel will be by
+       * passed here means there would have no further contact on to the RpcServer. So the
+       * ThreadLocal is never getting reset. We ran the new put as a super user (User.runAsLoginUser
+       * where the login user is the user who started RS process) but still as per the RPC context
+       * it is the old user. When AsyncProcess was used, the execute happen via another thread from
+       * pool and so old ThreadLocal variable is not accessible and so it looks as if no Rpc context
+       * and we were relying on the super user who starts the RS process.
+       */
+      t.put(Collections.singletonList(p));
     } finally {
       t.close();
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/cabdbf18/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
index d6f0af9..162e761 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
@@ -1231,6 +1231,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
 
   static abstract class BufferedMutatorTest extends Test {
     protected BufferedMutator mutator;
+    protected Table table;
 
     BufferedMutatorTest(Connection con, TestOptions options, Status status) {
       super(con, options, status);
@@ -1239,11 +1240,13 @@ public class PerformanceEvaluation extends Configured implements Tool {
     @Override
     void onStartup() throws IOException {
       this.mutator = connection.getBufferedMutator(TableName.valueOf(opts.tableName));
+      this.table = connection.getTable(TableName.valueOf(opts.tableName));
     }
 
     @Override
     void onTakedown() throws IOException {
       mutator.close();
+      table.close();
     }
   }
 
@@ -1465,9 +1468,10 @@ public class PerformanceEvaluation extends Configured implements Tool {
         }
       }
       put.setDurability(opts.writeToWAL ? Durability.SYNC_WAL : Durability.SKIP_WAL);
-      mutator.mutate(put);
       if (opts.autoFlush) {
-        mutator.flush();
+        table.put(put);
+      } else {
+        mutator.mutate(put);
       }
     }
   }
@@ -1666,9 +1670,10 @@ public class PerformanceEvaluation extends Configured implements Tool {
         }
       }
       put.setDurability(opts.writeToWAL ? Durability.SYNC_WAL : Durability.SKIP_WAL);
-      mutator.mutate(put);
       if (opts.autoFlush) {
-        mutator.flush();
+        table.put(put);
+      } else {
+        mutator.mutate(put);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/cabdbf18/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientPushback.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientPushback.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientPushback.java
index 2c5e89d..cafb2f6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientPushback.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientPushback.java
@@ -93,7 +93,7 @@ public class TestClientPushback {
     Configuration conf = UTIL.getConfiguration();
 
     ClusterConnection conn = (ClusterConnection) ConnectionFactory.createConnection(conf);
-    Table table = conn.getTable(tableName);
+    BufferedMutatorImpl mutator = (BufferedMutatorImpl) conn.getBufferedMutator(tableName);
 
     HRegionServer rs = UTIL.getHBaseCluster().getRegionServer(0);
     Region region = rs.getOnlineRegions(tableName).get(0);
@@ -102,7 +102,8 @@ public class TestClientPushback {
     // write some data
     Put p = new Put(Bytes.toBytes("row"));
     p.addColumn(family, qualifier, Bytes.toBytes("value1"));
-    table.put(p);
+    mutator.mutate(p);
+    mutator.flush();
 
     // get the current load on RS. Hopefully memstore isn't flushed since we wrote the the data
     int load = (int) ((((HRegion) region).addAndGetMemstoreSize(new MemstoreSize(0, 0)) * 100)
@@ -138,7 +139,6 @@ public class TestClientPushback {
     final CountDownLatch latch = new CountDownLatch(1);
     final AtomicLong endTime = new AtomicLong();
     long startTime = EnvironmentEdgeManager.currentTime();
-    BufferedMutatorImpl mutator = ((HTable) table).mutator;
     Batch.Callback<Result> callback = (byte[] r, byte[] row, Result result) -> {
         endTime.set(EnvironmentEdgeManager.currentTime());
         latch.countDown();

http://git-wip-us.apache.org/repos/asf/hbase/blob/cabdbf18/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
index 8a3841e..fcda723 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
@@ -4068,8 +4068,8 @@ public class TestFromClientSide {
       Put p = new Put(ROW);
       p.addColumn(BAD_FAM, QUALIFIER, VAL);
       table.put(p);
-    } catch (RetriesExhaustedWithDetailsException e) {
-      caughtNSCFE = e.getCause(0) instanceof NoSuchColumnFamilyException;
+    } catch (Exception e) {
+      caughtNSCFE = e instanceof NoSuchColumnFamilyException;
     }
     assertTrue("Should throw NoSuchColumnFamilyException", caughtNSCFE);
 
@@ -4110,7 +4110,6 @@ public class TestFromClientSide {
     final int NB_BATCH_ROWS = 10;
     Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()),
         new byte[][] { CONTENTS_FAMILY, SMALL_FAMILY });
-    table.setWriteBufferSize(10);
     ArrayList<Put> rowsUpdate = new ArrayList<Put>();
     for (int i = 0; i < NB_BATCH_ROWS * 10; i++) {
       byte[] row = Bytes.toBytes("row" + i);

http://git-wip-us.apache.org/repos/asf/hbase/blob/cabdbf18/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 1ef6c60..1f19cf2 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
@@ -85,7 +85,6 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 import org.junit.rules.TestRule;
-
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
 
 /**
@@ -442,7 +441,7 @@ public class TestHCM {
       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) {
+    } catch (SocketTimeoutException 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.

http://git-wip-us.apache.org/repos/asf/hbase/blob/cabdbf18/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
index 0454ed8..a3c9649 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
@@ -266,7 +266,6 @@ public class TestMultiParallel {
     // Load the data
     LOG.info("get new table");
     Table table = UTIL.getConnection().getTable(TEST_TABLE);
-    table.setWriteBufferSize(10 * 1024 * 1024);
 
     LOG.info("constructPutRequests");
     List<Put> puts = constructPutRequests();

http://git-wip-us.apache.org/repos/asf/hbase/blob/cabdbf18/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestServerBusyException.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestServerBusyException.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestServerBusyException.java
index c318ffc..dde7018 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestServerBusyException.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestServerBusyException.java
@@ -141,12 +141,10 @@ public class TestServerBusyException {
     public void run() {
       try {
         Put p = new Put(ROW);
-        p.addColumn(FAM_NAM, new byte[]{0}, new byte[]{0});
+        p.addColumn(FAM_NAM, new byte[] { 0 }, new byte[] { 0 });
         table.put(p);
-      } catch (RetriesExhaustedWithDetailsException e) {
-        if (e.exceptions.get(0) instanceof ServerTooBusyException) {
-          getServerBusyException = 1;
-        }
+      } catch (ServerTooBusyException e) {
+        getServerBusyException = 1;
       } catch (IOException ignore) {
       }
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/cabdbf18/hbase-server/src/test/java/org/apache/hadoop/hbase/constraint/TestConstraint.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/constraint/TestConstraint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/constraint/TestConstraint.java
index c3f6eed..c2a16af 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/constraint/TestConstraint.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/constraint/TestConstraint.java
@@ -124,13 +124,8 @@ public class TestConstraint {
     try {
       table.put(put);
       fail("This put should not have suceeded - AllFailConstraint was not run!");
-    } catch (RetriesExhaustedWithDetailsException e) {
-      List<Throwable> causes = e.getCauses();
-      assertEquals(
-          "More than one failure cause - should only be the failure constraint exception",
-          1, causes.size());
-      Throwable t = causes.get(0);
-      assertEquals(ConstraintException.class, t.getClass());
+    } catch (ConstraintException e) {
+      // expected
     }
     table.close();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/cabdbf18/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestHTableWrapper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestHTableWrapper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestHTableWrapper.java
index 4f80876..326b3c0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestHTableWrapper.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestHTableWrapper.java
@@ -148,7 +148,6 @@ public class TestHTableWrapper {
   private void checkHTableInterfaceMethods() throws Exception {
     checkConf();
     checkNameAndDescriptor();
-    checkBufferSize();
     checkExists();
     checkAppend();
     checkPutsAndDeletes();
@@ -175,13 +174,6 @@ public class TestHTableWrapper {
     assertEquals(table.getTableDescriptor(), hTableInterface.getTableDescriptor());
   }
 
-  private void checkBufferSize() throws IOException {
-    long initialWriteBufferSize = hTableInterface.getWriteBufferSize();
-    hTableInterface.setWriteBufferSize(12345L);
-    assertEquals(12345L, hTableInterface.getWriteBufferSize());
-    hTableInterface.setWriteBufferSize(initialWriteBufferSize);
-  }
-
   private void checkExists() throws IOException {
     boolean ex = hTableInterface.exists(new Get(ROW_A).addColumn(TEST_FAMILY, qualifierCol1));
     assertTrue(ex);

http://git-wip-us.apache.org/repos/asf/hbase/blob/cabdbf18/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/RegionAsTable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/RegionAsTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/RegionAsTable.java
index ea1d4c0..89885b6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/RegionAsTable.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/RegionAsTable.java
@@ -303,16 +303,6 @@ public class RegionAsTable implements Table {
   }
 
   @Override
-  public long getWriteBufferSize() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void setWriteBufferSize(long writeBufferSize) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
   public <R extends Message> Map<byte[], R> batchCoprocessorService(MethodDescriptor
       methodDescriptor, Message request,
       byte[] startKey, byte[] endKey, R responsePrototype)

http://git-wip-us.apache.org/repos/asf/hbase/blob/cabdbf18/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 a6f79b3..0fe79d1 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
@@ -558,7 +558,6 @@ public class TestMasterReplication {
     Table[] htables = new Table[numClusters];
     for (int i = 0; i < numClusters; i++) {
       Table htable = ConnectionFactory.createConnection(configurations[i]).getTable(tableName);
-      htable.setWriteBufferSize(1024);
       htables[i] = htable;
     }
     return htables;

http://git-wip-us.apache.org/repos/asf/hbase/blob/cabdbf18/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 4978b17..a9896ce 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
@@ -140,11 +140,8 @@ public class TestMultiSlaveReplication {
     utility2.getAdmin().createTable(table);
     utility3.getAdmin().createTable(table);
     Table htable1 = utility1.getConnection().getTable(tableName);
-    htable1.setWriteBufferSize(1024);
     Table htable2 = utility2.getConnection().getTable(tableName);
-    htable2.setWriteBufferSize(1024);
     Table htable3 = utility3.getConnection().getTable(tableName);
-    htable3.setWriteBufferSize(1024);
 
     ReplicationPeerConfig rpc = new ReplicationPeerConfig();
     rpc.setClusterKey(utility2.getClusterKey());

http://git-wip-us.apache.org/repos/asf/hbase/blob/cabdbf18/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 9cf80d4..87918ee 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
@@ -160,7 +160,6 @@ public class TestReplicationBase {
     utility1.waitUntilAllRegionsAssigned(tableName);
     utility2.waitUntilAllRegionsAssigned(tableName);
     htable1 = connection1.getTable(tableName);
-    htable1.setWriteBufferSize(1024);
     htable2 = connection2.getTable(tableName);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/cabdbf18/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 a3c5526..e1cb8ba 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
@@ -426,7 +426,6 @@ public class TestReplicationSmallTests extends TestReplicationBase {
       put.addColumn(famName, row, row);
       puts.add(put);
     }
-    htable1.setWriteBufferSize(1024);
     // The puts will be iterated through and flushed only when the buffer
     // size is reached.
     htable1.put(puts);

http://git-wip-us.apache.org/repos/asf/hbase/blob/cabdbf18/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 9ec9b99..edfff9a 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
@@ -198,15 +198,11 @@ public class TestReplicationSyncUpTool extends TestReplicationBase {
 
     // Get HTable from Master
     ht1Source = connection1.getTable(t1_su);
-    ht1Source.setWriteBufferSize(1024);
     ht2Source = connection1.getTable(t2_su);
-    ht1Source.setWriteBufferSize(1024);
 
     // Get HTable from Peer1
     ht1TargetAtPeer1 = connection2.getTable(t1_su);
-    ht1TargetAtPeer1.setWriteBufferSize(1024);
     ht2TargetAtPeer1 = connection2.getTable(t2_su);
-    ht2TargetAtPeer1.setWriteBufferSize(1024);
 
     /**
      * set M-S : Master: utility1 Slave1: utility2

http://git-wip-us.apache.org/repos/asf/hbase/blob/cabdbf18/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
index 5627016..18290f5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
@@ -64,9 +64,9 @@ import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.access.Permission.Action;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
-
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
+
 import com.google.protobuf.BlockingRpcChannel;
 import com.google.protobuf.ServiceException;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/cabdbf18/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java
index 7889138..27128cc 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java
@@ -145,6 +145,7 @@ public class TestNamespaceCommands extends SecureTestUtil {
         User.createUserForTesting(conf, "user_group_write", new String[] { GROUP_WRITE });
     // TODO: other table perms
 
+    UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 2);
     UTIL.startMiniCluster();
     // Wait for the ACL table to become available
     UTIL.waitTableAvailable(AccessControlLists.ACL_TABLE_NAME.getName(), 30 * 1000);


[24/50] [abbrv] hbase git commit: HBASE-18593 Tell m2eclipse what to do w/ replacer plugin

Posted by st...@apache.org.
HBASE-18593 Tell m2eclipse what to do w/ replacer plugin


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

Branch: refs/heads/HBASE-14070.HLC
Commit: bd40073094b248f74ac9a3c0fff7ef6668265feb
Parents: 424dff2
Author: Michael Stack <st...@apache.org>
Authored: Mon Aug 14 09:13:27 2017 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Mon Aug 14 09:13:27 2017 -0700

----------------------------------------------------------------------
 hbase-protocol-shaded/pom.xml | 17 +++++++++++++++++
 1 file changed, 17 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/bd400730/hbase-protocol-shaded/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/pom.xml b/hbase-protocol-shaded/pom.xml
index b28c03e..4c72eca 100644
--- a/hbase-protocol-shaded/pom.xml
+++ b/hbase-protocol-shaded/pom.xml
@@ -192,6 +192,23 @@
                     <ignore/>
                   </action>
                 </pluginExecution>
+                <pluginExecution>
+                  <pluginExecutionFilter>
+                    <groupId>
+                      com.google.code.maven-replacer-plugin
+                    </groupId>
+                    <artifactId>replacer</artifactId>
+                    <versionRange>[1.5.3,)</versionRange>
+                    <goals>
+                      <goal>replace</goal>
+                    </goals>
+                  </pluginExecutionFilter>
+                  <action>
+                    <execute>
+                     <runOnIncremental>false</runOnIncremental>
+                    </execute>
+                  </action>
+                </pluginExecution>
               </pluginExecutions>
             </lifecycleMappingMetadata>
           </configuration>


[41/50] [abbrv] hbase git commit: HBASE-17994 Add async client test to Performance Evaluation tool

Posted by st...@apache.org.
HBASE-17994 Add async client test to Performance Evaluation tool


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

Branch: refs/heads/HBASE-14070.HLC
Commit: 2a9cdd5e75ab6e51e9ca0539caba29cc4350b98e
Parents: 9da4e69
Author: Guanghao Zhang <zg...@apache.org>
Authored: Wed Jul 12 15:37:55 2017 +0800
Committer: Guanghao Zhang <zg...@apache.org>
Committed: Wed Aug 16 14:05:07 2017 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/PerformanceEvaluation.java     | 390 +++++++++++++++++--
 1 file changed, 351 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2a9cdd5e/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
index 162e761..eebb0f3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
@@ -53,6 +53,8 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Append;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.AsyncTable;
 import org.apache.hadoop.hbase.client.BufferedMutator;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
@@ -62,6 +64,7 @@ import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Increment;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RawAsyncTable;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.RowMutations;
@@ -99,9 +102,9 @@ import org.apache.htrace.Sampler;
 import org.apache.htrace.Trace;
 import org.apache.htrace.TraceScope;
 import org.apache.htrace.impl.ProbabilitySampler;
-
 import org.apache.hadoop.hbase.shaded.com.google.common.base.MoreObjects;
 import org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.ThreadFactoryBuilder;
+
 import com.codahale.metrics.Histogram;
 import com.codahale.metrics.UniformReservoir;
 
@@ -153,6 +156,16 @@ public class PerformanceEvaluation extends Configured implements Tool {
   private static final Path PERF_EVAL_DIR = new Path("performance_evaluation");
 
   static {
+    addCommandDescriptor(AsyncRandomReadTest.class, "asyncRandomRead",
+        "Run async random read test");
+    addCommandDescriptor(AsyncRandomWriteTest.class, "asyncRandomWrite",
+        "Run async random write test");
+    addCommandDescriptor(AsyncSequentialReadTest.class, "asyncSequentialRead",
+        "Run async sequential read test");
+    addCommandDescriptor(AsyncSequentialWriteTest.class, "asyncSequentialWrite",
+        "Run async sequential write test");
+    addCommandDescriptor(AsyncScanTest.class, "asyncScan",
+        "Run async scan test (read every row)");
     addCommandDescriptor(RandomReadTest.class, RANDOM_READ,
       "Run random read test");
     addCommandDescriptor(RandomSeekScanTest.class, RANDOM_SEEK_SCAN,
@@ -226,7 +239,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
     super(conf);
   }
 
-  protected static void addCommandDescriptor(Class<? extends Test> cmdClass,
+  protected static void addCommandDescriptor(Class<? extends TestBase> cmdClass,
       String name, String description) {
     CmdDescriptor cmdDescriptor = new CmdDescriptor(cmdClass, name, description);
     COMMANDS.put(name, cmdDescriptor);
@@ -295,9 +308,15 @@ public class PerformanceEvaluation extends Configured implements Tool {
       TestOptions opts = mapper.readValue(value.toString(), TestOptions.class);
       Configuration conf = HBaseConfiguration.create(context.getConfiguration());
       final Connection con = ConnectionFactory.createConnection(conf);
+      AsyncConnection asyncCon = null;
+      try {
+        asyncCon = ConnectionFactory.createAsyncConnection(conf).get();
+      } catch (ExecutionException e) {
+        throw new IOException(e);
+      }
 
       // Evaluation task
-      RunResult result = PerformanceEvaluation.runOneClient(this.cmd, conf, con, opts, status);
+      RunResult result = PerformanceEvaluation.runOneClient(this.cmd, conf, con, asyncCon, opts, status);
       // Collect how much time the thing took. Report as map output and
       // to the ELAPSED_TIME counter.
       context.getCounter(Counter.ELAPSED_TIME).increment(result.duration);
@@ -412,8 +431,8 @@ public class PerformanceEvaluation extends Configured implements Tool {
    * Run all clients in this vm each to its own thread.
    */
   static RunResult[] doLocalClients(final TestOptions opts, final Configuration conf)
-      throws IOException, InterruptedException {
-    final Class<? extends Test> cmd = determineCommandClass(opts.cmdName);
+      throws IOException, InterruptedException, ExecutionException {
+    final Class<? extends TestBase> cmd = determineCommandClass(opts.cmdName);
     assert cmd != null;
     @SuppressWarnings("unchecked")
     Future<RunResult>[] threads = new Future[opts.numClientThreads];
@@ -421,6 +440,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
     ExecutorService pool = Executors.newFixedThreadPool(opts.numClientThreads,
       new ThreadFactoryBuilder().setNameFormat("TestClient-%s").build());
     final Connection con = ConnectionFactory.createConnection(conf);
+    final AsyncConnection asyncCon = ConnectionFactory.createAsyncConnection(conf).get();
     for (int i = 0; i < threads.length; i++) {
       final int index = i;
       threads[i] = pool.submit(new Callable<RunResult>() {
@@ -428,7 +448,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
         public RunResult call() throws Exception {
           TestOptions threadOpts = new TestOptions(opts);
           if (threadOpts.startRow == 0) threadOpts.startRow = index * threadOpts.perClientRunRows;
-          RunResult run = runOneClient(cmd, conf, con, threadOpts, new Status() {
+          RunResult run = runOneClient(cmd, conf, con, asyncCon, threadOpts, new Status() {
             @Override
             public void setStatus(final String msg) throws IOException {
               LOG.info(msg);
@@ -463,6 +483,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
       + "\tAvg: " + (total / results.length) + "ms");
 
     con.close();
+    asyncCon.close();
 
     return results;
   }
@@ -476,7 +497,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
    */
   static Job doMapReduce(TestOptions opts, final Configuration conf)
       throws IOException, InterruptedException, ClassNotFoundException {
-    final Class<? extends Test> cmd = determineCommandClass(opts.cmdName);
+    final Class<? extends TestBase> cmd = determineCommandClass(opts.cmdName);
     assert cmd != null;
     Path inputDir = writeInputFile(conf, opts);
     conf.set(EvaluationMapTask.CMD_KEY, cmd.getName());
@@ -567,17 +588,17 @@ public class PerformanceEvaluation extends Configured implements Tool {
    * Describes a command.
    */
   static class CmdDescriptor {
-    private Class<? extends Test> cmdClass;
+    private Class<? extends TestBase> cmdClass;
     private String name;
     private String description;
 
-    CmdDescriptor(Class<? extends Test> cmdClass, String name, String description) {
+    CmdDescriptor(Class<? extends TestBase> cmdClass, String name, String description) {
       this.cmdClass = cmdClass;
       this.name = name;
       this.description = description;
     }
 
-    public Class<? extends Test> getCmdClass() {
+    public Class<? extends TestBase> getCmdClass() {
       return cmdClass;
     }
 
@@ -1001,7 +1022,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
    * A test.
    * Subclass to particularize what happens per row.
    */
-  static abstract class Test {
+  static abstract class TestBase {
     // Below is make it so when Tests are all running in the one
     // jvm, that they each have a differently seeded Random.
     private static final Random randomSeed = new Random(System.currentTimeMillis());
@@ -1018,8 +1039,6 @@ public class PerformanceEvaluation extends Configured implements Tool {
     private final Status status;
     private final Sampler<?> traceSampler;
     private final SpanReceiverHost receiverHost;
-    protected Connection connection;
-//    protected Table table;
 
     private String testName;
     private Histogram latencyHistogram;
@@ -1030,9 +1049,8 @@ public class PerformanceEvaluation extends Configured implements Tool {
      * Note that all subclasses of this class must provide a public constructor
      * that has the exact same list of arguments.
      */
-    Test(final Connection con, final TestOptions options, final Status status) {
-      this.connection = con;
-      this.conf = con == null ? HBaseConfiguration.create() : this.connection.getConfiguration();
+    TestBase(final Configuration conf, final TestOptions options, final Status status) {
+      this.conf = conf;
       this.receiverHost = this.conf == null? null: SpanReceiverHost.getInstance(conf);
       this.opts = options;
       this.status = status;
@@ -1098,14 +1116,14 @@ public class PerformanceEvaluation extends Configured implements Tool {
     }
 
     void testSetup() throws IOException {
-      if (!opts.oneCon) {
-        this.connection = ConnectionFactory.createConnection(conf);
-      }
+      createConnection();
       onStartup();
       latencyHistogram = YammerHistogramUtils.newHistogram(new UniformReservoir(1024 * 500));
       valueSizeHistogram = YammerHistogramUtils.newHistogram(new UniformReservoir(1024 * 500));
     }
 
+    abstract void createConnection() throws IOException;
+
     abstract void onStartup() throws IOException;
 
     void testTakedown() throws IOException {
@@ -1124,14 +1142,14 @@ public class PerformanceEvaluation extends Configured implements Tool {
         status.setStatus("Num measures (ValueSize): " + valueSizeHistogram.getCount());
         status.setStatus(YammerHistogramUtils.getPrettyHistogramReport(valueSizeHistogram));
       }
-      if (!opts.oneCon) {
-        connection.close();
-      }
+      closeConnection();
       receiverHost.closeReceivers();
     }
 
     abstract void onTakedown() throws IOException;
 
+    abstract void closeConnection() throws IOException;
+
     /*
      * Run test
      * @return Elapsed time.
@@ -1211,6 +1229,56 @@ public class PerformanceEvaluation extends Configured implements Tool {
     abstract void testRow(final int i) throws IOException, InterruptedException;
   }
 
+  static abstract class Test extends TestBase {
+    protected Connection connection;
+
+    Test(final Connection con, final TestOptions options, final Status status) {
+      super(con == null ? HBaseConfiguration.create() : con.getConfiguration(), options, status);
+      this.connection = con;
+    }
+
+    @Override
+    void createConnection() throws IOException {
+      if (!opts.isOneCon()) {
+        this.connection = ConnectionFactory.createConnection(conf);
+      }
+    }
+
+    @Override
+    void closeConnection() throws IOException {
+      if (!opts.isOneCon()) {
+        this.connection.close();
+      }
+    }
+  }
+
+  static abstract class AsyncTest extends TestBase {
+    protected AsyncConnection connection;
+
+    AsyncTest(final AsyncConnection con, final TestOptions options, final Status status) {
+      super(con == null ? HBaseConfiguration.create() : con.getConfiguration(), options, status);
+      this.connection = con;
+    }
+
+    @Override
+    void createConnection() {
+      if (!opts.isOneCon()) {
+        try {
+          this.connection = ConnectionFactory.createAsyncConnection(conf).get();
+        } catch (InterruptedException | ExecutionException e) {
+          LOG.error("Failed to create async connection", e);
+        }
+      }
+    }
+
+    @Override
+    void closeConnection() throws IOException {
+      if (!opts.isOneCon()) {
+        this.connection.close();
+      }
+    }
+  }
+
   static abstract class TableTest extends Test {
     protected Table table;
 
@@ -1229,6 +1297,242 @@ public class PerformanceEvaluation extends Configured implements Tool {
     }
   }
 
+  static abstract class AsyncTableTest extends AsyncTest {
+    protected RawAsyncTable table;
+
+    AsyncTableTest(AsyncConnection con, TestOptions options, Status status) {
+      super(con, options, status);
+    }
+
+    @Override
+    void onStartup() throws IOException {
+      this.table = connection.getRawTable(TableName.valueOf(opts.tableName));
+    }
+
+    @Override
+    void onTakedown() throws IOException {
+    }
+  }
+
+  static class AsyncRandomReadTest extends AsyncTableTest {
+    private final Consistency consistency;
+    private ArrayList<Get> gets;
+    private Random rd = new Random();
+
+    AsyncRandomReadTest(AsyncConnection con, TestOptions options, Status status) {
+      super(con, options, status);
+      consistency = options.replicas == DEFAULT_OPTS.replicas ? null : Consistency.TIMELINE;
+      if (opts.multiGet > 0) {
+        LOG.info("MultiGet enabled. Sending GETs in batches of " + opts.multiGet + ".");
+        this.gets = new ArrayList<>(opts.multiGet);
+      }
+    }
+
+    @Override
+    void testRow(final int i) throws IOException, InterruptedException {
+      if (opts.randomSleep > 0) {
+        Thread.sleep(rd.nextInt(opts.randomSleep));
+      }
+      Get get = new Get(getRandomRow(this.rand, opts.totalRows));
+      if (opts.addColumns) {
+        get.addColumn(FAMILY_NAME, QUALIFIER_NAME);
+      } else {
+        get.addFamily(FAMILY_NAME);
+      }
+      if (opts.filterAll) {
+        get.setFilter(new FilterAllFilter());
+      }
+      get.setConsistency(consistency);
+      if (LOG.isTraceEnabled()) LOG.trace(get.toString());
+      try {
+        if (opts.multiGet > 0) {
+          this.gets.add(get);
+          if (this.gets.size() == opts.multiGet) {
+            Result[] rs =
+                this.table.get(this.gets).stream().map(f -> propagate(f::get)).toArray(Result[]::new);
+            updateValueSize(rs);
+            this.gets.clear();
+          }
+        } else {
+          updateValueSize(this.table.get(get).get());
+        }
+      } catch (ExecutionException e) {
+        throw new IOException(e);
+      }
+    }
+
+    public static RuntimeException runtime(Throwable e) {
+      if (e instanceof RuntimeException) {
+        return (RuntimeException) e;
+      }
+      return new RuntimeException(e);
+    }
+
+    public static <V> V propagate(Callable<V> callable) {
+      try {
+        return callable.call();
+      } catch (Exception e) {
+        throw runtime(e);
+      }
+    }
+
+    @Override
+    protected int getReportingPeriod() {
+      int period = opts.perClientRunRows / 10;
+      return period == 0 ? opts.perClientRunRows : period;
+    }
+
+    @Override
+    protected void testTakedown() throws IOException {
+      if (this.gets != null && this.gets.size() > 0) {
+        this.table.get(gets);
+        this.gets.clear();
+      }
+      super.testTakedown();
+    }
+  }
+
+  static class AsyncRandomWriteTest extends AsyncTableTest {
+    AsyncRandomWriteTest(AsyncConnection con, TestOptions options, Status status) {
+      super(con, options, status);
+    }
+
+    @Override
+    void testRow(final int i) throws IOException, InterruptedException {
+      byte[] row = getRandomRow(this.rand, opts.totalRows);
+      Put put = new Put(row);
+      for (int column = 0; column < opts.columns; column++) {
+        byte[] qualifier = column == 0 ? COLUMN_ZERO : Bytes.toBytes("" + column);
+        byte[] value = generateData(this.rand, getValueLength(this.rand));
+        if (opts.useTags) {
+          byte[] tag = generateData(this.rand, TAG_LENGTH);
+          Tag[] tags = new Tag[opts.noOfTags];
+          for (int n = 0; n < opts.noOfTags; n++) {
+            Tag t = new ArrayBackedTag((byte) n, tag);
+            tags[n] = t;
+          }
+          KeyValue kv =
+              new KeyValue(row, FAMILY_NAME, qualifier, HConstants.LATEST_TIMESTAMP, value, tags);
+          put.add(kv);
+          updateValueSize(kv.getValueLength());
+        } else {
+          put.addColumn(FAMILY_NAME, qualifier, value);
+          updateValueSize(value.length);
+        }
+      }
+      put.setDurability(opts.writeToWAL ? Durability.SYNC_WAL : Durability.SKIP_WAL);
+      try {
+        table.put(put).get();
+      } catch (ExecutionException e) {
+        throw new IOException(e);
+      }
+    }
+  }
+
+  static class AsyncScanTest extends AsyncTableTest {
+    private ResultScanner testScanner;
+    private AsyncTable asyncTable;
+
+    AsyncScanTest(AsyncConnection con, TestOptions options, Status status) {
+      super(con, options, status);
+    }
+
+    @Override
+    void onStartup() throws IOException {
+      this.asyncTable =
+          connection.getTable(TableName.valueOf(opts.tableName),
+            Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors()));
+    }
+
+    @Override
+    void testTakedown() throws IOException {
+      if (this.testScanner != null) {
+        this.testScanner.close();
+      }
+      super.testTakedown();
+    }
+
+    @Override
+    void testRow(final int i) throws IOException {
+      if (this.testScanner == null) {
+        Scan scan =
+            new Scan().withStartRow(format(opts.startRow)).setCaching(opts.caching)
+                .setCacheBlocks(opts.cacheBlocks).setAsyncPrefetch(opts.asyncPrefetch)
+                .setReadType(opts.scanReadType);
+        if (opts.addColumns) {
+          scan.addColumn(FAMILY_NAME, QUALIFIER_NAME);
+        } else {
+          scan.addFamily(FAMILY_NAME);
+        }
+        if (opts.filterAll) {
+          scan.setFilter(new FilterAllFilter());
+        }
+        this.testScanner = asyncTable.getScanner(scan);
+      }
+      Result r = testScanner.next();
+      updateValueSize(r);
+    }
+  }
+
+  static class AsyncSequentialReadTest extends AsyncTableTest {
+    AsyncSequentialReadTest(AsyncConnection con, TestOptions options, Status status) {
+      super(con, options, status);
+    }
+
+    @Override
+    void testRow(final int i) throws IOException, InterruptedException {
+      Get get = new Get(format(i));
+      if (opts.addColumns) {
+        get.addColumn(FAMILY_NAME, QUALIFIER_NAME);
+      }
+      if (opts.filterAll) {
+        get.setFilter(new FilterAllFilter());
+      }
+      try {
+        updateValueSize(table.get(get).get());
+      } catch (ExecutionException e) {
+        throw new IOException(e);
+      }
+    }
+  }
+
+  static class AsyncSequentialWriteTest extends AsyncTableTest {
+    AsyncSequentialWriteTest(AsyncConnection con, TestOptions options, Status status) {
+      super(con, options, status);
+    }
+
+    @Override
+    void testRow(final int i) throws IOException, InterruptedException {
+      byte[] row = format(i);
+      Put put = new Put(row);
+      for (int column = 0; column < opts.columns; column++) {
+        byte [] qualifier = column == 0? COLUMN_ZERO: Bytes.toBytes("" + column);
+        byte[] value = generateData(this.rand, getValueLength(this.rand));
+        if (opts.useTags) {
+          byte[] tag = generateData(this.rand, TAG_LENGTH);
+          Tag[] tags = new Tag[opts.noOfTags];
+          for (int n = 0; n < opts.noOfTags; n++) {
+            Tag t = new ArrayBackedTag((byte) n, tag);
+            tags[n] = t;
+          }
+          KeyValue kv = new KeyValue(row, FAMILY_NAME, qualifier, HConstants.LATEST_TIMESTAMP,
+              value, tags);
+          put.add(kv);
+          updateValueSize(kv.getValueLength());
+        } else {
+          put.addColumn(FAMILY_NAME, qualifier, value);
+          updateValueSize(value.length);
+        }
+      }
+      put.setDurability(opts.writeToWAL ? Durability.SYNC_WAL : Durability.SKIP_WAL);
+      try {
+        table.put(put).get();
+      } catch (ExecutionException e) {
+        throw new IOException(e);
+      }
+    }
+  }
+
   static abstract class BufferedMutatorTest extends Test {
     protected BufferedMutator mutator;
     protected Table table;
@@ -1789,23 +2093,31 @@ public class PerformanceEvaluation extends Configured implements Tool {
     return random.nextInt(Integer.MAX_VALUE) % totalRows;
   }
 
-  static RunResult runOneClient(final Class<? extends Test> cmd, Configuration conf, Connection con,
-                           TestOptions opts, final Status status)
+  static RunResult runOneClient(final Class<? extends TestBase> cmd, Configuration conf,
+      Connection con, AsyncConnection asyncCon, TestOptions opts, final Status status)
       throws IOException, InterruptedException {
-    status.setStatus("Start " + cmd + " at offset " + opts.startRow + " for " +
-      opts.perClientRunRows + " rows");
+    status.setStatus("Start " + cmd + " at offset " + opts.startRow + " for "
+        + opts.perClientRunRows + " rows");
     long totalElapsedTime;
 
-    final Test t;
+    final TestBase t;
     try {
-      Constructor<? extends Test> constructor =
-        cmd.getDeclaredConstructor(Connection.class, TestOptions.class, Status.class);
-      t = constructor.newInstance(con, opts, status);
+      if (AsyncTest.class.isAssignableFrom(cmd)) {
+        Class<? extends AsyncTest> newCmd = (Class<? extends AsyncTest>) cmd;
+        Constructor<? extends AsyncTest> constructor =
+            newCmd.getDeclaredConstructor(AsyncConnection.class, TestOptions.class, Status.class);
+        t = constructor.newInstance(asyncCon, opts, status);
+      } else {
+        Class<? extends Test> newCmd = (Class<? extends Test>) cmd;
+        Constructor<? extends Test> constructor =
+            newCmd.getDeclaredConstructor(Connection.class, TestOptions.class, Status.class);
+        t = constructor.newInstance(con, opts, status);
+      }
     } catch (NoSuchMethodException e) {
-      throw new IllegalArgumentException("Invalid command class: " +
-          cmd.getName() + ".  It does not provide a constructor as described by " +
-          "the javadoc comment.  Available constructors are: " +
-          Arrays.toString(cmd.getConstructors()));
+      throw new IllegalArgumentException("Invalid command class: " + cmd.getName()
+          + ".  It does not provide a constructor as described by "
+          + "the javadoc comment.  Available constructors are: "
+          + Arrays.toString(cmd.getConstructors()));
     } catch (Exception e) {
       throw new IllegalStateException("Failed to construct command class", e);
     }
@@ -1823,8 +2135,8 @@ public class PerformanceEvaluation extends Configured implements Tool {
     return opts.valueRandom? opts.valueSize/2: opts.valueSize;
   }
 
-  private void runTest(final Class<? extends Test> cmd, TestOptions opts) throws IOException,
-      InterruptedException, ClassNotFoundException {
+  private void runTest(final Class<? extends TestBase> cmd, TestOptions opts) throws IOException,
+      InterruptedException, ClassNotFoundException, ExecutionException {
     // Log the configuration we're going to run with. Uses JSON mapper because lazy. It'll do
     // the TestOptions introspection for us and dump the output in a readable format.
     LOG.info(cmd.getSimpleName() + " test run options=" + MAPPER.writeValueAsString(opts));
@@ -1944,7 +2256,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
     System.err.println();
     System.err.println("Command:");
     for (CmdDescriptor command : COMMANDS.values()) {
-      System.err.println(String.format(" %-15s %s", command.getName(), command.getDescription()));
+      System.err.println(String.format(" %-20s %s", command.getName(), command.getDescription()));
     }
     System.err.println();
     System.err.println("Args:");
@@ -2285,7 +2597,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
         return errCode;
       }
 
-      Class<? extends Test> cmdClass = determineCommandClass(opts.cmdName);
+      Class<? extends TestBase> cmdClass = determineCommandClass(opts.cmdName);
       if (cmdClass != null) {
         runTest(cmdClass, opts);
         errCode = 0;
@@ -2302,7 +2614,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
     return COMMANDS.containsKey(cmd);
   }
 
-  private static Class<? extends Test> determineCommandClass(String cmd) {
+  private static Class<? extends TestBase> determineCommandClass(String cmd) {
     CmdDescriptor descriptor = COMMANDS.get(cmd);
     return descriptor != null ? descriptor.getCmdClass() : null;
   }


[17/50] [abbrv] hbase git commit: HBASE-18271 Shade netty Purge mention of netty-all.

Posted by st...@apache.org.
HBASE-18271 Shade netty Purge mention of netty-all.

Add in hbase-thirdparty hbase-shaded-netty instead.
s/io.netty/org.apache.hadoop.hbase.shaded.io.netty/ everywhere in hbase.

Also set a system property when running tests and when starting
hbase; required by netty so can find the relocation files in the
bundled .so.


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

Branch: refs/heads/HBASE-14070.HLC
Commit: c6ac04ab39dafad8ebc7033b852f535a932d4ce6
Parents: 923195c
Author: Michael Stack <st...@apache.org>
Authored: Wed Aug 2 14:47:51 2017 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Sun Aug 13 12:46:21 2017 -0700

----------------------------------------------------------------------
 bin/hbase                                       | 12 ++++-
 hbase-client/pom.xml                            |  4 +-
 .../client/AsyncAdminRequestRetryingCaller.java |  2 +-
 .../client/AsyncBatchRpcRetryingCaller.java     |  2 +-
 .../hbase/client/AsyncConnectionImpl.java       |  2 +-
 .../hadoop/hbase/client/AsyncHBaseAdmin.java    |  4 +-
 .../AsyncMasterRequestRpcRetryingCaller.java    |  2 +-
 .../hadoop/hbase/client/AsyncRegionLocator.java |  4 +-
 .../hbase/client/AsyncRpcRetryingCaller.java    |  2 +-
 .../client/AsyncRpcRetryingCallerFactory.java   |  2 +-
 .../AsyncScanSingleRegionRpcRetryingCaller.java |  4 +-
 .../AsyncServerRequestRpcRetryingCaller.java    |  2 +-
 .../AsyncSingleRequestRpcRetryingCaller.java    |  2 +-
 .../hbase/client/ClusterStatusListener.java     | 20 ++++-----
 .../hadoop/hbase/client/RawAsyncHBaseAdmin.java |  4 +-
 .../hadoop/hbase/ipc/AbstractRpcClient.java     |  2 +-
 .../hbase/ipc/BufferCallBeforeInitHandler.java  |  6 +--
 .../java/org/apache/hadoop/hbase/ipc/Call.java  |  2 +-
 .../hadoop/hbase/ipc/CellBlockBuilder.java      |  6 +--
 .../hbase/ipc/DefaultNettyEventLoopConfig.java  | 10 ++---
 .../apache/hadoop/hbase/ipc/NettyRpcClient.java | 10 ++---
 .../hbase/ipc/NettyRpcClientConfigHelper.java   |  4 +-
 .../hadoop/hbase/ipc/NettyRpcConnection.java    | 34 +++++++--------
 .../hadoop/hbase/ipc/NettyRpcDuplexHandler.java | 16 +++----
 .../apache/hadoop/hbase/ipc/RpcConnection.java  |  6 +--
 .../hbase/security/CryptoAESUnwrapHandler.java  | 10 ++---
 .../hbase/security/CryptoAESWrapHandler.java    | 16 +++----
 .../NettyHBaseRpcConnectionHeaderHandler.java   | 12 ++---
 .../hbase/security/NettyHBaseSaslRpcClient.java |  4 +-
 .../NettyHBaseSaslRpcClientHandler.java         |  8 ++--
 .../hbase/security/SaslChallengeDecoder.java    |  6 +--
 .../hbase/security/SaslUnwrapHandler.java       | 10 ++---
 .../hadoop/hbase/security/SaslWrapHandler.java  | 16 +++----
 .../hbase/client/example/HttpProxyExample.java  | 44 +++++++++----------
 hbase-prefix-tree/pom.xml                       |  8 ++--
 hbase-rsgroup/pom.xml                           |  4 +-
 hbase-server/pom.xml                            |  7 +--
 .../hbase/io/asyncfs/AsyncFSOutputHelper.java   |  4 +-
 .../asyncfs/FanOutOneBlockAsyncDFSOutput.java   | 32 +++++++-------
 .../FanOutOneBlockAsyncDFSOutputHelper.java     | 46 ++++++++++----------
 .../FanOutOneBlockAsyncDFSOutputSaslHelper.java | 38 ++++++++--------
 .../apache/hadoop/hbase/ipc/NettyRpcServer.java | 30 ++++++-------
 .../ipc/NettyRpcServerPreambleHandler.java      |  8 ++--
 .../hbase/ipc/NettyRpcServerRequestDecoder.java |  8 ++--
 .../ipc/NettyRpcServerResponseEncoder.java      |  8 ++--
 .../hbase/ipc/NettyServerRpcConnection.java     |  4 +-
 .../hbase/mapreduce/TableMapReduceUtil.java     |  2 +-
 .../hbase/master/ClusterStatusPublisher.java    | 30 ++++++-------
 .../hbase/regionserver/wal/AsyncFSWAL.java      |  6 +--
 .../wal/AsyncProtobufLogWriter.java             |  4 +-
 .../wal/SecureAsyncProtobufLogWriter.java       |  4 +-
 .../hbase/util/NettyEventLoopGroupConfig.java   | 20 ++++-----
 .../hadoop/hbase/wal/AsyncFSWALProvider.java    | 12 ++---
 .../hbase/wal/NettyAsyncFSWALConfigHelper.java  |  4 +-
 .../TestFanOutOneBlockAsyncDFSOutput.java       | 10 ++---
 .../hbase/io/asyncfs/TestLocalAsyncOutput.java  |  8 ++--
 .../TestSaslFanOutOneBlockAsyncDFSOutput.java   | 10 ++---
 .../apache/hadoop/hbase/ipc/TestNettyIPC.java   |  8 ++--
 .../hbase/regionserver/wal/TestAsyncFSWAL.java  |  8 ++--
 .../regionserver/wal/TestAsyncProtobufLog.java  |  8 ++--
 .../regionserver/wal/TestAsyncWALReplay.java    |  8 ++--
 pom.xml                                         | 19 +++++---
 62 files changed, 330 insertions(+), 318 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/bin/hbase
----------------------------------------------------------------------
diff --git a/bin/hbase b/bin/hbase
index d7a8069..7eeaf2d 100755
--- a/bin/hbase
+++ b/bin/hbase
@@ -473,8 +473,16 @@ fi
 HEAP_SETTINGS="$JAVA_HEAP_MAX $JAVA_OFFHEAP_MAX"
 # Exec unless HBASE_NOEXEC is set.
 export CLASSPATH
+# Netty is shaded in hbase. The shaded netty is pulled in with org.apache.hbase.thirdparty:hbase-shaded-netty.
+# It has a .so in it. Shading requires rename of the .so and then passing a system property so netty finds the
+# renamed .so and associates it w/ the relocated netty files.
+# So we can find the relocated .so, we need to add a system property pointing at new location. Trick is from:
+#  https://stackoverflow.com/questions/33825743/rename-files-inside-a-jar-using-some-maven-plugin
+# See toward the end of this issue for how to pass config: https://github.com/netty/netty/issues/6665
+#
+NETTY_PROP="-Dorg.apache.hadoop.hbase.shaded.io.netty.packagePrefix=org.apache.hadoop.hbase.shaded."
 if [ "${HBASE_NOEXEC}" != "" ]; then
-  "$JAVA" -Dproc_$COMMAND -XX:OnOutOfMemoryError="kill -9 %p" $HEAP_SETTINGS $HBASE_OPTS $CLASS "$@"
+  "$JAVA" -Dproc_$COMMAND "$NETTY_PROP" -XX:OnOutOfMemoryError="kill -9 %p" $HEAP_SETTINGS $HBASE_OPTS $CLASS "$@"
 else
-  exec "$JAVA" -Dproc_$COMMAND -XX:OnOutOfMemoryError="kill -9 %p" $HEAP_SETTINGS $HBASE_OPTS $CLASS "$@"
+  exec "$JAVA" -Dproc_$COMMAND  "$NETTY_PROP" -XX:OnOutOfMemoryError="kill -9 %p" $HEAP_SETTINGS $HBASE_OPTS $CLASS "$@"
 fi

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-client/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-client/pom.xml b/hbase-client/pom.xml
index 747391a..45c43e9 100644
--- a/hbase-client/pom.xml
+++ b/hbase-client/pom.xml
@@ -160,8 +160,8 @@
       <artifactId>protobuf-java</artifactId>
     </dependency>
     <dependency>
-      <groupId>io.netty</groupId>
-      <artifactId>netty-all</artifactId>
+      <groupId>org.apache.hbase.thirdparty</groupId>
+      <artifactId>hbase-shaded-netty</artifactId>
     </dependency>
     <dependency>
       <groupId>org.apache.zookeeper</groupId>

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdminRequestRetryingCaller.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdminRequestRetryingCaller.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdminRequestRetryingCaller.java
index afb72ea..2871c4d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdminRequestRetryingCaller.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdminRequestRetryingCaller.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.hbase.client;
 import java.io.IOException;
 import java.util.concurrent.CompletableFuture;
 
-import io.netty.util.HashedWheelTimer;
+import org.apache.hadoop.hbase.shaded.io.netty.util.HashedWheelTimer;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBatchRpcRetryingCaller.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBatchRpcRetryingCaller.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBatchRpcRetryingCaller.java
index 9b362d1..c52fd2f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBatchRpcRetryingCaller.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBatchRpcRetryingCaller.java
@@ -24,7 +24,7 @@ import static org.apache.hadoop.hbase.client.ConnectionUtils.resetController;
 import static org.apache.hadoop.hbase.client.ConnectionUtils.translateException;
 import static org.apache.hadoop.hbase.util.CollectionUtils.computeIfAbsent;
 
-import io.netty.util.HashedWheelTimer;
+import org.apache.hadoop.hbase.shaded.io.netty.util.HashedWheelTimer;
 
 import java.io.IOException;
 import java.util.ArrayList;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
index 8611de5..9913af9 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
@@ -23,7 +23,7 @@ import static org.apache.hadoop.hbase.client.NonceGenerator.CLIENT_NONCES_ENABLE
 
 import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 
-import io.netty.util.HashedWheelTimer;
+import org.apache.hadoop.hbase.shaded.io.netty.util.HashedWheelTimer;
 
 import java.io.IOException;
 import java.util.concurrent.CompletableFuture;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
index fbd8690..097f22a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
@@ -30,8 +30,8 @@ import java.util.stream.Collectors;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 
-import io.netty.util.Timeout;
-import io.netty.util.TimerTask;
+import org.apache.hadoop.hbase.shaded.io.netty.util.Timeout;
+import org.apache.hadoop.hbase.shaded.io.netty.util.TimerTask;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMasterRequestRpcRetryingCaller.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMasterRequestRpcRetryingCaller.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMasterRequestRpcRetryingCaller.java
index e2a3fee..7a598d7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMasterRequestRpcRetryingCaller.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMasterRequestRpcRetryingCaller.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import io.netty.util.HashedWheelTimer;
+import org.apache.hadoop.hbase.shaded.io.netty.util.HashedWheelTimer;
 
 import java.util.concurrent.CompletableFuture;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java
index 1114e2c..153bea4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java
@@ -21,8 +21,8 @@ import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
 import static org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil.findException;
 import static org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil.isMetaClearingException;
 
-import io.netty.util.HashedWheelTimer;
-import io.netty.util.Timeout;
+import org.apache.hadoop.hbase.shaded.io.netty.util.HashedWheelTimer;
+import org.apache.hadoop.hbase.shaded.io.netty.util.Timeout;
 
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.TimeUnit;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCaller.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCaller.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCaller.java
index c753383..3b01fe0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCaller.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCaller.java
@@ -23,7 +23,7 @@ import static org.apache.hadoop.hbase.client.ConnectionUtils.SLEEP_DELTA_NS;
 import static org.apache.hadoop.hbase.client.ConnectionUtils.getPauseTime;
 import static org.apache.hadoop.hbase.client.ConnectionUtils.resetController;
 import static org.apache.hadoop.hbase.client.ConnectionUtils.translateException;
-import io.netty.util.HashedWheelTimer;
+import org.apache.hadoop.hbase.shaded.io.netty.util.HashedWheelTimer;
 
 import java.util.ArrayList;
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCallerFactory.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCallerFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCallerFactory.java
index 729fb94..1aa3fbd 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCallerFactory.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCallerFactory.java
@@ -21,7 +21,7 @@ import static org.apache.hadoop.hbase.shaded.com.google.common.base.Precondition
 import static org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions.checkNotNull;
 import static org.apache.hadoop.hbase.client.ConnectionUtils.retries2Attempts;
 
-import io.netty.util.HashedWheelTimer;
+import org.apache.hadoop.hbase.shaded.io.netty.util.HashedWheelTimer;
 
 import java.util.List;
 import java.util.concurrent.CompletableFuture;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncScanSingleRegionRpcRetryingCaller.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncScanSingleRegionRpcRetryingCaller.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncScanSingleRegionRpcRetryingCaller.java
index 89b3afc..02a4357 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncScanSingleRegionRpcRetryingCaller.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncScanSingleRegionRpcRetryingCaller.java
@@ -30,8 +30,8 @@ import static org.apache.hadoop.hbase.client.ConnectionUtils.updateServerSideMet
 
 import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
 
-import io.netty.util.HashedWheelTimer;
-import io.netty.util.Timeout;
+import org.apache.hadoop.hbase.shaded.io.netty.util.HashedWheelTimer;
+import org.apache.hadoop.hbase.shaded.io.netty.util.Timeout;
 
 import java.io.IOException;
 import java.util.ArrayList;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncServerRequestRpcRetryingCaller.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncServerRequestRpcRetryingCaller.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncServerRequestRpcRetryingCaller.java
index 72241ea..40aa9be 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncServerRequestRpcRetryingCaller.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncServerRequestRpcRetryingCaller.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import io.netty.util.HashedWheelTimer;
+import org.apache.hadoop.hbase.shaded.io.netty.util.HashedWheelTimer;
 
 import java.io.IOException;
 import java.util.concurrent.CompletableFuture;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncSingleRequestRpcRetryingCaller.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncSingleRequestRpcRetryingCaller.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncSingleRequestRpcRetryingCaller.java
index e1c06d7..5071ec3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncSingleRequestRpcRetryingCaller.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncSingleRequestRpcRetryingCaller.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import io.netty.util.HashedWheelTimer;
+import org.apache.hadoop.hbase.shaded.io.netty.util.HashedWheelTimer;
 
 import java.io.IOException;
 import java.util.concurrent.CompletableFuture;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterStatusListener.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterStatusListener.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterStatusListener.java
index 240587b..9364944 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterStatusListener.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterStatusListener.java
@@ -20,16 +20,16 @@
 package org.apache.hadoop.hbase.client;
 
 
-import io.netty.bootstrap.Bootstrap;
-import io.netty.buffer.ByteBufInputStream;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.ChannelOption;
-import io.netty.channel.EventLoopGroup;
-import io.netty.channel.SimpleChannelInboundHandler;
-import io.netty.channel.nio.NioEventLoopGroup;
-import io.netty.channel.socket.DatagramChannel;
-import io.netty.channel.socket.DatagramPacket;
-import io.netty.channel.socket.nio.NioDatagramChannel;
+import org.apache.hadoop.hbase.shaded.io.netty.bootstrap.Bootstrap;
+import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBufInputStream;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelHandlerContext;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelOption;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoopGroup;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.SimpleChannelInboundHandler;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoopGroup;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.DatagramChannel;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.DatagramPacket;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.nio.NioDatagramChannel;
 
 import java.io.Closeable;
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
index 03f64fc..0070e0e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
@@ -42,8 +42,8 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.protobuf.Message;
 import com.google.protobuf.RpcChannel;
 
-import io.netty.util.Timeout;
-import io.netty.util.TimerTask;
+import org.apache.hadoop.hbase.shaded.io.netty.util.Timeout;
+import org.apache.hadoop.hbase.shaded.io.netty.util.TimerTask;
 
 import java.util.stream.Stream;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
index 55895f2..7925505 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
@@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback;
 
-import io.netty.util.HashedWheelTimer;
+import org.apache.hadoop.hbase.shaded.io.netty.util.HashedWheelTimer;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BufferCallBeforeInitHandler.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BufferCallBeforeInitHandler.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BufferCallBeforeInitHandler.java
index c628c31..221d57f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BufferCallBeforeInitHandler.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BufferCallBeforeInitHandler.java
@@ -17,9 +17,9 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
-import io.netty.channel.ChannelDuplexHandler;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.ChannelPromise;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelDuplexHandler;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelHandlerContext;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelPromise;
 
 import java.io.IOException;
 import java.util.HashMap;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Call.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Call.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Call.java
index 04d7238..b286c0e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Call.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Call.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
-import io.netty.util.Timeout;
+import org.apache.hadoop.hbase.shaded.io.netty.util.Timeout;
 
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.Message;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CellBlockBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CellBlockBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CellBlockBuilder.java
index d00490b..7ec313e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CellBlockBuilder.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CellBlockBuilder.java
@@ -17,9 +17,9 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.ByteBufAllocator;
-import io.netty.buffer.ByteBufOutputStream;
+import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBuf;
+import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBufAllocator;
+import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBufOutputStream;
 
 import java.io.ByteArrayInputStream;
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/DefaultNettyEventLoopConfig.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/DefaultNettyEventLoopConfig.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/DefaultNettyEventLoopConfig.java
index f710d54..efc74e4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/DefaultNettyEventLoopConfig.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/DefaultNettyEventLoopConfig.java
@@ -17,11 +17,11 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
-import io.netty.channel.Channel;
-import io.netty.channel.EventLoopGroup;
-import io.netty.channel.nio.NioEventLoopGroup;
-import io.netty.channel.socket.nio.NioSocketChannel;
-import io.netty.util.concurrent.DefaultThreadFactory;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoopGroup;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoopGroup;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.nio.NioSocketChannel;
+import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.DefaultThreadFactory;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.util.Pair;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClient.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClient.java
index cde453f..2fea562 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClient.java
@@ -17,11 +17,11 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
-import io.netty.channel.Channel;
-import io.netty.channel.EventLoopGroup;
-import io.netty.channel.nio.NioEventLoopGroup;
-import io.netty.channel.socket.nio.NioSocketChannel;
-import io.netty.util.concurrent.DefaultThreadFactory;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoopGroup;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoopGroup;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.nio.NioSocketChannel;
+import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.DefaultThreadFactory;
 
 import java.io.IOException;
 import java.net.SocketAddress;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClientConfigHelper.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClientConfigHelper.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClientConfigHelper.java
index a8aca52..0e12f27 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClientConfigHelper.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClientConfigHelper.java
@@ -19,8 +19,8 @@ package org.apache.hadoop.hbase.ipc;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
 
-import io.netty.channel.Channel;
-import io.netty.channel.EventLoopGroup;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoopGroup;
 
 import java.util.HashMap;
 import java.util.Map;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java
index c879990..54e2b2c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java
@@ -22,26 +22,26 @@ import static org.apache.hadoop.hbase.ipc.CallEvent.Type.TIMEOUT;
 import static org.apache.hadoop.hbase.ipc.IPCUtil.setCancelled;
 import static org.apache.hadoop.hbase.ipc.IPCUtil.toIOE;
 
-import io.netty.handler.timeout.ReadTimeoutHandler;
+import org.apache.hadoop.hbase.shaded.io.netty.handler.timeout.ReadTimeoutHandler;
 import org.apache.hadoop.hbase.security.NettyHBaseRpcConnectionHeaderHandler;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback;
 
-import io.netty.bootstrap.Bootstrap;
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.ByteBufOutputStream;
-import io.netty.buffer.Unpooled;
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelFuture;
-import io.netty.channel.ChannelFutureListener;
-import io.netty.channel.ChannelHandler;
-import io.netty.channel.ChannelOption;
-import io.netty.channel.ChannelPipeline;
-import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
-import io.netty.handler.timeout.IdleStateHandler;
-import io.netty.util.ReferenceCountUtil;
-import io.netty.util.concurrent.Future;
-import io.netty.util.concurrent.FutureListener;
-import io.netty.util.concurrent.Promise;
+import org.apache.hadoop.hbase.shaded.io.netty.bootstrap.Bootstrap;
+import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBuf;
+import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBufOutputStream;
+import org.apache.hadoop.hbase.shaded.io.netty.buffer.Unpooled;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelFuture;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelFutureListener;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelHandler;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelOption;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelPipeline;
+import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.LengthFieldBasedFrameDecoder;
+import org.apache.hadoop.hbase.shaded.io.netty.handler.timeout.IdleStateHandler;
+import org.apache.hadoop.hbase.shaded.io.netty.util.ReferenceCountUtil;
+import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.Future;
+import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.FutureListener;
+import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.Promise;
 
 import java.io.IOException;
 import java.util.concurrent.Executors;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java
index 78964d6..f904b4f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java
@@ -21,14 +21,14 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.Message;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.Message.Builder;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.TextFormat;
 
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.ByteBufInputStream;
-import io.netty.buffer.ByteBufOutputStream;
-import io.netty.channel.ChannelDuplexHandler;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.ChannelPromise;
-import io.netty.handler.timeout.IdleStateEvent;
-import io.netty.util.concurrent.PromiseCombiner;
+import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBuf;
+import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBufInputStream;
+import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBufOutputStream;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelDuplexHandler;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelHandlerContext;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelPromise;
+import org.apache.hadoop.hbase.shaded.io.netty.handler.timeout.IdleStateEvent;
+import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.PromiseCombiner;
 
 import java.io.IOException;
 import java.util.HashMap;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnection.java
index 98d2256..d3558ee 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnection.java
@@ -17,9 +17,9 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
-import io.netty.util.HashedWheelTimer;
-import io.netty.util.Timeout;
-import io.netty.util.TimerTask;
+import org.apache.hadoop.hbase.shaded.io.netty.util.HashedWheelTimer;
+import org.apache.hadoop.hbase.shaded.io.netty.util.Timeout;
+import org.apache.hadoop.hbase.shaded.io.netty.util.TimerTask;
 
 import java.io.IOException;
 import java.net.UnknownHostException;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-client/src/main/java/org/apache/hadoop/hbase/security/CryptoAESUnwrapHandler.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/CryptoAESUnwrapHandler.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/CryptoAESUnwrapHandler.java
index 31abeba..dab9098 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/CryptoAESUnwrapHandler.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/CryptoAESUnwrapHandler.java
@@ -18,16 +18,16 @@
 
 package org.apache.hadoop.hbase.security;
 
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.Unpooled;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.SimpleChannelInboundHandler;
+import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBuf;
+import org.apache.hadoop.hbase.shaded.io.netty.buffer.Unpooled;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelHandlerContext;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.SimpleChannelInboundHandler;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.io.crypto.aes.CryptoAES;
 
 /**
  * Unwrap messages with Crypto AES. Should be placed after a
- * {@link io.netty.handler.codec.LengthFieldBasedFrameDecoder}
+ * io.netty.handler.codec.LengthFieldBasedFrameDecoder
  */
 @InterfaceAudience.Private
 public class CryptoAESUnwrapHandler extends SimpleChannelInboundHandler<ByteBuf> {

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-client/src/main/java/org/apache/hadoop/hbase/security/CryptoAESWrapHandler.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/CryptoAESWrapHandler.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/CryptoAESWrapHandler.java
index 6c74ed8..47e1905 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/CryptoAESWrapHandler.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/CryptoAESWrapHandler.java
@@ -18,14 +18,14 @@
 
 package org.apache.hadoop.hbase.security;
 
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.Unpooled;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.ChannelOutboundHandlerAdapter;
-import io.netty.channel.ChannelPromise;
-import io.netty.channel.CoalescingBufferQueue;
-import io.netty.util.ReferenceCountUtil;
-import io.netty.util.concurrent.PromiseCombiner;
+import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBuf;
+import org.apache.hadoop.hbase.shaded.io.netty.buffer.Unpooled;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelHandlerContext;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelOutboundHandlerAdapter;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelPromise;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.CoalescingBufferQueue;
+import org.apache.hadoop.hbase.shaded.io.netty.util.ReferenceCountUtil;
+import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.PromiseCombiner;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.io.crypto.aes.CryptoAES;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseRpcConnectionHeaderHandler.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseRpcConnectionHeaderHandler.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseRpcConnectionHeaderHandler.java
index 5608874..c515e83 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseRpcConnectionHeaderHandler.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseRpcConnectionHeaderHandler.java
@@ -18,12 +18,12 @@
 
 package org.apache.hadoop.hbase.security;
 
-import io.netty.buffer.ByteBuf;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.ChannelPipeline;
-import io.netty.channel.SimpleChannelInboundHandler;
-import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
-import io.netty.util.concurrent.Promise;
+import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBuf;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelHandlerContext;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelPipeline;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.SimpleChannelInboundHandler;
+import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.LengthFieldBasedFrameDecoder;
+import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.Promise;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.io.crypto.aes.CryptoAES;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseSaslRpcClient.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseSaslRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseSaslRpcClient.java
index 9ae31a4..937fccd 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseSaslRpcClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseSaslRpcClient.java
@@ -17,8 +17,8 @@
  */
 package org.apache.hadoop.hbase.security;
 
-import io.netty.channel.ChannelPipeline;
-import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelPipeline;
+import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.LengthFieldBasedFrameDecoder;
 
 import java.io.IOException;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseSaslRpcClientHandler.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseSaslRpcClientHandler.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseSaslRpcClientHandler.java
index 4525aef..2f64ae2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseSaslRpcClientHandler.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseSaslRpcClientHandler.java
@@ -17,10 +17,10 @@
  */
 package org.apache.hadoop.hbase.security;
 
-import io.netty.buffer.ByteBuf;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.SimpleChannelInboundHandler;
-import io.netty.util.concurrent.Promise;
+import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBuf;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelHandlerContext;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.SimpleChannelInboundHandler;
+import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.Promise;
 
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslChallengeDecoder.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslChallengeDecoder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslChallengeDecoder.java
index d818097..b156251 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslChallengeDecoder.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslChallengeDecoder.java
@@ -17,9 +17,9 @@
  */
 package org.apache.hadoop.hbase.security;
 
-import io.netty.buffer.ByteBuf;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.handler.codec.ByteToMessageDecoder;
+import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBuf;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelHandlerContext;
+import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.ByteToMessageDecoder;
 
 import java.io.IOException;
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslUnwrapHandler.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslUnwrapHandler.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslUnwrapHandler.java
index e631478..2171647 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslUnwrapHandler.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslUnwrapHandler.java
@@ -17,10 +17,10 @@
  */
 package org.apache.hadoop.hbase.security;
 
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.Unpooled;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.SimpleChannelInboundHandler;
+import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBuf;
+import org.apache.hadoop.hbase.shaded.io.netty.buffer.Unpooled;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelHandlerContext;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.SimpleChannelInboundHandler;
 
 import javax.security.sasl.SaslClient;
 
@@ -28,7 +28,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 
 /**
  * Unwrap sasl messages. Should be placed after a
- * {@link io.netty.handler.codec.LengthFieldBasedFrameDecoder}
+ * io.netty.handler.codec.LengthFieldBasedFrameDecoder
  */
 @InterfaceAudience.Private
 public class SaslUnwrapHandler extends SimpleChannelInboundHandler<ByteBuf> {

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslWrapHandler.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslWrapHandler.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslWrapHandler.java
index 14ecf2e..fb03737 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslWrapHandler.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslWrapHandler.java
@@ -17,14 +17,14 @@
  */
 package org.apache.hadoop.hbase.security;
 
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.Unpooled;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.ChannelOutboundHandlerAdapter;
-import io.netty.channel.ChannelPromise;
-import io.netty.channel.CoalescingBufferQueue;
-import io.netty.util.ReferenceCountUtil;
-import io.netty.util.concurrent.PromiseCombiner;
+import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBuf;
+import org.apache.hadoop.hbase.shaded.io.netty.buffer.Unpooled;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelHandlerContext;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelOutboundHandlerAdapter;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelPromise;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.CoalescingBufferQueue;
+import org.apache.hadoop.hbase.shaded.io.netty.util.ReferenceCountUtil;
+import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.PromiseCombiner;
 
 import java.io.IOException;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-examples/src/main/java/org/apache/hadoop/hbase/client/example/HttpProxyExample.java
----------------------------------------------------------------------
diff --git a/hbase-examples/src/main/java/org/apache/hadoop/hbase/client/example/HttpProxyExample.java b/hbase-examples/src/main/java/org/apache/hadoop/hbase/client/example/HttpProxyExample.java
index c940c2a..bb83bac 100644
--- a/hbase-examples/src/main/java/org/apache/hadoop/hbase/client/example/HttpProxyExample.java
+++ b/hbase-examples/src/main/java/org/apache/hadoop/hbase/client/example/HttpProxyExample.java
@@ -20,28 +20,28 @@ package org.apache.hadoop.hbase.client.example;
 import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
 import org.apache.hadoop.hbase.shaded.com.google.common.base.Throwables;
 
-import io.netty.bootstrap.ServerBootstrap;
-import io.netty.buffer.ByteBuf;
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.ChannelInitializer;
-import io.netty.channel.ChannelOption;
-import io.netty.channel.EventLoopGroup;
-import io.netty.channel.SimpleChannelInboundHandler;
-import io.netty.channel.group.ChannelGroup;
-import io.netty.channel.group.DefaultChannelGroup;
-import io.netty.channel.nio.NioEventLoopGroup;
-import io.netty.channel.socket.nio.NioServerSocketChannel;
-import io.netty.channel.socket.nio.NioSocketChannel;
-import io.netty.handler.codec.http.DefaultFullHttpResponse;
-import io.netty.handler.codec.http.FullHttpRequest;
-import io.netty.handler.codec.http.HttpHeaderNames;
-import io.netty.handler.codec.http.HttpObjectAggregator;
-import io.netty.handler.codec.http.HttpResponseStatus;
-import io.netty.handler.codec.http.HttpServerCodec;
-import io.netty.handler.codec.http.HttpVersion;
-import io.netty.handler.codec.http.QueryStringDecoder;
-import io.netty.util.concurrent.GlobalEventExecutor;
+import org.apache.hadoop.hbase.shaded.io.netty.bootstrap.ServerBootstrap;
+import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBuf;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelHandlerContext;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelInitializer;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelOption;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoopGroup;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.SimpleChannelInboundHandler;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.group.ChannelGroup;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.group.DefaultChannelGroup;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoopGroup;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.nio.NioServerSocketChannel;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.nio.NioSocketChannel;
+import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.http.DefaultFullHttpResponse;
+import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.http.FullHttpRequest;
+import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.http.HttpHeaderNames;
+import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.http.HttpObjectAggregator;
+import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.http.HttpResponseStatus;
+import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.http.HttpServerCodec;
+import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.http.HttpVersion;
+import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.http.QueryStringDecoder;
+import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.GlobalEventExecutor;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-prefix-tree/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-prefix-tree/pom.xml b/hbase-prefix-tree/pom.xml
index b851331..a44a59e 100644
--- a/hbase-prefix-tree/pom.xml
+++ b/hbase-prefix-tree/pom.xml
@@ -116,12 +116,12 @@
       <artifactId>hbase-shaded-miscellaneous</artifactId>
     </dependency>
     <dependency>
-      <groupId>commons-logging</groupId>
-      <artifactId>commons-logging</artifactId>
+      <groupId>org.apache.hbase.thirdparty</groupId>
+      <artifactId>hbase-shaded-netty</artifactId>
     </dependency>
     <dependency>
-      <groupId>io.netty</groupId>
-      <artifactId>netty-all</artifactId>
+      <groupId>commons-logging</groupId>
+      <artifactId>commons-logging</artifactId>
     </dependency>
   </dependencies>
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-rsgroup/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/pom.xml b/hbase-rsgroup/pom.xml
index a3e710c..bbb489d 100644
--- a/hbase-rsgroup/pom.xml
+++ b/hbase-rsgroup/pom.xml
@@ -164,8 +164,8 @@
       <artifactId>protobuf-java</artifactId>
     </dependency>
     <dependency>
-      <groupId>io.netty</groupId>
-      <artifactId>netty-all</artifactId>
+      <groupId>org.apache.hbase.thirdparty</groupId>
+      <artifactId>hbase-shaded-netty</artifactId>
     </dependency>
     <dependency>
       <groupId>org.apache.zookeeper</groupId>

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-server/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml
index 647d531..e377874 100644
--- a/hbase-server/pom.xml
+++ b/hbase-server/pom.xml
@@ -264,6 +264,7 @@
           </properties>
           <systemPropertyVariables>
             <test.build.webapps>target/test-classes/webapps</test.build.webapps>
+            <org.apache.hadoop.hbase.shaded.io.netty.packagePrefix>org.apache.hadoop.hbase.shaded.</org.apache.hadoop.hbase.shaded.io.netty.packagePrefix>
           </systemPropertyVariables>
         </configuration>
       </plugin>
@@ -522,10 +523,6 @@
       <artifactId>jamon-runtime</artifactId>
     </dependency>
     <dependency>
-      <groupId>io.netty</groupId>
-      <artifactId>netty-all</artifactId>
-    </dependency>
-    <dependency>
       <groupId>javax.ws.rs</groupId>
       <artifactId>javax.ws.rs-api</artifactId>
     </dependency>
@@ -832,7 +829,7 @@
         <dependency>
           <groupId>io.netty</groupId>
           <artifactId>netty</artifactId>
-          <version>3.6.2.Final</version>
+          <version>${netty.hadoop.version}</version>
           <scope>test</scope>
         </dependency>
       </dependencies>

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/AsyncFSOutputHelper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/AsyncFSOutputHelper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/AsyncFSOutputHelper.java
index 5c2d4bf..653e0ca 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/AsyncFSOutputHelper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/AsyncFSOutputHelper.java
@@ -20,8 +20,8 @@ package org.apache.hadoop.hbase.io.asyncfs;
 import org.apache.hadoop.hbase.shaded.com.google.common.base.Throwables;
 import org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.ThreadFactoryBuilder;
 
-import io.netty.channel.Channel;
-import io.netty.channel.EventLoop;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoop;
 
 import java.io.IOException;
 import java.io.InterruptedIOException;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java
index a493565..5fcbe96 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java
@@ -17,8 +17,8 @@
  */
 package org.apache.hadoop.hbase.io.asyncfs;
 
-import static io.netty.handler.timeout.IdleState.READER_IDLE;
-import static io.netty.handler.timeout.IdleState.WRITER_IDLE;
+import static org.apache.hadoop.hbase.shaded.io.netty.handler.timeout.IdleState.READER_IDLE;
+import static org.apache.hadoop.hbase.shaded.io.netty.handler.timeout.IdleState.WRITER_IDLE;
 import static org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper.HEART_BEAT_SEQNO;
 import static org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper.READ_TIMEOUT;
 import static org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper.completeFile;
@@ -27,20 +27,20 @@ import static org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHel
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY;
 
 
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.ByteBufAllocator;
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelHandler.Sharable;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.EventLoop;
-import io.netty.channel.SimpleChannelInboundHandler;
-import io.netty.handler.codec.protobuf.ProtobufDecoder;
-import io.netty.handler.codec.protobuf.ProtobufVarint32FrameDecoder;
-import io.netty.handler.timeout.IdleStateEvent;
-import io.netty.handler.timeout.IdleStateHandler;
-import io.netty.util.concurrent.Future;
-import io.netty.util.concurrent.Promise;
-import io.netty.util.concurrent.PromiseCombiner;
+import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBuf;
+import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBufAllocator;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelHandler.Sharable;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelHandlerContext;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoop;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.SimpleChannelInboundHandler;
+import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.protobuf.ProtobufDecoder;
+import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.protobuf.ProtobufVarint32FrameDecoder;
+import org.apache.hadoop.hbase.shaded.io.netty.handler.timeout.IdleStateEvent;
+import org.apache.hadoop.hbase.shaded.io.netty.handler.timeout.IdleStateHandler;
+import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.Future;
+import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.Promise;
+import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.PromiseCombiner;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java
index 85b71e5..16412d3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java
@@ -17,8 +17,8 @@
  */
 package org.apache.hadoop.hbase.io.asyncfs;
 
-import static io.netty.channel.ChannelOption.CONNECT_TIMEOUT_MILLIS;
-import static io.netty.handler.timeout.IdleState.READER_IDLE;
+import static org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelOption.CONNECT_TIMEOUT_MILLIS;
+import static org.apache.hadoop.hbase.shaded.io.netty.handler.timeout.IdleState.READER_IDLE;
 import static org.apache.hadoop.fs.CreateFlag.CREATE;
 import static org.apache.hadoop.fs.CreateFlag.OVERWRITE;
 import static org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputSaslHelper.createEncryptor;
@@ -32,27 +32,27 @@ import org.apache.hadoop.hbase.shaded.com.google.common.base.Throwables;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableMap;
 import com.google.protobuf.CodedOutputStream;
 
-import io.netty.bootstrap.Bootstrap;
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.ByteBufAllocator;
-import io.netty.buffer.ByteBufOutputStream;
-import io.netty.buffer.PooledByteBufAllocator;
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelFuture;
-import io.netty.channel.ChannelFutureListener;
-import io.netty.channel.ChannelHandler;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.ChannelInitializer;
-import io.netty.channel.ChannelPipeline;
-import io.netty.channel.EventLoop;
-import io.netty.channel.SimpleChannelInboundHandler;
-import io.netty.handler.codec.protobuf.ProtobufDecoder;
-import io.netty.handler.codec.protobuf.ProtobufVarint32FrameDecoder;
-import io.netty.handler.timeout.IdleStateEvent;
-import io.netty.handler.timeout.IdleStateHandler;
-import io.netty.util.concurrent.Future;
-import io.netty.util.concurrent.FutureListener;
-import io.netty.util.concurrent.Promise;
+import org.apache.hadoop.hbase.shaded.io.netty.bootstrap.Bootstrap;
+import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBuf;
+import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBufAllocator;
+import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBufOutputStream;
+import org.apache.hadoop.hbase.shaded.io.netty.buffer.PooledByteBufAllocator;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelFuture;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelFutureListener;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelHandler;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelHandlerContext;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelInitializer;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelPipeline;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoop;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.SimpleChannelInboundHandler;
+import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.protobuf.ProtobufDecoder;
+import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.protobuf.ProtobufVarint32FrameDecoder;
+import org.apache.hadoop.hbase.shaded.io.netty.handler.timeout.IdleStateEvent;
+import org.apache.hadoop.hbase.shaded.io.netty.handler.timeout.IdleStateHandler;
+import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.Future;
+import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.FutureListener;
+import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.Promise;
 
 import java.io.IOException;
 import java.lang.reflect.InvocationTargetException;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java
index c1e4c24..bc6ab3c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hbase.io.asyncfs;
 
-import static io.netty.handler.timeout.IdleState.READER_IDLE;
+import static org.apache.hadoop.hbase.shaded.io.netty.handler.timeout.IdleState.READER_IDLE;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_CIPHER_SUITES_KEY;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.base.Charsets;
@@ -26,24 +26,24 @@ import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableSet;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
 import com.google.protobuf.CodedOutputStream;
 
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.ByteBufOutputStream;
-import io.netty.buffer.CompositeByteBuf;
-import io.netty.buffer.Unpooled;
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelDuplexHandler;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.ChannelOutboundHandlerAdapter;
-import io.netty.channel.ChannelPipeline;
-import io.netty.channel.ChannelPromise;
-import io.netty.channel.SimpleChannelInboundHandler;
-import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
-import io.netty.handler.codec.MessageToByteEncoder;
-import io.netty.handler.codec.protobuf.ProtobufDecoder;
-import io.netty.handler.codec.protobuf.ProtobufVarint32FrameDecoder;
-import io.netty.handler.timeout.IdleStateEvent;
-import io.netty.handler.timeout.IdleStateHandler;
-import io.netty.util.concurrent.Promise;
+import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBuf;
+import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBufOutputStream;
+import org.apache.hadoop.hbase.shaded.io.netty.buffer.CompositeByteBuf;
+import org.apache.hadoop.hbase.shaded.io.netty.buffer.Unpooled;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelDuplexHandler;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelHandlerContext;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelOutboundHandlerAdapter;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelPipeline;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelPromise;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.SimpleChannelInboundHandler;
+import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.LengthFieldBasedFrameDecoder;
+import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.MessageToByteEncoder;
+import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.protobuf.ProtobufDecoder;
+import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.protobuf.ProtobufVarint32FrameDecoder;
+import org.apache.hadoop.hbase.shaded.io.netty.handler.timeout.IdleStateEvent;
+import org.apache.hadoop.hbase.shaded.io.netty.handler.timeout.IdleStateHandler;
+import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.Promise;
 
 import java.io.IOException;
 import java.lang.reflect.Field;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java
index fafc53f..ffd6d6e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java
@@ -17,21 +17,21 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
-import io.netty.bootstrap.ServerBootstrap;
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelInitializer;
-import io.netty.channel.ChannelOption;
-import io.netty.channel.ChannelPipeline;
-import io.netty.channel.EventLoopGroup;
-import io.netty.channel.ServerChannel;
-import io.netty.channel.group.ChannelGroup;
-import io.netty.channel.group.DefaultChannelGroup;
-import io.netty.channel.nio.NioEventLoopGroup;
-import io.netty.channel.socket.nio.NioServerSocketChannel;
-import io.netty.handler.codec.FixedLengthFrameDecoder;
-import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
-import io.netty.util.concurrent.DefaultThreadFactory;
-import io.netty.util.concurrent.GlobalEventExecutor;
+import org.apache.hadoop.hbase.shaded.io.netty.bootstrap.ServerBootstrap;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelInitializer;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelOption;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelPipeline;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoopGroup;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ServerChannel;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.group.ChannelGroup;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.group.DefaultChannelGroup;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoopGroup;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.nio.NioServerSocketChannel;
+import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.FixedLengthFrameDecoder;
+import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.LengthFieldBasedFrameDecoder;
+import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.DefaultThreadFactory;
+import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.GlobalEventExecutor;
 
 import java.io.IOException;
 import java.io.InterruptedIOException;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerPreambleHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerPreambleHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerPreambleHandler.java
index 3754d44..07cf6c6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerPreambleHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerPreambleHandler.java
@@ -17,10 +17,10 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
-import io.netty.buffer.ByteBuf;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.ChannelPipeline;
-import io.netty.channel.SimpleChannelInboundHandler;
+import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBuf;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelHandlerContext;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelPipeline;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.SimpleChannelInboundHandler;
 
 import java.nio.ByteBuffer;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerRequestDecoder.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerRequestDecoder.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerRequestDecoder.java
index a40e9d3..fde8e46 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerRequestDecoder.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerRequestDecoder.java
@@ -17,10 +17,10 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
-import io.netty.buffer.ByteBuf;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.ChannelInboundHandlerAdapter;
-import io.netty.channel.group.ChannelGroup;
+import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBuf;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelHandlerContext;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelInboundHandlerAdapter;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.group.ChannelGroup;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerResponseEncoder.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerResponseEncoder.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerResponseEncoder.java
index b5b6a6b..5c106f8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerResponseEncoder.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerResponseEncoder.java
@@ -17,10 +17,10 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
-import io.netty.buffer.Unpooled;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.ChannelOutboundHandlerAdapter;
-import io.netty.channel.ChannelPromise;
+import org.apache.hadoop.hbase.shaded.io.netty.buffer.Unpooled;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelHandlerContext;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelOutboundHandlerAdapter;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelPromise;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyServerRpcConnection.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyServerRpcConnection.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyServerRpcConnection.java
index 61e12ab..35ce4c7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyServerRpcConnection.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyServerRpcConnection.java
@@ -17,8 +17,8 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
-import io.netty.buffer.ByteBuf;
-import io.netty.channel.Channel;
+import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBuf;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
 
 import java.io.IOException;
 import java.net.InetAddress;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
index 61244cc..ff458ff 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
@@ -792,7 +792,7 @@ public class TableMapReduceUtil {
       prefixTreeCodecClass, //  hbase-prefix-tree (if null will be skipped)
       // pull necessary dependencies
       org.apache.zookeeper.ZooKeeper.class,
-      io.netty.channel.Channel.class,
+      org.apache.hadoop.hbase.shaded.io.netty.channel.Channel.class,
       com.google.protobuf.Message.class,
       org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists.class,
       org.apache.htrace.Trace.class,

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterStatusPublisher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterStatusPublisher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterStatusPublisher.java
index 2c903fb..31a8ba2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterStatusPublisher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterStatusPublisher.java
@@ -21,21 +21,21 @@
 package org.apache.hadoop.hbase.master;
 
 
-import io.netty.bootstrap.Bootstrap;
-import io.netty.bootstrap.ChannelFactory;
-import io.netty.buffer.Unpooled;
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelException;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.ChannelOption;
-import io.netty.channel.EventLoopGroup;
-import io.netty.channel.nio.NioEventLoopGroup;
-import io.netty.channel.socket.DatagramChannel;
-import io.netty.channel.socket.DatagramPacket;
-import io.netty.channel.socket.InternetProtocolFamily;
-import io.netty.channel.socket.nio.NioDatagramChannel;
-import io.netty.handler.codec.MessageToMessageEncoder;
-import io.netty.util.internal.StringUtil;
+import org.apache.hadoop.hbase.shaded.io.netty.bootstrap.Bootstrap;
+import org.apache.hadoop.hbase.shaded.io.netty.bootstrap.ChannelFactory;
+import org.apache.hadoop.hbase.shaded.io.netty.buffer.Unpooled;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelException;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelHandlerContext;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelOption;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoopGroup;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoopGroup;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.DatagramChannel;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.DatagramPacket;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.InternetProtocolFamily;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.nio.NioDatagramChannel;
+import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.MessageToMessageEncoder;
+import org.apache.hadoop.hbase.shaded.io.netty.util.internal.StringUtil;
 
 import java.io.Closeable;
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
index e8d1f92..a4b6b3c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
@@ -24,9 +24,9 @@ import com.lmax.disruptor.RingBuffer;
 import com.lmax.disruptor.Sequence;
 import com.lmax.disruptor.Sequencer;
 
-import io.netty.channel.Channel;
-import io.netty.channel.EventLoop;
-import io.netty.util.concurrent.SingleThreadEventExecutor;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoop;
+import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.SingleThreadEventExecutor;
 
 import java.io.IOException;
 import java.io.InterruptedIOException;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncProtobufLogWriter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncProtobufLogWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncProtobufLogWriter.java
index fa7eddd..9221fc4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncProtobufLogWriter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncProtobufLogWriter.java
@@ -19,8 +19,8 @@ package org.apache.hadoop.hbase.regionserver.wal;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.base.Throwables;
 
-import io.netty.channel.Channel;
-import io.netty.channel.EventLoop;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoop;
 
 import java.io.IOException;
 import java.io.InterruptedIOException;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SecureAsyncProtobufLogWriter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SecureAsyncProtobufLogWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SecureAsyncProtobufLogWriter.java
index 22c8aa8..1e43726 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SecureAsyncProtobufLogWriter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SecureAsyncProtobufLogWriter.java
@@ -25,8 +25,8 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.io.crypto.Encryptor;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALHeader;
 
-import io.netty.channel.Channel;
-import io.netty.channel.EventLoop;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoop;
 
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
 public class SecureAsyncProtobufLogWriter extends AsyncProtobufLogWriter {

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-server/src/main/java/org/apache/hadoop/hbase/util/NettyEventLoopGroupConfig.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/NettyEventLoopGroupConfig.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/NettyEventLoopGroupConfig.java
index 30caf72..83a7122 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/NettyEventLoopGroupConfig.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/NettyEventLoopGroupConfig.java
@@ -17,16 +17,16 @@
  */
 package org.apache.hadoop.hbase.util;
 
-import io.netty.channel.Channel;
-import io.netty.channel.EventLoopGroup;
-import io.netty.channel.ServerChannel;
-import io.netty.channel.epoll.EpollEventLoopGroup;
-import io.netty.channel.epoll.EpollServerSocketChannel;
-import io.netty.channel.epoll.EpollSocketChannel;
-import io.netty.channel.nio.NioEventLoopGroup;
-import io.netty.channel.socket.nio.NioServerSocketChannel;
-import io.netty.channel.socket.nio.NioSocketChannel;
-import io.netty.util.concurrent.DefaultThreadFactory;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoopGroup;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.ServerChannel;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.epoll.EpollEventLoopGroup;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.epoll.EpollServerSocketChannel;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.epoll.EpollSocketChannel;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoopGroup;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.nio.NioServerSocketChannel;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.nio.NioSocketChannel;
+import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.DefaultThreadFactory;
 
 import java.util.concurrent.ThreadFactory;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java
index f0f1671..ac7a4f8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java
@@ -19,12 +19,12 @@ package org.apache.hadoop.hbase.wal;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.base.Throwables;
 
-import io.netty.channel.Channel;
-import io.netty.channel.EventLoop;
-import io.netty.channel.EventLoopGroup;
-import io.netty.channel.nio.NioEventLoopGroup;
-import io.netty.channel.socket.nio.NioSocketChannel;
-import io.netty.util.concurrent.DefaultThreadFactory;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoop;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoopGroup;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoopGroup;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.nio.NioSocketChannel;
+import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.DefaultThreadFactory;
 
 import java.io.IOException;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/NettyAsyncFSWALConfigHelper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/NettyAsyncFSWALConfigHelper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/NettyAsyncFSWALConfigHelper.java
index 273fc37..5b29dbd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/NettyAsyncFSWALConfigHelper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/NettyAsyncFSWALConfigHelper.java
@@ -19,8 +19,8 @@ package org.apache.hadoop.hbase.wal;
 
 import com.google.common.base.Preconditions;
 
-import io.netty.channel.Channel;
-import io.netty.channel.EventLoopGroup;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoopGroup;
 
 import java.util.HashMap;
 import java.util.Map;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestFanOutOneBlockAsyncDFSOutput.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestFanOutOneBlockAsyncDFSOutput.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestFanOutOneBlockAsyncDFSOutput.java
index 43a279e..4377196 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestFanOutOneBlockAsyncDFSOutput.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestFanOutOneBlockAsyncDFSOutput.java
@@ -24,11 +24,11 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.fail;
 
-import io.netty.channel.Channel;
-import io.netty.channel.EventLoop;
-import io.netty.channel.EventLoopGroup;
-import io.netty.channel.nio.NioEventLoopGroup;
-import io.netty.channel.socket.nio.NioSocketChannel;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoop;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoopGroup;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoopGroup;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.nio.NioSocketChannel;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestLocalAsyncOutput.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestLocalAsyncOutput.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestLocalAsyncOutput.java
index 4da778e..1fb5d37 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestLocalAsyncOutput.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestLocalAsyncOutput.java
@@ -20,10 +20,10 @@ package org.apache.hadoop.hbase.io.asyncfs;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 
-import io.netty.channel.Channel;
-import io.netty.channel.EventLoopGroup;
-import io.netty.channel.nio.NioEventLoopGroup;
-import io.netty.channel.socket.nio.NioSocketChannel;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoopGroup;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoopGroup;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.nio.NioSocketChannel;
 
 import java.io.IOException;
 import java.util.concurrent.ExecutionException;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6ac04ab/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestSaslFanOutOneBlockAsyncDFSOutput.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestSaslFanOutOneBlockAsyncDFSOutput.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestSaslFanOutOneBlockAsyncDFSOutput.java
index 7e67a90..8e5df03 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestSaslFanOutOneBlockAsyncDFSOutput.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestSaslFanOutOneBlockAsyncDFSOutput.java
@@ -31,11 +31,11 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIP
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY;
 
-import io.netty.channel.Channel;
-import io.netty.channel.EventLoop;
-import io.netty.channel.EventLoopGroup;
-import io.netty.channel.nio.NioEventLoopGroup;
-import io.netty.channel.socket.nio.NioSocketChannel;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoop;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoopGroup;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoopGroup;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.nio.NioSocketChannel;
 
 import java.io.File;
 import java.io.IOException;


[48/50] [abbrv] hbase git commit: HBASE-18608 AsyncConnection should return AsyncAdmin interface instead of the implemenation

Posted by st...@apache.org.
HBASE-18608 AsyncConnection should return AsyncAdmin interface instead of the implemenation


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

Branch: refs/heads/HBASE-14070.HLC
Commit: 092dc6de8483eea1b4e0d960cf22e65359379da1
Parents: a17ed03
Author: Guanghao Zhang <zg...@apache.org>
Authored: Wed Aug 16 18:00:53 2017 +0800
Committer: Guanghao Zhang <zg...@apache.org>
Committed: Thu Aug 17 09:47:39 2017 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/client/AsyncAdminBuilder.java      | 16 ++++++++--------
 .../hadoop/hbase/client/AsyncAdminBuilderBase.java  | 12 ++++++------
 .../apache/hadoop/hbase/client/AsyncConnection.java |  4 ++--
 .../hadoop/hbase/client/AsyncConnectionImpl.java    | 12 ++++++------
 .../hadoop/hbase/client/RawAsyncHBaseAdmin.java     |  2 +-
 .../hadoop/hbase/client/TestAsyncAdminBuilder.java  |  6 +++---
 6 files changed, 26 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/092dc6de/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdminBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdminBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdminBuilder.java
index d706949..fb0aefd 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdminBuilder.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdminBuilder.java
@@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
  * create a new AsyncAdmin instance.
  */
 @InterfaceAudience.Public
-public interface AsyncAdminBuilder<T extends AsyncAdmin> {
+public interface AsyncAdminBuilder {
 
   /**
    * Set timeout for a whole admin operation. Operation timeout and max attempt times(or max retry
@@ -39,7 +39,7 @@ public interface AsyncAdminBuilder<T extends AsyncAdmin> {
    * @param unit
    * @return this for invocation chaining
    */
-  AsyncAdminBuilder<T> setOperationTimeout(long timeout, TimeUnit unit);
+  AsyncAdminBuilder setOperationTimeout(long timeout, TimeUnit unit);
 
   /**
    * Set timeout for each rpc request.
@@ -47,7 +47,7 @@ public interface AsyncAdminBuilder<T extends AsyncAdmin> {
    * @param unit
    * @return this for invocation chaining
    */
-  AsyncAdminBuilder<T> setRpcTimeout(long timeout, TimeUnit unit);
+  AsyncAdminBuilder setRpcTimeout(long timeout, TimeUnit unit);
 
   /**
    * Set the base pause time for retrying. We use an exponential policy to generate sleep time when
@@ -56,7 +56,7 @@ public interface AsyncAdminBuilder<T extends AsyncAdmin> {
    * @param unit
    * @return this for invocation chaining
    */
-  AsyncAdminBuilder<T> setRetryPause(long timeout, TimeUnit unit);
+  AsyncAdminBuilder setRetryPause(long timeout, TimeUnit unit);
 
   /**
    * Set the max retry times for an admin operation. Usually it is the max attempt times minus 1.
@@ -65,7 +65,7 @@ public interface AsyncAdminBuilder<T extends AsyncAdmin> {
    * @param maxRetries
    * @return this for invocation chaining
    */
-  default AsyncAdminBuilder<T> setMaxRetries(int maxRetries) {
+  default AsyncAdminBuilder setMaxRetries(int maxRetries) {
     return setMaxAttempts(retries2Attempts(maxRetries));
   }
 
@@ -76,18 +76,18 @@ public interface AsyncAdminBuilder<T extends AsyncAdmin> {
    * @param maxAttempts
    * @return this for invocation chaining
    */
-  AsyncAdminBuilder<T> setMaxAttempts(int maxAttempts);
+  AsyncAdminBuilder setMaxAttempts(int maxAttempts);
 
   /**
    * Set the number of retries that are allowed before we start to log.
    * @param startLogErrorsCnt
    * @return this for invocation chaining
    */
-  AsyncAdminBuilder<T> setStartLogErrorsCnt(int startLogErrorsCnt);
+  AsyncAdminBuilder setStartLogErrorsCnt(int startLogErrorsCnt);
 
   /**
    * Create a {@link AsyncAdmin} instance.
    * @return a {@link AsyncAdmin} instance
    */
-  T build();
+  AsyncAdmin build();
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/092dc6de/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdminBuilderBase.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdminBuilderBase.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdminBuilderBase.java
index 013e8d7..77ff88d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdminBuilderBase.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdminBuilderBase.java
@@ -25,7 +25,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
  * Base class for all asynchronous admin builders.
  */
 @InterfaceAudience.Private
-abstract class AsyncAdminBuilderBase<T extends AsyncAdmin> implements AsyncAdminBuilder<T> {
+abstract class AsyncAdminBuilderBase implements AsyncAdminBuilder {
 
   protected long rpcTimeoutNs;
 
@@ -46,31 +46,31 @@ abstract class AsyncAdminBuilderBase<T extends AsyncAdmin> implements AsyncAdmin
   }
 
   @Override
-  public AsyncAdminBuilder<T> setOperationTimeout(long timeout, TimeUnit unit) {
+  public AsyncAdminBuilder setOperationTimeout(long timeout, TimeUnit unit) {
     this.operationTimeoutNs = unit.toNanos(timeout);
     return this;
   }
 
   @Override
-  public AsyncAdminBuilder<T> setRpcTimeout(long timeout, TimeUnit unit) {
+  public AsyncAdminBuilder setRpcTimeout(long timeout, TimeUnit unit) {
     this.rpcTimeoutNs = unit.toNanos(timeout);
     return this;
   }
 
   @Override
-  public AsyncAdminBuilder<T> setRetryPause(long timeout, TimeUnit unit) {
+  public AsyncAdminBuilder setRetryPause(long timeout, TimeUnit unit) {
     this.pauseNs = unit.toNanos(timeout);
     return this;
   }
 
   @Override
-  public AsyncAdminBuilder<T> setMaxAttempts(int maxAttempts) {
+  public AsyncAdminBuilder setMaxAttempts(int maxAttempts) {
     this.maxAttempts = maxAttempts;
     return this;
   }
 
   @Override
-  public AsyncAdminBuilder<T> setStartLogErrorsCnt(int startLogErrorsCnt) {
+  public AsyncAdminBuilder setStartLogErrorsCnt(int startLogErrorsCnt) {
     this.startLogErrorsCnt = startLogErrorsCnt;
     return this;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/092dc6de/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnection.java
index 04ef78e..24907ba 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnection.java
@@ -117,7 +117,7 @@ public interface AsyncConnection extends Closeable {
    * framework's callback thread, so typically you should not do any time consuming work inside
    * these methods.
    */
-  AsyncAdminBuilder<RawAsyncHBaseAdmin> getAdminBuilder();
+  AsyncAdminBuilder getAdminBuilder();
 
   /**
    * Retrieve an {@link AsyncAdmin} implementation to administer an HBase cluster.
@@ -135,5 +135,5 @@ public interface AsyncConnection extends Closeable {
    * Returns an {@link AsyncAdminBuilder} for creating {@link AsyncAdmin}.
    * @param pool the thread pool to use for executing callback
    */
-  AsyncAdminBuilder<AsyncHBaseAdmin> getAdminBuilder(ExecutorService pool);
+  AsyncAdminBuilder getAdminBuilder(ExecutorService pool);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/092dc6de/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
index 9913af9..d8f051f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
@@ -278,20 +278,20 @@ class AsyncConnectionImpl implements AsyncConnection {
   }
 
   @Override
-  public AsyncAdminBuilder<RawAsyncHBaseAdmin> getAdminBuilder() {
-    return new AsyncAdminBuilderBase<RawAsyncHBaseAdmin>(connConf) {
+  public AsyncAdminBuilder getAdminBuilder() {
+    return new AsyncAdminBuilderBase(connConf) {
       @Override
-      public RawAsyncHBaseAdmin build() {
+      public AsyncAdmin build() {
         return new RawAsyncHBaseAdmin(AsyncConnectionImpl.this, this);
       }
     };
   }
 
   @Override
-  public AsyncAdminBuilder<AsyncHBaseAdmin> getAdminBuilder(ExecutorService pool) {
-    return new AsyncAdminBuilderBase<AsyncHBaseAdmin>(connConf) {
+  public AsyncAdminBuilder getAdminBuilder(ExecutorService pool) {
+    return new AsyncAdminBuilderBase(connConf) {
       @Override
-      public AsyncHBaseAdmin build() {
+      public AsyncAdmin build() {
         RawAsyncHBaseAdmin rawAdmin = new RawAsyncHBaseAdmin(AsyncConnectionImpl.this, this);
         return new AsyncHBaseAdmin(rawAdmin, pool);
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/092dc6de/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
index 0070e0e..ba68a96 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
@@ -278,7 +278,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
 
   private final NonceGenerator ng;
 
-  RawAsyncHBaseAdmin(AsyncConnectionImpl connection, AsyncAdminBuilderBase<?> builder) {
+  RawAsyncHBaseAdmin(AsyncConnectionImpl connection, AsyncAdminBuilderBase builder) {
     this.connection = connection;
     this.metaTable = connection.getRawTable(META_TABLE_NAME);
     this.rpcTimeoutNs = builder.rpcTimeoutNs;

http://git-wip-us.apache.org/repos/asf/hbase/blob/092dc6de/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminBuilder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminBuilder.java
index ea25ee4..1f7346b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminBuilder.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminBuilder.java
@@ -59,13 +59,13 @@ public class TestAsyncAdminBuilder {
   private static AsyncConnection ASYNC_CONN;
 
   @Parameter
-  public Supplier<AsyncAdminBuilder<?>> getAdminBuilder;
+  public Supplier<AsyncAdminBuilder> getAdminBuilder;
 
-  private static AsyncAdminBuilder<RawAsyncHBaseAdmin> getRawAsyncAdminBuilder() {
+  private static AsyncAdminBuilder getRawAsyncAdminBuilder() {
     return ASYNC_CONN.getAdminBuilder();
   }
 
-  private static AsyncAdminBuilder<AsyncHBaseAdmin> getAsyncAdminBuilder() {
+  private static AsyncAdminBuilder getAsyncAdminBuilder() {
     return ASYNC_CONN.getAdminBuilder(ForkJoinPool.commonPool());
   }
 


[20/50] [abbrv] hbase git commit: HBASE-18566 [RSGROUP]Log the client IP/port of the rsgroup admin

Posted by st...@apache.org.
HBASE-18566 [RSGROUP]Log the client IP/port of the rsgroup admin

Signed-off-by: tedyu <yu...@gmail.com>


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

Branch: refs/heads/HBASE-14070.HLC
Commit: c6bf4d51928985517c438d1efcaefdeddc9ac5ef
Parents: 05e6e56
Author: Guangxu Cheng <gu...@gmail.com>
Authored: Mon Aug 14 10:44:52 2017 +0800
Committer: tedyu <yu...@gmail.com>
Committed: Sun Aug 13 20:29:12 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java   | 15 +++++++++++++++
 .../java/org/apache/hadoop/hbase/master/HMaster.java |  2 +-
 .../apache/hadoop/hbase/master/MasterServices.java   |  2 ++
 .../hadoop/hbase/master/MockNoopMasterServices.java  |  5 +++++
 4 files changed, 23 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c6bf4d51/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
index 9fda3f0..0bc5c76 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
@@ -120,6 +120,8 @@ public class RSGroupAdminEndpoint implements MasterObserver, CoprocessorService
         GetRSGroupInfoRequest request, RpcCallback<GetRSGroupInfoResponse> done) {
       GetRSGroupInfoResponse.Builder builder = GetRSGroupInfoResponse.newBuilder();
       String groupName = request.getRSGroupName();
+      LOG.info(master.getClientIdAuditPrefix() + " initiates rsgroup info retrieval, group="
+              + groupName);
       try {
         RSGroupInfo rsGroupInfo = groupAdminServer.getRSGroupInfo(groupName);
         if (rsGroupInfo != null) {
@@ -137,6 +139,8 @@ public class RSGroupAdminEndpoint implements MasterObserver, CoprocessorService
       GetRSGroupInfoOfTableResponse.Builder builder = GetRSGroupInfoOfTableResponse.newBuilder();
       try {
         TableName tableName = ProtobufUtil.toTableName(request.getTableName());
+        LOG.info(master.getClientIdAuditPrefix() + " initiates rsgroup info retrieval, table="
+                + tableName);
         RSGroupInfo RSGroupInfo = groupAdminServer.getRSGroupInfoOfTable(tableName);
         if (RSGroupInfo != null) {
           builder.setRSGroupInfo(RSGroupProtobufUtil.toProtoGroupInfo(RSGroupInfo));
@@ -156,6 +160,8 @@ public class RSGroupAdminEndpoint implements MasterObserver, CoprocessorService
         for (HBaseProtos.ServerName el : request.getServersList()) {
           hostPorts.add(Address.fromParts(el.getHostName(), el.getPort()));
         }
+        LOG.info(master.getClientIdAuditPrefix() + " move servers " + hostPorts +" to rsgroup "
+                + request.getTargetGroup());
         groupAdminServer.moveServers(hostPorts, request.getTargetGroup());
       } catch (IOException e) {
         CoprocessorRpcUtils.setControllerException(controller, e);
@@ -172,6 +178,8 @@ public class RSGroupAdminEndpoint implements MasterObserver, CoprocessorService
         for (HBaseProtos.TableName tableName : request.getTableNameList()) {
           tables.add(ProtobufUtil.toTableName(tableName));
         }
+        LOG.info(master.getClientIdAuditPrefix() + " move tables " + tables +" to rsgroup "
+                + request.getTargetGroup());
         groupAdminServer.moveTables(tables, request.getTargetGroup());
       } catch (IOException e) {
         CoprocessorRpcUtils.setControllerException(controller, e);
@@ -183,6 +191,7 @@ public class RSGroupAdminEndpoint implements MasterObserver, CoprocessorService
     public void addRSGroup(RpcController controller, AddRSGroupRequest request,
         RpcCallback<AddRSGroupResponse> done) {
       AddRSGroupResponse.Builder builder = AddRSGroupResponse.newBuilder();
+      LOG.info(master.getClientIdAuditPrefix() + " add rsgroup " + request.getRSGroupName());
       try {
         groupAdminServer.addRSGroup(request.getRSGroupName());
       } catch (IOException e) {
@@ -196,6 +205,7 @@ public class RSGroupAdminEndpoint implements MasterObserver, CoprocessorService
         RemoveRSGroupRequest request, RpcCallback<RemoveRSGroupResponse> done) {
       RemoveRSGroupResponse.Builder builder =
           RemoveRSGroupResponse.newBuilder();
+      LOG.info(master.getClientIdAuditPrefix() + " remove rsgroup " + request.getRSGroupName());
       try {
         groupAdminServer.removeRSGroup(request.getRSGroupName());
       } catch (IOException e) {
@@ -208,6 +218,7 @@ public class RSGroupAdminEndpoint implements MasterObserver, CoprocessorService
     public void balanceRSGroup(RpcController controller,
         BalanceRSGroupRequest request, RpcCallback<BalanceRSGroupResponse> done) {
       BalanceRSGroupResponse.Builder builder = BalanceRSGroupResponse.newBuilder();
+      LOG.info(master.getClientIdAuditPrefix() + " balance rsgroup, group=" + request.getRSGroupName());
       try {
         builder.setBalanceRan(groupAdminServer.balanceRSGroup(request.getRSGroupName()));
       } catch (IOException e) {
@@ -221,6 +232,7 @@ public class RSGroupAdminEndpoint implements MasterObserver, CoprocessorService
     public void listRSGroupInfos(RpcController controller,
         ListRSGroupInfosRequest request, RpcCallback<ListRSGroupInfosResponse> done) {
       ListRSGroupInfosResponse.Builder builder = ListRSGroupInfosResponse.newBuilder();
+      LOG.info(master.getClientIdAuditPrefix() + " list rsgroup");
       try {
         for (RSGroupInfo RSGroupInfo : groupAdminServer.listRSGroups()) {
           builder.addRSGroupInfo(RSGroupProtobufUtil.toProtoGroupInfo(RSGroupInfo));
@@ -238,6 +250,7 @@ public class RSGroupAdminEndpoint implements MasterObserver, CoprocessorService
       try {
         Address hp = Address.fromParts(request.getServer().getHostName(),
             request.getServer().getPort());
+        LOG.info(master.getClientIdAuditPrefix() + " initiates rsgroup info retrieval, server=" + hp);
         RSGroupInfo RSGroupInfo = groupAdminServer.getRSGroupOfServer(hp);
         if (RSGroupInfo != null) {
           builder.setRSGroupInfo(RSGroupProtobufUtil.toProtoGroupInfo(RSGroupInfo));
@@ -261,6 +274,8 @@ public class RSGroupAdminEndpoint implements MasterObserver, CoprocessorService
         for (HBaseProtos.TableName tableName : request.getTableNameList()) {
           tables.add(ProtobufUtil.toTableName(tableName));
         }
+        LOG.info(master.getClientIdAuditPrefix() + " move servers " + hostPorts
+                + " and tables " + tables + " to rsgroup" + request.getTargetGroup());
         groupAdminServer.moveServersAndTables(hostPorts, tables, request.getTargetGroup());
       } catch (IOException e) {
         CoprocessorRpcUtils.setControllerException(controller, e);

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6bf4d51/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 faf0893..ce83838 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
@@ -1517,7 +1517,7 @@ public class HMaster extends HRegionServer implements MasterServices {
   /**
    * @return Client info for use as prefix on an audit log string; who did an action
    */
-  String getClientIdAuditPrefix() {
+  public String getClientIdAuditPrefix() {
     return "Client=" + RpcServer.getRequestUserName() + "/" + RpcServer.getRemoteAddress();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6bf4d51/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
index f7f5d06..6e97bf4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
@@ -508,4 +508,6 @@ public interface MasterServices extends Server {
    * method to make sure meta is initialized.
    */
   boolean recoverMeta() throws IOException;
+
+  String getClientIdAuditPrefix();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c6bf4d51/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
index aac0e21..27c9a5f 100644
--- 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
@@ -460,6 +460,11 @@ public class MockNoopMasterServices implements MasterServices, Server {
   }
 
   @Override
+  public String getClientIdAuditPrefix() {
+    return null;
+  }
+
+  @Override
   public ProcedureEvent getInitializedEvent() {
     // TODO Auto-generated method stub
     return null;


[46/50] [abbrv] hbase git commit: HBASE-18573 Update Append and Delete to use Mutation#getCellList(family)

Posted by st...@apache.org.
HBASE-18573 Update Append and Delete to use Mutation#getCellList(family)

Signed-off-by: Jerry He <je...@apache.org>


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

Branch: refs/heads/HBASE-14070.HLC
Commit: 4c3a64db13b086ad3d8a6ffa1be8ba2f5a24719c
Parents: 5d2c3dd
Author: Xiang Li <wa...@gmail.com>
Authored: Thu Aug 17 00:39:35 2017 +0800
Committer: Jerry He <je...@apache.org>
Committed: Wed Aug 16 14:50:46 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/client/Append.java  |  9 +++---
 .../org/apache/hadoop/hbase/client/Delete.java  | 31 ++++----------------
 2 files changed, 10 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/4c3a64db/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Append.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Append.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Append.java
index 2bd0860..6947313 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Append.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Append.java
@@ -134,11 +134,10 @@ public class Append extends Mutation {
   public Append add(final Cell cell) {
     // Presume it is KeyValue for now.
     byte [] family = CellUtil.cloneFamily(cell);
-    List<Cell> list = this.familyMap.get(family);
-    if (list == null) {
-      list  = new ArrayList<>(1);
-      this.familyMap.put(family, list);
-    }
+
+    // Get cell list for the family
+    List<Cell> list = getCellList(family);
+
     // find where the new entry should be placed in the List
     list.add(cell);
     return this;

http://git-wip-us.apache.org/repos/asf/hbase/blob/4c3a64db/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java
index bf5241c..66b6cfc 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java
@@ -180,11 +180,7 @@ public class Delete extends Mutation implements Comparable<Row> {
         " doesn't match the original one " +  Bytes.toStringBinary(this.row));
     }
     byte [] family = CellUtil.cloneFamily(kv);
-    List<Cell> list = familyMap.get(family);
-    if (list == null) {
-      list = new ArrayList<>(1);
-      familyMap.put(family, list);
-    }
+    List<Cell> list = getCellList(family);
     list.add(kv);
     return this;
   }
@@ -216,11 +212,8 @@ public class Delete extends Mutation implements Comparable<Row> {
     if (timestamp < 0) {
       throw new IllegalArgumentException("Timestamp cannot be negative. ts=" + timestamp);
     }
-    List<Cell> list = familyMap.get(family);
-    if(list == null) {
-      list = new ArrayList<>(1);
-      familyMap.put(family, list);
-    } else if(!list.isEmpty()) {
+    List<Cell> list = getCellList(family);
+    if(!list.isEmpty()) {
       list.clear();
     }
     KeyValue kv = new KeyValue(row, family, null, timestamp, KeyValue.Type.DeleteFamily);
@@ -236,11 +229,7 @@ public class Delete extends Mutation implements Comparable<Row> {
    * @return this for invocation chaining
    */
   public Delete addFamilyVersion(final byte [] family, final long timestamp) {
-    List<Cell> list = familyMap.get(family);
-    if(list == null) {
-      list = new ArrayList<>(1);
-      familyMap.put(family, list);
-    }
+    List<Cell> list = getCellList(family);
     list.add(new KeyValue(row, family, null, timestamp,
           KeyValue.Type.DeleteFamilyVersion));
     return this;
@@ -269,11 +258,7 @@ public class Delete extends Mutation implements Comparable<Row> {
     if (timestamp < 0) {
       throw new IllegalArgumentException("Timestamp cannot be negative. ts=" + timestamp);
     }
-    List<Cell> list = familyMap.get(family);
-    if (list == null) {
-      list = new ArrayList<>(1);
-      familyMap.put(family, list);
-    }
+    List<Cell> list = getCellList(family);
     list.add(new KeyValue(this.row, family, qualifier, timestamp,
         KeyValue.Type.DeleteColumn));
     return this;
@@ -304,11 +289,7 @@ public class Delete extends Mutation implements Comparable<Row> {
     if (timestamp < 0) {
       throw new IllegalArgumentException("Timestamp cannot be negative. ts=" + timestamp);
     }
-    List<Cell> list = familyMap.get(family);
-    if(list == null) {
-      list = new ArrayList<>(1);
-      familyMap.put(family, list);
-    }
+    List<Cell> list = getCellList(family);
     KeyValue kv = new KeyValue(this.row, family, qualifier, timestamp, KeyValue.Type.Delete);
     list.add(kv);
     return this;


[22/50] [abbrv] hbase git commit: HBASE-18588 Verify we're using netty .so epolling on linux post HBASE-18271

Posted by st...@apache.org.
HBASE-18588 Verify we're using netty .so epolling on linux post HBASE-18271


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

Branch: refs/heads/HBASE-14070.HLC
Commit: ddbaf56ca8c712dc44608d3323280f578c56aed2
Parents: e2b797b
Author: Michael Stack <st...@apache.org>
Authored: Mon Aug 14 09:09:56 2017 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Mon Aug 14 09:09:56 2017 -0700

----------------------------------------------------------------------
 hbase-protocol-shaded/pom.xml | 17 +++++++++++++++++
 1 file changed, 17 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ddbaf56c/hbase-protocol-shaded/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/pom.xml b/hbase-protocol-shaded/pom.xml
index b28c03e..4c72eca 100644
--- a/hbase-protocol-shaded/pom.xml
+++ b/hbase-protocol-shaded/pom.xml
@@ -192,6 +192,23 @@
                     <ignore/>
                   </action>
                 </pluginExecution>
+                <pluginExecution>
+                  <pluginExecutionFilter>
+                    <groupId>
+                      com.google.code.maven-replacer-plugin
+                    </groupId>
+                    <artifactId>replacer</artifactId>
+                    <versionRange>[1.5.3,)</versionRange>
+                    <goals>
+                      <goal>replace</goal>
+                    </goals>
+                  </pluginExecutionFilter>
+                  <action>
+                    <execute>
+                     <runOnIncremental>false</runOnIncremental>
+                    </execute>
+                  </action>
+                </pluginExecution>
               </pluginExecutions>
             </lifecycleMappingMetadata>
           </configuration>


[44/50] [abbrv] hbase git commit: HBASE-18511 Default no regions on master

Posted by st...@apache.org.
HBASE-18511 Default no regions on master

Changes the configuration hbase.balancer.tablesOnMaster from list of
table names to instead be a boolean; true if master carries
tables/regions and false if it does not.

Adds a new configuration hbase.balancer.tablesOnMaster.systemTablesOnly.
If true, hbase.balancer.tablesOnMaster is considered true but only
system tables are put on the master.

M hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
  Master was claiming itself active master though it had stopped. Fix
the activeMaster flag. Set it to false on exit.

M hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java
 Add new configs and convenience methods for getting current state of
settings.

M hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
 Move configs up into super Interface and now the settings mean
different, remove the no longer needed processing.


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

Branch: refs/heads/HBASE-14070.HLC
Commit: 473446719b7b81b56216862bf2a94a576ff90f60
Parents: acf9b87
Author: Michael Stack <st...@apache.org>
Authored: Wed Aug 2 22:54:21 2017 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Wed Aug 16 08:39:36 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/master/HMaster.java |  28 ++-
 .../hadoop/hbase/master/LoadBalancer.java       |  31 ++-
 .../hadoop/hbase/master/ServerManager.java      |  30 +--
 .../hbase/master/balancer/BaseLoadBalancer.java | 110 ++++------
 .../balancer/FavoredStochasticBalancer.java     |  11 +-
 .../hbase/regionserver/HRegionServer.java       |   5 +-
 .../hadoop/hbase/HBaseTestingUtility.java       |   2 +-
 .../apache/hadoop/hbase/MiniHBaseCluster.java   |   3 +-
 .../apache/hadoop/hbase/client/TestAdmin1.java  |   8 +-
 .../hbase/client/TestAsyncTableAdminApi.java    |  16 +-
 .../hbase/client/TestClientClusterStatus.java   |   5 +-
 .../hadoop/hbase/client/TestFromClientSide.java |   5 +-
 .../hadoop/hbase/fs/TestBlockReorder.java       |   4 +-
 .../hadoop/hbase/master/TestMasterMetrics.java  |  19 +-
 .../hbase/master/TestMasterMetricsWrapper.java  |  13 +-
 .../hbase/master/TestMasterNoCluster.java       |   7 +-
 .../master/balancer/TestBaseLoadBalancer.java   |  10 +-
 .../balancer/TestRegionsOnMasterOptions.java    | 200 +++++++++++++++++++
 .../hbase/regionserver/TestClusterId.java       |   4 +-
 .../TestRSKilledWhenInitializing.java           |  15 +-
 .../hbase/regionserver/TestRegionOpen.java      |   5 +-
 .../regionserver/TestRegionServerAbort.java     |  14 +-
 .../regionserver/TestRegionServerHostname.java  |  11 +-
 .../regionserver/TestRegionServerMetrics.java   |  57 ++++--
 .../TestRegionServerReadRequestMetrics.java     |  12 +-
 .../TestRegionServerReportForDuty.java          |  15 +-
 .../TestSplitTransactionOnCluster.java          |  16 +-
 .../TestFlushWithThroughputController.java      |   8 +-
 .../security/access/TestNamespaceCommands.java  |  13 +-
 29 files changed, 491 insertions(+), 186 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/47344671/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 ce83838..6b4d4e9 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
@@ -530,6 +530,17 @@ public class HMaster extends HRegionServer implements MasterServices {
     }
   }
 
+  // Main run loop. Calls through to the regionserver run loop.
+  @Override
+  public void run() {
+    try {
+      super.run();
+    } finally {
+      // If on way out, then we are no longer active master.
+      this.activeMaster = false;
+    }
+  }
+
   // return the actual infoPort, -1 means disable info server.
   private int putUpJettyServer() throws IOException {
     if (!conf.getBoolean("hbase.master.infoserver.redirect", true)) {
@@ -604,9 +615,8 @@ public class HMaster extends HRegionServer implements MasterServices {
    */
   @Override
   protected void waitForMasterActive(){
-    boolean tablesOnMaster = BaseLoadBalancer.tablesOnMaster(conf);
-    while (!(tablesOnMaster && activeMaster)
-        && !isStopped() && !isAborted()) {
+    boolean tablesOnMaster = LoadBalancer.isTablesOnMaster(conf);
+    while (!(tablesOnMaster && activeMaster) && !isStopped() && !isAborted()) {
       sleeper.sleep();
     }
   }
@@ -644,7 +654,7 @@ public class HMaster extends HRegionServer implements MasterServices {
   protected void configureInfoServer() {
     infoServer.addServlet("master-status", "/master-status", MasterStatusServlet.class);
     infoServer.setAttribute(MASTER, this);
-    if (BaseLoadBalancer.tablesOnMaster(conf)) {
+    if (LoadBalancer.isTablesOnMaster(conf)) {
       super.configureInfoServer();
     }
   }
@@ -796,14 +806,16 @@ public class HMaster extends HRegionServer implements MasterServices {
     sleeper.skipSleepCycle();
 
     // Wait for region servers to report in
-    status.setStatus("Wait for region servers to report in");
+    String statusStr = "Wait for region servers to report in";
+    status.setStatus(statusStr);
+    LOG.info(status);
     waitForRegionServers(status);
 
     if (this.balancer instanceof FavoredNodesPromoter) {
       favoredNodesManager = new FavoredNodesManager(this);
     }
     // Wait for regionserver to finish initialization.
-    if (BaseLoadBalancer.tablesOnMaster(conf)) {
+    if (LoadBalancer.isTablesOnMaster(conf)) {
       waitForServerOnline();
     }
 
@@ -1643,11 +1655,11 @@ public class HMaster extends HRegionServer implements MasterServices {
         LOG.debug("Unable to determine a plan to assign " + hri);
         return;
       }
+      // TODO: What is this? I don't get it.
       if (dest.equals(serverName) && balancer instanceof BaseLoadBalancer
           && !((BaseLoadBalancer)balancer).shouldBeOnMaster(hri)) {
         // To avoid unnecessary region moving later by balancer. Don't put user
-        // regions on master. Regions on master could be put on other region
-        // server intentionally by test however.
+        // regions on master.
         LOG.debug("Skipping move of region " + hri.getRegionNameAsString()
           + " to avoid unnecessary region moving later by load balancer,"
           + " because it should not be on master");

http://git-wip-us.apache.org/repos/asf/hbase/blob/47344671/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java
index 5025566..3e3234a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java
@@ -18,9 +18,10 @@
  */
 package org.apache.hadoop.hbase.master;
 
-import java.util.List;
-import java.util.Map;
+import java.util.*;
 
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.UnmodifiableIterator;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ClusterStatus;
@@ -33,6 +34,9 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.conf.ConfigurationObserver;
 
 import edu.umd.cs.findbugs.annotations.Nullable;
+import org.apache.hadoop.hbase.security.access.AccessControlLists;
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.util.StringUtils;
 
 /**
  * Makes decisions about the placement and movement of Regions across
@@ -50,6 +54,18 @@ import edu.umd.cs.findbugs.annotations.Nullable;
  */
 @InterfaceAudience.Private
 public interface LoadBalancer extends Configurable, Stoppable, ConfigurationObserver {
+  /**
+   * Master can carry regions as of hbase-2.0.0.
+   * By default, it carries no tables.
+   * TODO: Add any | system as flags to indicate what it can do.
+   */
+  public static final String TABLES_ON_MASTER = "hbase.balancer.tablesOnMaster";
+
+  /**
+   * Master carries system tables.
+   */
+  public static final String SYSTEM_TABLES_ON_MASTER =
+    "hbase.balancer.tablesOnMaster.systemTablesOnly";
 
   // Used to signal to the caller that the region(s) cannot be assigned
   // We deliberately use 'localhost' so the operation will fail fast
@@ -147,4 +163,15 @@ public interface LoadBalancer extends Configurable, Stoppable, ConfigurationObse
    * @param conf
    */
   void onConfigurationChange(Configuration conf);
+
+  /**
+   * @return true if Master carries regions
+   */
+  static boolean isTablesOnMaster(Configuration conf) {
+    return conf.getBoolean(TABLES_ON_MASTER, false);
+  }
+
+  static boolean isSystemTablesOnlyOnMaster(Configuration conf) {
+    return conf.getBoolean(SYSTEM_TABLES_ON_MASTER, false);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/47344671/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 f0e9b88..86177b8 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
@@ -35,6 +35,7 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentNavigableMap;
 import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.function.Predicate;
 
 import org.apache.commons.logging.Log;
@@ -109,7 +110,7 @@ public class ServerManager {
   private static final Log LOG = LogFactory.getLog(ServerManager.class);
 
   // Set if we are to shutdown the cluster.
-  private volatile boolean clusterShutdown = false;
+  private AtomicBoolean clusterShutdown = new AtomicBoolean(false);
 
   /**
    * The last flushed sequence id for a region.
@@ -423,7 +424,6 @@ public class ServerManager {
   /**
    * Adds the onlineServers list. onlineServers should be locked.
    * @param serverName The remote servers name.
-   * @param sl
    */
   @VisibleForTesting
   void recordNewServerWithLock(final ServerName serverName, final ServerLoad sl) {
@@ -583,7 +583,7 @@ public class ServerManager {
 
     // If cluster is going down, yes, servers are going to be expiring; don't
     // process as a dead server
-    if (this.clusterShutdown) {
+    if (this.clusterShutdown.get()) {
       LOG.info("Cluster shutdown set; " + serverName +
         " expired; onlineServers=" + this.onlineServers.size());
       if (this.onlineServers.isEmpty()) {
@@ -591,7 +591,7 @@ public class ServerManager {
       }
       return;
     }
-
+    LOG.info("Processing expiration of " + serverName + " on " + this.master.getServerName());
     master.getAssignmentManager().submitServerCrash(serverName, true);
 
     // Tell our listeners that a server was removed
@@ -790,12 +790,12 @@ public class ServerManager {
   private int getMinToStart() {
     // One server should be enough to get us off the ground.
     int requiredMinToStart = 1;
-    if (BaseLoadBalancer.tablesOnMaster(master.getConfiguration())) {
-      if (!BaseLoadBalancer.userTablesOnMaster(master.getConfiguration())) {
-        // If Master is carrying regions but NOT user-space regions (the current default),
-        // since the Master shows as a 'server', we need at least one more server to check
-        // in before we can start up so up defaultMinToStart to 2.
-        requiredMinToStart = 2;
+    if (LoadBalancer.isTablesOnMaster(master.getConfiguration())) {
+      if (LoadBalancer.isSystemTablesOnlyOnMaster(master.getConfiguration())) {
+        // If Master is carrying regions but NOT user-space regions, it
+        // still shows as a 'server'. We need at least one more server to check
+        // in before we can start up so set defaultMinToStart to 2.
+        requiredMinToStart = requiredMinToStart + 1;
       }
     }
     int minToStart = this.master.getConfiguration().getInt(WAIT_ON_REGIONSERVERS_MINTOSTART, -1);
@@ -944,12 +944,14 @@ public class ServerManager {
   }
 
   public void shutdownCluster() {
-    this.clusterShutdown = true;
-    this.master.stop("Cluster shutdown requested");
+    String statusStr = "Cluster shutdown requested of master=" + this.master.getServerName();
+    LOG.info(statusStr);
+    this.clusterShutdown.set(true);
+    this.master.stop(statusStr);
   }
 
   public boolean isClusterShutdown() {
-    return this.clusterShutdown;
+    return this.clusterShutdown.get();
   }
 
   /**
@@ -973,7 +975,7 @@ public class ServerManager {
   public List<ServerName> createDestinationServersList(final List<ServerName> serversToExclude){
     final List<ServerName> destServers = getOnlineServersList();
 
-    if (serversToExclude != null){
+    if (serversToExclude != null) {
       destServers.removeAll(serversToExclude);
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/47344671/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
index 30f59a9..f7203b9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
@@ -61,6 +61,7 @@ import org.apache.hadoop.hbase.shaded.com.google.common.base.Joiner;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.ArrayListMultimap;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
+import org.apache.zookeeper.KeeperException;
 
 /**
  * The base class for load balancers. It provides the the functions used to by
@@ -991,69 +992,22 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
 
   // slop for regions
   protected float slop;
-  // overallSlop to controll simpleLoadBalancer's cluster level threshold
+  // overallSlop to control simpleLoadBalancer's cluster level threshold
   protected float overallSlop;
   protected Configuration config;
   protected RackManager rackManager;
   private static final Random RANDOM = new Random(System.currentTimeMillis());
   private static final Log LOG = LogFactory.getLog(BaseLoadBalancer.class);
-
-  // Regions of these tables are put on the master by default.
-  private static final String[] DEFAULT_TABLES_ON_MASTER =
-    new String[] {AccessControlLists.ACL_TABLE_NAME.getNameAsString(),
-      TableName.NAMESPACE_TABLE_NAME.getNameAsString(),
-      TableName.META_TABLE_NAME.getNameAsString()};
-
-  public static final String TABLES_ON_MASTER =
-    "hbase.balancer.tablesOnMaster";
-
-  protected final Set<String> tablesOnMaster = new HashSet<>();
   protected MetricsBalancer metricsBalancer = null;
   protected ClusterStatus clusterStatus = null;
   protected ServerName masterServerName;
   protected MasterServices services;
-
-  /**
-   * By default, regions of some small system tables such as meta,
-   * namespace, and acl are assigned to the active master. If you don't
-   * want to assign any region to the active master, you need to
-   * configure "hbase.balancer.tablesOnMaster" to "none".
-   */
-  protected static String[] getTablesOnMaster(Configuration conf) {
-    String valueString = conf.get(TABLES_ON_MASTER);
-    if (valueString == null) {
-      return DEFAULT_TABLES_ON_MASTER;
-    }
-    valueString = valueString.trim();
-    if (valueString.equalsIgnoreCase("none")) {
-      return null;
-    }
-    return StringUtils.getStrings(valueString);
-  }
-
-  /**
-   * Check if configured to put any tables on the active master
-   */
-  public static boolean tablesOnMaster(Configuration conf) {
-    String[] tables = getTablesOnMaster(conf);
-    return tables != null && tables.length > 0;
-  }
-
-  public static boolean userTablesOnMaster(Configuration conf) {
-    String[] tables = getTablesOnMaster(conf);
-    if (tables == null || tables.length == 0) {
-      return false;
-    }
-    for (String tn:tables) {
-      if (!tn.startsWith("hbase:")) {
-        return true;
-      }
-    }
-    return false;
-  }
+  protected boolean tablesOnMaster;
+  protected boolean onlySystemTablesOnMaster;
 
   @Override
   public void setConf(Configuration conf) {
+    this.config = conf;
     setSlop(conf);
     if (slop < 0) slop = 0;
     else if (slop > 1) slop = 1;
@@ -1061,13 +1015,18 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
     if (overallSlop < 0) overallSlop = 0;
     else if (overallSlop > 1) overallSlop = 1;
 
-    this.config = conf;
-    String[] tables = getTablesOnMaster(conf);
-    if (tables != null && tables.length > 0) {
-      Collections.addAll(tablesOnMaster, tables);
+    this.tablesOnMaster = LoadBalancer.isTablesOnMaster(this.config);
+    this.onlySystemTablesOnMaster = LoadBalancer.isSystemTablesOnlyOnMaster(this.config);
+    // If system tables on master, implies tablesOnMaster = true.
+    if (this.onlySystemTablesOnMaster && !this.tablesOnMaster) {
+      LOG.warn("Set " + TABLES_ON_MASTER + "=true because " + SYSTEM_TABLES_ON_MASTER + "=true");
+      this.tablesOnMaster = true;
     }
     this.rackManager = new RackManager(getConf());
     regionFinder.setConf(conf);
+    // Print out base configs. Don't print overallSlop since it for simple balancer exclusively.
+    LOG.info("slop=" + this.slop + ", tablesOnMaster=" + this.tablesOnMaster +
+      ", systemTablesOnMaster=" + this.onlySystemTablesOnMaster);
   }
 
   protected void setSlop(Configuration conf) {
@@ -1076,21 +1035,18 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
   }
 
   /**
-   * Check if a region belongs to some small system table.
+   * Check if a region belongs to some system table.
    * If so, the primary replica may be expected to be put on the master regionserver.
    */
   public boolean shouldBeOnMaster(HRegionInfo region) {
-    return tablesOnMaster.contains(region.getTable().getNameAsString())
-        && region.getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID;
+    return this.onlySystemTablesOnMaster && region.isSystemTable();
   }
 
   /**
    * Balance the regions that should be on master regionserver.
    */
-  protected List<RegionPlan> balanceMasterRegions(
-      Map<ServerName, List<HRegionInfo>> clusterMap) {
-    if (masterServerName == null
-        || clusterMap == null || clusterMap.size() <= 1) return null;
+  protected List<RegionPlan> balanceMasterRegions(Map<ServerName, List<HRegionInfo>> clusterMap) {
+    if (masterServerName == null || clusterMap == null || clusterMap.size() <= 1) return null;
     List<RegionPlan> plans = null;
     List<HRegionInfo> regions = clusterMap.get(masterServerName);
     if (regions != null) {
@@ -1135,19 +1091,22 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
   }
 
   /**
-   * Assign the regions that should be on master regionserver.
+   * If master is configured to carry system tables only, in here is
+   * where we figure what to assign it.
    */
-  protected Map<ServerName, List<HRegionInfo>> assignMasterRegions(
+  protected Map<ServerName, List<HRegionInfo>> assignMasterSystemRegions(
       Collection<HRegionInfo> regions, List<ServerName> servers) {
     if (servers == null || regions == null || regions.isEmpty()) {
       return null;
     }
     Map<ServerName, List<HRegionInfo>> assignments = new TreeMap<>();
-    if (masterServerName != null && servers.contains(masterServerName)) {
-      assignments.put(masterServerName, new ArrayList<>());
-      for (HRegionInfo region: regions) {
-        if (shouldBeOnMaster(region)) {
-          assignments.get(masterServerName).add(region);
+    if (this.onlySystemTablesOnMaster) {
+      if (masterServerName != null && servers.contains(masterServerName)) {
+        assignments.put(masterServerName, new ArrayList<>());
+        for (HRegionInfo region : regions) {
+          if (shouldBeOnMaster(region)) {
+            assignments.get(masterServerName).add(region);
+          }
         }
       }
     }
@@ -1243,7 +1202,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
   public Map<ServerName, List<HRegionInfo>> roundRobinAssignment(List<HRegionInfo> regions,
       List<ServerName> servers) throws HBaseIOException {
     metricsBalancer.incrMiscInvocations();
-    Map<ServerName, List<HRegionInfo>> assignments = assignMasterRegions(regions, servers);
+    Map<ServerName, List<HRegionInfo>> assignments = assignMasterSystemRegions(regions, servers);
     if (assignments != null && !assignments.isEmpty()) {
       servers = new ArrayList<>(servers);
       // Guarantee not to put other regions on master
@@ -1350,9 +1309,11 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
       if (shouldBeOnMaster(regionInfo)) {
         return masterServerName;
       }
-      servers = new ArrayList<>(servers);
-      // Guarantee not to put other regions on master
-      servers.remove(masterServerName);
+      if (!LoadBalancer.isTablesOnMaster(getConf())) {
+        // Guarantee we do not put any regions on master
+        servers = new ArrayList<>(servers);
+        servers.remove(masterServerName);
+      }
     }
 
     int numServers = servers == null ? 0 : servers.size();
@@ -1396,8 +1357,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
       List<ServerName> servers) throws HBaseIOException {
     // Update metrics
     metricsBalancer.incrMiscInvocations();
-    Map<ServerName, List<HRegionInfo>> assignments
-      = assignMasterRegions(regions.keySet(), servers);
+    Map<ServerName, List<HRegionInfo>> assignments = assignMasterSystemRegions(regions.keySet(), servers);
     if (assignments != null && !assignments.isEmpty()) {
       servers = new ArrayList<>(servers);
       // Guarantee not to put other regions on master

http://git-wip-us.apache.org/repos/asf/hbase/blob/47344671/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredStochasticBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredStochasticBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredStochasticBalancer.java
index 30cf16a..86ccd47 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredStochasticBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredStochasticBalancer.java
@@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.favored.FavoredNodesManager;
 import org.apache.hadoop.hbase.favored.FavoredNodesPlan;
 import org.apache.hadoop.hbase.favored.FavoredNodesPlan.Position;
 import org.apache.hadoop.hbase.favored.FavoredNodesPromoter;
+import org.apache.hadoop.hbase.master.LoadBalancer;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.master.RegionPlan;
 import org.apache.hadoop.hbase.util.Pair;
@@ -112,7 +113,7 @@ public class FavoredStochasticBalancer extends StochasticLoadBalancer implements
     metricsBalancer.incrMiscInvocations();
 
     Set<HRegionInfo> regionSet = Sets.newHashSet(regions);
-    Map<ServerName, List<HRegionInfo>> assignmentMap = assignMasterRegions(regions, servers);
+    Map<ServerName, List<HRegionInfo>> assignmentMap = assignMasterSystemRegions(regions, servers);
     if (assignmentMap != null && !assignmentMap.isEmpty()) {
       servers = new ArrayList<>(servers);
       // Guarantee not to put other regions on master
@@ -311,9 +312,11 @@ public class FavoredStochasticBalancer extends StochasticLoadBalancer implements
         metricsBalancer.incrMiscInvocations();
         return masterServerName;
       }
-      servers = new ArrayList<>(servers);
-      // Guarantee not to put other regions on master
-      servers.remove(masterServerName);
+      if (!LoadBalancer.isTablesOnMaster(getConf())) {
+        // Guarantee we do not put any regions on master
+        servers = new ArrayList<>(servers);
+        servers.remove(masterServerName);
+      }
     }
 
     ServerName destination = null;

http://git-wip-us.apache.org/repos/asf/hbase/blob/47344671/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 0774df1..0c1814f 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
@@ -116,6 +116,7 @@ import org.apache.hadoop.hbase.ipc.RpcServerInterface;
 import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
 import org.apache.hadoop.hbase.ipc.ServerRpcController;
 import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.LoadBalancer;
 import org.apache.hadoop.hbase.master.RegionState.State;
 import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer;
 import org.apache.hadoop.hbase.mob.MobCacheConfig;
@@ -2899,8 +2900,8 @@ public class HRegionServer extends HasThread implements
   static private void createNewReplicationInstance(Configuration conf,
     HRegionServer server, FileSystem walFs, Path walDir, Path oldWALDir) throws IOException{
 
-    if ((server instanceof HMaster) &&
-        (!BaseLoadBalancer.userTablesOnMaster(conf))) {
+    if ((server instanceof HMaster) && (!LoadBalancer.isTablesOnMaster(conf) ||
+        LoadBalancer.isSystemTablesOnlyOnMaster(conf))) {
       return;
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/47344671/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index 44aeb88..18b1114 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -49,6 +49,7 @@ import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 
+import edu.umd.cs.findbugs.annotations.Nullable;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang.RandomStringUtils;
 import org.apache.commons.logging.Log;
@@ -137,7 +138,6 @@ import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.ZooKeeper.States;
 
-import edu.umd.cs.findbugs.annotations.Nullable;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;

http://git-wip-us.apache.org/repos/asf/hbase/blob/47344671/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
index 2914e4b..e5b6bb6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
@@ -665,7 +665,8 @@ public class MiniHBaseCluster extends HBaseCluster {
   }
 
   /**
-   * @return List of region server threads.
+   * @return List of region server threads. Does not return the master even though it is also
+   * a region server.
    */
   public List<JVMClusterUtil.RegionServerThread> getRegionServerThreads() {
     return this.hbaseCluster.getRegionServers();

http://git-wip-us.apache.org/repos/asf/hbase/blob/47344671/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
index d82c741..dfdd11e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
@@ -53,6 +53,7 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
+import org.apache.hadoop.hbase.master.LoadBalancer;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
@@ -596,7 +597,8 @@ public class TestAdmin1 {
       }
       regs.add(loc.getRegionInfo());
     }
-    if (numRS >= 2) {
+    boolean tablesOnMaster = LoadBalancer.isTablesOnMaster(TEST_UTIL.getConfiguration());
+    if (tablesOnMaster) {
       // Ignore the master region server,
       // which contains less regions by intention.
       numRS--;
@@ -605,7 +607,9 @@ public class TestAdmin1 {
     int min = (int)Math.floor(average);
     int max = (int)Math.ceil(average);
     for (List<HRegionInfo> regionList : server2Regions.values()) {
-      assertTrue(regionList.size() == min || regionList.size() == max);
+      assertTrue("numRS=" + numRS + ", min=" + min + ", max=" + max +
+        ", size=" + regionList.size() + ", tablesOnMaster=" + tablesOnMaster,
+      regionList.size() == min || regionList.size() == max);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/47344671/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java
index e6fc2f3..d17c782 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java
@@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotEnabledException;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder.ModifyableTableDescriptor;
+import org.apache.hadoop.hbase.master.LoadBalancer;
 import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
@@ -209,6 +210,7 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
         new byte[] { 4, 4, 4 }, new byte[] { 5, 5, 5 }, new byte[] { 6, 6, 6 },
         new byte[] { 7, 7, 7 }, new byte[] { 8, 8, 8 }, new byte[] { 9, 9, 9 }, };
     int expectedRegions = splitKeys.length + 1;
+    boolean tablesOnMaster = LoadBalancer.isTablesOnMaster(TEST_UTIL.getConfiguration());
     createTableWithDefaultConf(tableName, Optional.of(splitKeys));
 
     boolean tableAvailable = admin.isTableAvailable(tableName, splitKeys).get();
@@ -256,7 +258,9 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
     hri = hris.next().getRegionInfo();
     assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[8]));
     assertTrue(hri.getEndKey() == null || hri.getEndKey().length == 0);
-    verifyRoundRobinDistribution(regions, expectedRegions);
+    if (tablesOnMaster) {
+      verifyRoundRobinDistribution(regions, expectedRegions);
+    }
 
     // Now test using start/end with a number of regions
 
@@ -310,7 +314,10 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
     hri = hris.next().getRegionInfo();
     assertTrue(Bytes.equals(hri.getStartKey(), new byte[] { 9, 9, 9, 9, 9, 9, 9, 9, 9, 9 }));
     assertTrue(hri.getEndKey() == null || hri.getEndKey().length == 0);
-    verifyRoundRobinDistribution(regions, expectedRegions);
+    if (tablesOnMaster) {
+      // This don't work if master is not carrying regions. FIX. TODO.
+      verifyRoundRobinDistribution(regions, expectedRegions);
+    }
 
     // Try once more with something that divides into something infinite
     startKey = new byte[] { 0, 0, 0, 0, 0, 0 };
@@ -328,7 +335,10 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
       "Tried to create " + expectedRegions + " regions " + "but only found " + regions.size(),
       expectedRegions, regions.size());
     System.err.println("Found " + regions.size() + " regions");
-    verifyRoundRobinDistribution(regions, expectedRegions);
+    if (tablesOnMaster) {
+      // This don't work if master is not carrying regions. FIX. TODO.
+      verifyRoundRobinDistribution(regions, expectedRegions);
+    }
 
     // Try an invalid case where there are duplicate split keys
     splitKeys = new byte[][] { new byte[] { 1, 1, 1 }, new byte[] { 2, 2, 2 },

http://git-wip-us.apache.org/repos/asf/hbase/blob/47344671/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientClusterStatus.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientClusterStatus.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientClusterStatus.java
index 2cf7bc5..49b4ff2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientClusterStatus.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientClusterStatus.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.LoadBalancer;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
@@ -154,9 +155,9 @@ public class TestClientClusterStatus {
     Assert.assertNotNull(status);
     Assert.assertNotNull(status.getServers());
     // exclude a dead region server
-    Assert.assertEquals(SLAVES - 1, numRs);
+    Assert.assertEquals(SLAVES, numRs);
     // live servers = primary master + nums of regionservers
-    Assert.assertEquals(status.getServers().size() - 1, numRs);
+    Assert.assertEquals(status.getServers().size() + 1 /*Master*/, numRs);
     Assert.assertTrue(status.getRegionsCount() > 0);
     Assert.assertNotNull(status.getDeadServerNames());
     Assert.assertEquals(1, status.getDeadServersSize());

http://git-wip-us.apache.org/repos/asf/hbase/blob/47344671/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
index fcda723..48cb812 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
@@ -95,6 +95,7 @@ import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.io.hfile.BlockCache;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
+import org.apache.hadoop.hbase.master.LoadBalancer;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
@@ -4328,9 +4329,11 @@ public class TestFromClientSide {
 
     // test that the same unmanaged connection works with a new
     // Admin and can connect to the new master;
+    boolean tablesOnMaster = LoadBalancer.isTablesOnMaster(TEST_UTIL.getConfiguration());
     try (Admin admin = conn.getAdmin()) {
       assertTrue(admin.tableExists(tableName));
-      assertTrue(admin.getClusterStatus().getServersSize() == SLAVES + 1);
+      assertTrue(admin.getClusterStatus().getServersSize() ==
+        SLAVES + (tablesOnMaster? 1: 0));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/47344671/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java
index 3f6982a..5b9ef2e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.master.LoadBalancer;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.Region;
@@ -259,7 +260,8 @@ public class TestBlockReorder {
 
     MiniHBaseCluster hbm = htu.startMiniHBaseCluster(1, 1);
     hbm.waitForActiveAndReadyMaster();
-    HRegionServer targetRs = hbm.getMaster();
+    HRegionServer targetRs = LoadBalancer.isTablesOnMaster(hbm.getConf())? hbm.getMaster():
+      hbm.getRegionServer(0);
 
     // We want to have a datanode with the same name as the region server, so
     //  we're going to get the regionservername, and start a new datanode with this name.

http://git-wip-us.apache.org/repos/asf/hbase/blob/47344671/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetrics.java
index 77404d4..04cfc65 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetrics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetrics.java
@@ -100,7 +100,13 @@ public class TestMasterMetrics {
     request.setLoad(sl);
 
     master.getMasterRpcServices().regionServerReport(null, request.build());
-    metricsHelper.assertCounter("cluster_requests", expectedRequestNumber, masterSource);
+    boolean tablesOnMaster = LoadBalancer.isTablesOnMaster(TEST_UTIL.getConfiguration());
+    if (tablesOnMaster) {
+      metricsHelper.assertCounter("cluster_requests", expectedRequestNumber, masterSource);
+    } else {
+      metricsHelper.assertCounterGt("cluster_requests", expectedRequestNumber, masterSource);
+
+    }
 
     expectedRequestNumber = 15000;
 
@@ -110,7 +116,11 @@ public class TestMasterMetrics {
     request.setLoad(sl);
 
     master.getMasterRpcServices().regionServerReport(null, request.build());
-    metricsHelper.assertCounter("cluster_requests", expectedRequestNumber, masterSource);
+    if (tablesOnMaster) {
+      metricsHelper.assertCounter("cluster_requests", expectedRequestNumber, masterSource);
+    } else {
+      metricsHelper.assertCounterGt("cluster_requests", expectedRequestNumber, masterSource);
+    }
 
     master.stopMaster();
   }
@@ -118,8 +128,9 @@ public class TestMasterMetrics {
   @Test
   public void testDefaultMasterMetrics() throws Exception {
     MetricsMasterSource masterSource = master.getMasterMetrics().getMetricsSource();
-    metricsHelper.assertGauge( "numRegionServers", 2, masterSource);
-    metricsHelper.assertGauge( "averageLoad", 1, masterSource);
+    boolean tablesOnMaster = LoadBalancer.isTablesOnMaster(TEST_UTIL.getConfiguration());
+    metricsHelper.assertGauge( "numRegionServers",1 + (tablesOnMaster? 1: 0), masterSource);
+    metricsHelper.assertGauge( "averageLoad", 1 + (tablesOnMaster? 0: 1), masterSource);
     metricsHelper.assertGauge( "numDeadRegionServers", 0, masterSource);
 
     metricsHelper.assertGauge("masterStartTime", master.getMasterStartTime(), masterSource);

http://git-wip-us.apache.org/repos/asf/hbase/blob/47344671/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetricsWrapper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetricsWrapper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetricsWrapper.java
index 8c1138a..9acd2f7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetricsWrapper.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetricsWrapper.java
@@ -40,10 +40,11 @@ public class TestMasterMetricsWrapper {
   private static final Log LOG = LogFactory.getLog(TestMasterMetricsWrapper.class);
 
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static final int NUM_RS = 4;
 
   @BeforeClass
   public static void setup() throws Exception {
-    TEST_UTIL.startMiniCluster(1, 4);
+    TEST_UTIL.startMiniCluster(1, NUM_RS);
   }
 
   @AfterClass
@@ -63,7 +64,9 @@ public class TestMasterMetricsWrapper {
     assertEquals(master.getMasterStartTime(), info.getStartTime());
     assertEquals(master.getMasterCoprocessors().length, info.getCoprocessors().length);
     assertEquals(master.getServerManager().getOnlineServersList().size(), info.getNumRegionServers());
-    assertEquals(5, info.getNumRegionServers());
+    int regionServerCount =
+      NUM_RS + (LoadBalancer.isTablesOnMaster(TEST_UTIL.getConfiguration())? 1: 0);
+    assertEquals(regionServerCount, info.getNumRegionServers());
 
     String zkServers = info.getZookeeperQuorum();
     assertEquals(zkServers.split(",").length, TEST_UTIL.getZkCluster().getZooKeeperServerNum());
@@ -74,11 +77,11 @@ public class TestMasterMetricsWrapper {
     TEST_UTIL.getMiniHBaseCluster().waitOnRegionServer(index);
     // We stopped the regionserver but could take a while for the master to notice it so hang here
     // until it does... then move forward to see if metrics wrapper notices.
-    while (TEST_UTIL.getHBaseCluster().getMaster().getServerManager().getOnlineServers().size() !=
-        4) {
+    while (TEST_UTIL.getHBaseCluster().getMaster().getServerManager().getOnlineServers().size() ==
+        regionServerCount ) {
       Threads.sleep(10);
     }
-    assertEquals(4, info.getNumRegionServers());
+    assertEquals(regionServerCount - 1, info.getNumRegionServers());
     assertEquals(1, info.getNumDeadRegionServers());
     assertEquals(1, info.getNumWALFiles());
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/47344671/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
index 04ab29c..480ba9a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
@@ -26,7 +26,6 @@ import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Set;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -65,6 +64,7 @@ import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
+import org.junit.Ignore;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 import org.junit.rules.TestRule;
@@ -82,6 +82,7 @@ import org.mockito.Mockito;
 public class TestMasterNoCluster {
   private static final Log LOG = LogFactory.getLog(TestMasterNoCluster.class);
   private static final HBaseTestingUtility TESTUTIL = new HBaseTestingUtility();
+
   @Rule public final TestRule timeout = CategoryBasedTimeout.builder().
       withTimeout(this.getClass()).withLookingForStuckThread(true).build();
 
@@ -149,7 +150,7 @@ public class TestMasterNoCluster {
    * @throws InterruptedException
    * @throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException 
    */
-  @Test
+  @Ignore @Test // Disabled since HBASE-18511. Reenable when master can carry regions.
   public void testFailover() throws Exception {
     final long now = System.currentTimeMillis();
     // Names for our three servers.  Make the port numbers match hostname.
@@ -253,7 +254,7 @@ public class TestMasterNoCluster {
     }
   }
 
-  @Test
+  @Ignore @Test // Disabled since HBASE-18511. Reenable when master can carry regions.
   public void testNotPullingDeadRegionServerFromZK()
       throws IOException, KeeperException, InterruptedException {
     final Configuration conf = TESTUTIL.getConfiguration();

http://git-wip-us.apache.org/repos/asf/hbase/blob/47344671/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBaseLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBaseLoadBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBaseLoadBalancer.java
index 400c9d9..023554d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBaseLoadBalancer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBaseLoadBalancer.java
@@ -107,7 +107,6 @@ public class TestBaseLoadBalancer extends BalancerTestBase {
   }
 
   public static class MockBalancer extends BaseLoadBalancer {
-
     @Override
     public List<RegionPlan> balanceCluster(Map<ServerName, List<HRegionInfo>> clusterState) {
       return null;
@@ -118,7 +117,6 @@ public class TestBaseLoadBalancer extends BalancerTestBase {
         Map<ServerName, List<HRegionInfo>> clusterState) throws HBaseIOException {
       return null;
     }
-
   }
 
   /**
@@ -149,8 +147,10 @@ public class TestBaseLoadBalancer extends BalancerTestBase {
     hris.add(HRegionInfo.FIRST_META_REGIONINFO);
     tmp.add(master);
     Map<ServerName, List<HRegionInfo>> plans = loadBalancer.roundRobinAssignment(hris, tmp);
-    assertTrue(plans.get(master).contains(HRegionInfo.FIRST_META_REGIONINFO));
-    assertEquals(1, plans.get(master).size());
+    if (LoadBalancer.isTablesOnMaster(loadBalancer.getConf())) {
+      assertTrue(plans.get(master).contains(HRegionInfo.FIRST_META_REGIONINFO));
+      assertEquals(1, plans.get(master).size());
+    }
     int totalRegion = 0;
     for (List<HRegionInfo> regions: plans.values()) {
       totalRegion += regions.size();
@@ -541,4 +541,4 @@ public class TestBaseLoadBalancer extends BalancerTestBase {
     assertEquals(1, cluster.regionLocations[r43].length);
     assertEquals(-1, cluster.regionLocations[r43][0]);
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/47344671/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionsOnMasterOptions.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionsOnMasterOptions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionsOnMasterOptions.java
new file mode 100644
index 0000000..d590db5
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionsOnMasterOptions.java
@@ -0,0 +1,200 @@
+/**
+ * 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.balancer;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.*;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.LoadBalancer;
+import org.apache.hadoop.hbase.regionserver.Region;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.JVMClusterUtil;
+import org.apache.hadoop.hbase.util.Threads;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+import org.junit.rules.TestRule;
+
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test options for regions on master; none, system, or any (i.e. master is like any other
+ * regionserver). Checks how regions are deployed when each of the options are enabled.
+ * It then does kill combinations to make sure the distribution is more than just for startup.
+ */
+@Category({MediumTests.class})
+public class TestRegionsOnMasterOptions {
+  private static final Log LOG = LogFactory.getLog(TestRegionsOnMasterOptions.class);
+  @Rule public TestName name = new TestName();
+  @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()).
+    withLookingForStuckThread(true).build();
+  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private Configuration c;
+  private String tablesOnMasterOldValue;
+  private String systemTablesOnMasterOldValue;
+  private static final int SLAVES = 3;
+  private static final int MASTERS = 2;
+  // Make the count of REGIONS high enough so I can distingush case where master is only carrying
+  // system regions from the case where it is carrying any region; i.e. 2 system regions vs more
+  // if user + system.
+  private static final int REGIONS = 12;
+  private static final int SYSTEM_REGIONS = 2; // ns and meta -- no acl unless enabled.
+
+  @Before
+  public void setup() {
+    this.c = TEST_UTIL.getConfiguration();
+    this.tablesOnMasterOldValue = c.get(LoadBalancer.TABLES_ON_MASTER);
+    this.systemTablesOnMasterOldValue = c.get(LoadBalancer.SYSTEM_TABLES_ON_MASTER);
+  }
+
+  @After
+  public void tearDown() {
+    unset(LoadBalancer.TABLES_ON_MASTER, this.tablesOnMasterOldValue);
+    unset(LoadBalancer.SYSTEM_TABLES_ON_MASTER, this.systemTablesOnMasterOldValue);
+  }
+
+  private void unset(final String key, final String value) {
+    if (value == null) {
+      c.unset(key);
+    } else {
+      c.set(key, value);
+    }
+  }
+
+  @Test
+  public void testRegionsOnAllServers() throws Exception {
+    c.setBoolean(LoadBalancer.TABLES_ON_MASTER, true);
+    c.setBoolean(LoadBalancer.SYSTEM_TABLES_ON_MASTER, false);
+    int rsCount = (REGIONS + SYSTEM_REGIONS)/(SLAVES + 1/*Master*/);
+    checkBalance(rsCount, rsCount);
+  }
+
+  @Test
+  public void testNoRegionOnMaster() throws Exception {
+    c.setBoolean(LoadBalancer.TABLES_ON_MASTER, false);
+    c.setBoolean(LoadBalancer.SYSTEM_TABLES_ON_MASTER, false);
+    int rsCount = (REGIONS + SYSTEM_REGIONS)/SLAVES;
+    checkBalance(0, rsCount);
+  }
+
+  @Test
+  public void testSystemTablesOnMaster() throws Exception {
+    c.setBoolean(LoadBalancer.TABLES_ON_MASTER, true);
+    c.setBoolean(LoadBalancer.SYSTEM_TABLES_ON_MASTER, true);
+    // IS THIS SHORT-CIRCUIT RPC? Yes. Here is how it looks currently if I have an exception
+    // thrown in doBatchMutate inside a Region.
+    //
+    //    java.lang.Exception
+    //    at org.apache.hadoop.hbase.regionserver.HRegion.doBatchMutate(HRegion.java:3845)
+    //    at org.apache.hadoop.hbase.regionserver.HRegion.put(HRegion.java:2972)
+    //    at org.apache.hadoop.hbase.regionserver.RSRpcServices.mutate(RSRpcServices.java:2751)
+    //    at org.apache.hadoop.hbase.client.ClientServiceCallable.doMutate(ClientServiceCallable.java:55)
+    //    at org.apache.hadoop.hbase.client.HTable$3.rpcCall(HTable.java:585)
+    //    at org.apache.hadoop.hbase.client.HTable$3.rpcCall(HTable.java:579)
+    //    at org.apache.hadoop.hbase.client.RegionServerCallable.call(RegionServerCallable.java:126)
+    //    at org.apache.hadoop.hbase.client.RpcRetryingCallerImpl.callWithRetries(RpcRetryingCallerImpl.java:106)
+    //    at org.apache.hadoop.hbase.client.HTable.put(HTable.java:589)
+    //    at org.apache.hadoop.hbase.master.TableNamespaceManager.insertIntoNSTable(TableNamespaceManager.java:156)
+    //    at org.apache.hadoop.hbase.master.procedure.CreateNamespaceProcedure.insertIntoNSTable(CreateNamespaceProcedure.java:222)
+    //    at org.apache.hadoop.hbase.master.procedure.CreateNamespaceProcedure.executeFromState(CreateNamespaceProcedure.java:76)
+    //    at org.apache.hadoop.hbase.master.procedure.CreateNamespaceProcedure.executeFromState(CreateNamespaceProcedure.java:40)
+    //    at org.apache.hadoop.hbase.procedure2.StateMachineProcedure.execute(StateMachineProcedure.java:181)
+    //    at org.apache.hadoop.hbase.procedure2.Procedure.doExecute(Procedure.java:847)
+    //    at org.apache.hadoop.hbase.procedure2.ProcedureExecutor.execProcedure(ProcedureExecutor.java:1440)
+    //    at org.apache.hadoop.hbase.procedure2.ProcedureExecutor.executeProcedure(ProcedureExecutor.java:1209)
+    //    at org.apache.hadoop.hbase.procedure2.ProcedureExecutor.access$800(ProcedureExecutor.java:79)
+    //    at org.apache.hadoop.hbase.procedure2.ProcedureExecutor$WorkerThread.run(ProcedureExecutor.java:1719)
+    //
+    // If I comment out the ConnectionUtils ConnectionImplementation content, I see this:
+    //
+    //    java.lang.Exception
+    //    at org.apache.hadoop.hbase.regionserver.HRegion.doBatchMutate(HRegion.java:3845)
+    //    at org.apache.hadoop.hbase.regionserver.HRegion.put(HRegion.java:2972)
+    //    at org.apache.hadoop.hbase.regionserver.RSRpcServices.mutate(RSRpcServices.java:2751)
+    //    at org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:41546)
+    //    at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:406)
+    //    at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:133)
+    //    at org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:278)
+    //    at org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:258)
+
+    checkBalance(SYSTEM_REGIONS, REGIONS/SLAVES);
+  }
+
+  private void checkBalance(int masterCount, int rsCount) throws Exception {
+    MiniHBaseCluster cluster = TEST_UTIL.startMiniCluster(MASTERS, SLAVES);
+    TableName tn = TableName.valueOf(this.name.getMethodName());
+    try {
+      Table t = TEST_UTIL.createMultiRegionTable(tn, HConstants.CATALOG_FAMILY, REGIONS);
+      LOG.info("Server: " + cluster.getMaster().getServerManager().getOnlineServersList());
+      List<Region> regions = cluster.getMaster().getOnlineRegions();
+      int mActualCount = regions.size();
+      if (masterCount == 0 || masterCount == SYSTEM_REGIONS) {
+        // 0 means no regions on master.
+        assertEquals(masterCount, mActualCount);
+      } else {
+        // This is master as a regionserver scenario.
+        checkCount(masterCount, mActualCount);
+      }
+      // Allow that balance is not exact. FYI, getRegionServerThreads does not include master
+      // thread though it is a regionserver so we have to check master and then below the
+      // regionservers.
+      for (JVMClusterUtil.RegionServerThread rst: cluster.getRegionServerThreads()) {
+        regions = rst.getRegionServer().getOnlineRegions();
+        int rsActualCount = regions.size();
+        checkCount(rsActualCount, rsCount);
+      }
+      HMaster oldMaster = cluster.getMaster();
+      cluster.killMaster(oldMaster.getServerName());
+      oldMaster.join();
+      while (cluster.getMaster() == null ||
+          cluster.getMaster().getServerName().equals(oldMaster.getServerName())) {
+        Threads.sleep(10);
+      }
+      while (!cluster.getMaster().isInitialized()) {
+        Threads.sleep(10);
+      }
+      LOG.info("Cluster is up; running balancer");
+      cluster.getMaster().balance();
+      regions = cluster.getMaster().getOnlineRegions();
+      int mNewActualCount = regions.size();
+      if (masterCount == 0 || masterCount == SYSTEM_REGIONS) {
+        // 0 means no regions on master. After crash, should still be no regions on master.
+        // If masterCount == SYSTEM_REGIONS, means master only carrying system regions and should
+        // still only carry system regions post crash.
+        assertEquals(masterCount, mNewActualCount);
+      }
+    } finally {
+      LOG.info("Running shutdown of cluster");
+      TEST_UTIL.shutdownMiniCluster();
+    }
+  }
+
+  private void checkCount(int actual, int expected) {
+    assertTrue("Actual=" + actual + ", expected=" + expected,
+    actual >= (expected - 2) && actual <= (expected + 2)); // Lots of slop +/- 2
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/47344671/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestClusterId.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestClusterId.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestClusterId.java
index 33ca73d..2240226 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestClusterId.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestClusterId.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.CoordinatedStateManager;
 import org.apache.hadoop.hbase.CoordinatedStateManagerFactory;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.master.LoadBalancer;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.master.HMaster;
@@ -111,7 +112,8 @@ public class TestClusterId {
     }
     TEST_UTIL.startMiniHBaseCluster(1, 1);
     HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
-    assertEquals(2, master.getServerManager().getOnlineServersList().size());
+    int expected = LoadBalancer.isTablesOnMaster(TEST_UTIL.getConfiguration())? 2: 1;
+    assertEquals(expected, master.getServerManager().getOnlineServersList().size());
   }
   
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/47344671/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSKilledWhenInitializing.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSKilledWhenInitializing.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSKilledWhenInitializing.java
index 304bfe7..f682f79 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSKilledWhenInitializing.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSKilledWhenInitializing.java
@@ -39,8 +39,10 @@ import org.apache.hadoop.hbase.LocalHBaseCluster;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.LoadBalancer;
 import org.apache.hadoop.hbase.master.ServerListener;
 import org.apache.hadoop.hbase.master.ServerManager;
+import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
@@ -74,7 +76,7 @@ public class TestRSKilledWhenInitializing {
   private static final int NUM_RS = 2;
 
   /**
-   * Test verifies whether a region server is removing from online servers list in master if it went
+   * Test verifies whether a region server is removed from online servers list in master if it went
    * down after registering with master. Test will TIMEOUT if an error!!!!
    * @throws Exception
    */
@@ -98,18 +100,18 @@ public class TestRSKilledWhenInitializing {
       for (int i = 0; i < NUM_RS; i++) {
         cluster.getRegionServers().get(i).start();
       }
-      // Now wait on master to see NUM_RS + 1 servers as being online, thats NUM_RS plus
-      // the Master itself (because Master hosts hbase:meta and checks in as though it a RS).
+      // Expected total regionservers depends on whether Master can host regions or not.
+      int expectedTotalRegionServers = NUM_RS + (LoadBalancer.isTablesOnMaster(conf)? 1: 0);
       List<ServerName> onlineServersList = null;
       do {
         onlineServersList = master.getMaster().getServerManager().getOnlineServersList();
-      } while (onlineServersList.size() < (NUM_RS + 1));
+      } while (onlineServersList.size() < expectedTotalRegionServers);
       // Wait until killedRS is set. Means RegionServer is starting to go down.
       while (killedRS.get() == null) {
         Threads.sleep(1);
       }
       // Wait on the RegionServer to fully die.
-      while (cluster.getLiveRegionServers().size() > NUM_RS) {
+      while (cluster.getLiveRegionServers().size() >= expectedTotalRegionServers) {
         Threads.sleep(1);
       }
       // Make sure Master is fully up before progressing. Could take a while if regions
@@ -134,7 +136,8 @@ public class TestRSKilledWhenInitializing {
       }
       // Try moving region to the killed server. It will fail. As by-product, we will
       // remove the RS from Master online list because no corresponding znode.
-      assertEquals(NUM_RS + 1, master.getMaster().getServerManager().getOnlineServersList().size());
+      assertEquals(expectedTotalRegionServers,
+        master.getMaster().getServerManager().getOnlineServersList().size());
       LOG.info("Move " + hri.getEncodedName() + " to " + killedRS.get());
       master.getMaster().move(hri.getEncodedNameAsBytes(),
           Bytes.toBytes(killedRS.get().toString()));

http://git-wip-us.apache.org/repos/asf/hbase/blob/47344671/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionOpen.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionOpen.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionOpen.java
index 3a1769e..1f8d16b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionOpen.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionOpen.java
@@ -83,8 +83,7 @@ public class TestRegionOpen {
     final TableName tableName = TableName.valueOf(TestRegionOpen.class.getSimpleName());
     ThreadPoolExecutor exec = getRS().getExecutorService()
         .getExecutorThreadPool(ExecutorType.RS_OPEN_PRIORITY_REGION);
-
-    assertEquals(0, exec.getCompletedTaskCount());
+    long completed = exec.getCompletedTaskCount();
 
     HTableDescriptor htd = new HTableDescriptor(tableName);
     htd.setPriority(HConstants.HIGH_QOS);
@@ -94,7 +93,7 @@ public class TestRegionOpen {
       admin.createTable(htd);
     }
 
-    assertEquals(1, exec.getCompletedTaskCount());
+    assertEquals(completed + 1, exec.getCompletedTaskCount());
   }
 
   @Test(timeout = 60000)

http://git-wip-us.apache.org/repos/asf/hbase/blob/47344671/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java
index 81dd630..6df3063 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.RegionObserver;
 import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.RegionServerObserver;
+import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
@@ -94,16 +95,19 @@ public class TestRegionServerAbort {
 
   @After
   public void tearDown() throws Exception {
+    String className = StopBlockingRegionObserver.class.getName();
     for (JVMClusterUtil.RegionServerThread t : cluster.getRegionServerThreads()) {
       HRegionServer rs = t.getRegionServer();
       RegionServerCoprocessorHost cpHost = rs.getRegionServerCoprocessorHost();
-      StopBlockingRegionObserver cp = (StopBlockingRegionObserver)
-          cpHost.findCoprocessor(StopBlockingRegionObserver.class.getName());
+      StopBlockingRegionObserver cp = (StopBlockingRegionObserver)cpHost.findCoprocessor(className);
       cp.setStopAllowed(true);
     }
-    ((StopBlockingRegionObserver) cluster.getMaster().getRegionServerCoprocessorHost().findCoprocessor(
-        StopBlockingRegionObserver.class.getName()
-    )).setStopAllowed(true);
+    HMaster master = cluster.getMaster();
+    RegionServerCoprocessorHost host = master.getRegionServerCoprocessorHost();
+    if (host != null) {
+      StopBlockingRegionObserver obs = (StopBlockingRegionObserver) host.findCoprocessor(className);
+      if (obs != null) obs.setStopAllowed(true);
+    }
     testUtil.shutdownMiniCluster();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/47344671/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerHostname.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerHostname.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerHostname.java
index 317a3a2..8eee9b6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerHostname.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerHostname.java
@@ -32,6 +32,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.master.LoadBalancer;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
@@ -101,7 +102,8 @@ public class TestRegionServerHostname {
           ZooKeeperWatcher zkw = TEST_UTIL.getZooKeeperWatcher();
           List<String> servers = ZKUtil.listChildrenNoWatch(zkw, zkw.znodePaths.rsZNode);
           // there would be NUM_RS+1 children - one for the master
-          assertTrue(servers.size() == NUM_RS+1);
+          assertTrue(servers.size() ==
+            NUM_RS + (LoadBalancer.isTablesOnMaster(TEST_UTIL.getConfiguration())? 1: 0));
           for (String server : servers) {
             assertTrue("From zookeeper: " + server + " hostname: " + hostName,
               server.startsWith(hostName.toLowerCase(Locale.ROOT)+","));
@@ -153,11 +155,14 @@ public class TestRegionServerHostname {
 
   @Test(timeout=30000)
   public void testRegionServerHostnameReportedToMaster() throws Exception {
-    TEST_UTIL.getConfiguration().setBoolean(HRegionServer.RS_HOSTNAME_DISABLE_MASTER_REVERSEDNS_KEY, true);
+    TEST_UTIL.getConfiguration().setBoolean(HRegionServer.RS_HOSTNAME_DISABLE_MASTER_REVERSEDNS_KEY,
+    true);
     TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS);
+    boolean tablesOnMaster = LoadBalancer.isTablesOnMaster(TEST_UTIL.getConfiguration());
+    int expectedRS = NUM_RS + (tablesOnMaster? 1: 0);
     try (ZooKeeperWatcher zkw = TEST_UTIL.getZooKeeperWatcher()) {
       List<String> servers = ZKUtil.listChildrenNoWatch(zkw, zkw.znodePaths.rsZNode);
-      assertEquals("should be NUM_RS+1 children - one for master", NUM_RS + 1, servers.size());
+      assertEquals(expectedRS, servers.size());
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/47344671/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
index 2812e2e..437caac 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
@@ -55,6 +55,7 @@ import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.master.LoadBalancer;
 import org.apache.hadoop.hbase.test.MetricsAssertHelper;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
@@ -103,11 +104,13 @@ public class TestRegionServerMetrics {
   private static byte[] qualifier = Bytes.toBytes("qual");
   private static byte[] val = Bytes.toBytes("val");
   private static Admin admin;
+  private static boolean TABLES_ON_MASTER;
 
   @BeforeClass
   public static void startCluster() throws Exception {
     metricsHelper = CompatibilityFactory.getInstance(MetricsAssertHelper.class);
     TEST_UTIL = new HBaseTestingUtility();
+    TABLES_ON_MASTER = LoadBalancer.isTablesOnMaster(TEST_UTIL.getConfiguration());
     conf = TEST_UTIL.getConfiguration();
     conf.getLong("hbase.splitlog.max.resubmit", 0);
     // Make the failure test faster
@@ -241,7 +244,7 @@ public class TestRegionServerMetrics {
 
   @Test
   public void testRegionCount() throws Exception {
-    metricsHelper.assertGauge("regionCount", 1, serverSource);
+    metricsHelper.assertGauge("regionCount", TABLES_ON_MASTER? 1: 3, serverSource);
   }
 
   @Test
@@ -283,32 +286,42 @@ public class TestRegionServerMetrics {
     doNGets(10, true);  // true = batch
 
     metricsRegionServer.getRegionServerWrapper().forceRecompute();
-    assertCounter("totalRequestCount", requests + 41);
-    assertCounter("totalRowActionRequestCount", rowActionRequests + 50);
-    assertCounter("readRequestCount", readRequests + 20);
+    if (TABLES_ON_MASTER) {
+      assertCounter("totalRequestCount", requests + 41);
+      assertCounter("totalRowActionRequestCount", rowActionRequests + 50);
+      assertCounter("readRequestCount", readRequests + 20);
+    }
+
+
     assertCounter("writeRequestCount", writeRequests + 30);
 
     doNPuts(30, true);
 
     metricsRegionServer.getRegionServerWrapper().forceRecompute();
-    assertCounter("totalRequestCount", requests + 42);
-    assertCounter("totalRowActionRequestCount", rowActionRequests + 80);
-    assertCounter("readRequestCount", readRequests + 20);
+    if (TABLES_ON_MASTER) {
+      assertCounter("totalRequestCount", requests + 42);
+      assertCounter("totalRowActionRequestCount", rowActionRequests + 80);
+      assertCounter("readRequestCount", readRequests + 20);
+    }
     assertCounter("writeRequestCount", writeRequests + 60);
 
     doScan(10, false); // test after batch put so we have enough lines
     metricsRegionServer.getRegionServerWrapper().forceRecompute();
-    assertCounter("totalRequestCount", requests + 52);
-    assertCounter("totalRowActionRequestCount", rowActionRequests + 90);
-    assertCounter("readRequestCount", readRequests + 30);
+    if (TABLES_ON_MASTER) {
+      assertCounter("totalRequestCount", requests + 52);
+      assertCounter("totalRowActionRequestCount", rowActionRequests + 90);
+      assertCounter("readRequestCount", readRequests + 30);
+    }
     assertCounter("writeRequestCount", writeRequests + 60);
     numScanNext += 10;
 
     doScan(10, true); // true = caching
     metricsRegionServer.getRegionServerWrapper().forceRecompute();
-    assertCounter("totalRequestCount", requests + 53);
-    assertCounter("totalRowActionRequestCount", rowActionRequests + 100);
-    assertCounter("readRequestCount", readRequests + 40);
+    if (TABLES_ON_MASTER) {
+      assertCounter("totalRequestCount", requests + 53);
+      assertCounter("totalRowActionRequestCount", rowActionRequests + 100);
+      assertCounter("readRequestCount", readRequests + 40);
+    }
     assertCounter("writeRequestCount", writeRequests + 60);
     numScanNext += 1;
   }
@@ -341,7 +354,7 @@ public class TestRegionServerMetrics {
     TEST_UTIL.getAdmin().flush(tableName);
 
     metricsRegionServer.getRegionServerWrapper().forceRecompute();
-    assertGauge("storeCount", 1);
+    assertGauge("storeCount", TABLES_ON_MASTER? 1: 7);
     assertGauge("storeFileCount", 1);
   }
 
@@ -424,7 +437,9 @@ public class TestRegionServerMetrics {
     }
     numScanNext += NUM_SCAN_NEXT;
     assertRegionMetrics("scanCount", NUM_SCAN_NEXT);
-    assertCounter("ScanSize_num_ops", numScanNext);
+    if (TABLES_ON_MASTER) {
+      assertCounter("ScanSize_num_ops", numScanNext);
+    }
   }
 
   @Test
@@ -442,7 +457,9 @@ public class TestRegionServerMetrics {
     }
     numScanNext += NUM_SCAN_NEXT;
     assertRegionMetrics("scanCount", NUM_SCAN_NEXT);
-    assertCounter("ScanTime_num_ops", numScanNext);
+    if (TABLES_ON_MASTER) {
+      assertCounter("ScanTime_num_ops", numScanNext);
+    }
   }
 
   @Test
@@ -456,11 +473,15 @@ public class TestRegionServerMetrics {
     for (int nextCount = 0; nextCount < NUM_SCAN_NEXT; nextCount++) {
       Result result = resultScanners.next();
       assertNotNull(result);
-      assertEquals(1, result.size());
+      if (TABLES_ON_MASTER) {
+        assertEquals(1, result.size());
+      }
     }
     numScanNext += NUM_SCAN_NEXT;
     assertRegionMetrics("scanCount", NUM_SCAN_NEXT);
-    assertCounter("ScanSize_num_ops", numScanNext);
+    if (TABLES_ON_MASTER) {
+      assertCounter("ScanSize_num_ops", numScanNext);
+    }
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hbase/blob/47344671/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReadRequestMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReadRequestMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReadRequestMetrics.java
index 9242c0c..297aa50 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReadRequestMetrics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReadRequestMetrics.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.filter.BinaryComparator;
 import org.apache.hadoop.hbase.filter.CompareFilter;
 import org.apache.hadoop.hbase.filter.RowFilter;
 import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.master.LoadBalancer;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.AfterClass;
@@ -87,6 +88,7 @@ public class TestRegionServerReadRequestMetrics {
 
   @BeforeClass
   public static void setUpOnce() throws Exception {
+    // Default starts one regionserver only.
     TEST_UTIL.startMiniCluster();
     admin = TEST_UTIL.getAdmin();
     serverNames = admin.getClusterStatus().getServers();
@@ -121,8 +123,16 @@ public class TestRegionServerReadRequestMetrics {
 
     assertEquals(expectedReadRequests,
       requestsMap.get(Metric.REGION_READ) - requestsMapPrev.get(Metric.REGION_READ));
-    assertEquals(expectedReadRequests,
+    boolean tablesOnMaster = LoadBalancer.isTablesOnMaster(TEST_UTIL.getConfiguration());
+    if (tablesOnMaster) {
+      // If NO tables on master, then the single regionserver in this test carries user-space
+      // tables and the meta table. The first time through, the read will be inflated by meta
+      // lookups. We don't know which test will be first through since junit randomizes. This
+      // method is used by a bunch of tests. Just do this check if master is hosting (system)
+      // regions only.
+      assertEquals(expectedReadRequests,
       requestsMap.get(Metric.SERVER_READ) - requestsMapPrev.get(Metric.SERVER_READ));
+    }
     assertEquals(expectedFilteredReadRequests,
       requestsMap.get(Metric.FILTERED_REGION_READ)
         - requestsMapPrev.get(Metric.FILTERED_REGION_READ));

http://git-wip-us.apache.org/repos/asf/hbase/blob/47344671/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 a4e90e6..6f8a23b 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
@@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.LocalHBaseCluster;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.MiniHBaseCluster.MiniHBaseClusterRegionServer;
+import org.apache.hadoop.hbase.master.LoadBalancer;
 import org.apache.hadoop.hbase.master.ServerManager;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
@@ -83,8 +84,9 @@ public class TestRegionServerReportForDuty {
     // Use a random unique port
     cluster.getConfiguration().setInt(HConstants.MASTER_PORT, HBaseTestingUtility.randomFreePort());
     // master has a rs. defaultMinToStart = 2
-    cluster.getConfiguration().setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, 2);
-    cluster.getConfiguration().setInt(ServerManager.WAIT_ON_REGIONSERVERS_MAXTOSTART, 2);
+    boolean tablesOnMaster = LoadBalancer.isTablesOnMaster(testUtil.getConfiguration());
+    cluster.getConfiguration().setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, tablesOnMaster? 2: 1);
+    cluster.getConfiguration().setInt(ServerManager.WAIT_ON_REGIONSERVERS_MAXTOSTART, tablesOnMaster? 2: 1);
     master = cluster.addMaster();
     rs = cluster.addRegionServer();
     LOG.debug("Starting master: " + master.getMaster().getServerName());
@@ -110,8 +112,10 @@ public class TestRegionServerReportForDuty {
     // Start a new master and use another random unique port
     // Also let it wait for exactly 2 region severs to report in.
     cluster.getConfiguration().setInt(HConstants.MASTER_PORT, HBaseTestingUtility.randomFreePort());
-    cluster.getConfiguration().setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, 3);
-    cluster.getConfiguration().setInt(ServerManager.WAIT_ON_REGIONSERVERS_MAXTOSTART, 3);
+    cluster.getConfiguration().setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART,
+      tablesOnMaster? 3: 2);
+    cluster.getConfiguration().setInt(ServerManager.WAIT_ON_REGIONSERVERS_MAXTOSTART,
+      tablesOnMaster? 3: 2);
     backupMaster = cluster.addMaster();
     LOG.debug("Starting new master: " + backupMaster.getMaster().getServerName());
     backupMaster.start();
@@ -121,7 +125,8 @@ public class TestRegionServerReportForDuty {
     // Do some checking/asserts here.
     assertTrue(backupMaster.getMaster().isActiveMaster());
     assertTrue(backupMaster.getMaster().isInitialized());
-    assertEquals(backupMaster.getMaster().getServerManager().getOnlineServersList().size(), 3);
+    assertEquals(backupMaster.getMaster().getServerManager().getOnlineServersList().size(),
+      tablesOnMaster? 3: 2);
 
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/47344671/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
index aa517b5..69bc9a7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
@@ -70,10 +70,7 @@ import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.MasterObserver;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
-import org.apache.hadoop.hbase.master.HMaster;
-import org.apache.hadoop.hbase.master.MasterRpcServices;
-import org.apache.hadoop.hbase.master.NoSuchProcedureException;
-import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.master.*;
 import org.apache.hadoop.hbase.master.RegionState.State;
 import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.master.assignment.RegionStates;
@@ -830,9 +827,14 @@ public class TestSplitTransactionOnCluster {
     // hbase:meta  We don't want hbase:meta replay polluting our test when we later crash
     // the table region serving server.
     int metaServerIndex = cluster.getServerWithMeta();
-    assertTrue(metaServerIndex == -1); // meta is on master now
-    // TODO: When we change master so it doesn't carry regions, be careful here.
-    HRegionServer metaRegionServer = cluster.getMaster();
+    boolean tablesOnMaster = LoadBalancer.isTablesOnMaster(TESTING_UTIL.getConfiguration());
+    if (tablesOnMaster) {
+      // Need to check master is supposed to host meta... perhaps it is not.
+      throw new UnsupportedOperationException();
+      // TODO: assertTrue(metaServerIndex == -1); // meta is on master now
+    }
+    HRegionServer metaRegionServer = tablesOnMaster?
+      cluster.getMaster(): cluster.getRegionServer(metaServerIndex);
     int tableRegionIndex = cluster.getServerWith(hri.getRegionName());
     assertTrue(tableRegionIndex != -1);
     HRegionServer tableRegionServer = cluster.getRegionServer(tableRegionIndex);

http://git-wip-us.apache.org/repos/asf/hbase/blob/47344671/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/throttle/TestFlushWithThroughputController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/throttle/TestFlushWithThroughputController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/throttle/TestFlushWithThroughputController.java
index fb11756..d790eff 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/throttle/TestFlushWithThroughputController.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/throttle/TestFlushWithThroughputController.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.master.LoadBalancer;
 import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.Region;
@@ -169,7 +170,12 @@ public class TestFlushWithThroughputController {
     }
     assertEquals(0.0, regionServer.getFlushPressure(), EPSILON);
     Thread.sleep(5000);
-    assertEquals(10L * 1024 * 1024, throughputController.getMaxThroughput(), EPSILON);
+    boolean tablesOnMaster = LoadBalancer.isTablesOnMaster(hbtu.getConfiguration());
+    if (tablesOnMaster) {
+      // If no tables on the master, this math is off and I'm not sure what it is supposed to be
+      // when meta is on the regionserver and not on the master.
+      assertEquals(10L * 1024 * 1024, throughputController.getMaxThroughput(), EPSILON);
+    }
     Table table = conn.getTable(tableName);
     Random rand = new Random();
     for (int i = 0; i < 10; i++) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/47344671/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java
index 27128cc..dad071a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java
@@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.security.access.Permission.Action;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.SecurityTests;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.JVMClusterUtil;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -150,9 +151,15 @@ public class TestNamespaceCommands extends SecureTestUtil {
     // Wait for the ACL table to become available
     UTIL.waitTableAvailable(AccessControlLists.ACL_TABLE_NAME.getName(), 30 * 1000);
 
-    ACCESS_CONTROLLER = (AccessController) UTIL.getMiniHBaseCluster().getMaster()
-      .getRegionServerCoprocessorHost()
-        .findCoprocessor(AccessController.class.getName());
+    // Find the Access Controller CP. Could be on master or if master is not serving regions, is
+    // on an arbitrary server.
+    for (JVMClusterUtil.RegionServerThread rst:
+        UTIL.getMiniHBaseCluster().getLiveRegionServerThreads()) {
+      ACCESS_CONTROLLER = (AccessController)rst.getRegionServer().getRegionServerCoprocessorHost().
+        findCoprocessor(AccessController.class.getName());
+      if (ACCESS_CONTROLLER != null) break;
+    }
+    if (ACCESS_CONTROLLER == null) throw new NullPointerException();
 
     UTIL.getAdmin().createNamespace(NamespaceDescriptor.create(TEST_NAMESPACE).build());
     UTIL.getAdmin().createNamespace(NamespaceDescriptor.create(TEST_NAMESPACE2).build());


[43/50] [abbrv] hbase git commit: HBASE-18493 [AMv2] Skipped re-assignment of regions on crashed server through AssignmentManager.checkIfShouldMoveSystemRegionAsync() as those regions are handled by ServerCrashProcedure

Posted by st...@apache.org.
HBASE-18493 [AMv2] Skipped re-assignment of regions on crashed server through AssignmentManager.checkIfShouldMoveSystemRegionAsync() as those regions are handled by ServerCrashProcedure

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/acf9b87d
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/acf9b87d
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/acf9b87d

Branch: refs/heads/HBASE-14070.HLC
Commit: acf9b87dca2cd190f4b5318efd5dc48e19b317f4
Parents: 4c74a73
Author: Umesh Agashe <ua...@cloudera.com>
Authored: Tue Aug 15 14:00:04 2017 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Wed Aug 16 08:12:05 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/master/assignment/AssignmentManager.java   | 9 +++++++++
 1 file changed, 9 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/acf9b87d/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
index 54cb1ca..0b23f47 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
@@ -480,6 +480,15 @@ public class AssignmentManager implements ServerListener {
         synchronized (checkIfShouldMoveSystemRegionLock) {
           List<RegionPlan> plans = new ArrayList<>();
           for (ServerName server : getExcludedServersForSystemTable()) {
+            if (master.getServerManager().isServerDead(server)) {
+              // TODO: See HBASE-18494 and HBASE-18495. Though getExcludedServersForSystemTable()
+              // considers only online servers, the server could be queued for dead server
+              // processing. As region assignments for crashed server is handled by
+              // ServerCrashProcedure, do NOT handle them here. The goal is to handle this through
+              // regular flow of LoadBalancer as a favored node and not to have this special
+              // handling.
+              continue;
+            }
             List<HRegionInfo> regionsShouldMove = getCarryingSystemTables(server);
             if (!regionsShouldMove.isEmpty()) {
               for (HRegionInfo regionInfo : regionsShouldMove) {


[49/50] [abbrv] hbase git commit: HBASE-18375: Fix the bug where the pool chunks from ChunkCreator are deallocated and not returned to pool, because there is no reference to them

Posted by st...@apache.org.
HBASE-18375: Fix the bug where the pool chunks from ChunkCreator are deallocated and not returned to pool, because there is no reference to them


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

Branch: refs/heads/HBASE-14070.HLC
Commit: 75a6b36849c58d6a751f57226ab0c8f7884a9e87
Parents: 092dc6d
Author: anastas <an...@yahoo-inc.com>
Authored: Thu Aug 17 18:23:19 2017 +0300
Committer: anastas <an...@yahoo-inc.com>
Committed: Thu Aug 17 18:23:19 2017 +0300

----------------------------------------------------------------------
 .../regionserver/CellChunkImmutableSegment.java |   5 +-
 .../hadoop/hbase/regionserver/ChunkCreator.java | 171 +++++++++----------
 .../hbase/regionserver/CompactionPipeline.java  |  19 ++-
 .../hbase/regionserver/MemStoreLABImpl.java     |  27 ++-
 .../hbase/regionserver/TestMemStoreLAB.java     |  12 +-
 .../TestMemstoreLABWithoutPool.java             |   3 +-
 6 files changed, 126 insertions(+), 111 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/75a6b368/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellChunkImmutableSegment.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellChunkImmutableSegment.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellChunkImmutableSegment.java
index cdda279..3653166 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellChunkImmutableSegment.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellChunkImmutableSegment.java
@@ -176,10 +176,7 @@ public class CellChunkImmutableSegment extends ImmutableSegment {
   private int createCellReference(ByteBufferKeyValue cell, ByteBuffer idxBuffer, int idxOffset) {
     int offset = idxOffset;
     int dataChunkID = cell.getChunkId();
-    // ensure strong pointer to data chunk, as index is no longer directly points to it
-    Chunk c = ChunkCreator.getInstance().saveChunkFromGC(dataChunkID);
-    // if c is null, it means that this cell chunks was already released shouldn't happen
-    assert (c!=null);
+
     offset = ByteBufferUtils.putInt(idxBuffer, offset, dataChunkID);    // write data chunk id
     offset = ByteBufferUtils.putInt(idxBuffer, offset, cell.getOffset());          // offset
     offset = ByteBufferUtils.putInt(idxBuffer, offset, KeyValueUtil.length(cell)); // length

http://git-wip-us.apache.org/repos/asf/hbase/blob/75a6b368/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java
index 7e5395c..e818426 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java
@@ -18,7 +18,6 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
-import java.lang.ref.WeakReference;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Set;
@@ -58,21 +57,8 @@ public class ChunkCreator {
   // the header size need to be changed in case chunk id size is changed
   public static final int SIZEOF_CHUNK_HEADER = Bytes.SIZEOF_INT;
 
-  // An object pointed by a weak reference can be garbage collected, in opposite to an object
-  // referenced by a strong (regular) reference. Every chunk created via ChunkCreator is referenced
-  // from either weakChunkIdMap or strongChunkIdMap.
-  // Upon chunk C creation, C's ID is mapped into weak reference to C, in order not to disturb C's
-  // GC in case all other reference to C are going to be removed.
-  // When chunk C is referenced from CellChunkMap (via C's ID) it is possible to GC the chunk C.
-  // To avoid that upon inserting C into CellChunkMap, C's ID is mapped into strong (regular)
-  // reference to C.
-
-  // map that doesn't influence GC
-  private Map<Integer, WeakReference<Chunk>> weakChunkIdMap =
-      new ConcurrentHashMap<Integer, WeakReference<Chunk>>();
-
-  // map that keeps chunks from garbage collection
-  private Map<Integer, Chunk> strongChunkIdMap = new ConcurrentHashMap<Integer, Chunk>();
+  // mapping from chunk IDs to chunks
+  private Map<Integer, Chunk> chunkIdMap = new ConcurrentHashMap<Integer, Chunk>();
 
   private final int chunkSize;
   private final boolean offheap;
@@ -95,7 +81,7 @@ public class ChunkCreator {
   }
 
   /**
-   * Initializes the instance of MSLABChunkCreator
+   * Initializes the instance of ChunkCreator
    * @param chunkSize the chunkSize
    * @param offheap indicates if the chunk is to be created offheap or not
    * @param globalMemStoreSize  the global memstore size
@@ -120,10 +106,19 @@ public class ChunkCreator {
   }
 
   /**
-   * Creates and inits a chunk.
+   * Creates and inits a chunk. The default implementation.
    * @return the chunk that was initialized
    */
   Chunk getChunk() {
+    return getChunk(CompactingMemStore.IndexType.ARRAY_MAP);
+  }
+
+  /**
+   * Creates and inits a chunk.
+   * @return the chunk that was initialized
+   * @param chunkIndexType whether the requested chunk is going to be used with CellChunkMap index
+   */
+  Chunk getChunk(CompactingMemStore.IndexType chunkIndexType) {
     Chunk chunk = null;
     if (pool != null) {
       //  the pool creates the chunk internally. The chunk#init() call happens here
@@ -137,68 +132,45 @@ public class ChunkCreator {
       }
     }
     if (chunk == null) {
-      chunk = createChunk();
+      // the second boolean parameter means:
+      // if CellChunkMap index is requested, put allocated on demand chunk mapping into chunkIdMap
+      chunk = createChunk(false, chunkIndexType);
     }
-    // put this chunk initially into the weakChunkIdMap
-    this.weakChunkIdMap.put(chunk.getId(), new WeakReference<>(chunk));
+
     // now we need to actually do the expensive memory allocation step in case of a new chunk,
     // else only the offset is set to the beginning of the chunk to accept allocations
     chunk.init();
     return chunk;
   }
 
-  private Chunk createChunk() {
-    return createChunk(false);
-  }
-
   /**
    * Creates the chunk either onheap or offheap
    * @param pool indicates if the chunks have to be created which will be used by the Pool
+   * @param chunkIndexType
    * @return the chunk
    */
-  private Chunk createChunk(boolean pool) {
+  private Chunk createChunk(boolean pool, CompactingMemStore.IndexType chunkIndexType) {
+    Chunk chunk = null;
     int id = chunkID.getAndIncrement();
     assert id > 0;
     // do not create offheap chunk on demand
     if (pool && this.offheap) {
-      return new OffheapChunk(chunkSize, id, pool);
+      chunk = new OffheapChunk(chunkSize, id, pool);
     } else {
-      return new OnheapChunk(chunkSize, id, pool);
+      chunk = new OnheapChunk(chunkSize, id, pool);
     }
+    if (pool || (chunkIndexType == CompactingMemStore.IndexType.CHUNK_MAP)) {
+      // put the pool chunk into the chunkIdMap so it is not GC-ed
+      this.chunkIdMap.put(chunk.getId(), chunk);
+    }
+    return chunk;
   }
 
   @VisibleForTesting
   // Used to translate the ChunkID into a chunk ref
   Chunk getChunk(int id) {
-    WeakReference<Chunk> ref = weakChunkIdMap.get(id);
-    if (ref != null) {
-      return ref.get();
-    }
-    // check also the strong mapping
-    return strongChunkIdMap.get(id);
-  }
-
-  // transfer the weak pointer to be a strong chunk pointer
-  Chunk saveChunkFromGC(int chunkID) {
-    Chunk c = strongChunkIdMap.get(chunkID); // check whether the chunk is already protected
-    if (c != null)                           // with strong pointer
-      return c;
-    WeakReference<Chunk> ref = weakChunkIdMap.get(chunkID);
-    if (ref != null) {
-      c = ref.get();
-    }
-    if (c != null) {
-      // put this strong reference to chunk into the strongChunkIdMap
-      // the read of the weakMap is always happening before the read of the strongMap
-      // so no synchronization issues here
-      this.strongChunkIdMap.put(chunkID, c);
-      this.weakChunkIdMap.remove(chunkID);
-      return c;
-    }
-    // we should actually never return null as someone should not ask to save from GC a chunk,
-    // which is already released. However, we are not asserting it here and we let the caller
-    // to deal with the return value an assert if needed
-    return null;
+    // can return null if chunk was never mapped
+    return chunkIdMap.get(id);
   }
 
   int getChunkSize() {
@@ -210,30 +182,23 @@ public class ChunkCreator {
   }
 
   private void removeChunks(Set<Integer> chunkIDs) {
-    this.weakChunkIdMap.keySet().removeAll(chunkIDs);
-    this.strongChunkIdMap.keySet().removeAll(chunkIDs);
+    this.chunkIdMap.keySet().removeAll(chunkIDs);
   }
 
   Chunk removeChunk(int chunkId) {
-    WeakReference<Chunk> weak = this.weakChunkIdMap.remove(chunkId);
-    Chunk strong = this.strongChunkIdMap.remove(chunkId);
-    if (weak != null) {
-      return weak.get();
-    }
-    return strong;
+    return this.chunkIdMap.remove(chunkId);
   }
 
   @VisibleForTesting
-  // the chunks in the weakChunkIdMap may already be released so we shouldn't relay
+  // the chunks in the chunkIdMap may already be released so we shouldn't relay
   // on this counting for strong correctness. This method is used only in testing.
-  int size() {
-    return this.weakChunkIdMap.size()+this.strongChunkIdMap.size();
+  int numberOfMappedChunks() {
+    return this.chunkIdMap.size();
   }
 
   @VisibleForTesting
   void clearChunkIds() {
-    this.strongChunkIdMap.clear();
-    this.weakChunkIdMap.clear();
+    this.chunkIdMap.clear();
   }
 
   /**
@@ -262,7 +227,9 @@ public class ChunkCreator {
       this.poolSizePercentage = poolSizePercentage;
       this.reclaimedChunks = new LinkedBlockingQueue<>();
       for (int i = 0; i < initialCount; i++) {
-        Chunk chunk = createChunk(true);
+        // Chunks from pool are covered with strong references anyway
+        // TODO: change to CHUNK_MAP if it is generally defined
+        Chunk chunk = createChunk(true, CompactingMemStore.IndexType.ARRAY_MAP);
         chunk.init();
         reclaimedChunks.add(chunk);
       }
@@ -281,7 +248,7 @@ public class ChunkCreator {
      * then.
      * Note: Chunks returned by this pool must be put back to the pool after its use.
      * @return a chunk
-     * @see #putbackChunks(Set)
+     * @see #putbackChunks(Chunk)
      */
     Chunk getChunk() {
       Chunk chunk = reclaimedChunks.poll();
@@ -294,7 +261,8 @@ public class ChunkCreator {
           long created = this.chunkCount.get();
           if (created < this.maxCount) {
             if (this.chunkCount.compareAndSet(created, created + 1)) {
-              chunk = createChunk(true);
+              // TODO: change to CHUNK_MAP if it is generally defined
+              chunk = createChunk(true, CompactingMemStore.IndexType.ARRAY_MAP);
               break;
             }
           } else {
@@ -308,21 +276,16 @@ public class ChunkCreator {
     /**
      * Add the chunks to the pool, when the pool achieves the max size, it will skip the remaining
      * chunks
-     * @param chunks
+     * @param c
      */
-    private void putbackChunks(Set<Integer> chunks) {
-      int toAdd = Math.min(chunks.size(), this.maxCount - reclaimedChunks.size());
-      Iterator<Integer> iterator = chunks.iterator();
-      while (iterator.hasNext()) {
-        Integer chunkId = iterator.next();
-        // remove the chunks every time though they are from the pool or not
-        Chunk chunk = ChunkCreator.this.removeChunk(chunkId);
-        if (chunk != null) {
-          if (chunk.isFromPool() && toAdd > 0) {
-            reclaimedChunks.add(chunk);
-          }
-          toAdd--;
-        }
+    private void putbackChunks(Chunk c) {
+      int toAdd = this.maxCount - reclaimedChunks.size();
+      if (c.isFromPool() && toAdd > 0) {
+        reclaimedChunks.add(c);
+      } else {
+        // remove the chunk (that is not going to pool)
+        // though it is initially from the pool or not
+        ChunkCreator.this.removeChunk(c.getId());
       }
     }
 
@@ -433,6 +396,20 @@ public class ChunkCreator {
     return 0;
   }
 
+  @VisibleForTesting
+  boolean isChunkInPool(int chunkId) {
+    if (pool != null) {
+      // chunks that are from pool will return true chunk reference not null
+      Chunk c = getChunk(chunkId);
+      if (c==null) {
+        return false;
+      }
+      return pool.reclaimedChunks.contains(c);
+    }
+
+    return false;
+  }
+
   /*
    * Only used in testing
    */
@@ -444,10 +421,24 @@ public class ChunkCreator {
   }
 
   synchronized void putbackChunks(Set<Integer> chunks) {
-    if (pool != null) {
-      pool.putbackChunks(chunks);
-    } else {
+    // if there is no pool just try to clear the chunkIdMap in case there is something
+    if ( pool == null ) {
       this.removeChunks(chunks);
+      return;
     }
+
+    // if there is pool, go over all chunk IDs that came back, the chunks may be from pool or not
+    for (int chunkID : chunks) {
+      // translate chunk ID to chunk, if chunk initially wasn't in pool
+      // this translation will (most likely) return null
+      Chunk chunk = ChunkCreator.this.getChunk(chunkID);
+      if (chunk != null) {
+        pool.putbackChunks(chunk);
+      }
+      // if chunk is null, it was never covered by the chunkIdMap (and so wasn't in pool also),
+      // so we have nothing to do on its release
+    }
+    return;
   }
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/75a6b368/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionPipeline.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionPipeline.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionPipeline.java
index 5136f24..f281392 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionPipeline.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionPipeline.java
@@ -43,10 +43,10 @@ import org.apache.hadoop.hbase.util.ClassSize;
  * suffix of the pipeline.
  *
  * The synchronization model is copy-on-write. Methods which change the structure of the
- * pipeline (pushHead() and swap()) apply their changes in the context of a lock. They also make
- * a read-only copy of the pipeline's list. Read methods read from a read-only copy. If a read
- * method accesses the read-only copy more than once it makes a local copy of it
- * to ensure it accesses the same copy.
+ * pipeline (pushHead(), flattenOneSegment() and swap()) apply their changes in the context of a
+ * lock. They also make a read-only copy of the pipeline's list. Read methods read from a
+ * read-only copy. If a read method accesses the read-only copy more than once it makes a local
+ * copy of it to ensure it accesses the same copy.
  *
  * The methods getVersionedList(), getVersionedTail(), and flattenOneSegment() are also
  * protected by a lock since they need to have a consistent (atomic) view of the pipeline list
@@ -261,6 +261,8 @@ public class CompactionPipeline {
 
   private void swapSuffix(List<? extends Segment> suffix, ImmutableSegment segment,
       boolean closeSegmentsInSuffix) {
+    pipeline.removeAll(suffix);
+    if(segment != null) pipeline.addLast(segment);
     // During index merge we won't be closing the segments undergoing the merge. Segment#close()
     // will release the MSLAB chunks to pool. But in case of index merge there wont be any data copy
     // from old MSLABs. So the new cells in new segment also refers to same chunks. In case of data
@@ -272,15 +274,20 @@ public class CompactionPipeline {
         itemInSuffix.close();
       }
     }
-    pipeline.removeAll(suffix);
-    if(segment != null) pipeline.addLast(segment);
   }
 
   // replacing one segment in the pipeline with a new one exactly at the same index
   // need to be called only within synchronized block
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="VO_VOLATILE_INCREMENT",
+      justification="replaceAtIndex is invoked under a synchronize block so safe")
   private void replaceAtIndex(int idx, ImmutableSegment newSegment) {
     pipeline.set(idx, newSegment);
     readOnlyCopy = new LinkedList<>(pipeline);
+    // the version increment is indeed needed, because the swap uses removeAll() method of the
+    // linked-list that compares the objects to find what to remove.
+    // The flattening changes the segment object completely (creation pattern) and so
+    // swap will not proceed correctly after concurrent flattening.
+    version++;
   }
 
   public Segment getTail() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/75a6b368/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java
index 85e2abe..2ae665e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java
@@ -78,6 +78,7 @@ public class MemStoreLABImpl implements MemStoreLAB {
   private final int chunkSize;
   private final int maxAlloc;
   private final ChunkCreator chunkCreator;
+  private final CompactingMemStore.IndexType idxType; // what index is used for corresponding segment
 
   // This flag is for closing this instance, its set when clearing snapshot of
   // memstore
@@ -100,6 +101,9 @@ public class MemStoreLABImpl implements MemStoreLAB {
     // if we don't exclude allocations >CHUNK_SIZE, we'd infiniteloop on one!
     Preconditions.checkArgument(maxAlloc <= chunkSize,
         MAX_ALLOC_KEY + " must be less than " + CHUNK_SIZE_KEY);
+    idxType = CompactingMemStore.IndexType.valueOf(conf.get(
+        CompactingMemStore.COMPACTING_MEMSTORE_INDEX_KEY,
+        CompactingMemStore.COMPACTING_MEMSTORE_INDEX_DEFAULT));
   }
 
   @Override
@@ -239,7 +243,7 @@ public class MemStoreLABImpl implements MemStoreLAB {
         if (c != null) {
           return c;
         }
-        c = this.chunkCreator.getChunk();
+        c = this.chunkCreator.getChunk(idxType);
         if (c != null) {
           // set the curChunk. No need of CAS as only one thread will be here
           curChunk.set(c);
@@ -253,12 +257,15 @@ public class MemStoreLABImpl implements MemStoreLAB {
     return null;
   }
 
-  // Returning a new chunk, without replacing current chunk,
-  // meaning MSLABImpl does not make the returned chunk as CurChunk.
-  // The space on this chunk will be allocated externally
-  // The interface is only for external callers
+  /* Creating chunk to be used as index chunk in CellChunkMap, part of the chunks array.
+  ** Returning a new chunk, without replacing current chunk,
+  ** meaning MSLABImpl does not make the returned chunk as CurChunk.
+  ** The space on this chunk will be allocated externally.
+  ** The interface is only for external callers
+  */
   @Override
   public Chunk getNewExternalChunk() {
+    // the new chunk is going to be part of the chunk array and will always be referenced
     Chunk c = this.chunkCreator.getChunk();
     chunks.add(c.getId());
     return c;
@@ -280,4 +287,14 @@ public class MemStoreLABImpl implements MemStoreLAB {
     }
     return pooledChunks;
   }
+
+  @VisibleForTesting Integer getNumOfChunksReturnedToPool() {
+    int i = 0;
+    for (Integer id : this.chunks) {
+      if (chunkCreator.isChunkInPool(id)) {
+        i++;
+      }
+    }
+    return i;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/75a6b368/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java
index f171dd0..06b9c40 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java
@@ -247,14 +247,16 @@ public class TestMemStoreLAB {
         }
       }
       // none of the chunkIds would have been returned back
-      assertTrue("All the chunks must have been cleared", ChunkCreator.INSTANCE.size() != 0);
+      assertTrue("All the chunks must have been cleared",
+          ChunkCreator.INSTANCE.numberOfMappedChunks() != 0);
+      int pooledChunksNum = mslab.getPooledChunks().size();
       // close the mslab
       mslab.close();
-      // make sure all chunks reclaimed or removed from chunk queue
-      int queueLength = mslab.getPooledChunks().size();
+      // make sure all chunks where reclaimed back to pool
+      int queueLength = mslab.getNumOfChunksReturnedToPool();
       assertTrue("All chunks in chunk queue should be reclaimed or removed"
-          + " after mslab closed but actually: " + queueLength,
-        queueLength == 0);
+          + " after mslab closed but actually: " + (pooledChunksNum-queueLength),
+          pooledChunksNum-queueLength == 0);
     } finally {
       ChunkCreator.INSTANCE = oldInstance;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/75a6b368/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemstoreLABWithoutPool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemstoreLABWithoutPool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemstoreLABWithoutPool.java
index 96be8ec..d3f9bc1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemstoreLABWithoutPool.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemstoreLABWithoutPool.java
@@ -140,7 +140,8 @@ public class TestMemstoreLABWithoutPool {
       mslab[i].close();
     }
     // all of the chunkIds would have been returned back
-    assertTrue("All the chunks must have been cleared", ChunkCreator.INSTANCE.size() == 0);
+    assertTrue("All the chunks must have been cleared",
+        ChunkCreator.INSTANCE.numberOfMappedChunks() == 0);
   }
 
   private Thread getChunkQueueTestThread(final MemStoreLABImpl mslab, String threadName,


[26/50] [abbrv] hbase git commit: HBASE-18238 rubocop autocorrect for bin/

Posted by st...@apache.org.
HBASE-18238 rubocop autocorrect for bin/


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

Branch: refs/heads/HBASE-14070.HLC
Commit: ea8fa59a4c2fe7633ebe70df622098bfe36b5df9
Parents: 096dac2
Author: Mike Drob <md...@apache.org>
Authored: Wed Jul 19 12:05:26 2017 -0500
Committer: Mike Drob <md...@apache.org>
Committed: Mon Aug 14 13:53:37 2017 -0500

----------------------------------------------------------------------
 bin/draining_servers.rb             | 108 +++++++++++++++----------------
 bin/get-active-master.rb            |   6 +-
 bin/hirb.rb                         |  46 +++++++------
 bin/region_mover.rb                 |   2 +-
 bin/region_status.rb                |  50 +++++++-------
 bin/replication/copy_tables_desc.rb |  47 +++++++-------
 bin/shutdown_regionserver.rb        |  16 ++---
 7 files changed, 132 insertions(+), 143 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ea8fa59a/bin/draining_servers.rb
----------------------------------------------------------------------
diff --git a/bin/draining_servers.rb b/bin/draining_servers.rb
index 8e1b250..ea74c30 100644
--- a/bin/draining_servers.rb
+++ b/bin/draining_servers.rb
@@ -16,7 +16,7 @@
 # limitations under the License.
 #
 
-# Add or remove servers from draining mode via zookeeper 
+# Add or remove servers from draining mode via zookeeper
 
 require 'optparse'
 include Java
@@ -29,13 +29,13 @@ java_import org.apache.commons.logging.Log
 java_import org.apache.commons.logging.LogFactory
 
 # Name of this script
-NAME = "draining_servers"
+NAME = 'draining_servers'.freeze
 
 # Do command-line parsing
 options = {}
 optparse = OptionParser.new do |opts|
   opts.banner = "Usage: ./hbase org.jruby.Main #{NAME}.rb [options] add|remove|list <hostname>|<host:port>|<servername> ..."
-  opts.separator 'Add remove or list servers in draining mode. Can accept either hostname to drain all region servers' +
+  opts.separator 'Add remove or list servers in draining mode. Can accept either hostname to drain all region servers' \
                  'in that host, a host:port pair or a host,port,startCode triplet. More than one server can be given separated by space'
   opts.on('-h', '--help', 'Display usage information') do
     puts opts
@@ -51,117 +51,117 @@ optparse.parse!
 # Return array of servernames where servername is hostname+port+startcode
 # comma-delimited
 def getServers(admin)
-  serverInfos = admin.getClusterStatus().getServers()
+  serverInfos = admin.getClusterStatus.getServers
   servers = []
   for server in serverInfos
-    servers << server.getServerName()
+    servers << server.getServerName
   end
-  return servers
+  servers
 end
 
 def getServerNames(hostOrServers, config)
   ret = []
   connection = ConnectionFactory.createConnection(config)
-  
+
   for hostOrServer in hostOrServers
     # check whether it is already serverName. No need to connect to cluster
     parts = hostOrServer.split(',')
-    if parts.size() == 3
+    if parts.size == 3
       ret << hostOrServer
-    else 
-      admin = connection.getAdmin() if not admin
+    else
+      admin = connection.getAdmin unless admin
       servers = getServers(admin)
 
-      hostOrServer = hostOrServer.gsub(/:/, ",")
-      for server in servers 
+      hostOrServer = hostOrServer.tr(':', ',')
+      for server in servers
         ret << server if server.start_with?(hostOrServer)
       end
     end
   end
-  
-  admin.close() if admin
-  connection.close()
-  return ret
+
+  admin.close if admin
+  connection.close
+  ret
 end
 
-def addServers(options, hostOrServers)
-  config = HBaseConfiguration.create()
+def addServers(_options, hostOrServers)
+  config = HBaseConfiguration.create
   servers = getServerNames(hostOrServers, config)
-  
-  zkw = org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.new(config, "draining_servers", nil)
+
+  zkw = org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.new(config, 'draining_servers', nil)
   parentZnode = zkw.znodePaths.drainingZNode
-  
+
   begin
     for server in servers
       node = ZKUtil.joinZNode(parentZnode, server)
       ZKUtil.createAndFailSilent(zkw, node)
     end
   ensure
-    zkw.close()
+    zkw.close
   end
 end
 
-def removeServers(options, hostOrServers)
-  config = HBaseConfiguration.create()
+def removeServers(_options, hostOrServers)
+  config = HBaseConfiguration.create
   servers = getServerNames(hostOrServers, config)
-  
-  zkw = org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.new(config, "draining_servers", nil)
+
+  zkw = org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.new(config, 'draining_servers', nil)
   parentZnode = zkw.znodePaths.drainingZNode
-  
+
   begin
     for server in servers
       node = ZKUtil.joinZNode(parentZnode, server)
       ZKUtil.deleteNodeFailSilent(zkw, node)
     end
   ensure
-    zkw.close()
+    zkw.close
   end
 end
 
 # list servers in draining mode
-def listServers(options)
-  config = HBaseConfiguration.create()
-  
-  zkw = org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.new(config, "draining_servers", nil)
+def listServers(_options)
+  config = HBaseConfiguration.create
+
+  zkw = org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.new(config, 'draining_servers', nil)
   parentZnode = zkw.znodePaths.drainingZNode
 
   servers = ZKUtil.listChildrenNoWatch(zkw, parentZnode)
-  servers.each {|server| puts server}
+  servers.each { |server| puts server }
 end
 
-hostOrServers = ARGV[1..ARGV.size()]
+hostOrServers = ARGV[1..ARGV.size]
 
 # Create a logger and disable the DEBUG-level annoying client logging
 def configureLogging(options)
   apacheLogger = LogFactory.getLog(NAME)
   # Configure log4j to not spew so much
-  unless (options[:debug]) 
-    logger = org.apache.log4j.Logger.getLogger("org.apache.hadoop.hbase")
+  unless options[:debug]
+    logger = org.apache.log4j.Logger.getLogger('org.apache.hadoop.hbase')
     logger.setLevel(org.apache.log4j.Level::WARN)
-    logger = org.apache.log4j.Logger.getLogger("org.apache.zookeeper")
+    logger = org.apache.log4j.Logger.getLogger('org.apache.zookeeper')
     logger.setLevel(org.apache.log4j.Level::WARN)
   end
-  return apacheLogger
+  apacheLogger
 end
 
 # Create a logger and save it to ruby global
 $LOG = configureLogging(options)
 case ARGV[0]
-  when 'add'
-    if ARGV.length < 2
-      puts optparse
-      exit 1
-    end
-    addServers(options, hostOrServers)
-  when 'remove'
-    if ARGV.length < 2
-      puts optparse
-      exit 1
-    end
-    removeServers(options, hostOrServers)
-  when 'list'
-    listServers(options)
-  else
+when 'add'
+  if ARGV.length < 2
     puts optparse
-    exit 3
+    exit 1
+  end
+  addServers(options, hostOrServers)
+when 'remove'
+  if ARGV.length < 2
+    puts optparse
+    exit 1
+  end
+  removeServers(options, hostOrServers)
+when 'list'
+  listServers(options)
+else
+  puts optparse
+  exit 3
 end

http://git-wip-us.apache.org/repos/asf/hbase/blob/ea8fa59a/bin/get-active-master.rb
----------------------------------------------------------------------
diff --git a/bin/get-active-master.rb b/bin/get-active-master.rb
index 4479425..0cd3d02 100644
--- a/bin/get-active-master.rb
+++ b/bin/get-active-master.rb
@@ -17,7 +17,7 @@
 
 # Prints the hostname of the machine running the active master.
 
-include Java 
+include Java
 java_import org.apache.hadoop.hbase.HBaseConfiguration
 java_import org.apache.hadoop.hbase.ServerName
 java_import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher
@@ -32,7 +32,7 @@ config = HBaseConfiguration.create
 
 zk = ZooKeeperWatcher.new(config, 'get-active-master', nil)
 begin
-  puts MasterAddressTracker.getMasterAddress(zk).getHostname()
+  puts MasterAddressTracker.getMasterAddress(zk).getHostname
 ensure
-  zk.close()
+  zk.close
 end

http://git-wip-us.apache.org/repos/asf/hbase/blob/ea8fa59a/bin/hirb.rb
----------------------------------------------------------------------
diff --git a/bin/hirb.rb b/bin/hirb.rb
index 841ab54..c06c83b 100644
--- a/bin/hirb.rb
+++ b/bin/hirb.rb
@@ -72,13 +72,13 @@ log_level = org.apache.log4j.Level::ERROR
 interactive = true
 for arg in ARGV
   if arg =~ /^--format=(.+)/i
-    format = $1
+    format = Regexp.last_match(1)
     if format =~ /^html$/i
-      raise NoMethodError.new("Not yet implemented")
+      raise NoMethodError, 'Not yet implemented'
     elsif format =~ /^console$/i
       # This is default
     else
-      raise ArgumentError.new("Unsupported format " + arg)
+      raise ArgumentError, 'Unsupported format ' + arg
     end
     found.push(arg)
   elsif arg == '-h' || arg == '--help'
@@ -89,7 +89,7 @@ for arg in ARGV
     $fullBackTrace = true
     @shell_debug = true
     found.push(arg)
-    puts "Setting DEBUG log level..."
+    puts 'Setting DEBUG log level...'
   elsif arg == '-n' || arg == '--noninteractive'
     interactive = false
     found.push(arg)
@@ -106,13 +106,11 @@ end
 # Delete all processed args
 found.each { |arg| ARGV.delete(arg) }
 # Make sure debug flag gets back to IRB
-if @shell_debug
-  ARGV.unshift('-d')
-end
+ARGV.unshift('-d') if @shell_debug
 
 # Set logging level to avoid verboseness
-org.apache.log4j.Logger.getLogger("org.apache.zookeeper").setLevel(log_level)
-org.apache.log4j.Logger.getLogger("org.apache.hadoop.hbase").setLevel(log_level)
+org.apache.log4j.Logger.getLogger('org.apache.zookeeper').setLevel(log_level)
+org.apache.log4j.Logger.getLogger('org.apache.hadoop.hbase').setLevel(log_level)
 
 # Require HBase now after setting log levels
 require 'hbase_constants'
@@ -155,8 +153,8 @@ def debug
     conf.back_trace_limit = 100
     log_level = org.apache.log4j.Level::DEBUG
   end
-  org.apache.log4j.Logger.getLogger("org.apache.zookeeper").setLevel(log_level)
-  org.apache.log4j.Logger.getLogger("org.apache.hadoop.hbase").setLevel(log_level)
+  org.apache.log4j.Logger.getLogger('org.apache.zookeeper').setLevel(log_level)
+  org.apache.log4j.Logger.getLogger('org.apache.hadoop.hbase').setLevel(log_level)
   debug?
 end
 
@@ -176,23 +174,23 @@ if interactive
   # Output a banner message that tells users where to go for help
   @shell.print_banner
 
-  require "irb"
+  require 'irb'
   require 'irb/hirb'
 
   module IRB
     def self.start(ap_path = nil)
-      $0 = File::basename(ap_path, ".rb") if ap_path
+      $0 = File.basename(ap_path, '.rb') if ap_path
 
       IRB.setup(ap_path)
       @CONF[:IRB_NAME] = 'hbase'
       @CONF[:AP_NAME] = 'hbase'
       @CONF[:BACK_TRACE_LIMIT] = 0 unless $fullBackTrace
 
-      if @CONF[:SCRIPT]
-        hirb = HIRB.new(nil, @CONF[:SCRIPT])
-      else
-        hirb = HIRB.new
-      end
+      hirb = if @CONF[:SCRIPT]
+               HIRB.new(nil, @CONF[:SCRIPT])
+             else
+               HIRB.new
+             end
 
       @CONF[:IRB_RC].call(hirb.context) if @CONF[:IRB_RC]
       @CONF[:MAIN_CONTEXT] = hirb.context
@@ -211,9 +209,9 @@ else
     #     in order to maintain compatibility with previous behavior where
     #     a user could pass in script2run and then still pipe commands on
     #     stdin.
-    require "irb/ruby-lex"
-    require "irb/workspace"
-    workspace = IRB::WorkSpace.new(binding())
+    require 'irb/ruby-lex'
+    require 'irb/workspace'
+    workspace = IRB::WorkSpace.new(binding)
     scanner = RubyLex.new
 
     # RubyLex claims to take an IO but really wants an InputMethod
@@ -226,7 +224,7 @@ else
 
     scanner.set_input(STDIN)
     scanner.each_top_level_statement do |statement, linenum|
-       puts(workspace.evaluate(nil, statement, 'stdin', linenum))
+      puts(workspace.evaluate(nil, statement, 'stdin', linenum))
     end
   # XXX We're catching Exception on purpose, because we want to include
   #     unwrapped java exceptions, syntax errors, eval failures, etc.
@@ -234,8 +232,8 @@ else
     message = exception.to_s
     # exception unwrapping in shell means we'll have to handle Java exceptions
     # as a special case in order to format them properly.
-    if exception.kind_of? java.lang.Exception
-      $stderr.puts "java exception"
+    if exception.is_a? java.lang.Exception
+      $stderr.puts 'java exception'
       message = exception.get_message
     end
     # Include the 'ERROR' string to try to make transition easier for scripts that

http://git-wip-us.apache.org/repos/asf/hbase/blob/ea8fa59a/bin/region_mover.rb
----------------------------------------------------------------------
diff --git a/bin/region_mover.rb b/bin/region_mover.rb
index 614056f..6756145 100644
--- a/bin/region_mover.rb
+++ b/bin/region_mover.rb
@@ -20,5 +20,5 @@
 # not move a new region until successful confirm of region loading in new
 # location. Presumes balancer is disabled when we run (not harmful if its
 # on but this script and balancer will end up fighting each other).
-$BIN=File.dirname(__FILE__)
+$BIN = File.dirname(__FILE__)
 exec "#{$BIN}/hbase org.apache.hadoop.hbase.util.RegionMover #{ARGV.join(' ')}"

http://git-wip-us.apache.org/repos/asf/hbase/blob/ea8fa59a/bin/region_status.rb
----------------------------------------------------------------------
diff --git a/bin/region_status.rb b/bin/region_status.rb
index aab7e12..abd19dd 100644
--- a/bin/region_status.rb
+++ b/bin/region_status.rb
@@ -22,11 +22,10 @@
 #
 #  ${HBASE_HOME}/bin/hbase org.jruby.Main region_status.rb [wait] [--table <table_name>]
 
-
 require 'optparse'
 
-usage = 'Usage : ./hbase org.jruby.Main region_status.rb [wait]' +
-  '[--table <table_name>]\n'
+usage = 'Usage : ./hbase org.jruby.Main region_status.rb [wait]' \
+        '[--table <table_name>]\n'
 OptionParser.new do |o|
   o.banner = usage
   o.on('-t', '--table TABLENAME', 'Only process TABLENAME') do |tablename|
@@ -37,12 +36,11 @@ OptionParser.new do |o|
 end
 
 SHOULD_WAIT = ARGV[0] == 'wait'
-if ARGV[0] and not SHOULD_WAIT
+if ARGV[0] && !SHOULD_WAIT
   print usage
   exit 1
 end
 
-
 require 'java'
 
 java_import org.apache.hadoop.hbase.HBaseConfiguration
@@ -61,17 +59,17 @@ java_import org.apache.hadoop.hbase.client.ConnectionFactory
 
 # disable debug logging on this script for clarity
 log_level = org.apache.log4j.Level::ERROR
-org.apache.log4j.Logger.getLogger("org.apache.zookeeper").setLevel(log_level)
-org.apache.log4j.Logger.getLogger("org.apache.hadoop.hbase").setLevel(log_level)
+org.apache.log4j.Logger.getLogger('org.apache.zookeeper').setLevel(log_level)
+org.apache.log4j.Logger.getLogger('org.apache.hadoop.hbase').setLevel(log_level)
 
 config = HBaseConfiguration.create
 config.set 'fs.defaultFS', config.get(HConstants::HBASE_DIR)
 connection = ConnectionFactory.createConnection(config)
 # wait until the master is running
 admin = nil
-while true
+loop do
   begin
-    admin = connection.getAdmin()
+    admin = connection.getAdmin
     break
   rescue MasterNotRunningException => e
     print 'Waiting for master to start...\n'
@@ -99,7 +97,7 @@ REGION_INFO = 'regioninfo'.to_java_bytes
 scan.addColumn INFO, REGION_INFO
 table = nil
 iter = nil
-while true
+loop do
   begin
     table = connection.getTable(TableName.valueOf('hbase:meta'))
     scanner = table.getScanner(scan)
@@ -112,14 +110,14 @@ while true
 end
 while iter.hasNext
   result = iter.next
-  rowid = Bytes.toString(result.getRow())
+  rowid = Bytes.toString(result.getRow)
   rowidStr = java.lang.String.new(rowid)
-  if not $tablename.nil? and not rowidStr.startsWith(tableNameMetaPrefix)
+  if !$tablename.nil? && !rowidStr.startsWith(tableNameMetaPrefix)
     # Gone too far, break
     break
   end
-  region = MetaTableAccessor::getHRegionInfo(result)
-  if not region.isOffline
+  region = MetaTableAccessor.getHRegionInfo(result)
+  unless region.isOffline
     # only include regions that should be online
     meta_count += 1
   end
@@ -127,30 +125,26 @@ end
 scanner.close
 # If we're trying to see the status of all HBase tables, we need to include the
 # hbase:meta table, that is not included in our scan
-if $tablename.nil?
-  meta_count += 1
-end
+meta_count += 1 if $tablename.nil?
 
 # query the master to see how many regions are on region servers
-if not $tablename.nil?
-  $TableName = TableName.valueOf($tablename.to_java_bytes)
-end
-while true
+$TableName = TableName.valueOf($tablename.to_java_bytes) unless $tablename.nil?
+loop do
   if $tablename.nil?
-    server_count = admin.getClusterStatus().getRegionsCount()
+    server_count = admin.getClusterStatus.getRegionsCount
   else
-    connection = ConnectionFactory::createConnection(config);
-    server_count = MetaTableAccessor::allTableRegions(connection, $TableName).size()
+    connection = ConnectionFactory.createConnection(config)
+    server_count = MetaTableAccessor.allTableRegions(connection, $TableName).size
   end
   print "Region Status: #{server_count} / #{meta_count}\n"
-  if SHOULD_WAIT and server_count < meta_count
-    #continue this loop until server & meta count match
+  if SHOULD_WAIT && server_count < meta_count
+    # continue this loop until server & meta count match
     sleep 10
   else
     break
   end
 end
-admin.close()
-connection.close()
+admin.close
+connection.close
 
 exit server_count == meta_count ? 0 : 1

http://git-wip-us.apache.org/repos/asf/hbase/blob/ea8fa59a/bin/replication/copy_tables_desc.rb
----------------------------------------------------------------------
diff --git a/bin/replication/copy_tables_desc.rb b/bin/replication/copy_tables_desc.rb
index a265bf1..c494765 100644
--- a/bin/replication/copy_tables_desc.rb
+++ b/bin/replication/copy_tables_desc.rb
@@ -34,20 +34,20 @@ java_import org.apache.hadoop.hbase.client.ConnectionFactory
 java_import org.apache.hadoop.hbase.client.HBaseAdmin
 
 # Name of this script
-NAME = "copy_tables_desc"
+NAME = 'copy_tables_desc'.freeze
 
 # Print usage for this script
 def usage
-  puts 'Usage: %s.rb master_zookeeper.quorum.peers:clientport:znode_parent slave_zookeeper.quorum.peers:clientport:znode_parent [table1,table2,table3,...]' % NAME
+  puts format('Usage: %s.rb master_zookeeper.quorum.peers:clientport:znode_parent slave_zookeeper.quorum.peers:clientport:znode_parent [table1,table2,table3,...]', NAME)
   exit!
 end
 
-def copy (src, dst, table)
+def copy(src, dst, table)
   # verify if table exists in source cluster
   begin
     t = src.getTableDescriptor(TableName.valueOf(table))
   rescue org.apache.hadoop.hbase.TableNotFoundException
-    puts "Source table \"%s\" doesn't exist, skipping." % table
+    puts format("Source table \"%s\" doesn't exist, skipping.", table)
     return
   end
 
@@ -55,45 +55,42 @@ def copy (src, dst, table)
   begin
     dst.createTable(t)
   rescue org.apache.hadoop.hbase.TableExistsException
-    puts "Destination table \"%s\" exists in remote cluster, skipping." % table
+    puts format('Destination table "%s" exists in remote cluster, skipping.', table)
     return
   end
 
-  puts "Schema for table \"%s\" was succesfully copied to remote cluster." % table
+  puts format('Schema for table "%s" was succesfully copied to remote cluster.', table)
 end
 
-
-if ARGV.size < 2 || ARGV.size > 3
-  usage
-end
+usage if ARGV.size < 2 || ARGV.size > 3
 
 LOG = LogFactory.getLog(NAME)
 
-parts1 = ARGV[0].split(":")
+parts1 = ARGV[0].split(':')
 
-parts2 = ARGV[1].split(":")
+parts2 = ARGV[1].split(':')
 
-parts3 = ARGV[2].split(",") unless ARGV[2].nil?
+parts3 = ARGV[2].split(',') unless ARGV[2].nil?
 
-c1 = HBaseConfiguration.create()
+c1 = HBaseConfiguration.create
 c1.set(HConstants::ZOOKEEPER_QUORUM, parts1[0])
-c1.set("hbase.zookeeper.property.clientPort", parts1[1])
+c1.set('hbase.zookeeper.property.clientPort', parts1[1])
 c1.set(HConstants::ZOOKEEPER_ZNODE_PARENT, parts1[2])
 
 connection1 = ConnectionFactory.createConnection(c1)
-admin1 = connection1.getAdmin()
+admin1 = connection1.getAdmin
 
-c2 = HBaseConfiguration.create()
+c2 = HBaseConfiguration.create
 c2.set(HConstants::ZOOKEEPER_QUORUM, parts2[0])
-c2.set("hbase.zookeeper.property.clientPort", parts2[1])
+c2.set('hbase.zookeeper.property.clientPort', parts2[1])
 c2.set(HConstants::ZOOKEEPER_ZNODE_PARENT, parts2[2])
 
 connection2 = ConnectionFactory.createConnection(c2)
-admin2 = connection2.getAdmin()
+admin2 = connection2.getAdmin
 
 if parts3.nil?
-  admin1.listTableNames().each do |t|
-    copy(admin1, admin2, t.nameAsString())
+  admin1.listTableNames.each do |t|
+    copy(admin1, admin2, t.nameAsString)
   end
 else
   parts3.each do |t|
@@ -101,7 +98,7 @@ else
   end
 end
 
-admin1.close()
-admin2.close()
-connection1.close()
-connection2.close()
+admin1.close
+admin2.close
+connection1.close
+connection2.close

http://git-wip-us.apache.org/repos/asf/hbase/blob/ea8fa59a/bin/shutdown_regionserver.rb
----------------------------------------------------------------------
diff --git a/bin/shutdown_regionserver.rb b/bin/shutdown_regionserver.rb
index 9d514a7..fd1af30 100644
--- a/bin/shutdown_regionserver.rb
+++ b/bin/shutdown_regionserver.rb
@@ -27,24 +27,24 @@ java_import org.apache.hadoop.hbase.HBaseConfiguration
 java_import org.apache.hadoop.hbase.client.HBaseAdmin
 java_import org.apache.hadoop.hbase.client.ConnectionFactory
 
-def usage(msg=nil)
+def usage(msg = nil)
   $stderr.puts 'Usage: shutdown_regionserver.rb <host:port>..'
   $stderr.puts
   $stderr.puts 'Stops the specified regionservers via RPC'
-  $stderr.puts 'Error: %s' % msg if msg
+  $stderr.puts format('Error: %s', msg) if msg
   abort
 end
 
-usage if ARGV.length < 1
+usage if ARGV.empty?
 
 ARGV.each do |x|
-  usage 'Invalid host:port: %s' % x unless x.include? ':'
+  usage format('Invalid host:port: %s', x) unless x.include? ':'
 end
 
-config = HBaseConfiguration.create()
+config = HBaseConfiguration.create
 connection = ConnectionFactory.createConnection(config)
 begin
-  admin = connection.getAdmin()
+  admin = connection.getAdmin
 rescue
   abort "Error: Couldn't instantiate HBaseAdmin"
 end
@@ -52,5 +52,5 @@ end
 ARGV.each do |hostport|
   admin.stopRegionServer(hostport)
 end
-admin.close()
-connection.close()
+admin.close
+connection.close


[06/50] [abbrv] hbase git commit: HBASE-18469 Correct RegionServer metric of totalRequestCount

Posted by st...@apache.org.
HBASE-18469 Correct RegionServer metric of totalRequestCount


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

Branch: refs/heads/HBASE-14070.HLC
Commit: 679f34e88108d870b79f3175b195e8e7d803c331
Parents: 8da77b4
Author: Yu Li <li...@apache.org>
Authored: Fri Aug 11 14:13:18 2017 +0800
Committer: Yu Li <li...@apache.org>
Committed: Fri Aug 11 14:13:18 2017 +0800

----------------------------------------------------------------------
 .../regionserver/MetricsRegionServerSource.java |  3 ++
 .../MetricsRegionServerWrapper.java             |  2 +
 .../MetricsRegionServerSourceImpl.java          |  2 +
 .../MetricsRegionServerWrapperImpl.java         |  5 +++
 .../hbase/regionserver/RSRpcServices.java       | 14 ++++++-
 .../MetricsRegionServerWrapperStub.java         |  5 +++
 .../regionserver/TestMetricsRegionServer.java   |  4 ++
 .../regionserver/TestRegionServerMetrics.java   | 40 +++++++++++++++++++-
 8 files changed, 71 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/679f34e8/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java
index df522d3..9656894 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java
@@ -244,6 +244,9 @@ public interface MetricsRegionServerSource extends BaseSource, JvmPauseMonitorSo
   String TOTAL_REQUEST_COUNT = "totalRequestCount";
   String TOTAL_REQUEST_COUNT_DESC =
       "Total number of requests this RegionServer has answered.";
+  String TOTAL_ROW_ACTION_REQUEST_COUNT = "totalRowActionRequestCount";
+  String TOTAL_ROW_ACTION_REQUEST_COUNT_DESC =
+      "Total number of region requests this RegionServer has answered, count by row-level action";
   String READ_REQUEST_COUNT = "readRequestCount";
   String READ_REQUEST_COUNT_DESC =
       "Number of read requests this region server has answered.";

http://git-wip-us.apache.org/repos/asf/hbase/blob/679f34e8/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java
index 0aa625c..ccb9de2 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java
@@ -457,4 +457,6 @@ public interface MetricsRegionServerWrapper {
   long getDeleteFamilyBloomHitCount();
 
   long getTrailerHitCount();
+
+  long getTotalRowActionRequestCount();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/679f34e8/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
index 94b21bc..e69e17c 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
@@ -354,6 +354,8 @@ public class MetricsRegionServerSourceImpl
           .addGauge(Interns.info(AVERAGE_REGION_SIZE, AVERAGE_REGION_SIZE_DESC), rsWrap.getAverageRegionSize())
           .addCounter(Interns.info(TOTAL_REQUEST_COUNT, TOTAL_REQUEST_COUNT_DESC),
               rsWrap.getTotalRequestCount())
+          .addCounter(Interns.info(TOTAL_ROW_ACTION_REQUEST_COUNT, TOTAL_ROW_ACTION_REQUEST_COUNT_DESC),
+              rsWrap.getTotalRowActionRequestCount())
           .addCounter(Interns.info(READ_REQUEST_COUNT, READ_REQUEST_COUNT_DESC),
               rsWrap.getReadRequestsCount())
           .addCounter(Interns.info(FILTERED_READ_REQUEST_COUNT, FILTERED_READ_REQUEST_COUNT_DESC),

http://git-wip-us.apache.org/repos/asf/hbase/blob/679f34e8/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
index 882044c..f43db93 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
@@ -219,6 +219,11 @@ class MetricsRegionServerWrapperImpl
   }
 
   @Override
+  public long getTotalRowActionRequestCount() {
+    return regionServer.rpcServices.requestRowActionCount.sum();
+  }
+
+  @Override
   public int getSplitQueueSize() {
     if (this.regionServer.compactSplitThread == null) {
       return 0;

http://git-wip-us.apache.org/repos/asf/hbase/blob/679f34e8/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index cc85ddb..696bd76 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -256,8 +256,13 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
   static final int BATCH_ROWS_THRESHOLD_DEFAULT = 5000;
 
   // Request counter. (Includes requests that are not serviced by regions.)
+  // Count only once for requests with multiple actions like multi/caching-scan/replayBatch
   final LongAdder requestCount = new LongAdder();
 
+  // Request counter. (Excludes requests that are not serviced by regions.)
+  // Count rows for requests with multiple actions like multi/caching-scan/replayBatch
+  final LongAdder requestRowActionCount = new LongAdder();
+
   // Request counter for rpc get
   final LongAdder rpcGetRequestCount = new LongAdder();
 
@@ -1091,7 +1096,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
           it.remove();
         }
       }
-      requestCount.add(mutations.size());
+      requestCount.increment();
+      requestRowActionCount.add(mutations.size());
       if (!region.getRegionInfo().isMetaTable()) {
         regionServer.cacheFlusher.reclaimMemStoreMemory();
       }
@@ -2393,6 +2399,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     try {
       checkOpen();
       requestCount.increment();
+      requestRowActionCount.increment();
       rpcGetRequestCount.increment();
       Region region = getRegion(request.getRegion());
 
@@ -2549,11 +2556,12 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     RegionScannersCloseCallBack closeCallBack = null;
     RpcCallContext context = RpcServer.getCurrentCall();
     this.rpcMultiRequestCount.increment();
+    this.requestCount.increment();
     Map<RegionSpecifier, ClientProtos.RegionLoadStats> regionStats = new HashMap<>(request
       .getRegionActionCount());
     ActivePolicyEnforcement spaceQuotaEnforcement = getSpaceQuotaManager().getActiveEnforcements();
     for (RegionAction regionAction : request.getRegionActionList()) {
-      this.requestCount.add(regionAction.getActionCount());
+      this.requestRowActionCount.add(regionAction.getActionCount());
       OperationQuota quota;
       Region region;
       regionActionResultBuilder.clear();
@@ -2687,6 +2695,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     try {
       checkOpen();
       requestCount.increment();
+      requestRowActionCount.increment();
       rpcMutateRequestCount.increment();
       Region region = getRegion(request.getRegion());
       MutateResponse.Builder builder = MutateResponse.newBuilder();
@@ -3133,6 +3142,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
         }
       }
       region.updateReadRequestsCount(numOfResults);
+      requestRowActionCount.add(numOfResults);
       long end = EnvironmentEdgeManager.currentTime();
       long responseCellSize = context != null ? context.getResponseCellSize() : 0;
       region.getMetrics().updateScanTime(end - before);

http://git-wip-us.apache.org/repos/asf/hbase/blob/679f34e8/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java
index 6e4828c..1c3f6f3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java
@@ -101,6 +101,11 @@ public class MetricsRegionServerWrapperStub implements MetricsRegionServerWrappe
   }
 
   @Override
+  public long getTotalRowActionRequestCount() {
+    return getReadRequestsCount() + getWriteRequestsCount();
+  }
+
+  @Override
   public long getReadRequestsCount() {
     return 997;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/679f34e8/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServer.java
index 8ed9f08..7ce7701 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServer.java
@@ -72,6 +72,10 @@ public class TestMetricsRegionServer {
     HELPER.assertGauge("memstoreSize", 1025, serverSource);
     HELPER.assertGauge("storeFileSize", 1900, serverSource);
     HELPER.assertCounter("totalRequestCount", 899, serverSource);
+    HELPER.assertCounter("totalRowActionRequestCount",
+      HELPER.getCounter("readRequestCount", serverSource)
+          + HELPER.getCounter("writeRequestCount", serverSource),
+      serverSource);
     HELPER.assertCounter("readRequestCount", 997, serverSource);
     HELPER.assertCounter("filteredReadRequestCount", 1997, serverSource);
     HELPER.assertCounter("writeRequestCount", 707, serverSource);

http://git-wip-us.apache.org/repos/asf/hbase/blob/679f34e8/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
index 4be76e6..2812e2e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
@@ -224,6 +224,21 @@ public class TestRegionServerMetrics {
     }
   }
 
+  public void doScan(int n, boolean caching) throws IOException {
+    Scan scan = new Scan();
+    if (caching) {
+      scan.setCaching(n);
+    } else {
+      scan.setCaching(1);
+    }
+    ResultScanner scanner = table.getScanner(scan);
+    for (int i = 0; i < n; i++) {
+      Result res = scanner.next();
+      LOG.debug(
+        "Result row: " + Bytes.toString(res.getRow()) + ", value: " + res.getValue(cf, qualifier));
+    }
+  }
+
   @Test
   public void testRegionCount() throws Exception {
     metricsHelper.assertGauge("regionCount", 1, serverSource);
@@ -242,6 +257,7 @@ public class TestRegionServerMetrics {
 
     metricsRegionServer.getRegionServerWrapper().forceRecompute();
     long requests = metricsHelper.getCounter("totalRequestCount", serverSource);
+    long rowActionRequests = metricsHelper.getCounter("totalRowActionRequestCount", serverSource);
     long readRequests = metricsHelper.getCounter("readRequestCount", serverSource);
     long writeRequests = metricsHelper.getCounter("writeRequestCount", serverSource);
 
@@ -249,6 +265,7 @@ public class TestRegionServerMetrics {
 
     metricsRegionServer.getRegionServerWrapper().forceRecompute();
     assertCounter("totalRequestCount", requests + 30);
+    assertCounter("totalRowActionRequestCount", rowActionRequests + 30);
     assertCounter("readRequestCount", readRequests);
     assertCounter("writeRequestCount", writeRequests + 30);
 
@@ -256,6 +273,7 @@ public class TestRegionServerMetrics {
 
     metricsRegionServer.getRegionServerWrapper().forceRecompute();
     assertCounter("totalRequestCount", requests + 40);
+    assertCounter("totalRowActionRequestCount", rowActionRequests + 40);
     assertCounter("readRequestCount", readRequests + 10);
     assertCounter("writeRequestCount", writeRequests + 30);
 
@@ -265,16 +283,34 @@ public class TestRegionServerMetrics {
     doNGets(10, true);  // true = batch
 
     metricsRegionServer.getRegionServerWrapper().forceRecompute();
-    assertCounter("totalRequestCount", requests + 50);
+    assertCounter("totalRequestCount", requests + 41);
+    assertCounter("totalRowActionRequestCount", rowActionRequests + 50);
     assertCounter("readRequestCount", readRequests + 20);
     assertCounter("writeRequestCount", writeRequests + 30);
 
     doNPuts(30, true);
 
     metricsRegionServer.getRegionServerWrapper().forceRecompute();
-    assertCounter("totalRequestCount", requests + 80);
+    assertCounter("totalRequestCount", requests + 42);
+    assertCounter("totalRowActionRequestCount", rowActionRequests + 80);
     assertCounter("readRequestCount", readRequests + 20);
     assertCounter("writeRequestCount", writeRequests + 60);
+
+    doScan(10, false); // test after batch put so we have enough lines
+    metricsRegionServer.getRegionServerWrapper().forceRecompute();
+    assertCounter("totalRequestCount", requests + 52);
+    assertCounter("totalRowActionRequestCount", rowActionRequests + 90);
+    assertCounter("readRequestCount", readRequests + 30);
+    assertCounter("writeRequestCount", writeRequests + 60);
+    numScanNext += 10;
+
+    doScan(10, true); // true = caching
+    metricsRegionServer.getRegionServerWrapper().forceRecompute();
+    assertCounter("totalRequestCount", requests + 53);
+    assertCounter("totalRowActionRequestCount", rowActionRequests + 100);
+    assertCounter("readRequestCount", readRequests + 40);
+    assertCounter("writeRequestCount", writeRequests + 60);
+    numScanNext += 1;
   }
 
   @Test