[23/41] hbase git commit: HBASE-18640 Move mapreduce out of hbase-server into separate module.

2017-08-25 Thread appy
http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatScanBase.java
--
diff --git 
a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatScanBase.java
 
b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatScanBase.java
new file mode 100644
index 000..13b6a96
--- /dev/null
+++ 
b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatScanBase.java
@@ -0,0 +1,287 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.mapreduce;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.NavigableMap;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+
+
+/**
+ * 
+ * Tests various scan start and stop row scenarios. This is set in a scan and
+ * tested in a MapReduce job to see if that is handed over and done properly
+ * too.
+ * 
+ * 
+ * This test is broken into two parts in order to side-step the test timeout
+ * period of 900, as documented in HBASE-8326.
+ * 
+ */
+public abstract class TestTableInputFormatScanBase {
+
+  private static final Log LOG = 
LogFactory.getLog(TestTableInputFormatScanBase.class);
+  static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+  static final TableName TABLE_NAME = TableName.valueOf("scantest");
+  static final byte[][] INPUT_FAMILYS = {Bytes.toBytes("content1"), 
Bytes.toBytes("content2")};
+  static final String KEY_STARTROW = "startRow";
+  static final String KEY_LASTROW = "stpRow";
+
+  private static Table table = null;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+// test intermittently fails under hadoop2 (2.0.2-alpha) if 
shortcircuit-read (scr) is on.
+// this turns it off for this test.  TODO: Figure out why scr breaks 
recovery.
+System.setProperty("hbase.tests.use.shortcircuit.reads", "false");
+
+// switch TIF to log at DEBUG level
+TEST_UTIL.enableDebug(TableInputFormat.class);
+TEST_UTIL.enableDebug(TableInputFormatBase.class);
+// start mini hbase cluster
+TEST_UTIL.startMiniCluster(3);
+// create and fill table
+table = TEST_UTIL.createMultiRegionTable(TABLE_NAME, INPUT_FAMILYS);
+TEST_UTIL.loadTable(table, INPUT_FAMILYS, null, false);
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+TEST_UTIL.shutdownMiniCluster();
+  }
+
+  /**
+   * Pass the key and value to reduce.
+   */
+  public static class ScanMapper
+  extends TableMapper {
+
+/**
+ * Pass the key and value to reduce.
+ *
+ * @param key  The key, here "aaa", "aab" etc.
+ * @param value  The value is the same as the key.
+ * @param context  The task context.
+ * @throws IOException When reading the rows fails.
+ */
+@Override
+public void map(ImmutableBytesWritable key, Result value,
+  Context context)
+throws IOException, InterruptedException {
+  if (value.size() != 2) {
+throw new IOException("There should be two input columns");
+  }
+  Map

[02/41] hbase git commit: HBASE-18640 Move mapreduce out of hbase-server into separate module.

2017-08-25 Thread appy
http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TsvImporterCustomTestMapperForOprAttr.java
--
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TsvImporterCustomTestMapperForOprAttr.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TsvImporterCustomTestMapperForOprAttr.java
deleted file mode 100644
index 9d8b8f0..000
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TsvImporterCustomTestMapperForOprAttr.java
+++ /dev/null
@@ -1,58 +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.mapreduce;
-
-import java.io.IOException;
-
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.client.Put;
-import 
org.apache.hadoop.hbase.mapreduce.ImportTsv.TsvParser.BadTsvLineException;
-import org.apache.hadoop.hbase.mapreduce.ImportTsv.TsvParser.ParsedLine;
-import org.apache.hadoop.hbase.util.Bytes;
-
-/**
- * 
- * Just shows a simple example of how the attributes can be extracted and added
- * to the puts
- */
-public class TsvImporterCustomTestMapperForOprAttr extends TsvImporterMapper {
-  @Override
-  protected void populatePut(byte[] lineBytes, ParsedLine parsed, Put put, int 
i)
-  throws BadTsvLineException, IOException {
-KeyValue kv;
-kv = new KeyValue(lineBytes, parsed.getRowKeyOffset(), 
parsed.getRowKeyLength(),
-parser.getFamily(i), 0, parser.getFamily(i).length, 
parser.getQualifier(i), 0,
-parser.getQualifier(i).length, ts, KeyValue.Type.Put, lineBytes, 
parsed.getColumnOffset(i),
-parsed.getColumnLength(i));
-if (parsed.getIndividualAttributes() != null) {
-  String[] attributes = parsed.getIndividualAttributes();
-  for (String attr : attributes) {
-String[] split = attr.split(ImportTsv.DEFAULT_ATTRIBUTES_SEPERATOR);
-if (split == null || split.length <= 1) {
-  throw new BadTsvLineException("Invalid attributes seperator 
specified" + attributes);
-} else {
-  if (split[0].length() <= 0 || split[1].length() <= 0) {
-throw new BadTsvLineException("Invalid attributes seperator 
specified" + attributes);
-  }
-  put.setAttribute(split[0], Bytes.toBytes(split[1]));
-}
-  }
-}
-put.add(kv);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
--
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
index f641887..a81d268 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
@@ -65,7 +65,6 @@ 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.mapreduce.TableInputFormatBase;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.TableNamespaceManager;
@@ -336,7 +335,7 @@ public class TestNamespaceAuditor {
 byte[] columnFamily = Bytes.toBytes("info");
 HTableDescriptor tableDescOne = new HTableDescriptor(tableTwo);
 tableDescOne.addFamily(new HColumnDescriptor(columnFamily));
-ADMIN.createTable(tableDescOne, Bytes.toBytes("1"), Bytes.toBytes("2000"), 
initialRegions);
+ADMIN.createTable(tableDescOne, Bytes.toBytes("0"), Bytes.toBytes("9"), 
initialRegions);
 Connection connection = 
ConnectionFactory.createConnection(UTIL.getConfiguration());
 try (Table table = connection.getTable(tableTwo)) {
   UTIL.loadNumericRows(table, 

[14/41] hbase git commit: HBASE-18640 Move mapreduce out of hbase-server into separate module.

2017-08-25 Thread appy
http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/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
deleted file mode 100644
index ff458ff..000
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
+++ /dev/null
@@ -1,1027 +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.mapreduce;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.URL;
-import java.net.URLDecoder;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Enumeration;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.zip.ZipEntry;
-import java.util.zip.ZipFile;
-
-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.HBaseConfiguration;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.MetaTableAccessor;
-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.Put;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
-import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.security.UserProvider;
-import org.apache.hadoop.hbase.security.token.TokenUtil;
-import org.apache.hadoop.hbase.util.Base64;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.zookeeper.ZKConfig;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.util.StringUtils;
-
-import com.codahale.metrics.MetricRegistry;
-
-/**
- * Utility for {@link TableMapper} and {@link TableReducer}
- */
-@SuppressWarnings({ "rawtypes", "unchecked" })
-@InterfaceAudience.Public
-public class TableMapReduceUtil {
-  private static final Log LOG = LogFactory.getLog(TableMapReduceUtil.class);
-
-  /**
-   * Use this before submitting a TableMap job. It will appropriately set up
-   * the job.
-   *
-   * @param table  The table name to read from.
-   * @param scan  The scan instance with the columns, time range etc.
-   * @param mapper  The mapper class to use.
-   * @param outputKeyClass  The class of the output key.
-   * @param outputValueClass  The class of the output value.
-   * @param job  The current job to adjust.  Make sure the passed job is
-   * carrying all necessary HBase configuration.
-   * @throws IOException When setting up the details fails.
-   */
-  public static void initTableMapperJob(String table, Scan scan,
-  Class mapper,
-  Class outputKeyClass,
-  Class outputValueClass, Job job)
-  throws IOException {
-initTableMapperJob(table, scan, mapper, outputKeyClass, outputValueClass,
-job, true);
-  }
-
-
-  /**
-   * Use this before submitting a TableMap job. It will appropriately set up
-   * the job.
-   *
-   * @param table  The table name to read from.
-   * @param scan  The scan instance with the columns, time range etc.
-   * @param mapper  The mapper class to use.
-   * @param outputKeyClass  The class of the output key.
-   * @param outputValueClass  The class of the output value.
-   * @param job  The current job to adjust.  Make sure the passed job is
-   * carrying all necessary HBase configuration.
-   * @throws IOException When setting up the details fails.
-   */
-  public static void initTableMapperJob(TableName table,
-  Scan 

[36/41] hbase git commit: HBASE-18640 Move mapreduce out of hbase-server into separate module.

2017-08-25 Thread appy
http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableSnapshotInputFormatImpl.java
--
diff --git 
a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableSnapshotInputFormatImpl.java
 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableSnapshotInputFormatImpl.java
new file mode 100644
index 000..4331c0f
--- /dev/null
+++ 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableSnapshotInputFormatImpl.java
@@ -0,0 +1,252 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.mapreduce;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
+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.HRegionInfo;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
+import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
+import org.apache.hadoop.hbase.util.ConfigurationUtil;
+import org.apache.hadoop.hbase.util.FSUtils;
+
+import java.io.IOException;
+import java.util.AbstractMap;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+/**
+ * Shared implementation of mapreduce code over multiple table snapshots.
+ * Utilized by both mapreduce ({@link org.apache.hadoop.hbase.mapreduce
+ * .MultiTableSnapshotInputFormat} and mapred
+ * ({@link org.apache.hadoop.hbase.mapred.MultiTableSnapshotInputFormat} 
implementations.
+ */
+@InterfaceAudience.LimitedPrivate({ "HBase" })
+@InterfaceStability.Evolving
+public class MultiTableSnapshotInputFormatImpl {
+
+  private static final Log LOG = 
LogFactory.getLog(MultiTableSnapshotInputFormatImpl.class);
+
+  public static final String RESTORE_DIRS_KEY =
+  "hbase.MultiTableSnapshotInputFormat.restore.snapshotDirMapping";
+  public static final String SNAPSHOT_TO_SCANS_KEY =
+  "hbase.MultiTableSnapshotInputFormat.snapshotsToScans";
+
+  /**
+   * Configure conf to read from snapshotScans, with snapshots restored to a 
subdirectory of
+   * restoreDir.
+   * Sets: {@link #RESTORE_DIRS_KEY}, {@link #SNAPSHOT_TO_SCANS_KEY}
+   *
+   * @param conf
+   * @param snapshotScans
+   * @param restoreDir
+   * @throws IOException
+   */
+  public void setInput(Configuration conf, Map 
snapshotScans,
+  Path restoreDir) throws IOException {
+Path rootDir = FSUtils.getRootDir(conf);
+FileSystem fs = rootDir.getFileSystem(conf);
+
+setSnapshotToScans(conf, snapshotScans);
+Map restoreDirs =
+generateSnapshotToRestoreDirMapping(snapshotScans.keySet(), 
restoreDir);
+setSnapshotDirs(conf, restoreDirs);
+restoreSnapshots(conf, restoreDirs, fs);
+  }
+
+  /**
+   * Return the list of splits extracted from the scans/snapshots pushed to 
conf by
+   * {@link
+   * #setInput(org.apache.hadoop.conf.Configuration, java.util.Map, 
org.apache.hadoop.fs.Path)}
+   *
+   * @param conf Configuration to determine splits from
+   * @return Return the list of splits extracted from the scans/snapshots 
pushed to conf
+   * @throws IOException
+   */
+  public List getSplits(Configuration 
conf)
+  throws IOException {
+Path rootDir = FSUtils.getRootDir(conf);
+FileSystem fs = rootDir.getFileSystem(conf);
+
+List rtn = Lists.newArrayList();
+
+Map snapshotsToScans = getSnapshotsToScans(conf);
+Map snapshotsToRestoreDirs = getSnapshotDirs(conf);
+for (Map.Entry entry : 
snapshotsToScans.entrySet()) {
+  String snapshotName = entry.getKey();
+
+  Path restoreDir = snapshotsToRestoreDirs.get(snapshotName);
+
+  SnapshotManifest 

[17/41] hbase git commit: HBASE-18640 Move mapreduce out of hbase-server into separate module.

2017-08-25 Thread appy
http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
deleted file mode 100644
index b5bb2ec..000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
+++ /dev/null
@@ -1,780 +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.mapreduce;
-
-import java.io.ByteArrayInputStream;
-import java.io.DataInput;
-import java.io.DataInputStream;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.TreeMap;
-import java.util.UUID;
-
-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.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellComparator;
-import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValueUtil;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.ZooKeeperConnectionException;
-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.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Durability;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.RegionLocator;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.filter.Filter;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
-import org.apache.hadoop.io.RawComparator;
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.io.WritableComparator;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Partitioner;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.hadoop.mapreduce.TaskCounter;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
-import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.mapreduce.lib.partition.TotalOrderPartitioner;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-import org.apache.zookeeper.KeeperException;
-
-
-/**
- * Import data written by {@link Export}.
- */
-@InterfaceAudience.Public
-public class Import extends Configured implements Tool {
-  private static final Log LOG = LogFactory.getLog(Import.class);
-  final static String NAME = "import";
-  public final static String CF_RENAME_PROP = "HBASE_IMPORTER_RENAME_CFS";
-  public final static String BULK_OUTPUT_CONF_KEY = "import.bulk.output";
-  public final static String FILTER_CLASS_CONF_KEY = "import.filter.class";
-  public final static String FILTER_ARGS_CONF_KEY = "import.filter.args";
-  public final static String TABLE_NAME = "import.table.name";
-  public final static String WAL_DURABILITY = "import.wal.durability";
-  public final static String HAS_LARGE_RESULT= "import.bulk.hasLargeResult";
-
-  private final static String JOB_NAME_CONF_KEY = "mapreduce.job.name";
-
-  public static class KeyValueWritableComparablePartitioner 
-  extends Partitioner {
-private static 

[32/41] hbase git commit: HBASE-18640 Move mapreduce out of hbase-server into separate module.

2017-08-25 Thread appy
http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
--
diff --git 
a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
new file mode 100644
index 000..acf6ff8
--- /dev/null
+++ 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
@@ -0,0 +1,700 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.mapreduce.replication;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+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.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableSnapshotScanner;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.PrefixFilter;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.mapreduce.TableInputFormat;
+import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
+import org.apache.hadoop.hbase.mapreduce.TableSnapshotInputFormat;
+import org.apache.hadoop.hbase.mapreduce.TableMapper;
+import org.apache.hadoop.hbase.mapreduce.TableSplit;
+import org.apache.hadoop.hbase.replication.ReplicationException;
+import org.apache.hadoop.hbase.replication.ReplicationFactory;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.ReplicationPeerZKImpl;
+import org.apache.hadoop.hbase.replication.ReplicationPeers;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.Threads;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+import 
org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * This map-only job compares the data from a local table with a remote one.
+ * Every cell is compared and must have exactly the same keys (even timestamp)
+ * as well as same value. It is possible to restrict the job by time range and
+ * families. The peer id that's provided must match the one given when the
+ * replication stream was setup.
+ * 
+ * Two counters are provided, Verifier.Counters.GOODROWS and BADROWS. The 
reason
+ * for a why a row is different is shown in the map's log.
+ */
+public class VerifyReplication extends Configured implements Tool {
+
+  private static final Log LOG =
+  LogFactory.getLog(VerifyReplication.class);
+
+  public final static String NAME = "verifyrep";
+  private final static String PEER_CONFIG_PREFIX = NAME + ".peer.";
+  long startTime = 0;
+  long endTime = Long.MAX_VALUE;
+  int batch = -1;
+  int versions = -1;
+  String tableName = null;
+  String families = null;
+  String delimiter = "";
+  String peerId = null;
+  String rowPrefixes = null;
+  int sleepMsBeforeReCompare = 0;
+  boolean verbose = false;
+  boolean includeDeletedCells = false;
+  //Source table 

[01/41] hbase git commit: HBASE-18640 Move mapreduce out of hbase-server into separate module.

2017-08-25 Thread appy
Repository: hbase
Updated Branches:
  refs/heads/master 8d33949b8 -> 664b6be0e


http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java
--
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java
deleted file mode 100644
index ad832e3..000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java
+++ /dev/null
@@ -1,968 +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.util;
-
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.lang.reflect.Constructor;
-import java.net.InetAddress;
-import java.security.SecureRandom;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Locale;
-import java.util.Properties;
-import java.util.Random;
-import java.util.concurrent.atomic.AtomicReference;
-
-import javax.crypto.spec.SecretKeySpec;
-
-import org.apache.commons.cli.CommandLine;
-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.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.TableName;
-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.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.Durability;
-import org.apache.hadoop.hbase.client.TableDescriptor;
-import org.apache.hadoop.hbase.io.compress.Compression;
-import org.apache.hadoop.hbase.io.crypto.Cipher;
-import org.apache.hadoop.hbase.io.crypto.Encryption;
-import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
-import org.apache.hadoop.hbase.regionserver.BloomType;
-import org.apache.hadoop.hbase.security.EncryptionUtil;
-import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.security.access.AccessControlClient;
-import org.apache.hadoop.hbase.security.access.Permission;
-import org.apache.hadoop.hbase.util.test.LoadTestDataGenerator;
-import org.apache.hadoop.hbase.util.test.LoadTestDataGeneratorWithACL;
-import org.apache.hadoop.security.SecurityUtil;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.util.ToolRunner;
-
-/**
- * A command-line utility that reads, writes, and verifies data. Unlike
- * {@link org.apache.hadoop.hbase.PerformanceEvaluation}, this tool validates 
the data written,
- * and supports simultaneously writing and reading the same set of keys.
- */
-@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
-public class LoadTestTool extends AbstractHBaseTool {
-
-  private static final Log LOG = LogFactory.getLog(LoadTestTool.class);
-  private static final String COLON = ":";
-
-  /** Table name for the test */
-  private TableName tableName;
-
-  /** Column families for the test */
-  private byte[][] families;
-
-  /** Table name to use of not overridden on the command line */
-  protected static final String DEFAULT_TABLE_NAME = "cluster_test";
-
-  /** Column family used by the test */
-  public static byte[] DEFAULT_COLUMN_FAMILY = Bytes.toBytes("test_cf");
-
-  /** Column families used by the test */
-  public static final byte[][] DEFAULT_COLUMN_FAMILIES = { 
DEFAULT_COLUMN_FAMILY };
-
-  /** The default data size if not specified */
-  protected static final int DEFAULT_DATA_SIZE = 64;
-
-  /** The number of reader/writer threads if not specified */
-  protected static final int DEFAULT_NUM_THREADS = 20;
-
-  /** Usage string for the load option */
-  protected static final String OPT_USAGE_LOAD =
-  ":" +
-  "[:<#threads=" + DEFAULT_NUM_THREADS + ">]";
-
-  /** Usage string for the read option */
-  protected static final String OPT_USAGE_READ 

[24/41] hbase git commit: HBASE-18640 Move mapreduce out of hbase-server into separate module.

2017-08-25 Thread appy
http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultithreadedTableMapper.java
--
diff --git 
a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultithreadedTableMapper.java
 
b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultithreadedTableMapper.java
new file mode 100644
index 000..694a359
--- /dev/null
+++ 
b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultithreadedTableMapper.java
@@ -0,0 +1,264 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.mapreduce;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableMap;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.MapReduceTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test Map/Reduce job over HBase tables. The map/reduce process we're testing
+ * on our tables is simple - take every row in the table, reverse the value of
+ * a particular cell, and write it back to the table.
+ */
+@Category({MapReduceTests.class, LargeTests.class})
+public class TestMultithreadedTableMapper {
+  private static final Log LOG = 
LogFactory.getLog(TestMultithreadedTableMapper.class);
+  private static final HBaseTestingUtility UTIL =
+  new HBaseTestingUtility();
+  static final TableName MULTI_REGION_TABLE_NAME = TableName.valueOf("mrtest");
+  static final byte[] INPUT_FAMILY = Bytes.toBytes("contents");
+  static final byte[] OUTPUT_FAMILY = Bytes.toBytes("text");
+  static final intNUMBER_OF_THREADS = 10;
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+// Up the handlers; this test needs more than usual.
+
UTIL.getConfiguration().setInt(HConstants.REGION_SERVER_HIGH_PRIORITY_HANDLER_COUNT,
 10);
+UTIL.startMiniCluster();
+Table table =
+UTIL.createMultiRegionTable(MULTI_REGION_TABLE_NAME, new byte[][] { 
INPUT_FAMILY,
+OUTPUT_FAMILY });
+UTIL.loadTable(table, INPUT_FAMILY, false);
+UTIL.waitUntilAllRegionsAssigned(MULTI_REGION_TABLE_NAME);
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+UTIL.shutdownMiniCluster();
+  }
+
+  /**
+   * Pass the given key and processed record reduce
+   */
+  public static class ProcessContentsMapper
+  extends TableMapper {
+
+/**
+ * Pass the key, and reversed value to reduce
+ *
+ * @param key
+ * @param value
+ * @param context
+ * @throws IOException
+ */
+@Override
+public void map(ImmutableBytesWritable key, Result value,
+Context context)
+throws IOException, InterruptedException {
+  if (value.size() != 1) {
+throw new IOException("There should only be one input column");
+  }
+  Map>>
+  cf = value.getMap();
+  if(!cf.containsKey(INPUT_FAMILY)) {
+throw new IOException("Wrong input columns. 

[09/41] hbase git commit: HBASE-18640 Move mapreduce out of hbase-server into separate module.

2017-08-25 Thread appy
http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/hbase-server/src/test/java/org/apache/hadoop/hbase/ScanPerformanceEvaluation.java
--
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/ScanPerformanceEvaluation.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/ScanPerformanceEvaluation.java
deleted file mode 100644
index e669f14..000
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/ScanPerformanceEvaluation.java
+++ /dev/null
@@ -1,406 +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;
-
-import java.io.IOException;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.commons.cli.CommandLine;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-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.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.client.TableSnapshotScanner;
-import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
-import org.apache.hadoop.hbase.mapreduce.TableMapper;
-import org.apache.hadoop.hbase.util.AbstractHBaseTool;
-import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.Counters;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.util.ToolRunner;
-
-import org.apache.hadoop.hbase.shaded.com.google.common.base.Stopwatch;
-
-/**
- * A simple performance evaluation tool for single client and MR scans
- * and snapshot scans.
- */
-@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
-public class ScanPerformanceEvaluation extends AbstractHBaseTool {
-
-  private static final String HBASE_COUNTER_GROUP_NAME = "HBase Counters";
-
-  private String type;
-  private String file;
-  private String tablename;
-  private String snapshotName;
-  private String restoreDir;
-  private String caching;
-
-  @Override
-  public void setConf(Configuration conf) {
-super.setConf(conf);
-Path rootDir;
-try {
-  rootDir = FSUtils.getRootDir(conf);
-  rootDir.getFileSystem(conf);
-} catch (IOException ex) {
-  throw new RuntimeException(ex);
-}
-  }
-
-  @Override
-  protected void addOptions() {
-this.addRequiredOptWithArg("t", "type", "the type of the test. One of the 
following: streaming|scan|snapshotscan|scanmapreduce|snapshotscanmapreduce");
-this.addOptWithArg("f", "file", "the filename to read from");
-this.addOptWithArg("tn", "table", "the tablename to read from");
-this.addOptWithArg("sn", "snapshot", "the snapshot name to read from");
-this.addOptWithArg("rs", "restoredir", "the directory to restore the 
snapshot");
-this.addOptWithArg("ch", "caching", "scanner caching value");
-  }
-
-  @Override
-  protected void processOptions(CommandLine cmd) {
-type = cmd.getOptionValue("type");
-file = cmd.getOptionValue("file");
-tablename = cmd.getOptionValue("table");
-snapshotName = cmd.getOptionValue("snapshot");
-restoreDir = cmd.getOptionValue("restoredir");
-caching = cmd.getOptionValue("caching");
-  }
-
-  protected void testHdfsStreaming(Path filename) throws IOException {
-byte[] buf = new byte[1024];
-FileSystem fs = filename.getFileSystem(getConf());
-
-// read the file from start to finish
-Stopwatch fileOpenTimer = Stopwatch.createUnstarted();
-Stopwatch streamTimer = Stopwatch.createUnstarted();
-
-fileOpenTimer.start();
-FSDataInputStream in = fs.open(filename);
-fileOpenTimer.stop();
-
-long totalBytes = 0;
-

[04/41] hbase git commit: HBASE-18640 Move mapreduce out of hbase-server into separate module.

2017-08-25 Thread appy
http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultithreadedTableMapper.java
--
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultithreadedTableMapper.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultithreadedTableMapper.java
deleted file mode 100644
index 694a359..000
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultithreadedTableMapper.java
+++ /dev/null
@@ -1,264 +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.mapreduce;
-
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.NavigableMap;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.apache.hadoop.hbase.testclassification.MapReduceTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-/**
- * Test Map/Reduce job over HBase tables. The map/reduce process we're testing
- * on our tables is simple - take every row in the table, reverse the value of
- * a particular cell, and write it back to the table.
- */
-@Category({MapReduceTests.class, LargeTests.class})
-public class TestMultithreadedTableMapper {
-  private static final Log LOG = 
LogFactory.getLog(TestMultithreadedTableMapper.class);
-  private static final HBaseTestingUtility UTIL =
-  new HBaseTestingUtility();
-  static final TableName MULTI_REGION_TABLE_NAME = TableName.valueOf("mrtest");
-  static final byte[] INPUT_FAMILY = Bytes.toBytes("contents");
-  static final byte[] OUTPUT_FAMILY = Bytes.toBytes("text");
-  static final intNUMBER_OF_THREADS = 10;
-
-  @BeforeClass
-  public static void beforeClass() throws Exception {
-// Up the handlers; this test needs more than usual.
-
UTIL.getConfiguration().setInt(HConstants.REGION_SERVER_HIGH_PRIORITY_HANDLER_COUNT,
 10);
-UTIL.startMiniCluster();
-Table table =
-UTIL.createMultiRegionTable(MULTI_REGION_TABLE_NAME, new byte[][] { 
INPUT_FAMILY,
-OUTPUT_FAMILY });
-UTIL.loadTable(table, INPUT_FAMILY, false);
-UTIL.waitUntilAllRegionsAssigned(MULTI_REGION_TABLE_NAME);
-  }
-
-  @AfterClass
-  public static void afterClass() throws Exception {
-UTIL.shutdownMiniCluster();
-  }
-
-  /**
-   * Pass the given key and processed record reduce
-   */
-  public static class ProcessContentsMapper
-  extends TableMapper {
-
-/**
- * Pass the key, and reversed value to reduce
- *
- * @param key
- * @param value
- * @param context
- * @throws IOException
- */
-@Override
-public void map(ImmutableBytesWritable key, Result value,
-Context context)
-throws IOException, InterruptedException {
-  if (value.size() != 1) {
-throw new IOException("There should only be one input column");
-  }
-  Map>>
-  cf = value.getMap();
-  if(!cf.containsKey(INPUT_FAMILY)) {
-throw new IOException("Wrong input columns. Missing: '" +

[15/41] hbase git commit: HBASE-18640 Move mapreduce out of hbase-server into separate module.

2017-08-25 Thread appy
http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/SyncTable.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/SyncTable.java 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/SyncTable.java
deleted file mode 100644
index c72a0c3..000
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/SyncTable.java
+++ /dev/null
@@ -1,786 +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.mapreduce;
-
-import java.io.IOException;
-import java.util.Iterator;
-import java.util.Collections;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellComparator;
-import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.mapreduce.Counters;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
-import org.apache.hadoop.util.GenericOptionsParser;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-
-import org.apache.hadoop.hbase.shaded.com.google.common.base.Throwables;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Iterators;
-
-public class SyncTable extends Configured implements Tool {
-
-  private static final Log LOG = LogFactory.getLog(SyncTable.class);
-
-  static final String SOURCE_HASH_DIR_CONF_KEY = "sync.table.source.hash.dir";
-  static final String SOURCE_TABLE_CONF_KEY = "sync.table.source.table.name";
-  static final String TARGET_TABLE_CONF_KEY = "sync.table.target.table.name";
-  static final String SOURCE_ZK_CLUSTER_CONF_KEY = 
"sync.table.source.zk.cluster";
-  static final String TARGET_ZK_CLUSTER_CONF_KEY = 
"sync.table.target.zk.cluster";
-  static final String DRY_RUN_CONF_KEY="sync.table.dry.run";
-
-  Path sourceHashDir;
-  String sourceTableName;
-  String targetTableName;
-
-  String sourceZkCluster;
-  String targetZkCluster;
-  boolean dryRun;
-
-  Counters counters;
-
-  public SyncTable(Configuration conf) {
-super(conf);
-  }
-
-  public Job createSubmittableJob(String[] args) throws IOException {
-FileSystem fs = sourceHashDir.getFileSystem(getConf());
-if (!fs.exists(sourceHashDir)) {
-  throw new IOException("Source hash dir not found: " + sourceHashDir);
-}
-
-HashTable.TableHash tableHash = HashTable.TableHash.read(getConf(), 
sourceHashDir);
-LOG.info("Read source hash manifest: " + tableHash);
-LOG.info("Read " + tableHash.partitions.size() + " partition keys");
-if (!tableHash.tableName.equals(sourceTableName)) {
-  LOG.warn("Table name mismatch - manifest indicates hash was taken from: "
-  + tableHash.tableName + " but job is reading from: " + 
sourceTableName);
-}
-if (tableHash.numHashFiles != tableHash.partitions.size() + 1) {
-  throw new RuntimeException("Hash data appears corrupt. The number of of 
hash files created"
-  + " should be 1 more than the number of partition keys.  However, 
the manifest file "
-  + " says numHashFiles=" + tableHash.numHashFiles + " but the number 
of partition keys"
-  + " found in the partitions file is " + 

[22/41] hbase git commit: HBASE-18640 Move mapreduce out of hbase-server into separate module.

2017-08-25 Thread appy
http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TsvImporterCustomTestMapperForOprAttr.java
--
diff --git 
a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TsvImporterCustomTestMapperForOprAttr.java
 
b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TsvImporterCustomTestMapperForOprAttr.java
new file mode 100644
index 000..a9da98b
--- /dev/null
+++ 
b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TsvImporterCustomTestMapperForOprAttr.java
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.mapreduce;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Put;
+import 
org.apache.hadoop.hbase.mapreduce.ImportTsv.TsvParser.BadTsvLineException;
+import org.apache.hadoop.hbase.mapreduce.ImportTsv.TsvParser.ParsedLine;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * Just shows a simple example of how the attributes can be extracted and added
+ * to the puts
+ */
+public class TsvImporterCustomTestMapperForOprAttr extends TsvImporterMapper {
+  @Override
+  protected void populatePut(byte[] lineBytes, ParsedLine parsed, Put put, int 
i)
+  throws BadTsvLineException, IOException {
+KeyValue kv;
+kv = new KeyValue(lineBytes, parsed.getRowKeyOffset(), 
parsed.getRowKeyLength(),
+parser.getFamily(i), 0, parser.getFamily(i).length, 
parser.getQualifier(i), 0,
+parser.getQualifier(i).length, ts, KeyValue.Type.Put, lineBytes, 
parsed.getColumnOffset(i),
+parsed.getColumnLength(i));
+if (parsed.getIndividualAttributes() != null) {
+  String[] attributes = parsed.getIndividualAttributes();
+  for (String attr : attributes) {
+String[] split = attr.split(ImportTsv.DEFAULT_ATTRIBUTES_SEPERATOR);
+if (split == null || split.length <= 1) {
+  throw new BadTsvLineException("Invalid attributes seperator 
specified" + attributes);
+} else {
+  if (split[0].length() <= 0 || split[1].length() <= 0) {
+throw new BadTsvLineException("Invalid attributes seperator 
specified" + attributes);
+  }
+  put.setAttribute(split[0], Bytes.toBytes(split[1]));
+}
+  }
+}
+put.add(kv);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
--
diff --git 
a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
 
b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
new file mode 100644
index 000..69c4c7c
--- /dev/null
+++ 
b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
@@ -0,0 +1,1059 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.replication;
+
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.NavigableMap;
+import java.util.TreeMap;
+
+import org.apache.commons.logging.Log;
+import 

[11/41] hbase git commit: HBASE-18640 Move mapreduce out of hbase-server into separate module.

2017-08-25 Thread appy
http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
deleted file mode 100644
index e80410f..000
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
+++ /dev/null
@@ -1, +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.snapshot;
-
-import java.io.BufferedInputStream;
-import java.io.FileNotFoundException;
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.LinkedList;
-import java.util.List;
-
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.Option;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileChecksum;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.io.FileLink;
-import org.apache.hadoop.hbase.io.HFileLink;
-import org.apache.hadoop.hbase.io.WALLink;
-import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
-import org.apache.hadoop.hbase.mob.MobUtils;
-import 
org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
-import 
org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotFileInfo;
-import 
org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
-import org.apache.hadoop.hbase.util.AbstractHBaseTool;
-import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.util.HFileArchiveUtil;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.io.BytesWritable;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
-import org.apache.hadoop.mapreduce.security.TokenCache;
-import org.apache.hadoop.hbase.io.hadoopbackport.ThrottledInputStream;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.util.Tool;
-
-/**
- * Export the specified snapshot to a given FileSystem.
- *
- * The .snapshot/name folder is copied to the destination cluster
- * and then all the hfiles/wals are copied using a Map-Reduce Job in the 
.archive/ location.
- * When everything is done, the second cluster can restore the snapshot.
- */
-@InterfaceAudience.Public
-public class ExportSnapshot extends AbstractHBaseTool implements Tool {
-  public static final String NAME = "exportsnapshot";
-  /** Configuration prefix for overrides for the source filesystem */
-  public static final String CONF_SOURCE_PREFIX = NAME + ".from.";
-  /** Configuration prefix for overrides for the destination filesystem */
-  public static final String CONF_DEST_PREFIX = NAME + ".to.";
-
-  private static final Log LOG = LogFactory.getLog(ExportSnapshot.class);
-
-  private static final String MR_NUM_MAPS = "mapreduce.job.maps";
-  private static final 

[13/41] hbase git commit: HBASE-18640 Move mapreduce out of hbase-server into separate module.

2017-08-25 Thread appy
http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatImpl.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatImpl.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatImpl.java
deleted file mode 100644
index bf11473..000
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatImpl.java
+++ /dev/null
@@ -1,412 +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.mapreduce;
-
-import org.apache.hadoop.hbase.client.TableDescriptor;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
-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.CellUtil;
-import org.apache.hadoop.hbase.HDFSBlocksDistribution;
-import org.apache.hadoop.hbase.HDFSBlocksDistribution.HostAndWeight;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.client.ClientSideRegionScanner;
-import org.apache.hadoop.hbase.client.IsolationLevel;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import 
org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.TableSnapshotRegionSplit;
-import 
org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
-import 
org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
-import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
-import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.io.Writable;
-
-import java.io.ByteArrayOutputStream;
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.UUID;
-
-/**
- * Hadoop MR API-agnostic implementation for mapreduce over table snapshots.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class TableSnapshotInputFormatImpl {
-  // TODO: Snapshots files are owned in fs by the hbase user. There is no
-  // easy way to delegate access.
-
-  public static final Log LOG = 
LogFactory.getLog(TableSnapshotInputFormatImpl.class);
-
-  private static final String SNAPSHOT_NAME_KEY = 
"hbase.TableSnapshotInputFormat.snapshot.name";
-  // key for specifying the root dir of the restored snapshot
-  protected static final String RESTORE_DIR_KEY = 
"hbase.TableSnapshotInputFormat.restore.dir";
-
-  /** See {@link #getBestLocations(Configuration, HDFSBlocksDistribution)} */
-  private static final String LOCALITY_CUTOFF_MULTIPLIER =
-"hbase.tablesnapshotinputformat.locality.cutoff.multiplier";
-  private static final float DEFAULT_LOCALITY_CUTOFF_MULTIPLIER = 0.8f;
-
-  /**
-   * Implementation class for InputSplit logic common between mapred and 
mapreduce.
-   */
-  public static class InputSplit implements Writable {
-
-private TableDescriptor htd;
-private HRegionInfo regionInfo;
-private String[] locations;
-private String scan;
-private String restoreDir;
-
-// constructor for mapreduce framework / Writable
-public InputSplit() {}
-
-public InputSplit(TableDescriptor htd, HRegionInfo regionInfo, 
List locations,
-Scan scan, Path restoreDir) {
-  this.htd = htd;
-  this.regionInfo = regionInfo;
-  if (locations == null || locations.isEmpty()) {
-this.locations = new String[0];
- 

[27/41] hbase git commit: HBASE-18640 Move mapreduce out of hbase-server into separate module.

2017-08-25 Thread appy
http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
--
diff --git 
a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
 
b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
new file mode 100644
index 000..c6a8761
--- /dev/null
+++ 
b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
@@ -0,0 +1,1496 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.mapreduce;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.Callable;
+
+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.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.hbase.ArrayBackedTag;
+import org.apache.hadoop.hbase.CategoryBasedTimeout;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HDFSBlocksDistribution;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.HadoopShims;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.PerformanceEvaluation;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.TagType;
+import org.apache.hadoop.hbase.TagUtil;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.io.compress.Compression;
+import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
+import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.io.hfile.HFile.Reader;
+import org.apache.hadoop.hbase.io.hfile.HFileScanner;
+import org.apache.hadoop.hbase.regionserver.BloomType;
+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.regionserver.TestHRegionFileSystem;
+import org.apache.hadoop.hbase.regionserver.TimeRangeTracker;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.ReflectionUtils;
+import org.apache.hadoop.hbase.util.Writables;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+import 

[21/41] hbase git commit: HBASE-18640 Move mapreduce out of hbase-server into separate module.

2017-08-25 Thread appy
http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java
--
diff --git 
a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java 
b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java
new file mode 100644
index 000..6b5cbe2
--- /dev/null
+++ 
b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java
@@ -0,0 +1,915 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.lang.reflect.Constructor;
+import java.security.SecureRandom;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicReference;
+
+import javax.crypto.spec.SecretKeySpec;
+
+import org.apache.commons.cli.CommandLine;
+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.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+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.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Durability;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.io.compress.Compression;
+import org.apache.hadoop.hbase.io.crypto.Cipher;
+import org.apache.hadoop.hbase.io.crypto.Encryption;
+import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
+import org.apache.hadoop.hbase.regionserver.BloomType;
+import org.apache.hadoop.hbase.security.EncryptionUtil;
+import org.apache.hadoop.hbase.security.HBaseKerberosUtils;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.security.access.AccessControlClient;
+import org.apache.hadoop.hbase.security.access.Permission;
+import org.apache.hadoop.hbase.util.test.LoadTestDataGenerator;
+import org.apache.hadoop.hbase.util.test.LoadTestDataGeneratorWithACL;
+import org.apache.hadoop.util.ToolRunner;
+
+/**
+ * A command-line utility that reads, writes, and verifies data. Unlike
+ * {@link org.apache.hadoop.hbase.PerformanceEvaluation}, this tool validates 
the data written,
+ * and supports simultaneously writing and reading the same set of keys.
+ */
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
+public class LoadTestTool extends AbstractHBaseTool {
+
+  private static final Log LOG = LogFactory.getLog(LoadTestTool.class);
+  private static final String COLON = ":";
+
+  /** Table name for the test */
+  private TableName tableName;
+
+  /** Column families for the test */
+  private byte[][] families;
+
+  /** Table name to use of not overridden on the command line */
+  protected static final String DEFAULT_TABLE_NAME = "cluster_test";
+
+  /** The default data size if not specified */
+  protected static final int DEFAULT_DATA_SIZE = 64;
+
+  /** The number of reader/writer threads if not specified */
+  protected static final int DEFAULT_NUM_THREADS = 20;
+
+  /** Usage string for the load option */
+  protected static final String OPT_USAGE_LOAD =
+  ":" +
+  "[:<#threads=" + DEFAULT_NUM_THREADS + ">]";
+
+  /** Usage string for the read option */
+  protected static final String OPT_USAGE_READ =
+  "[:<#threads=" + DEFAULT_NUM_THREADS + ">]";
+
+  /** Usage string for the update option */
+  protected static final String OPT_USAGE_UPDATE =
+  "[:<#threads=" + DEFAULT_NUM_THREADS
+  + ">][:<#whether to ignore nonce collisions=0>]";
+
+  protected static final String OPT_USAGE_BLOOM = "Bloom filter type, one of " 
+
+  Arrays.toString(BloomType.values());
+
+  protected static final String 

[37/41] hbase git commit: HBASE-18640 Move mapreduce out of hbase-server into separate module.

2017-08-25 Thread appy
http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java
--
diff --git 
a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java
 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java
new file mode 100644
index 000..b64271e
--- /dev/null
+++ 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java
@@ -0,0 +1,793 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.mapreduce;
+
+import static java.lang.String.format;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+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.conf.Configured;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+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.TableNotEnabledException;
+import org.apache.hadoop.hbase.TableNotFoundException;
+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.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Put;
+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.util.Base64;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Splitter;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+
+/**
+ * Tool to import data from a TSV file.
+ *
+ * This tool is rather simplistic - it doesn't do any quoting or
+ * escaping, but is useful for many data loads.
+ *
+ * @see ImportTsv#usage(String)
+ */
+@InterfaceAudience.Public
+public class ImportTsv extends Configured implements Tool {
+
+  protected static final Log LOG = LogFactory.getLog(ImportTsv.class);
+
+  final static String NAME = "importtsv";
+
+  public final static String MAPPER_CONF_KEY = "importtsv.mapper.class";
+  public final static String BULK_OUTPUT_CONF_KEY = "importtsv.bulk.output";
+  public final static String TIMESTAMP_CONF_KEY = "importtsv.timestamp";
+  public final static String JOB_NAME_CONF_KEY = "mapreduce.job.name";
+  // TODO: the rest of these configs are used exclusively by TsvImporterMapper.
+  // Move them out of the tool and let the mapper handle its own validation.
+  public final static String DRY_RUN_CONF_KEY = "importtsv.dry.run";
+  // If true, bad lines are logged to stderr. Default: false.
+  public final static String LOG_BAD_LINES_CONF_KEY = 
"importtsv.log.bad.lines";
+  public final static String SKIP_LINES_CONF_KEY = "importtsv.skip.bad.lines";
+  public final static String SKIP_EMPTY_COLUMNS = 
"importtsv.skip.empty.columns";
+  public final static String COLUMNS_CONF_KEY = "importtsv.columns";
+  public final static String SEPARATOR_CONF_KEY = "importtsv.separator";
+  public final static String 

[30/41] hbase git commit: HBASE-18640 Move mapreduce out of hbase-server into separate module.

2017-08-25 Thread appy
http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
--
diff --git 
a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
 
b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
new file mode 100644
index 000..23a70a9
--- /dev/null
+++ 
b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
@@ -0,0 +1,2627 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import static 
org.codehaus.jackson.map.SerializationConfig.Feature.SORT_PROPERTIES_ALPHABETICALLY;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.lang.reflect.Constructor;
+import java.math.BigDecimal;
+import java.math.MathContext;
+import java.text.DecimalFormat;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Date;
+import java.util.LinkedList;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Random;
+import java.util.TreeMap;
+import java.util.NoSuchElementException;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+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.conf.Configured;
+import org.apache.hadoop.fs.FileSystem;
+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;
+import org.apache.hadoop.hbase.client.Consistency;
+import org.apache.hadoop.hbase.client.Delete;
+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;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.filter.BinaryComparator;
+import org.apache.hadoop.hbase.filter.CompareFilter;
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterAllFilter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.PageFilter;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.filter.WhileMatchFilter;
+import org.apache.hadoop.hbase.io.compress.Compression;
+import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
+import org.apache.hadoop.hbase.io.hfile.RandomDistribution;
+import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
+import org.apache.hadoop.hbase.regionserver.BloomType;
+import org.apache.hadoop.hbase.regionserver.CompactingMemStore;
+import org.apache.hadoop.hbase.regionserver.TestHRegionFileSystem;
+import org.apache.hadoop.hbase.trace.HBaseHTraceConfiguration;
+import org.apache.hadoop.hbase.trace.SpanReceiverHost;
+import org.apache.hadoop.hbase.util.*;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.lib.input.NLineInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
+import 

[08/41] hbase git commit: HBASE-18640 Move mapreduce out of hbase-server into separate module.

2017-08-25 Thread appy
http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableMapReduceUtil.java
--
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableMapReduceUtil.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableMapReduceUtil.java
deleted file mode 100644
index ac2f20d..000
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableMapReduceUtil.java
+++ /dev/null
@@ -1,272 +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.mapred;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.apache.hadoop.hbase.testclassification.MapReduceTests;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.mapred.JobClient;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.MapReduceBase;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.mapred.RunningJob;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableMap;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableSet;
-
-@Category({MapReduceTests.class, LargeTests.class})
-public class TestTableMapReduceUtil {
-
-  private static final Log LOG = LogFactory
-  .getLog(TestTableMapReduceUtil.class);
-
-  private static Table presidentsTable;
-  private static final String TABLE_NAME = "People";
-
-  private static final byte[] COLUMN_FAMILY = Bytes.toBytes("info");
-  private static final byte[] COLUMN_QUALIFIER = Bytes.toBytes("name");
-
-  private static ImmutableSet presidentsRowKeys = ImmutableSet.of(
-  "president1", "president2", "president3");
-  private static Iterator presidentNames = ImmutableSet.of(
-  "John F. Kennedy", "George W. Bush", "Barack Obama").iterator();
-
-  private static ImmutableSet actorsRowKeys = ImmutableSet.of("actor1",
-  "actor2");
-  private static Iterator actorNames = ImmutableSet.of(
-  "Jack Nicholson", "Martin Freeman").iterator();
-
-  private static String PRESIDENT_PATTERN = "president";
-  private static String ACTOR_PATTERN = "actor";
-  private static ImmutableMap relation = 
ImmutableMap
-  .of(PRESIDENT_PATTERN, presidentsRowKeys, ACTOR_PATTERN, actorsRowKeys);
-
-  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
-
-  @BeforeClass
-  public static void beforeClass() throws Exception {
-UTIL.startMiniCluster();
-presidentsTable = createAndFillTable(TableName.valueOf(TABLE_NAME));
-  }
-
-  @AfterClass
-  public static void afterClass() throws Exception {
-UTIL.shutdownMiniCluster();
-  }
-
-  @Before
-  public void before() throws IOException {
-LOG.info("before");
-UTIL.ensureSomeRegionServersAvailable(1);
-LOG.info("before done");
-  }
-
-  public static Table createAndFillTable(TableName tableName) throws 
IOException {
-Table table = UTIL.createTable(tableName, COLUMN_FAMILY);
-createPutCommand(table);
-return table;
-  }
-
-  private static void createPutCommand(Table table) throws IOException {
-for (String president : presidentsRowKeys) {
-  if 

[38/41] hbase git commit: HBASE-18640 Move mapreduce out of hbase-server into separate module.

2017-08-25 Thread appy
http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HRegionPartitioner.java
--
diff --git 
a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HRegionPartitioner.java
 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HRegionPartitioner.java
new file mode 100644
index 000..3c3060b
--- /dev/null
+++ 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HRegionPartitioner.java
@@ -0,0 +1,140 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.mapreduce;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.TableName;
+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.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.mapred.TableOutputFormat;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.mapreduce.Partitioner;
+
+/**
+ * This is used to partition the output keys into groups of keys.
+ * Keys are grouped according to the regions that currently exist
+ * so that each reducer fills a single region so load is distributed.
+ *
+ * This class is not suitable as partitioner creating hfiles
+ * for incremental bulk loads as region spread will likely change between time 
of
+ * hfile creation and load time. See {@link LoadIncrementalHFiles}
+ * and http://hbase.apache.org/book.html#arch.bulk.load;>Bulk 
Load.
+ *
+ * @param   The type of the key.
+ * @param   The type of the value.
+ */
+@InterfaceAudience.Public
+public class HRegionPartitioner
+extends Partitioner
+implements Configurable {
+
+  private static final Log LOG = LogFactory.getLog(HRegionPartitioner.class);
+  private Configuration conf = null;
+  // Connection and locator are not cleaned up; they just die when partitioner 
is done.
+  private Connection connection;
+  private RegionLocator locator;
+  private byte[][] startKeys;
+
+  /**
+   * Gets the partition number for a given key (hence record) given the total
+   * number of partitions i.e. number of reduce-tasks for the job.
+   *
+   * Typically a hash function on a all or a subset of the key.
+   *
+   * @param key  The key to be partitioned.
+   * @param value  The entry value.
+   * @param numPartitions  The total number of partitions.
+   * @return The partition number for the key.
+   * @see org.apache.hadoop.mapreduce.Partitioner#getPartition(
+   *   java.lang.Object, java.lang.Object, int)
+   */
+  @Override
+  public int getPartition(ImmutableBytesWritable key,
+  VALUE value, int numPartitions) {
+byte[] region = null;
+// Only one region return 0
+if (this.startKeys.length == 1){
+  return 0;
+}
+try {
+  // Not sure if this is cached after a split so we could have problems
+  // here if a region splits while mapping
+  region = 
this.locator.getRegionLocation(key.get()).getRegionInfo().getStartKey();
+} catch (IOException e) {
+  LOG.error(e);
+}
+for (int i = 0; i < this.startKeys.length; i++){
+  if (Bytes.compareTo(region, this.startKeys[i]) == 0 ){
+if (i >= numPartitions-1){
+  // cover if we have less reduces then regions.
+  return (Integer.toString(i).hashCode()
+  & Integer.MAX_VALUE) % numPartitions;
+}
+return i;
+  }
+}
+// if above fails to find start key that match we need to return something
+return 0;
+  }
+
+  /**
+   * Returns the current configuration.
+   *
+   * @return The current configuration.
+   * @see org.apache.hadoop.conf.Configurable#getConf()
+   */
+  @Override
+  public Configuration getConf() {
+return conf;
+  }
+
+  /**

[12/41] hbase git commit: HBASE-18640 Move mapreduce out of hbase-server into separate module.

2017-08-25 Thread appy
http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
deleted file mode 100644
index 8bb266e..000
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
+++ /dev/null
@@ -1,700 +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.mapreduce.replication;
-
-import java.io.IOException;
-import java.util.Arrays;
-
-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.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.client.TableSnapshotScanner;
-import org.apache.hadoop.hbase.filter.Filter;
-import org.apache.hadoop.hbase.filter.FilterList;
-import org.apache.hadoop.hbase.filter.PrefixFilter;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.mapreduce.TableInputFormat;
-import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
-import org.apache.hadoop.hbase.mapreduce.TableMapper;
-import org.apache.hadoop.hbase.mapreduce.TableSnapshotInputFormat;
-import org.apache.hadoop.hbase.mapreduce.TableSplit;
-import org.apache.hadoop.hbase.replication.ReplicationException;
-import org.apache.hadoop.hbase.replication.ReplicationFactory;
-import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
-import org.apache.hadoop.hbase.replication.ReplicationPeerZKImpl;
-import org.apache.hadoop.hbase.replication.ReplicationPeers;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.util.Threads;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.MRJobConfig;
-import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-
-import 
org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
-
-/**
- * This map-only job compares the data from a local table with a remote one.
- * Every cell is compared and must have exactly the same keys (even timestamp)
- * as well as same value. It is possible to restrict the job by time range and
- * families. The peer id that's provided must match the one given when the
- * replication stream was setup.
- * 
- * Two counters are provided, Verifier.Counters.GOODROWS and BADROWS. The 
reason
- * for a why a row is different is shown in the map's log.
- */
-public class VerifyReplication extends Configured implements Tool {
-
-  private static final Log LOG =
-  LogFactory.getLog(VerifyReplication.class);
-
-  public final static String NAME = "verifyrep";
-  private final static String PEER_CONFIG_PREFIX = NAME + ".peer.";
-  long startTime = 0;
-  long endTime = Long.MAX_VALUE;
-  int batch = -1;
-  int versions = -1;
-  String tableName = null;
-  String families = null;
-  String delimiter = "";
-  String peerId = null;
-  String rowPrefixes = null;
-  int sleepMsBeforeReCompare = 0;
-  boolean verbose = false;
-  boolean includeDeletedCells = false;
-  //Source table snapshot 

[07/41] hbase git commit: HBASE-18640 Move mapreduce out of hbase-server into separate module.

2017-08-25 Thread appy
http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
--
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
deleted file mode 100644
index 87522b6..000
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
+++ /dev/null
@@ -1,1495 +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.mapreduce;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNotSame;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.lang.reflect.Field;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Random;
-import java.util.Set;
-import java.util.concurrent.Callable;
-
-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.LocatedFileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.RemoteIterator;
-import org.apache.hadoop.hbase.ArrayBackedTag;
-import org.apache.hadoop.hbase.CategoryBasedTimeout;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HDFSBlocksDistribution;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.HadoopShims;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.PerformanceEvaluation;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.Tag;
-import org.apache.hadoop.hbase.TagType;
-import org.apache.hadoop.hbase.TagUtil;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.RegionLocator;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.io.compress.Compression;
-import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
-import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
-import org.apache.hadoop.hbase.io.hfile.CacheConfig;
-import org.apache.hadoop.hbase.io.hfile.HFile;
-import org.apache.hadoop.hbase.io.hfile.HFile.Reader;
-import org.apache.hadoop.hbase.io.hfile.HFileScanner;
-import org.apache.hadoop.hbase.regionserver.BloomType;
-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.regionserver.TimeRangeTracker;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.util.ReflectionUtils;
-import org.apache.hadoop.hbase.util.Writables;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
-import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
-import 

[40/41] hbase git commit: HBASE-18640 Move mapreduce out of hbase-server into separate module.

2017-08-25 Thread appy
http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java
--
diff --git 
a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java
 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java
new file mode 100644
index 000..9811a97
--- /dev/null
+++ 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java
@@ -0,0 +1,313 @@
+/**
+ *
+ * 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.mapred;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+
+/**
+ * A Base for {@link TableInputFormat}s. Receives a {@link Table}, a
+ * byte[] of input columns and optionally a {@link Filter}.
+ * Subclasses may use other TableRecordReader implementations.
+ *
+ * Subclasses MUST ensure initializeTable(Connection, TableName) is called for 
an instance to
+ * function properly. Each of the entry points to this class used by the 
MapReduce framework,
+ * {@link #getRecordReader(InputSplit, JobConf, Reporter)} and {@link 
#getSplits(JobConf, int)},
+ * will call {@link #initialize(JobConf)} as a convenient centralized location 
to handle
+ * retrieving the necessary configuration information. If your subclass 
overrides either of these
+ * methods, either call the parent version or call initialize yourself.
+ *
+ * 
+ * An example of a subclass:
+ * 
+ *   class ExampleTIF extends TableInputFormatBase {
+ *
+ * {@literal @}Override
+ * protected void initialize(JobConf context) throws IOException {
+ *   // We are responsible for the lifecycle of this connection until we 
hand it over in
+ *   // initializeTable.
+ *   Connection connection =
+ *  ConnectionFactory.createConnection(HBaseConfiguration.create(job));
+ *   TableName tableName = TableName.valueOf("exampleTable");
+ *   // mandatory. once passed here, TableInputFormatBase will handle 
closing the connection.
+ *   initializeTable(connection, tableName);
+ *   byte[][] inputColumns = new byte [][] { Bytes.toBytes("columnA"),
+ * Bytes.toBytes("columnB") };
+ *   // mandatory
+ *   setInputColumns(inputColumns);
+ *   // optional, by default we'll get everything for the given columns.
+ *   Filter exampleFilter = new RowFilter(CompareOp.EQUAL, new 
RegexStringComparator("aa.*"));
+ *   setRowFilter(exampleFilter);
+ * }
+ *   }
+ * 
+ */
+
+@InterfaceAudience.Public
+public abstract class TableInputFormatBase
+implements InputFormat {
+  private static final Log LOG = LogFactory.getLog(TableInputFormatBase.class);
+  private byte [][] inputColumns;
+  private Table table;
+  private RegionLocator regionLocator;
+  private Connection connection;
+  private TableRecordReader tableRecordReader;
+  private Filter rowFilter;
+
+  private static final String NOT_INITIALIZED = "The input format instance has 
not been properly " +
+  "initialized. Ensure you call initializeTable either in your constructor 
or initialize " +
+  "method";
+  private static final String INITIALIZATION_ERROR = "Cannot create a record 
reader because of a" +
+" previous error. Please look at the previous logs lines from" +
+" the task's full log for more 

[39/41] hbase git commit: HBASE-18640 Move mapreduce out of hbase-server into separate module.

2017-08-25 Thread appy
http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/CopyTable.java
--
diff --git 
a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/CopyTable.java
 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/CopyTable.java
new file mode 100644
index 000..9cccf8c
--- /dev/null
+++ 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/CopyTable.java
@@ -0,0 +1,386 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.mapreduce;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+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.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+/**
+ * Tool used to copy a table to another one which can be on a different setup.
+ * It is also configurable with a start and time as well as a specification
+ * of the region server implementation if different from the local cluster.
+ */
+@InterfaceAudience.Public
+public class CopyTable extends Configured implements Tool {
+  private static final Log LOG = LogFactory.getLog(CopyTable.class);
+
+  final static String NAME = "copytable";
+  long startTime = 0;
+  long endTime = HConstants.LATEST_TIMESTAMP;
+  int batch = Integer.MAX_VALUE;
+  int cacheRow = -1;
+  int versions = -1;
+  String tableName = null;
+  String startRow = null;
+  String stopRow = null;
+  String dstTableName = null;
+  String peerAddress = null;
+  String families = null;
+  boolean allCells = false;
+  static boolean shuffle = false;
+
+  boolean bulkload = false;
+  Path bulkloadDir = null;
+
+  private final static String JOB_NAME_CONF_KEY = "mapreduce.job.name";
+
+  /**
+   * 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 {
+if (!doCommandLine(args)) {
+  return null;
+}
+
+Job job = Job.getInstance(getConf(), getConf().get(JOB_NAME_CONF_KEY, NAME 
+ "_" + tableName));
+job.setJarByClass(CopyTable.class);
+Scan scan = new Scan();
+
+scan.setBatch(batch);
+scan.setCacheBlocks(false);
+
+if (cacheRow > 0) {
+  scan.setCaching(cacheRow);
+} else {
+  
scan.setCaching(getConf().getInt(HConstants.HBASE_CLIENT_SCANNER_CACHING, 100));
+}
+
+scan.setTimeRange(startTime, endTime);
+
+if (allCells) {
+  scan.setRaw(true);
+}
+if (shuffle) {
+  job.getConfiguration().set(TableInputFormat.SHUFFLE_MAPS, "true");
+}
+if (versions >= 0) {
+  scan.setMaxVersions(versions);
+}
+
+if (startRow != null) {
+  scan.setStartRow(Bytes.toBytesBinary(startRow));
+}
+
+if (stopRow != null) {
+  scan.setStopRow(Bytes.toBytesBinary(stopRow));
+}
+
+if(families != null) {
+  String[] fams = families.split(",");
+  Map cfRenameMap = new HashMap<>();
+  for(String fam : fams) {
+String sourceCf;
+if(fam.contains(":")) {
+// fam looks like "sourceCfName:destCfName"
+String[] srcAndDest = fam.split(":", 2);
+sourceCf = srcAndDest[0];
+String destCf = srcAndDest[1];
+cfRenameMap.put(sourceCf, destCf);
+} else {
+// fam is just "sourceCf"
+

[29/41] hbase git commit: HBASE-18640 Move mapreduce out of hbase-server into separate module.

2017-08-25 Thread appy
http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/ScanPerformanceEvaluation.java
--
diff --git 
a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/ScanPerformanceEvaluation.java
 
b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/ScanPerformanceEvaluation.java
new file mode 100644
index 000..e669f14
--- /dev/null
+++ 
b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/ScanPerformanceEvaluation.java
@@ -0,0 +1,406 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase;
+
+import java.io.IOException;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+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.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableSnapshotScanner;
+import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
+import org.apache.hadoop.hbase.mapreduce.TableMapper;
+import org.apache.hadoop.hbase.util.AbstractHBaseTool;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.Counters;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.ToolRunner;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Stopwatch;
+
+/**
+ * A simple performance evaluation tool for single client and MR scans
+ * and snapshot scans.
+ */
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
+public class ScanPerformanceEvaluation extends AbstractHBaseTool {
+
+  private static final String HBASE_COUNTER_GROUP_NAME = "HBase Counters";
+
+  private String type;
+  private String file;
+  private String tablename;
+  private String snapshotName;
+  private String restoreDir;
+  private String caching;
+
+  @Override
+  public void setConf(Configuration conf) {
+super.setConf(conf);
+Path rootDir;
+try {
+  rootDir = FSUtils.getRootDir(conf);
+  rootDir.getFileSystem(conf);
+} catch (IOException ex) {
+  throw new RuntimeException(ex);
+}
+  }
+
+  @Override
+  protected void addOptions() {
+this.addRequiredOptWithArg("t", "type", "the type of the test. One of the 
following: streaming|scan|snapshotscan|scanmapreduce|snapshotscanmapreduce");
+this.addOptWithArg("f", "file", "the filename to read from");
+this.addOptWithArg("tn", "table", "the tablename to read from");
+this.addOptWithArg("sn", "snapshot", "the snapshot name to read from");
+this.addOptWithArg("rs", "restoredir", "the directory to restore the 
snapshot");
+this.addOptWithArg("ch", "caching", "scanner caching value");
+  }
+
+  @Override
+  protected void processOptions(CommandLine cmd) {
+type = cmd.getOptionValue("type");
+file = cmd.getOptionValue("file");
+tablename = cmd.getOptionValue("table");
+snapshotName = cmd.getOptionValue("snapshot");
+restoreDir = cmd.getOptionValue("restoredir");
+caching = cmd.getOptionValue("caching");
+  }
+
+  protected void testHdfsStreaming(Path filename) throws IOException {
+byte[] buf = new byte[1024];
+FileSystem fs = filename.getFileSystem(getConf());
+
+// read the file from start to finish
+Stopwatch fileOpenTimer = Stopwatch.createUnstarted();
+Stopwatch streamTimer = Stopwatch.createUnstarted();
+
+fileOpenTimer.start();
+FSDataInputStream in = fs.open(filename);
+fileOpenTimer.stop();
+
+long totalBytes = 0;
+

[06/41] hbase git commit: HBASE-18640 Move mapreduce out of hbase-server into separate module.

2017-08-25 Thread appy
http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java
--
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java
deleted file mode 100644
index dc59817..000
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java
+++ /dev/null
@@ -1,727 +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.mapreduce;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import java.io.ByteArrayOutputStream;
-import java.io.File;
-import java.io.IOException;
-import java.io.PrintStream;
-import java.net.URL;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.NavigableMap;
-
-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.Cell;
-import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.KeepDeletedCells;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Durability;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.filter.Filter;
-import org.apache.hadoop.hbase.filter.FilterBase;
-import org.apache.hadoop.hbase.filter.PrefixFilter;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.mapreduce.Import.KeyValueImporter;
-import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
-import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
-import org.apache.hadoop.hbase.wal.WAL;
-import org.apache.hadoop.hbase.wal.WALKey;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.LauncherSecurityManager;
-import org.apache.hadoop.mapreduce.Mapper.Context;
-import org.apache.hadoop.util.ToolRunner;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.rules.TestName;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-/**
- * Tests the table import and table export MR job functionality
- */
-@Category({VerySlowMapReduceTests.class, MediumTests.class})
-public class TestImportExport {
-  private static final Log LOG = LogFactory.getLog(TestImportExport.class);
-  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
-  private static final byte[] ROW1 = Bytes.toBytesBinary("\\x32row1");
-  private static final byte[] ROW2 = Bytes.toBytesBinary("\\x32row2");
-  private static final byte[] ROW3 = Bytes.toBytesBinary("\\x32row3");
-  private static final String FAMILYA_STRING = "a";
-  private static final String FAMILYB_STRING = "b";
-  private static final byte[] FAMILYA = Bytes.toBytes(FAMILYA_STRING);
-  private 

[41/41] hbase git commit: HBASE-18640 Move mapreduce out of hbase-server into separate module.

2017-08-25 Thread appy
HBASE-18640 Move mapreduce out of hbase-server into separate module.

- Moves out o.a.h.h.{mapred, mapreduce} to new hbase-mapreduce module which 
depends
  on hbase-server because of classes like *Snapshot{Input,Output}Format.java, 
WALs, replication, etc
- hbase-backup depends on it for WALPlayer and MR job stuff
- A bunch of tools needed to be pulled into hbase-mapreduce becuase of their 
dependencies on MR.
  These are: CompactionTool, LoadTestTool, PerformanceEvaluation, ExportSnapshot
  This is better place of them than hbase-server. But ideal place would be in 
separate hbase-tools module.
- There were some tests in hbase-server which were digging into these tools for 
static util funtions or
  confs. Moved these to better/easily shared place. For eg. security related 
stuff to HBaseKerberosUtils.
- Note that hbase-mapreduce has secondPartExecution tests. On my machine they 
took like 20 min, so maybe
  more on apache jenkins. That's basically equal reduction of runtime of 
hbase-server tests, which is a
  big win!

Change-Id: Ieeb7235014717ca83ee5cb13b2a27fddfa6838e8


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

Branch: refs/heads/master
Commit: 664b6be0ef65218328847ea501fa88cb877e6759
Parents: 8d33949
Author: Apekshit Sharma 
Authored: Sun Aug 20 14:34:16 2017 -0700
Committer: Apekshit Sharma 
Committed: Fri Aug 25 18:38:48 2017 -0700

--
 hbase-assembly/pom.xml  |4 +
 .../src/main/assembly/hadoop-two-compat.xml |1 +
 hbase-assembly/src/main/assembly/src.xml|1 +
 hbase-backup/pom.xml|   10 +
 hbase-examples/pom.xml  |4 +
 hbase-it/pom.xml|   16 +
 .../hadoop/hbase/IntegrationTestIngest.java |5 +-
 .../IntegrationTestIngestStripeCompactions.java |4 +-
 .../hbase/IntegrationTestIngestWithMOB.java |5 +-
 .../hbase/IntegrationTestRegionReplicaPerf.java |3 +-
 .../mapreduce/IntegrationTestImportTsv.java |1 -
 .../test/IntegrationTestLoadAndVerify.java  |2 +-
 hbase-mapreduce/pom.xml |  316 +++
 .../org/apache/hadoop/hbase/mapred/Driver.java  |   52 +
 .../hadoop/hbase/mapred/GroupingTableMap.java   |  157 ++
 .../hadoop/hbase/mapred/HRegionPartitioner.java |   95 +
 .../hadoop/hbase/mapred/IdentityTableMap.java   |   76 +
 .../hbase/mapred/IdentityTableReduce.java   |   61 +
 .../mapred/MultiTableSnapshotInputFormat.java   |  128 +
 .../apache/hadoop/hbase/mapred/RowCounter.java  |  121 +
 .../hadoop/hbase/mapred/TableInputFormat.java   |   90 +
 .../hbase/mapred/TableInputFormatBase.java  |  313 +++
 .../apache/hadoop/hbase/mapred/TableMap.java|   38 +
 .../hadoop/hbase/mapred/TableMapReduceUtil.java |  376 +++
 .../hadoop/hbase/mapred/TableOutputFormat.java  |  134 +
 .../hadoop/hbase/mapred/TableRecordReader.java  |  139 +
 .../hbase/mapred/TableRecordReaderImpl.java |  259 ++
 .../apache/hadoop/hbase/mapred/TableReduce.java |   38 +
 .../hbase/mapred/TableSnapshotInputFormat.java  |  166 ++
 .../apache/hadoop/hbase/mapred/TableSplit.java  |  154 +
 .../hadoop/hbase/mapred/package-info.java   |   26 +
 .../hadoop/hbase/mapreduce/CellCounter.java |  333 +++
 .../hadoop/hbase/mapreduce/CellCreator.java |  134 +
 .../hadoop/hbase/mapreduce/CopyTable.java   |  386 +++
 .../DefaultVisibilityExpressionResolver.java|  144 +
 .../apache/hadoop/hbase/mapreduce/Driver.java   |   64 +
 .../apache/hadoop/hbase/mapreduce/Export.java   |  197 ++
 .../hbase/mapreduce/GroupingTableMapper.java|  177 ++
 .../hbase/mapreduce/HFileInputFormat.java   |  174 ++
 .../hbase/mapreduce/HFileOutputFormat2.java |  902 ++
 .../hbase/mapreduce/HRegionPartitioner.java |  140 +
 .../hadoop/hbase/mapreduce/HashTable.java   |  747 +
 .../hbase/mapreduce/IdentityTableMapper.java|   67 +
 .../hbase/mapreduce/IdentityTableReducer.java   |   79 +
 .../apache/hadoop/hbase/mapreduce/Import.java   |  780 ++
 .../hadoop/hbase/mapreduce/ImportTsv.java   |  793 ++
 .../hadoop/hbase/mapreduce/JarFinder.java   |  186 ++
 .../hbase/mapreduce/KeyValueSerialization.java  |   88 +
 .../hbase/mapreduce/KeyValueSortReducer.java|   57 +
 .../mapreduce/MultiTableHFileOutputFormat.java  |  122 +
 .../hbase/mapreduce/MultiTableInputFormat.java  |  104 +
 .../mapreduce/MultiTableInputFormatBase.java|  296 ++
 .../hbase/mapreduce/MultiTableOutputFormat.java |  176 ++
 .../MultiTableSnapshotInputFormat.java  |  106 +
 .../MultiTableSnapshotInputFormatImpl.java  |  252 ++
 .../mapreduce/MultithreadedTableMapper.java |  301 ++
 

[19/41] hbase git commit: HBASE-18640 Move mapreduce out of hbase-server into separate module.

2017-08-25 Thread appy
http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/CellCreator.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/CellCreator.java 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/CellCreator.java
deleted file mode 100644
index 1d4d37b..000
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/CellCreator.java
+++ /dev/null
@@ -1,134 +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.mapreduce;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.Tag;
-import org.apache.hadoop.util.ReflectionUtils;
-
-/**
- * Facade to create Cells for HFileOutputFormat. The created Cells are of 
Put type.
- */
-@InterfaceAudience.Public
-public class CellCreator {
-
-  public static final String VISIBILITY_EXP_RESOLVER_CLASS =
-  "hbase.mapreduce.visibility.expression.resolver.class";
-
-  private VisibilityExpressionResolver visExpResolver;
-
-  public CellCreator(Configuration conf) {
-Class clazz = conf.getClass(
-VISIBILITY_EXP_RESOLVER_CLASS, 
DefaultVisibilityExpressionResolver.class,
-VisibilityExpressionResolver.class);
-this.visExpResolver = ReflectionUtils.newInstance(clazz, conf);
-this.visExpResolver.init();
-  }
-
-  /**
-   * @param row row key
-   * @param roffset row offset
-   * @param rlength row length
-   * @param family family name
-   * @param foffset family offset
-   * @param flength family length
-   * @param qualifier column qualifier
-   * @param qoffset qualifier offset
-   * @param qlength qualifier length
-   * @param timestamp version timestamp
-   * @param value column value
-   * @param voffset value offset
-   * @param vlength value length
-   * @return created Cell
-   * @throws IOException
-   */
-  public Cell create(byte[] row, int roffset, int rlength, byte[] family, int 
foffset, int flength,
-  byte[] qualifier, int qoffset, int qlength, long timestamp, byte[] 
value, int voffset,
-  int vlength) throws IOException {
-return create(row, roffset, rlength, family, foffset, flength, qualifier, 
qoffset, qlength,
-timestamp, value, voffset, vlength, (List)null);
-  }
-
-  /**
-   * @param row row key
-   * @param roffset row offset
-   * @param rlength row length
-   * @param family family name
-   * @param foffset family offset
-   * @param flength family length
-   * @param qualifier column qualifier
-   * @param qoffset qualifier offset
-   * @param qlength qualifier length
-   * @param timestamp version timestamp
-   * @param value column value
-   * @param voffset value offset
-   * @param vlength value length
-   * @param visExpression visibility expression to be associated with cell
-   * @return created Cell
-   * @throws IOException
-   */
-  @Deprecated
-  public Cell create(byte[] row, int roffset, int rlength, byte[] family, int 
foffset, int flength,
-  byte[] qualifier, int qoffset, int qlength, long timestamp, byte[] 
value, int voffset,
-  int vlength, String visExpression) throws IOException {
-List visTags = null;
-if (visExpression != null) {
-  visTags = this.visExpResolver.createVisibilityExpTags(visExpression);
-}
-return new KeyValue(row, roffset, rlength, family, foffset, flength, 
qualifier, qoffset,
-qlength, timestamp, KeyValue.Type.Put, value, voffset, vlength, 
visTags);
-  }
-
-  /**
-   * @param row row key
-   * @param roffset row offset
-   * @param rlength row length
-   * @param family family name
-   * @param foffset family offset
-   * @param flength family length
-   * @param qualifier column qualifier
-   * @param qoffset qualifier offset
-   * @param qlength qualifier length
-   * @param timestamp version timestamp
-   * @param value column value
-   * @param voffset value offset
-   * @param vlength value length
-   * @param tags
-   * 

[28/41] hbase git commit: HBASE-18640 Move mapreduce out of hbase-server into separate module.

2017-08-25 Thread appy
http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestTableOutputFormatConnectionExhaust.java
--
diff --git 
a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestTableOutputFormatConnectionExhaust.java
 
b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestTableOutputFormatConnectionExhaust.java
new file mode 100644
index 000..835117c
--- /dev/null
+++ 
b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestTableOutputFormatConnectionExhaust.java
@@ -0,0 +1,104 @@
+/**
+ * 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.mapred;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordWriter;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.io.IOException;
+
+import static org.junit.Assert.fail;
+
+/**
+ * Spark creates many instances of TableOutputFormat within a single process.  
We need to make
+ * sure we can have many instances and not leak connections.
+ *
+ * This test creates a few TableOutputFormats and shouldn't fail due to ZK 
connection exhaustion.
+ */
+@Category(MediumTests.class)
+public class TestTableOutputFormatConnectionExhaust {
+
+  private static final Log LOG =
+  LogFactory.getLog(TestTableOutputFormatConnectionExhaust.class);
+
+  private final static HBaseTestingUtility UTIL = new HBaseTestingUtility();
+  static final String TABLE = "TestTableOutputFormatConnectionExhaust";
+  static final String FAMILY = "family";
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+// Default in ZookeeperMiniCluster is 1000, setting artificially low to 
trigger exhaustion.
+// need min of 7 to properly start the default mini HBase cluster
+UTIL.getConfiguration().setInt(HConstants.ZOOKEEPER_MAX_CLIENT_CNXNS, 10);
+UTIL.startMiniCluster();
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+UTIL.shutdownMiniCluster();
+  }
+
+  @Before
+  public void before() throws IOException {
+LOG.info("before");
+UTIL.ensureSomeRegionServersAvailable(1);
+LOG.info("before done");
+  }
+
+  /**
+   * Open and close a TableOutputFormat.  The closing the RecordWriter should 
release HBase
+   * Connection (ZK) resources, and will throw exception if they are exhausted.
+   */
+  static void openCloseTableOutputFormat(int iter)  throws IOException {
+LOG.info("Instantiating TableOutputFormat connection  " + iter);
+JobConf conf = new JobConf();
+conf.addResource(UTIL.getConfiguration());
+conf.set(TableOutputFormat.OUTPUT_TABLE, TABLE);
+TableMapReduceUtil.initTableMapJob(TABLE, FAMILY, TableMap.class,
+ImmutableBytesWritable.class, ImmutableBytesWritable.class, conf);
+TableOutputFormat tof = new TableOutputFormat();
+RecordWriter rw = tof.getRecordWriter(null, conf, TABLE, null);
+rw.close(null);
+  }
+
+  @Test
+  public void testConnectionExhaustion() throws IOException {
+int MAX_INSTANCES = 5; // fails on iteration 3 if zk connections leak
+for (int i = 0; i < MAX_INSTANCES; i++) {
+  final int iter = i;
+  try {
+openCloseTableOutputFormat(iter);
+  } catch (Exception e) {
+LOG.error("Exception encountered", e);
+fail("Failed on iteration " + i);
+  }
+}
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestTableSnapshotInputFormat.java
--
diff --git 
a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestTableSnapshotInputFormat.java
 

[34/41] hbase git commit: HBASE-18640 Move mapreduce out of hbase-server into separate module.

2017-08-25 Thread appy
http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
--
diff --git 
a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
new file mode 100644
index 000..ff458ff
--- /dev/null
+++ 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
@@ -0,0 +1,1027 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.mapreduce;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URL;
+import java.net.URLDecoder;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Enumeration;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipFile;
+
+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.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.MetaTableAccessor;
+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.Put;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.security.UserProvider;
+import org.apache.hadoop.hbase.security.token.TokenUtil;
+import org.apache.hadoop.hbase.util.Base64;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.zookeeper.ZKConfig;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.util.StringUtils;
+
+import com.codahale.metrics.MetricRegistry;
+
+/**
+ * Utility for {@link TableMapper} and {@link TableReducer}
+ */
+@SuppressWarnings({ "rawtypes", "unchecked" })
+@InterfaceAudience.Public
+public class TableMapReduceUtil {
+  private static final Log LOG = LogFactory.getLog(TableMapReduceUtil.class);
+
+  /**
+   * Use this before submitting a TableMap job. It will appropriately set up
+   * the job.
+   *
+   * @param table  The table name to read from.
+   * @param scan  The scan instance with the columns, time range etc.
+   * @param mapper  The mapper class to use.
+   * @param outputKeyClass  The class of the output key.
+   * @param outputValueClass  The class of the output value.
+   * @param job  The current job to adjust.  Make sure the passed job is
+   * carrying all necessary HBase configuration.
+   * @throws IOException When setting up the details fails.
+   */
+  public static void initTableMapperJob(String table, Scan scan,
+  Class mapper,
+  Class outputKeyClass,
+  Class outputValueClass, Job job)
+  throws IOException {
+initTableMapperJob(table, scan, mapper, outputKeyClass, outputValueClass,
+job, true);
+  }
+
+
+  /**
+   * Use this before submitting a TableMap job. It will appropriately set up
+   * the job.
+   *
+   * @param table  The table name to read from.
+   * @param scan  The scan instance with the columns, time range etc.
+   * @param mapper  The mapper class to use.
+   * @param outputKeyClass  The class of the output key.
+   * @param outputValueClass  The class of the output value.
+   * @param job  The current job to adjust.  Make sure the passed job is
+   * carrying all necessary HBase configuration.
+   * @throws IOException When setting up the details fails.
+   */
+  public static void initTableMapperJob(TableName table,
+   

[33/41] hbase git commit: HBASE-18640 Move mapreduce out of hbase-server into separate module.

2017-08-25 Thread appy
http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatImpl.java
--
diff --git 
a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatImpl.java
 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatImpl.java
new file mode 100644
index 000..403051f
--- /dev/null
+++ 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatImpl.java
@@ -0,0 +1,410 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.mapreduce;
+
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+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.CellUtil;
+import org.apache.hadoop.hbase.HDFSBlocksDistribution;
+import org.apache.hadoop.hbase.HDFSBlocksDistribution.HostAndWeight;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.ClientSideRegionScanner;
+import org.apache.hadoop.hbase.client.IsolationLevel;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.TableSnapshotRegionSplit;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
+import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
+import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.io.Writable;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+
+/**
+ * Hadoop MR API-agnostic implementation for mapreduce over table snapshots.
+ */
+@InterfaceAudience.Private
+public class TableSnapshotInputFormatImpl {
+  // TODO: Snapshots files are owned in fs by the hbase user. There is no
+  // easy way to delegate access.
+
+  public static final Log LOG = 
LogFactory.getLog(TableSnapshotInputFormatImpl.class);
+
+  private static final String SNAPSHOT_NAME_KEY = 
"hbase.TableSnapshotInputFormat.snapshot.name";
+  // key for specifying the root dir of the restored snapshot
+  protected static final String RESTORE_DIR_KEY = 
"hbase.TableSnapshotInputFormat.restore.dir";
+
+  /** See {@link #getBestLocations(Configuration, HDFSBlocksDistribution)} */
+  private static final String LOCALITY_CUTOFF_MULTIPLIER =
+"hbase.tablesnapshotinputformat.locality.cutoff.multiplier";
+  private static final float DEFAULT_LOCALITY_CUTOFF_MULTIPLIER = 0.8f;
+
+  /**
+   * Implementation class for InputSplit logic common between mapred and 
mapreduce.
+   */
+  public static class InputSplit implements Writable {
+
+private TableDescriptor htd;
+private HRegionInfo regionInfo;
+private String[] locations;
+private String scan;
+private String restoreDir;
+
+// constructor for mapreduce framework / Writable
+public InputSplit() {}
+
+public InputSplit(TableDescriptor htd, HRegionInfo regionInfo, 
List locations,
+Scan scan, Path restoreDir) {
+  this.htd = htd;
+  this.regionInfo = regionInfo;
+  if (locations == null || locations.isEmpty()) {
+this.locations = new String[0];
+  } else {
+this.locations = locations.toArray(new String[locations.size()]);

[20/41] hbase git commit: HBASE-18640 Move mapreduce out of hbase-server into separate module.

2017-08-25 Thread appy
http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/RowCounter.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/RowCounter.java 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/RowCounter.java
deleted file mode 100644
index 43560fd..000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/RowCounter.java
+++ /dev/null
@@ -1,121 +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.mapred;
-
-import java.io.IOException;
-
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.mapred.FileOutputFormat;
-import org.apache.hadoop.mapred.JobClient;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-
-/**
- * A job with a map to count rows.
- * Map outputs table rows IF the input row has columns that have content.
- * Uses a org.apache.hadoop.mapred.lib.IdentityReducer
- */
-@InterfaceAudience.Public
-public class RowCounter extends Configured implements Tool {
-  // Name of this 'program'
-  static final String NAME = "rowcounter";
-
-  /**
-   * Mapper that runs the count.
-   */
-  static class RowCounterMapper
-  implements TableMap {
-private static enum Counters {ROWS}
-
-public void map(ImmutableBytesWritable row, Result values,
-OutputCollector output,
-Reporter reporter)
-throws IOException {
-// Count every row containing data, whether it's in qualifiers or 
values
-reporter.incrCounter(Counters.ROWS, 1);
-}
-
-public void configure(JobConf jc) {
-  // Nothing to do.
-}
-
-public void close() throws IOException {
-  // Nothing to do.
-}
-  }
-
-  /**
-   * @param args
-   * @return the JobConf
-   * @throws IOException
-   */
-  public JobConf createSubmittableJob(String[] args) throws IOException {
-JobConf c = new JobConf(getConf(), getClass());
-c.setJobName(NAME);
-// Columns are space delimited
-StringBuilder sb = new StringBuilder();
-final int columnoffset = 2;
-for (int i = columnoffset; i < args.length; i++) {
-  if (i > columnoffset) {
-sb.append(" ");
-  }
-  sb.append(args[i]);
-}
-// Second argument is the table name.
-TableMapReduceUtil.initTableMapJob(args[1], sb.toString(),
-  RowCounterMapper.class, ImmutableBytesWritable.class, Result.class, c);
-c.setNumReduceTasks(0);
-// First arg is the output directory.
-FileOutputFormat.setOutputPath(c, new Path(args[0]));
-return c;
-  }
-
-  static int printUsage() {
-System.out.println(NAME +
-  "[...]");
-return -1;
-  }
-
-  public int run(final String[] args) throws Exception {
-// Make sure there are at least 3 parameters
-if (args.length < 3) {
-  System.err.println("ERROR: Wrong number of parameters: " + args.length);
-  return printUsage();
-}
-JobClient.runJob(createSubmittableJob(args));
-return 0;
-  }
-
-  /**
-   * @param args
-   * @throws Exception
-   */
-  public static void main(String[] args) throws Exception {
-int errCode = ToolRunner.run(HBaseConfiguration.create(), new 
RowCounter(), args);
-System.exit(errCode);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableInputFormat.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableInputFormat.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableInputFormat.java
deleted file mode 100644
index 

[10/41] hbase git commit: HBASE-18640 Move mapreduce out of hbase-server into separate module.

2017-08-25 Thread appy
http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/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
deleted file mode 100644
index eebb0f3..000
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
+++ /dev/null
@@ -1,2626 +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;
-
-import static 
org.codehaus.jackson.map.SerializationConfig.Feature.SORT_PROPERTIES_ALPHABETICALLY;
-
-import java.io.IOException;
-import java.io.PrintStream;
-import java.lang.reflect.Constructor;
-import java.math.BigDecimal;
-import java.math.MathContext;
-import java.text.DecimalFormat;
-import java.text.SimpleDateFormat;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Date;
-import java.util.LinkedList;
-import java.util.Locale;
-import java.util.Map;
-import java.util.Queue;
-import java.util.Random;
-import java.util.TreeMap;
-import java.util.NoSuchElementException;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-
-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.conf.Configured;
-import org.apache.hadoop.fs.FileSystem;
-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;
-import org.apache.hadoop.hbase.client.Consistency;
-import org.apache.hadoop.hbase.client.Delete;
-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;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.filter.BinaryComparator;
-import org.apache.hadoop.hbase.filter.CompareFilter;
-import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
-import org.apache.hadoop.hbase.filter.Filter;
-import org.apache.hadoop.hbase.filter.FilterAllFilter;
-import org.apache.hadoop.hbase.filter.FilterList;
-import org.apache.hadoop.hbase.filter.PageFilter;
-import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
-import org.apache.hadoop.hbase.filter.WhileMatchFilter;
-import org.apache.hadoop.hbase.io.compress.Compression;
-import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
-import org.apache.hadoop.hbase.io.hfile.RandomDistribution;
-import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
-import org.apache.hadoop.hbase.regionserver.BloomType;
-import org.apache.hadoop.hbase.regionserver.CompactingMemStore;
-import org.apache.hadoop.hbase.trace.HBaseHTraceConfiguration;
-import org.apache.hadoop.hbase.trace.SpanReceiverHost;
-import org.apache.hadoop.hbase.util.*;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.lib.input.NLineInputFormat;
-import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
-import org.apache.hadoop.mapreduce.lib.reduce.LongSumReducer;
-import org.apache.hadoop.util.Tool;
-import 

[25/41] hbase git commit: HBASE-18640 Move mapreduce out of hbase-server into separate module.

2017-08-25 Thread appy
http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java
--
diff --git 
a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java
 
b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java
new file mode 100644
index 000..7b6e684
--- /dev/null
+++ 
b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java
@@ -0,0 +1,571 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.mapreduce;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.io.hfile.HFileScanner;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.Utils.OutputFileUtils.OutputFilesFilter;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.ExpectedException;
+
+@Category({VerySlowMapReduceTests.class, LargeTests.class})
+public class TestImportTsv implements Configurable {
+
+  private static final Log LOG = LogFactory.getLog(TestImportTsv.class);
+  protected static final String NAME = TestImportTsv.class.getSimpleName();
+  protected static HBaseTestingUtility util = new HBaseTestingUtility();
+
+  // Delete the tmp directory after running doMROnTableTest. Boolean. Default 
is true.
+  protected static final String DELETE_AFTER_LOAD_CONF = NAME + 
".deleteAfterLoad";
+
+  /**
+   * Force use of combiner in doMROnTableTest. Boolean. Default is true.
+   */
+  protected static final String FORCE_COMBINER_CONF = NAME + ".forceCombiner";
+
+  private final String FAMILY = "FAM";
+  private TableName tn;
+  private Map args;
+
+  @Rule
+  public ExpectedException exception = ExpectedException.none();
+
+  public Configuration getConf() {
+return util.getConfiguration();
+  }
+
+  public void setConf(Configuration conf) {
+throw new IllegalArgumentException("setConf not supported");
+  }
+
+  @BeforeClass
+  public static void provisionCluster() throws Exception {
+util.startMiniCluster();
+  }
+
+  @AfterClass
+  public static void releaseCluster() throws Exception {
+util.shutdownMiniCluster();
+  }
+
+  @Before
+  public void setup() throws Exception {
+tn = TableName.valueOf("test-" + UUID.randomUUID());
+args = new 

[31/41] hbase git commit: HBASE-18640 Move mapreduce out of hbase-server into separate module.

2017-08-25 Thread appy
http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
--
diff --git 
a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
new file mode 100644
index 000..e80410f
--- /dev/null
+++ 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
@@ -0,0 +1, @@
+/**
+ * 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.snapshot;
+
+import java.io.BufferedInputStream;
+import java.io.FileNotFoundException;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.Option;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileChecksum;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.io.FileLink;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.io.WALLink;
+import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
+import org.apache.hadoop.hbase.mob.MobUtils;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotFileInfo;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
+import org.apache.hadoop.hbase.util.AbstractHBaseTool;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.HFileArchiveUtil;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
+import org.apache.hadoop.mapreduce.security.TokenCache;
+import org.apache.hadoop.hbase.io.hadoopbackport.ThrottledInputStream;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.Tool;
+
+/**
+ * Export the specified snapshot to a given FileSystem.
+ *
+ * The .snapshot/name folder is copied to the destination cluster
+ * and then all the hfiles/wals are copied using a Map-Reduce Job in the 
.archive/ location.
+ * When everything is done, the second cluster can restore the snapshot.
+ */
+@InterfaceAudience.Public
+public class ExportSnapshot extends AbstractHBaseTool implements Tool {
+  public static final String NAME = "exportsnapshot";
+  /** Configuration prefix for overrides for the source filesystem */
+  public static final String CONF_SOURCE_PREFIX = NAME + ".from.";
+  /** Configuration prefix for overrides for the destination filesystem */
+  public static final String CONF_DEST_PREFIX = NAME + ".to.";
+
+  private static final Log LOG = LogFactory.getLog(ExportSnapshot.class);
+
+  private static final String MR_NUM_MAPS = "mapreduce.job.maps";
+  private static 

[26/41] hbase git commit: HBASE-18640 Move mapreduce out of hbase-server into separate module.

2017-08-25 Thread appy
http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java
--
diff --git 
a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java
 
b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java
new file mode 100644
index 000..91d2696
--- /dev/null
+++ 
b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java
@@ -0,0 +1,726 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.mapreduce;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+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.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.KeepDeletedCells;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Durability;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterBase;
+import org.apache.hadoop.hbase.filter.PrefixFilter;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.mapreduce.Import.KeyValueImporter;
+import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.LauncherSecurityManager;
+import org.apache.hadoop.mapreduce.Mapper.Context;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+/**
+ * Tests the table import and table export MR job functionality
+ */
+@Category({VerySlowMapReduceTests.class, MediumTests.class})
+public class TestImportExport {
+  private static final Log LOG = LogFactory.getLog(TestImportExport.class);
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+  private static final byte[] ROW1 = Bytes.toBytesBinary("\\x32row1");
+  private static final byte[] ROW2 = Bytes.toBytesBinary("\\x32row2");
+  private static final byte[] ROW3 = Bytes.toBytesBinary("\\x32row3");
+  private static final String FAMILYA_STRING = "a";
+  private static final String FAMILYB_STRING = "b";
+  private static final byte[] FAMILYA = Bytes.toBytes(FAMILYA_STRING);
+  private static final byte[] 

[18/41] hbase git commit: HBASE-18640 Move mapreduce out of hbase-server into separate module.

2017-08-25 Thread appy
http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java
deleted file mode 100644
index 7fea254..000
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java
+++ /dev/null
@@ -1,902 +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.mapreduce;
-
-import java.io.IOException;
-import java.io.UnsupportedEncodingException;
-import java.net.InetSocketAddress;
-import java.net.URLDecoder;
-import java.net.URLEncoder;
-import java.nio.charset.StandardCharsets;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.TreeSet;
-import java.util.UUID;
-import java.util.function.Function;
-import java.util.stream.Collectors;
-
-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.Cell;
-import org.apache.hadoop.hbase.CellComparator;
-import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.client.TableDescriptor;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
-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.RegionLocator;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.fs.HFileSystem;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionLocation;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.io.compress.Compression;
-import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
-import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
-import org.apache.hadoop.hbase.io.hfile.CacheConfig;
-import org.apache.hadoop.hbase.io.hfile.HFile;
-import org.apache.hadoop.hbase.io.hfile.HFileContext;
-import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
-import org.apache.hadoop.hbase.io.hfile.HFileWriterImpl;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValueUtil;
-import org.apache.hadoop.hbase.regionserver.BloomType;
-import org.apache.hadoop.hbase.regionserver.HStore;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
-import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.mapreduce.lib.partition.TotalOrderPartitioner;
-
-import 
org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
-
-/**
- * Writes HFiles. Passed Cells must arrive in order.
- * Writes current time as the sequence id for the file. Sets the major 
compacted
- * attribute on created @{link {@link HFile}s. Calling write(null,null) will 
forcibly roll
- * all HFiles being written.
- * 
- * Using this class as part of a MapReduce job is best done
- 

[05/41] hbase git commit: HBASE-18640 Move mapreduce out of hbase-server into separate module.

2017-08-25 Thread appy
http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java
--
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java
deleted file mode 100644
index efcf91e..000
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java
+++ /dev/null
@@ -1,571 +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.mapreduce;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.io.hfile.CacheConfig;
-import org.apache.hadoop.hbase.io.hfile.HFile;
-import org.apache.hadoop.hbase.io.hfile.HFileScanner;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.Utils.OutputFileUtils.OutputFilesFilter;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.rules.ExpectedException;
-
-@Category({VerySlowMapReduceTests.class, LargeTests.class})
-public class TestImportTsv implements Configurable {
-
-  private static final Log LOG = LogFactory.getLog(TestImportTsv.class);
-  protected static final String NAME = TestImportTsv.class.getSimpleName();
-  protected static HBaseTestingUtility util = new HBaseTestingUtility();
-
-  // Delete the tmp directory after running doMROnTableTest. Boolean. Default 
is true.
-  protected static final String DELETE_AFTER_LOAD_CONF = NAME + 
".deleteAfterLoad";
-
-  /**
-   * Force use of combiner in doMROnTableTest. Boolean. Default is true.
-   */
-  protected static final String FORCE_COMBINER_CONF = NAME + ".forceCombiner";
-
-  private final String FAMILY = "FAM";
-  private TableName tn;
-  private Map args;
-
-  @Rule
-  public ExpectedException exception = ExpectedException.none();
-
-  public Configuration getConf() {
-return util.getConfiguration();
-  }
-
-  public void setConf(Configuration conf) {
-throw new IllegalArgumentException("setConf not supported");
-  }
-
-  @BeforeClass
-  public static void provisionCluster() throws Exception {
-util.startMiniCluster();
-  }
-
-  @AfterClass
-  public static void releaseCluster() throws Exception {
-util.shutdownMiniCluster();
-  }
-
-  @Before
-  public void setup() throws Exception {
-tn = TableName.valueOf("test-" + UUID.randomUUID());
-args = new HashMap<>();
-  

[03/41] hbase git commit: HBASE-18640 Move mapreduce out of hbase-server into separate module.

2017-08-25 Thread appy
http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatScanBase.java
--
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatScanBase.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatScanBase.java
deleted file mode 100644
index 0f49333..000
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatScanBase.java
+++ /dev/null
@@ -1,287 +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.mapreduce;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.NavigableMap;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-
-
-/**
- * 
- * Tests various scan start and stop row scenarios. This is set in a scan and
- * tested in a MapReduce job to see if that is handed over and done properly
- * too.
- * 
- * 
- * This test is broken into two parts in order to side-step the test timeout
- * period of 900, as documented in HBASE-8326.
- * 
- */
-public abstract class TestTableInputFormatScanBase {
-
-  private static final Log LOG = 
LogFactory.getLog(TestTableInputFormatScanBase.class);
-  static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-
-  static final TableName TABLE_NAME = TableName.valueOf("scantest");
-  static final byte[][] INPUT_FAMILYS = {Bytes.toBytes("content1"), 
Bytes.toBytes("content2")};
-  static final String KEY_STARTROW = "startRow";
-  static final String KEY_LASTROW = "stpRow";
-
-  private static Table table = null;
-
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-// test intermittently fails under hadoop2 (2.0.2-alpha) if 
shortcircuit-read (scr) is on.
-// this turns it off for this test.  TODO: Figure out why scr breaks 
recovery. 
-System.setProperty("hbase.tests.use.shortcircuit.reads", "false");
-
-// switch TIF to log at DEBUG level
-TEST_UTIL.enableDebug(TableInputFormat.class);
-TEST_UTIL.enableDebug(TableInputFormatBase.class);
-// start mini hbase cluster
-TEST_UTIL.startMiniCluster(3);
-// create and fill table
-table = TEST_UTIL.createMultiRegionTable(TABLE_NAME, INPUT_FAMILYS);
-TEST_UTIL.loadTable(table, INPUT_FAMILYS, null, false);
-  }
-
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-TEST_UTIL.shutdownMiniCluster();
-  }
-
-  /**
-   * Pass the key and value to reduce.
-   */
-  public static class ScanMapper
-  extends TableMapper {
-
-/**
- * Pass the key and value to reduce.
- *
- * @param key  The key, here "aaa", "aab" etc.
- * @param value  The value is the same as the key.
- * @param context  The task context.
- * @throws IOException When reading the rows fails.
- */
-@Override
-public void map(ImmutableBytesWritable key, Result value,
-  Context context)
-throws IOException, InterruptedException {
-  if (value.size() != 2) {
-throw new IOException("There should be two input columns");
-  }
-  Map

[16/41] hbase git commit: HBASE-18640 Move mapreduce out of hbase-server into separate module.

2017-08-25 Thread appy
http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableInputFormatBase.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableInputFormatBase.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableInputFormatBase.java
deleted file mode 100644
index e18b3aa..000
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableInputFormatBase.java
+++ /dev/null
@@ -1,297 +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.mapreduce;
-
-import java.io.IOException;
-import java.text.MessageFormat;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HRegionLocation;
-import org.apache.hadoop.hbase.TableName;
-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.Result;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.util.RegionSizeCalculator;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-
-import java.util.Map;
-import java.util.HashMap;
-import java.util.Iterator;
-/**
- * A base for {@link MultiTableInputFormat}s. Receives a list of
- * {@link Scan} instances that define the input tables and
- * filters etc. Subclasses may use other TableRecordReader implementations.
- */
-@InterfaceAudience.Public
-public abstract class MultiTableInputFormatBase extends
-InputFormat {
-
-  private static final Log LOG = 
LogFactory.getLog(MultiTableInputFormatBase.class);
-
-  /** Holds the set of scans used to define the input. */
-  private List scans;
-
-  /** The reader scanning the table, can be a custom one. */
-  private TableRecordReader tableRecordReader = null;
-
-  /**
-   * Builds a TableRecordReader. If no TableRecordReader was provided, uses the
-   * default.
-   *
-   * @param split The split to work with.
-   * @param context The current context.
-   * @return The newly created record reader.
-   * @throws IOException When creating the reader fails.
-   * @throws InterruptedException when record reader initialization fails
-   * @see org.apache.hadoop.mapreduce.InputFormat#createRecordReader(
-   *  org.apache.hadoop.mapreduce.InputSplit,
-   *  org.apache.hadoop.mapreduce.TaskAttemptContext)
-   */
-  @Override
-  public RecordReader createRecordReader(
-  InputSplit split, TaskAttemptContext context)
-  throws IOException, InterruptedException {
-TableSplit tSplit = (TableSplit) split;
-LOG.info(MessageFormat.format("Input split length: {0} bytes.", 
tSplit.getLength()));
-
-if (tSplit.getTable() == null) {
-  throw new IOException("Cannot create a record reader because of a"
-  + " previous error. Please look at the previous logs lines from"
-  + " the task's full log for more details.");
-}
-final Connection connection = 
ConnectionFactory.createConnection(context.getConfiguration());
-Table table = connection.getTable(tSplit.getTable());
-
-if (this.tableRecordReader == null) {
-  this.tableRecordReader = new TableRecordReader();
-}
-final TableRecordReader trr = this.tableRecordReader;
-
-try {
-  Scan sc = tSplit.getScan();
-  sc.setStartRow(tSplit.getStartRow());
-  sc.setStopRow(tSplit.getEndRow());
-  trr.setScan(sc);
-  

[35/41] hbase git commit: HBASE-18640 Move mapreduce out of hbase-server into separate module.

2017-08-25 Thread appy
http://git-wip-us.apache.org/repos/asf/hbase/blob/664b6be0/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/SyncTable.java
--
diff --git 
a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/SyncTable.java
 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/SyncTable.java
new file mode 100644
index 000..c72a0c3
--- /dev/null
+++ 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/SyncTable.java
@@ -0,0 +1,786 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.mapreduce;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.Collections;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.mapreduce.Counters;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
+import org.apache.hadoop.util.GenericOptionsParser;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Throwables;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Iterators;
+
+public class SyncTable extends Configured implements Tool {
+
+  private static final Log LOG = LogFactory.getLog(SyncTable.class);
+
+  static final String SOURCE_HASH_DIR_CONF_KEY = "sync.table.source.hash.dir";
+  static final String SOURCE_TABLE_CONF_KEY = "sync.table.source.table.name";
+  static final String TARGET_TABLE_CONF_KEY = "sync.table.target.table.name";
+  static final String SOURCE_ZK_CLUSTER_CONF_KEY = 
"sync.table.source.zk.cluster";
+  static final String TARGET_ZK_CLUSTER_CONF_KEY = 
"sync.table.target.zk.cluster";
+  static final String DRY_RUN_CONF_KEY="sync.table.dry.run";
+
+  Path sourceHashDir;
+  String sourceTableName;
+  String targetTableName;
+
+  String sourceZkCluster;
+  String targetZkCluster;
+  boolean dryRun;
+
+  Counters counters;
+
+  public SyncTable(Configuration conf) {
+super(conf);
+  }
+
+  public Job createSubmittableJob(String[] args) throws IOException {
+FileSystem fs = sourceHashDir.getFileSystem(getConf());
+if (!fs.exists(sourceHashDir)) {
+  throw new IOException("Source hash dir not found: " + sourceHashDir);
+}
+
+HashTable.TableHash tableHash = HashTable.TableHash.read(getConf(), 
sourceHashDir);
+LOG.info("Read source hash manifest: " + tableHash);
+LOG.info("Read " + tableHash.partitions.size() + " partition keys");
+if (!tableHash.tableName.equals(sourceTableName)) {
+  LOG.warn("Table name mismatch - manifest indicates hash was taken from: "
+  + tableHash.tableName + " but job is reading from: " + 
sourceTableName);
+}
+if (tableHash.numHashFiles != tableHash.partitions.size() + 1) {
+  throw new RuntimeException("Hash data appears corrupt. The number of of 
hash files created"
+  + " should be 1 more than the number of partition keys.  However, 
the manifest file "
+  + " says numHashFiles=" + tableHash.numHashFiles + " but the number 
of partition keys"
+  + " found in the partitions file is " + 

[1/2] hbase git commit: HBASE-16324 Remove LegacyScanQueryMatcher

2017-08-25 Thread zhangduo
Repository: hbase
Updated Branches:
  refs/heads/branch-2 b55b952d5 -> 95bc46452


http://git-wip-us.apache.org/repos/asf/hbase/blob/95bc4645/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java
--
diff --git 
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
index 1653728..4082818 100644
--- 
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
@@ -19,8 +19,13 @@
 
 package org.apache.hadoop.hbase.regionserver;
 
+import static org.apache.hadoop.hbase.CellUtil.createCell;
+import static org.apache.hadoop.hbase.KeyValueTestUtil.create;
 import static 
org.apache.hadoop.hbase.regionserver.KeyValueScanFixture.scanFixture;
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -28,6 +33,7 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 import java.util.NavigableSet;
+import java.util.OptionalInt;
 import java.util.TreeSet;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -42,7 +48,6 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeepDeletedCells;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValueTestUtil;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.filter.ColumnCountGetFilter;
@@ -51,7 +56,6 @@ import 
org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdge;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
-import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -66,11 +70,10 @@ public class TestStoreScanner {
   @Rule public final TestRule timeout = 
CategoryBasedTimeout.builder().withTimeout(this.getClass()).
   withLookingForStuckThread(true).build();
   private static final String CF_STR = "cf";
-  private static final byte [] CF = Bytes.toBytes(CF_STR);
+  private static final byte[] CF = Bytes.toBytes(CF_STR);
   static Configuration CONF = HBaseConfiguration.create();
   private ScanInfo scanInfo = new ScanInfo(CONF, CF, 0, Integer.MAX_VALUE, 
Long.MAX_VALUE,
   KeepDeletedCells.FALSE, HConstants.DEFAULT_BLOCKSIZE, 0, 
CellComparator.COMPARATOR, false);
-  private ScanType scanType = ScanType.USER_SCAN;
 
   /**
* From here on down, we have a bunch of defines and specific CELL_GRID of 
Cells. The
@@ -79,15 +82,15 @@ public class TestStoreScanner {
* {@link 
StoreScanner#optimize(org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher.MatchCode,
* Cell)} is not overly enthusiastic.
*/
-  private static final byte [] ZERO = new byte [] {'0'};
-  private static final byte [] ZERO_POINT_ZERO = new byte [] {'0', '.', '0'};
-  private static final byte [] ONE = new byte [] {'1'};
-  private static final byte [] TWO = new byte [] {'2'};
-  private static final byte [] TWO_POINT_TWO = new byte [] {'2', '.', '2'};
-  private static final byte [] THREE = new byte [] {'3'};
-  private static final byte [] FOUR = new byte [] {'4'};
-  private static final byte [] FIVE = new byte [] {'5'};
-  private static final byte [] VALUE = new byte [] {'v'};
+  private static final byte[] ZERO = new byte[] {'0'};
+  private static final byte[] ZERO_POINT_ZERO = new byte[] {'0', '.', '0'};
+  private static final byte[] ONE = new byte[] {'1'};
+  private static final byte[] TWO = new byte[] {'2'};
+  private static final byte[] TWO_POINT_TWO = new byte[] {'2', '.', '2'};
+  private static final byte[] THREE = new byte[] {'3'};
+  private static final byte[] FOUR = new byte[] {'4'};
+  private static final byte[] FIVE = new byte[] {'5'};
+  private static final byte[] VALUE = new byte[] {'v'};
   private static final int CELL_GRID_BLOCK2_BOUNDARY = 4;
   private static final int CELL_GRID_BLOCK3_BOUNDARY = 11;
   private static final int CELL_GRID_BLOCK4_BOUNDARY = 15;
@@ -100,32 +103,32 @@ public class TestStoreScanner {
* We will use this to test scan does the right thing as it
* we do Gets, StoreScanner#optimize, and what we do on (faked) block 
boundaries.
*/
-  private static final Cell [] CELL_GRID = new Cell [] {
-CellUtil.createCell(ONE, CF, ONE, 1L, KeyValue.Type.Put.getCode(), VALUE),
-CellUtil.createCell(ONE, CF, TWO, 1L, KeyValue.Type.Put.getCode(), VALUE),
-

[2/2] hbase git commit: HBASE-16324 Remove LegacyScanQueryMatcher

2017-08-25 Thread zhangduo
HBASE-16324 Remove LegacyScanQueryMatcher


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

Branch: refs/heads/branch-2
Commit: 95bc4645271a5fe0c88b4bcce2561da189882034
Parents: b55b952
Author: zhangduo 
Authored: Fri Aug 25 17:02:03 2017 +0800
Committer: zhangduo 
Committed: Sat Aug 26 08:00:20 2017 +0800

--
 .../example/ZooKeeperScanPolicyObserver.java|  10 +-
 .../hbase/mob/DefaultMobStoreCompactor.java |   6 +-
 .../compactions/PartitionedMobCompactor.java|   8 +-
 .../hadoop/hbase/regionserver/HMobStore.java|   1 -
 .../MemStoreCompactorSegmentsIterator.java  |  26 +-
 .../regionserver/ReversedStoreScanner.java  |   8 +-
 .../hadoop/hbase/regionserver/StoreFlusher.java |  10 +-
 .../hadoop/hbase/regionserver/StoreScanner.java | 217 +++---
 .../regionserver/compactions/Compactor.java |  18 +-
 .../querymatcher/LegacyScanQueryMatcher.java| 384 ---
 ...estAvoidCellReferencesIntoShippedBlocks.java |  11 +-
 .../hadoop/hbase/client/TestFromClientSide.java |   4 +-
 .../TestRegionObserverScannerOpenHook.java  |  31 +-
 .../TestPartitionedMobCompactor.java|   6 +-
 .../regionserver/NoOpScanPolicyObserver.java|  24 +-
 .../regionserver/TestCompactingMemStore.java|  34 +-
 .../hbase/regionserver/TestDefaultMemStore.java |  66 +-
 .../regionserver/TestMobStoreCompaction.java|   5 +-
 .../regionserver/TestReversibleScanners.java|  22 +-
 .../hbase/regionserver/TestStoreScanner.java| 682 +--
 .../hbase/util/TestCoprocessorScanPolicy.java   |  24 +-
 21 files changed, 552 insertions(+), 1045 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/95bc4645/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java
--
diff --git 
a/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java
 
b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java
index 35f85f7..b489fe4 100644
--- 
a/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java
+++ 
b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java
@@ -19,9 +19,9 @@
 package org.apache.hadoop.hbase.coprocessor.example;
 
 import java.io.IOException;
-import java.util.Collections;
 import java.util.List;
 import java.util.NavigableSet;
+import java.util.OptionalInt;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -194,9 +194,7 @@ public class ZooKeeperScanPolicyObserver implements 
RegionObserver {
   // take default action
   return null;
 }
-Scan scan = new Scan();
-scan.setMaxVersions(scanInfo.getMaxVersions());
-return new StoreScanner(store, scanInfo, scan, scanners,
+return new StoreScanner(store, scanInfo, OptionalInt.empty(), scanners,
 ScanType.COMPACT_RETAIN_DELETES, store.getSmallestReadPoint(), 
HConstants.OLDEST_TIMESTAMP);
   }
 
@@ -210,9 +208,7 @@ public class ZooKeeperScanPolicyObserver implements 
RegionObserver {
   // take default action
   return null;
 }
-Scan scan = new Scan();
-scan.setMaxVersions(scanInfo.getMaxVersions());
-return new StoreScanner(store, scanInfo, scan, scanners, scanType,
+return new StoreScanner(store, scanInfo, OptionalInt.empty(), scanners, 
scanType,
 store.getSmallestReadPoint(), earliestPutTs);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/95bc4645/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java
index c475b17..89d2958 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java
@@ -22,6 +22,7 @@ import java.io.InterruptedIOException;
 import java.util.ArrayList;
 import java.util.Date;
 import java.util.List;
+import java.util.OptionalInt;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -32,7 +33,6 @@ import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import 

[2/2] hbase git commit: HBASE-16324 Remove LegacyScanQueryMatcher

2017-08-25 Thread zhangduo
HBASE-16324 Remove LegacyScanQueryMatcher


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

Branch: refs/heads/master
Commit: 8d33949b8db072902783f63cd9aaa68cbd6b905f
Parents: 2773510
Author: zhangduo 
Authored: Fri Aug 25 17:02:03 2017 +0800
Committer: zhangduo 
Committed: Sat Aug 26 08:04:43 2017 +0800

--
 .../example/ZooKeeperScanPolicyObserver.java|  10 +-
 .../hbase/mob/DefaultMobStoreCompactor.java |   6 +-
 .../compactions/PartitionedMobCompactor.java|   8 +-
 .../hadoop/hbase/regionserver/HMobStore.java|   1 -
 .../MemStoreCompactorSegmentsIterator.java  |  26 +-
 .../regionserver/ReversedStoreScanner.java  |   8 +-
 .../hadoop/hbase/regionserver/StoreFlusher.java |  10 +-
 .../hadoop/hbase/regionserver/StoreScanner.java | 217 +++---
 .../regionserver/compactions/Compactor.java |  18 +-
 .../querymatcher/LegacyScanQueryMatcher.java| 384 ---
 ...estAvoidCellReferencesIntoShippedBlocks.java |  11 +-
 .../hadoop/hbase/client/TestFromClientSide.java |   4 +-
 .../TestRegionObserverScannerOpenHook.java  |  31 +-
 .../TestPartitionedMobCompactor.java|   6 +-
 .../regionserver/NoOpScanPolicyObserver.java|  24 +-
 .../regionserver/TestCompactingMemStore.java|  34 +-
 .../hbase/regionserver/TestDefaultMemStore.java |  66 +-
 .../regionserver/TestMobStoreCompaction.java|   5 +-
 .../regionserver/TestReversibleScanners.java|  22 +-
 .../hbase/regionserver/TestStoreScanner.java| 682 +--
 .../hbase/util/TestCoprocessorScanPolicy.java   |  24 +-
 21 files changed, 552 insertions(+), 1045 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/8d33949b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java
--
diff --git 
a/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java
 
b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java
index 35f85f7..b489fe4 100644
--- 
a/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java
+++ 
b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java
@@ -19,9 +19,9 @@
 package org.apache.hadoop.hbase.coprocessor.example;
 
 import java.io.IOException;
-import java.util.Collections;
 import java.util.List;
 import java.util.NavigableSet;
+import java.util.OptionalInt;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -194,9 +194,7 @@ public class ZooKeeperScanPolicyObserver implements 
RegionObserver {
   // take default action
   return null;
 }
-Scan scan = new Scan();
-scan.setMaxVersions(scanInfo.getMaxVersions());
-return new StoreScanner(store, scanInfo, scan, scanners,
+return new StoreScanner(store, scanInfo, OptionalInt.empty(), scanners,
 ScanType.COMPACT_RETAIN_DELETES, store.getSmallestReadPoint(), 
HConstants.OLDEST_TIMESTAMP);
   }
 
@@ -210,9 +208,7 @@ public class ZooKeeperScanPolicyObserver implements 
RegionObserver {
   // take default action
   return null;
 }
-Scan scan = new Scan();
-scan.setMaxVersions(scanInfo.getMaxVersions());
-return new StoreScanner(store, scanInfo, scan, scanners, scanType,
+return new StoreScanner(store, scanInfo, OptionalInt.empty(), scanners, 
scanType,
 store.getSmallestReadPoint(), earliestPutTs);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/8d33949b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java
index c475b17..89d2958 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java
@@ -22,6 +22,7 @@ import java.io.InterruptedIOException;
 import java.util.ArrayList;
 import java.util.Date;
 import java.util.List;
+import java.util.OptionalInt;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -32,7 +33,6 @@ import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import 

[1/2] hbase git commit: HBASE-16324 Remove LegacyScanQueryMatcher

2017-08-25 Thread zhangduo
Repository: hbase
Updated Branches:
  refs/heads/master 2773510f1 -> 8d33949b8


http://git-wip-us.apache.org/repos/asf/hbase/blob/8d33949b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java
--
diff --git 
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
index 1653728..4082818 100644
--- 
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
@@ -19,8 +19,13 @@
 
 package org.apache.hadoop.hbase.regionserver;
 
+import static org.apache.hadoop.hbase.CellUtil.createCell;
+import static org.apache.hadoop.hbase.KeyValueTestUtil.create;
 import static 
org.apache.hadoop.hbase.regionserver.KeyValueScanFixture.scanFixture;
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -28,6 +33,7 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 import java.util.NavigableSet;
+import java.util.OptionalInt;
 import java.util.TreeSet;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -42,7 +48,6 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeepDeletedCells;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValueTestUtil;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.filter.ColumnCountGetFilter;
@@ -51,7 +56,6 @@ import 
org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdge;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
-import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -66,11 +70,10 @@ public class TestStoreScanner {
   @Rule public final TestRule timeout = 
CategoryBasedTimeout.builder().withTimeout(this.getClass()).
   withLookingForStuckThread(true).build();
   private static final String CF_STR = "cf";
-  private static final byte [] CF = Bytes.toBytes(CF_STR);
+  private static final byte[] CF = Bytes.toBytes(CF_STR);
   static Configuration CONF = HBaseConfiguration.create();
   private ScanInfo scanInfo = new ScanInfo(CONF, CF, 0, Integer.MAX_VALUE, 
Long.MAX_VALUE,
   KeepDeletedCells.FALSE, HConstants.DEFAULT_BLOCKSIZE, 0, 
CellComparator.COMPARATOR, false);
-  private ScanType scanType = ScanType.USER_SCAN;
 
   /**
* From here on down, we have a bunch of defines and specific CELL_GRID of 
Cells. The
@@ -79,15 +82,15 @@ public class TestStoreScanner {
* {@link 
StoreScanner#optimize(org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher.MatchCode,
* Cell)} is not overly enthusiastic.
*/
-  private static final byte [] ZERO = new byte [] {'0'};
-  private static final byte [] ZERO_POINT_ZERO = new byte [] {'0', '.', '0'};
-  private static final byte [] ONE = new byte [] {'1'};
-  private static final byte [] TWO = new byte [] {'2'};
-  private static final byte [] TWO_POINT_TWO = new byte [] {'2', '.', '2'};
-  private static final byte [] THREE = new byte [] {'3'};
-  private static final byte [] FOUR = new byte [] {'4'};
-  private static final byte [] FIVE = new byte [] {'5'};
-  private static final byte [] VALUE = new byte [] {'v'};
+  private static final byte[] ZERO = new byte[] {'0'};
+  private static final byte[] ZERO_POINT_ZERO = new byte[] {'0', '.', '0'};
+  private static final byte[] ONE = new byte[] {'1'};
+  private static final byte[] TWO = new byte[] {'2'};
+  private static final byte[] TWO_POINT_TWO = new byte[] {'2', '.', '2'};
+  private static final byte[] THREE = new byte[] {'3'};
+  private static final byte[] FOUR = new byte[] {'4'};
+  private static final byte[] FIVE = new byte[] {'5'};
+  private static final byte[] VALUE = new byte[] {'v'};
   private static final int CELL_GRID_BLOCK2_BOUNDARY = 4;
   private static final int CELL_GRID_BLOCK3_BOUNDARY = 11;
   private static final int CELL_GRID_BLOCK4_BOUNDARY = 15;
@@ -100,32 +103,32 @@ public class TestStoreScanner {
* We will use this to test scan does the right thing as it
* we do Gets, StoreScanner#optimize, and what we do on (faked) block 
boundaries.
*/
-  private static final Cell [] CELL_GRID = new Cell [] {
-CellUtil.createCell(ONE, CF, ONE, 1L, KeyValue.Type.Put.getCode(), VALUE),
-CellUtil.createCell(ONE, CF, TWO, 1L, KeyValue.Type.Put.getCode(), VALUE),
-

[5/7] hbase git commit: HBASE-18679 Add a null check around the result of getCounters() in ITBLL

2017-08-25 Thread elserj
HBASE-18679 Add a null check around the result of getCounters() in ITBLL


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

Branch: refs/heads/branch-1.3
Commit: 4ce79774a13450f8a2f44cc3c4e437371d80f0fb
Parents: 22e8f87
Author: Josh Elser 
Authored: Thu Aug 24 17:52:13 2017 -0400
Committer: Josh Elser 
Committed: Fri Aug 25 19:05:46 2017 -0400

--
 .../hbase/test/IntegrationTestBigLinkedList.java   | 13 -
 1 file changed, 12 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/4ce79774/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
--
diff --git 
a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
 
b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
index 4ab0a58..0e3b198 100644
--- 
a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
+++ 
b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
@@ -815,6 +815,11 @@ public class IntegrationTestBigLinkedList extends 
IntegrationTestBase {
 public boolean verify() {
   try {
 Counters counters = job.getCounters();
+if (counters == null) {
+  LOG.info("Counters object was null, Generator verification cannot be 
performed."
+  + " This is commonly a result of insufficient YARN 
configuration.");
+  return false;
+}
 
 if (counters.findCounter(Counts.TERMINATING).getValue() > 0 ||
 counters.findCounter(Counts.UNDEFINED).getValue() > 0 ||
@@ -1306,7 +1311,8 @@ public class IntegrationTestBigLinkedList extends 
IntegrationTestBase {
   if (success) {
 Counters counters = job.getCounters();
 if (null == counters) {
-  LOG.warn("Counters were null, cannot verify Job completion");
+  LOG.warn("Counters were null, cannot verify Job completion."
+  + " This is commonly a result of insufficient YARN 
configuration.");
   // We don't have access to the counters to know if we have "bad" 
counts
   return 0;
 }
@@ -1328,6 +1334,11 @@ public class IntegrationTestBigLinkedList extends 
IntegrationTestBase {
   }
 
   Counters counters = job.getCounters();
+  if (counters == null) {
+LOG.info("Counters object was null, write verification cannot be 
performed."
+  + " This is commonly a result of insufficient YARN 
configuration.");
+return false;
+  }
 
   // Run through each check, even if we fail one early
   boolean success = verifyExpectedValues(expectedReferenced, counters);



[6/7] hbase git commit: HBASE-18679 Add a null check around the result of getCounters() in ITBLL

2017-08-25 Thread elserj
HBASE-18679 Add a null check around the result of getCounters() in ITBLL


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

Branch: refs/heads/branch-1.2
Commit: 5c6fb74b5fd835ea20ba984cd63749d0d1c70630
Parents: 1347635
Author: Josh Elser 
Authored: Thu Aug 24 17:52:13 2017 -0400
Committer: Josh Elser 
Committed: Fri Aug 25 19:09:00 2017 -0400

--
 .../hadoop/hbase/test/IntegrationTestBigLinkedList.java  | 8 +++-
 1 file changed, 7 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/5c6fb74b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
--
diff --git 
a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
 
b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
index 91f99f1..45b0d03 100644
--- 
a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
+++ 
b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
@@ -1106,7 +1106,8 @@ public class IntegrationTestBigLinkedList extends 
IntegrationTestBase {
   if (success) {
 Counters counters = job.getCounters();
 if (null == counters) {
-  LOG.warn("Counters were null, cannot verify Job completion");
+  LOG.warn("Counters were null, cannot verify Job completion."
+  + " This is commonly a result of insufficient YARN 
configuration.");
   // We don't have access to the counters to know if we have "bad" 
counts
   return 0;
 }
@@ -1128,6 +1129,11 @@ public class IntegrationTestBigLinkedList extends 
IntegrationTestBase {
   }
 
   Counters counters = job.getCounters();
+  if (counters == null) {
+LOG.info("Counters object was null, write verification cannot be 
performed."
+  + " This is commonly a result of insufficient YARN 
configuration.");
+return false;
+  }
 
   // Run through each check, even if we fail one early
   boolean success = verifyExpectedValues(expectedReferenced, counters);



[7/7] hbase git commit: HBASE-18679 Add a null check around the result of getCounters() in ITBLL

2017-08-25 Thread elserj
HBASE-18679 Add a null check around the result of getCounters() in ITBLL


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

Branch: refs/heads/branch-1.1
Commit: 8d6e37bb260a787e7d5d97495f5cc1ca260695d7
Parents: 43b0a66
Author: Josh Elser 
Authored: Thu Aug 24 17:52:13 2017 -0400
Committer: Josh Elser 
Committed: Fri Aug 25 19:11:49 2017 -0400

--
 .../hadoop/hbase/test/IntegrationTestBigLinkedList.java  | 8 +++-
 1 file changed, 7 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/8d6e37bb/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
--
diff --git 
a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
 
b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
index 849efa4..7aed9de 100644
--- 
a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
+++ 
b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
@@ -1104,7 +1104,8 @@ public class IntegrationTestBigLinkedList extends 
IntegrationTestBase {
   if (success) {
 Counters counters = job.getCounters();
 if (null == counters) {
-  LOG.warn("Counters were null, cannot verify Job completion");
+  LOG.warn("Counters were null, cannot verify Job completion."
+  + " This is commonly a result of insufficient YARN 
configuration.");
   // We don't have access to the counters to know if we have "bad" 
counts
   return 0;
 }
@@ -1126,6 +1127,11 @@ public class IntegrationTestBigLinkedList extends 
IntegrationTestBase {
   }
 
   Counters counters = job.getCounters();
+  if (counters == null) {
+LOG.info("Counters object was null, write verification cannot be 
performed."
+  + " This is commonly a result of insufficient YARN 
configuration.");
+return false;
+  }
 
   // Run through each check, even if we fail one early
   boolean success = verifyExpectedValues(expectedReferenced, counters);



[1/7] hbase git commit: HBASE-18679 Add a null check around the result of getCounters() in ITBLL

2017-08-25 Thread elserj
Repository: hbase
Updated Branches:
  refs/heads/branch-1 50c67e969 -> fd4387998
  refs/heads/branch-1.1 43b0a6612 -> 8d6e37bb2
  refs/heads/branch-1.2 134763530 -> 5c6fb74b5
  refs/heads/branch-1.3 22e8f87ae -> 4ce79774a
  refs/heads/branch-1.4 ca6714d54 -> a02521929
  refs/heads/branch-2 005693f0c -> b55b952d5
  refs/heads/master 439191ece -> 2773510f1


HBASE-18679 Add a null check around the result of getCounters() in ITBLL


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

Branch: refs/heads/master
Commit: 2773510f120730f926569fef30c3e7b766517e89
Parents: 439191e
Author: Josh Elser 
Authored: Thu Aug 24 17:52:13 2017 -0400
Committer: Josh Elser 
Committed: Fri Aug 25 18:40:02 2017 -0400

--
 .../hbase/test/IntegrationTestBigLinkedList.java   | 13 -
 1 file changed, 12 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/2773510f/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
--
diff --git 
a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
 
b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
index 2fdfab6..f05ef66 100644
--- 
a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
+++ 
b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
@@ -820,6 +820,11 @@ public class IntegrationTestBigLinkedList extends 
IntegrationTestBase {
 public boolean verify() {
   try {
 Counters counters = job.getCounters();
+if (counters == null) {
+  LOG.info("Counters object was null, Generator verification cannot be 
performed."
+  + " This is commonly a result of insufficient YARN 
configuration.");
+  return false;
+}
 
 if (counters.findCounter(Counts.TERMINATING).getValue() > 0 ||
 counters.findCounter(Counts.UNDEFINED).getValue() > 0 ||
@@ -1315,7 +1320,8 @@ public class IntegrationTestBigLinkedList extends 
IntegrationTestBase {
   if (success) {
 Counters counters = job.getCounters();
 if (null == counters) {
-  LOG.warn("Counters were null, cannot verify Job completion");
+  LOG.warn("Counters were null, cannot verify Job completion."
+  + " This is commonly a result of insufficient YARN 
configuration.");
   // We don't have access to the counters to know if we have "bad" 
counts
   return 0;
 }
@@ -1337,6 +1343,11 @@ public class IntegrationTestBigLinkedList extends 
IntegrationTestBase {
   }
 
   Counters counters = job.getCounters();
+  if (counters == null) {
+LOG.info("Counters object was null, write verification cannot be 
performed."
+  + " This is commonly a result of insufficient YARN 
configuration.");
+return false;
+  }
 
   // Run through each check, even if we fail one early
   boolean success = verifyExpectedValues(expectedReferenced, counters);



[4/7] hbase git commit: HBASE-18679 Add a null check around the result of getCounters() in ITBLL

2017-08-25 Thread elserj
HBASE-18679 Add a null check around the result of getCounters() in ITBLL


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

Branch: refs/heads/branch-1.4
Commit: a02521929806ff065753418e00292d422350a3e1
Parents: ca6714d
Author: Josh Elser 
Authored: Thu Aug 24 17:52:13 2017 -0400
Committer: Josh Elser 
Committed: Fri Aug 25 18:59:19 2017 -0400

--
 .../hbase/test/IntegrationTestBigLinkedList.java   | 13 -
 1 file changed, 12 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/a0252192/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
--
diff --git 
a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
 
b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
index 4ab0a58..0e3b198 100644
--- 
a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
+++ 
b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
@@ -815,6 +815,11 @@ public class IntegrationTestBigLinkedList extends 
IntegrationTestBase {
 public boolean verify() {
   try {
 Counters counters = job.getCounters();
+if (counters == null) {
+  LOG.info("Counters object was null, Generator verification cannot be 
performed."
+  + " This is commonly a result of insufficient YARN 
configuration.");
+  return false;
+}
 
 if (counters.findCounter(Counts.TERMINATING).getValue() > 0 ||
 counters.findCounter(Counts.UNDEFINED).getValue() > 0 ||
@@ -1306,7 +1311,8 @@ public class IntegrationTestBigLinkedList extends 
IntegrationTestBase {
   if (success) {
 Counters counters = job.getCounters();
 if (null == counters) {
-  LOG.warn("Counters were null, cannot verify Job completion");
+  LOG.warn("Counters were null, cannot verify Job completion."
+  + " This is commonly a result of insufficient YARN 
configuration.");
   // We don't have access to the counters to know if we have "bad" 
counts
   return 0;
 }
@@ -1328,6 +1334,11 @@ public class IntegrationTestBigLinkedList extends 
IntegrationTestBase {
   }
 
   Counters counters = job.getCounters();
+  if (counters == null) {
+LOG.info("Counters object was null, write verification cannot be 
performed."
+  + " This is commonly a result of insufficient YARN 
configuration.");
+return false;
+  }
 
   // Run through each check, even if we fail one early
   boolean success = verifyExpectedValues(expectedReferenced, counters);



[2/7] hbase git commit: HBASE-18679 Add a null check around the result of getCounters() in ITBLL

2017-08-25 Thread elserj
HBASE-18679 Add a null check around the result of getCounters() in ITBLL


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

Branch: refs/heads/branch-2
Commit: b55b952d5c5e90bde03f454500e867b7909fea75
Parents: 005693f
Author: Josh Elser 
Authored: Thu Aug 24 17:52:13 2017 -0400
Committer: Josh Elser 
Committed: Fri Aug 25 18:45:12 2017 -0400

--
 .../hbase/test/IntegrationTestBigLinkedList.java   | 13 -
 1 file changed, 12 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/b55b952d/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
--
diff --git 
a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
 
b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
index 2fdfab6..f05ef66 100644
--- 
a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
+++ 
b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
@@ -820,6 +820,11 @@ public class IntegrationTestBigLinkedList extends 
IntegrationTestBase {
 public boolean verify() {
   try {
 Counters counters = job.getCounters();
+if (counters == null) {
+  LOG.info("Counters object was null, Generator verification cannot be 
performed."
+  + " This is commonly a result of insufficient YARN 
configuration.");
+  return false;
+}
 
 if (counters.findCounter(Counts.TERMINATING).getValue() > 0 ||
 counters.findCounter(Counts.UNDEFINED).getValue() > 0 ||
@@ -1315,7 +1320,8 @@ public class IntegrationTestBigLinkedList extends 
IntegrationTestBase {
   if (success) {
 Counters counters = job.getCounters();
 if (null == counters) {
-  LOG.warn("Counters were null, cannot verify Job completion");
+  LOG.warn("Counters were null, cannot verify Job completion."
+  + " This is commonly a result of insufficient YARN 
configuration.");
   // We don't have access to the counters to know if we have "bad" 
counts
   return 0;
 }
@@ -1337,6 +1343,11 @@ public class IntegrationTestBigLinkedList extends 
IntegrationTestBase {
   }
 
   Counters counters = job.getCounters();
+  if (counters == null) {
+LOG.info("Counters object was null, write verification cannot be 
performed."
+  + " This is commonly a result of insufficient YARN 
configuration.");
+return false;
+  }
 
   // Run through each check, even if we fail one early
   boolean success = verifyExpectedValues(expectedReferenced, counters);



[3/7] hbase git commit: HBASE-18679 Add a null check around the result of getCounters() in ITBLL

2017-08-25 Thread elserj
HBASE-18679 Add a null check around the result of getCounters() in ITBLL


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

Branch: refs/heads/branch-1
Commit: fd43879985547eb05e04bed4c91feea7eac977c3
Parents: 50c67e9
Author: Josh Elser 
Authored: Thu Aug 24 17:52:13 2017 -0400
Committer: Josh Elser 
Committed: Fri Aug 25 18:54:01 2017 -0400

--
 .../hbase/test/IntegrationTestBigLinkedList.java   | 13 -
 1 file changed, 12 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/fd438799/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
--
diff --git 
a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
 
b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
index 4ab0a58..0e3b198 100644
--- 
a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
+++ 
b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
@@ -815,6 +815,11 @@ public class IntegrationTestBigLinkedList extends 
IntegrationTestBase {
 public boolean verify() {
   try {
 Counters counters = job.getCounters();
+if (counters == null) {
+  LOG.info("Counters object was null, Generator verification cannot be 
performed."
+  + " This is commonly a result of insufficient YARN 
configuration.");
+  return false;
+}
 
 if (counters.findCounter(Counts.TERMINATING).getValue() > 0 ||
 counters.findCounter(Counts.UNDEFINED).getValue() > 0 ||
@@ -1306,7 +1311,8 @@ public class IntegrationTestBigLinkedList extends 
IntegrationTestBase {
   if (success) {
 Counters counters = job.getCounters();
 if (null == counters) {
-  LOG.warn("Counters were null, cannot verify Job completion");
+  LOG.warn("Counters were null, cannot verify Job completion."
+  + " This is commonly a result of insufficient YARN 
configuration.");
   // We don't have access to the counters to know if we have "bad" 
counts
   return 0;
 }
@@ -1328,6 +1334,11 @@ public class IntegrationTestBigLinkedList extends 
IntegrationTestBase {
   }
 
   Counters counters = job.getCounters();
+  if (counters == null) {
+LOG.info("Counters object was null, write verification cannot be 
performed."
+  + " This is commonly a result of insufficient YARN 
configuration.");
+return false;
+  }
 
   // Run through each check, even if we fail one early
   boolean success = verifyExpectedValues(expectedReferenced, counters);



hbase git commit: HBASE-18687 Add @since 2.0.0 to new classes; AMENDMENT2

2017-08-25 Thread stack
Repository: hbase
Updated Branches:
  refs/heads/branch-2 34ca89e83 -> 005693f0c


HBASE-18687 Add @since 2.0.0 to new classes; AMENDMENT2


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

Branch: refs/heads/branch-2
Commit: 005693f0cae80432dc5ad3f419a960afcdebddc7
Parents: 34ca89e
Author: Michael Stack 
Authored: Fri Aug 25 14:44:01 2017 -0700
Committer: Michael Stack 
Committed: Fri Aug 25 14:44:28 2017 -0700

--
 .../java/org/apache/hadoop/hbase/backup/BackupClientFactory.java   | 2 ++
 1 file changed, 2 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/005693f0/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java
--
diff --git 
a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java
 
b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java
index 6db39f8..22e69a3 100644
--- 
a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java
+++ 
b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java
@@ -25,6 +25,8 @@ import 
org.apache.hadoop.hbase.backup.impl.IncrementalTableBackupClient;
 import org.apache.hadoop.hbase.backup.impl.TableBackupClient;
 import org.apache.hadoop.hbase.client.Connection;
 
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
 @InterfaceAudience.Private
 public class BackupClientFactory {
 



hbase git commit: HBASE-18687 Add @since 2.0.0 to new classes; AMENDMENT2

2017-08-25 Thread stack
Repository: hbase
Updated Branches:
  refs/heads/master 6859d4e20 -> 439191ece


HBASE-18687 Add @since 2.0.0 to new classes; AMENDMENT2


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

Branch: refs/heads/master
Commit: 439191ece6e024dd736cff000109748672219d18
Parents: 6859d4e
Author: Michael Stack 
Authored: Fri Aug 25 14:44:01 2017 -0700
Committer: Michael Stack 
Committed: Fri Aug 25 14:44:01 2017 -0700

--
 .../java/org/apache/hadoop/hbase/backup/BackupClientFactory.java   | 2 ++
 1 file changed, 2 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/439191ec/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java
--
diff --git 
a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java
 
b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java
index 6db39f8..22e69a3 100644
--- 
a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java
+++ 
b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java
@@ -25,6 +25,8 @@ import 
org.apache.hadoop.hbase.backup.impl.IncrementalTableBackupClient;
 import org.apache.hadoop.hbase.backup.impl.TableBackupClient;
 import org.apache.hadoop.hbase.client.Connection;
 
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
 @InterfaceAudience.Private
 public class BackupClientFactory {
 



hbase git commit: HBASE-18687 Add @since 2.0.0 to new classes; AMENDMENT

2017-08-25 Thread stack
Repository: hbase
Updated Branches:
  refs/heads/branch-2 3e1c598d8 -> 34ca89e83


HBASE-18687 Add @since 2.0.0 to new classes; AMENDMENT


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

Branch: refs/heads/branch-2
Commit: 34ca89e83e062f776ae3d7a384219cfcb31b539e
Parents: 3e1c598
Author: Michael Stack 
Authored: Fri Aug 25 14:14:51 2017 -0700
Committer: Michael Stack 
Committed: Fri Aug 25 14:15:51 2017 -0700

--
 .../java/org/apache/hadoop/hbase/backup/BackupClientFactory.java  | 1 +
 .../java/org/apache/hadoop/hbase/filter/BinaryComparator.java | 1 +
 .../main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java  | 1 +
 .../apache/hadoop/hbase/security/AbstractHBaseSaslRpcClient.java  | 1 +
 .../apache/hadoop/hbase/security/access/AccessControlUtil.java| 3 +++
 5 files changed, 7 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/34ca89e8/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java
--
diff --git 
a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java
 
b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java
index 21d73cc..6db39f8 100644
--- 
a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java
+++ 
b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java
@@ -25,6 +25,7 @@ import 
org.apache.hadoop.hbase.backup.impl.IncrementalTableBackupClient;
 import org.apache.hadoop.hbase.backup.impl.TableBackupClient;
 import org.apache.hadoop.hbase.client.Connection;
 
+@InterfaceAudience.Private
 public class BackupClientFactory {
 
   public static TableBackupClient create (Connection conn, String backupId, 
BackupRequest request)

http://git-wip-us.apache.org/repos/asf/hbase/blob/34ca89e8/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryComparator.java
--
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryComparator.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryComparator.java
index 87b622c..8a4aa34 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryComparator.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryComparator.java
@@ -33,6 +33,7 @@ import 
org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferE
 /**
  * A binary comparator which lexicographically compares against the specified
  * byte array using {@link 
org.apache.hadoop.hbase.util.Bytes#compareTo(byte[], byte[])}.
+ * @since 2.0.0
  */
 @InterfaceAudience.Public
 public class BinaryComparator extends 
org.apache.hadoop.hbase.filter.ByteArrayComparable {

http://git-wip-us.apache.org/repos/asf/hbase/blob/34ca89e8/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 7925505..12c829e 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
@@ -89,6 +89,7 @@ import org.apache.hadoop.security.token.TokenSelector;
  * outside the lock in {@link Call} and {@link HBaseRpcController} which means 
the implementations
  * of the callbacks are free to hold any lock.
  * 
+ * @since 2.0.0
  */
 @InterfaceAudience.Private
 public abstract class AbstractRpcClient implements 
RpcClient {

http://git-wip-us.apache.org/repos/asf/hbase/blob/34ca89e8/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AbstractHBaseSaslRpcClient.java
--
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AbstractHBaseSaslRpcClient.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AbstractHBaseSaslRpcClient.java
index cd2f4cd..de2c96e 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AbstractHBaseSaslRpcClient.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AbstractHBaseSaslRpcClient.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.security.token.TokenIdentifier;
 /**
  * A utility class that encapsulates SASL logic for RPC client. Copied from
  * org.apache.hadoop.security
+ * @since 2.0.0
  */
 

hbase git commit: HBASE-18687 Add @since 2.0.0 to new classes; AMENDMENT

2017-08-25 Thread stack
Repository: hbase
Updated Branches:
  refs/heads/master e62fdd9db -> 6859d4e20


HBASE-18687 Add @since 2.0.0 to new classes; AMENDMENT


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

Branch: refs/heads/master
Commit: 6859d4e207b261a8afb09eed9c94c94a7d7425ad
Parents: e62fdd9
Author: Michael Stack 
Authored: Fri Aug 25 14:14:51 2017 -0700
Committer: Michael Stack 
Committed: Fri Aug 25 14:14:51 2017 -0700

--
 .../java/org/apache/hadoop/hbase/backup/BackupClientFactory.java  | 1 +
 .../java/org/apache/hadoop/hbase/filter/BinaryComparator.java | 1 +
 .../main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java  | 1 +
 .../apache/hadoop/hbase/security/AbstractHBaseSaslRpcClient.java  | 1 +
 .../apache/hadoop/hbase/security/access/AccessControlUtil.java| 3 +++
 5 files changed, 7 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/6859d4e2/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java
--
diff --git 
a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java
 
b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java
index 21d73cc..6db39f8 100644
--- 
a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java
+++ 
b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java
@@ -25,6 +25,7 @@ import 
org.apache.hadoop.hbase.backup.impl.IncrementalTableBackupClient;
 import org.apache.hadoop.hbase.backup.impl.TableBackupClient;
 import org.apache.hadoop.hbase.client.Connection;
 
+@InterfaceAudience.Private
 public class BackupClientFactory {
 
   public static TableBackupClient create (Connection conn, String backupId, 
BackupRequest request)

http://git-wip-us.apache.org/repos/asf/hbase/blob/6859d4e2/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryComparator.java
--
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryComparator.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryComparator.java
index 87b622c..8a4aa34 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryComparator.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryComparator.java
@@ -33,6 +33,7 @@ import 
org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferE
 /**
  * A binary comparator which lexicographically compares against the specified
  * byte array using {@link 
org.apache.hadoop.hbase.util.Bytes#compareTo(byte[], byte[])}.
+ * @since 2.0.0
  */
 @InterfaceAudience.Public
 public class BinaryComparator extends 
org.apache.hadoop.hbase.filter.ByteArrayComparable {

http://git-wip-us.apache.org/repos/asf/hbase/blob/6859d4e2/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 7925505..12c829e 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
@@ -89,6 +89,7 @@ import org.apache.hadoop.security.token.TokenSelector;
  * outside the lock in {@link Call} and {@link HBaseRpcController} which means 
the implementations
  * of the callbacks are free to hold any lock.
  * 
+ * @since 2.0.0
  */
 @InterfaceAudience.Private
 public abstract class AbstractRpcClient implements 
RpcClient {

http://git-wip-us.apache.org/repos/asf/hbase/blob/6859d4e2/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AbstractHBaseSaslRpcClient.java
--
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AbstractHBaseSaslRpcClient.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AbstractHBaseSaslRpcClient.java
index cd2f4cd..de2c96e 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AbstractHBaseSaslRpcClient.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AbstractHBaseSaslRpcClient.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.security.token.TokenIdentifier;
 /**
  * A utility class that encapsulates SASL logic for RPC client. Copied from
  * org.apache.hadoop.security
+ * @since 2.0.0
  */
 

hbase git commit: Revert "So far -- fix this message" Revert miscommit

2017-08-25 Thread stack
Repository: hbase
Updated Branches:
  refs/heads/master 20d272bce -> e62fdd9db


Revert "So far -- fix this message"
Revert miscommit

This reverts commit 3bc64dac951a8bb40e8687dc2e60049ee75856f5.


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

Branch: refs/heads/master
Commit: e62fdd9db436c970f305ffb18c334dc420f9d75c
Parents: 20d272b
Author: Michael Stack 
Authored: Fri Aug 25 14:14:05 2017 -0700
Committer: Michael Stack 
Committed: Fri Aug 25 14:14:05 2017 -0700

--
 .../java/org/apache/hadoop/hbase/backup/BackupClientFactory.java  | 1 -
 .../java/org/apache/hadoop/hbase/filter/BinaryComparator.java | 1 -
 .../main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java  | 1 -
 .../apache/hadoop/hbase/security/AbstractHBaseSaslRpcClient.java  | 1 -
 .../apache/hadoop/hbase/security/access/AccessControlUtil.java| 3 ---
 5 files changed, 7 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/e62fdd9d/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java
--
diff --git 
a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java
 
b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java
index 6db39f8..21d73cc 100644
--- 
a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java
+++ 
b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java
@@ -25,7 +25,6 @@ import 
org.apache.hadoop.hbase.backup.impl.IncrementalTableBackupClient;
 import org.apache.hadoop.hbase.backup.impl.TableBackupClient;
 import org.apache.hadoop.hbase.client.Connection;
 
-@InterfaceAudience.Private
 public class BackupClientFactory {
 
   public static TableBackupClient create (Connection conn, String backupId, 
BackupRequest request)

http://git-wip-us.apache.org/repos/asf/hbase/blob/e62fdd9d/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryComparator.java
--
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryComparator.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryComparator.java
index 8a4aa34..87b622c 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryComparator.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryComparator.java
@@ -33,7 +33,6 @@ import 
org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferE
 /**
  * A binary comparator which lexicographically compares against the specified
  * byte array using {@link 
org.apache.hadoop.hbase.util.Bytes#compareTo(byte[], byte[])}.
- * @since 2.0.0
  */
 @InterfaceAudience.Public
 public class BinaryComparator extends 
org.apache.hadoop.hbase.filter.ByteArrayComparable {

http://git-wip-us.apache.org/repos/asf/hbase/blob/e62fdd9d/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 12c829e..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
@@ -89,7 +89,6 @@ import org.apache.hadoop.security.token.TokenSelector;
  * outside the lock in {@link Call} and {@link HBaseRpcController} which means 
the implementations
  * of the callbacks are free to hold any lock.
  * 
- * @since 2.0.0
  */
 @InterfaceAudience.Private
 public abstract class AbstractRpcClient implements 
RpcClient {

http://git-wip-us.apache.org/repos/asf/hbase/blob/e62fdd9d/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AbstractHBaseSaslRpcClient.java
--
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AbstractHBaseSaslRpcClient.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AbstractHBaseSaslRpcClient.java
index de2c96e..cd2f4cd 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AbstractHBaseSaslRpcClient.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AbstractHBaseSaslRpcClient.java
@@ -42,7 +42,6 @@ import org.apache.hadoop.security.token.TokenIdentifier;
 /**
  * A utility class that encapsulates SASL logic for RPC client. Copied from
  * 

hbase git commit: HBASE-18687 Add @since 2.0.0 to new classes

2017-08-25 Thread stack
Repository: hbase
Updated Branches:
  refs/heads/branch-2 f74cf679e -> 3e1c598d8


HBASE-18687 Add @since 2.0.0 to new classes


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

Branch: refs/heads/branch-2
Commit: 3e1c598d8eaf968d11d89c9c82c6a102e84b6e0c
Parents: f74cf67
Author: Michael Stack 
Authored: Fri Aug 25 13:12:20 2017 -0700
Committer: Michael Stack 
Committed: Fri Aug 25 13:12:52 2017 -0700

--
 .../main/java/org/apache/hadoop/hbase/AsyncMetaTableAccessor.java | 1 +
 .../src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java  | 1 +
 .../java/org/apache/hadoop/hbase/client/AsyncAdminBuilder.java| 1 +
 .../hadoop/hbase/client/AsyncAdminRequestRetryingCaller.java  | 3 +++
 .../main/java/org/apache/hadoop/hbase/client/AsyncConnection.java | 1 +
 .../main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java | 1 +
 .../hadoop/hbase/client/AsyncMasterRequestRpcRetryingCaller.java  | 1 +
 .../java/org/apache/hadoop/hbase/client/AsyncRequestFuture.java   | 3 ++-
 .../apache/hadoop/hbase/client/AsyncRpcRetryingCallerFactory.java | 1 +
 .../src/main/java/org/apache/hadoop/hbase/client/AsyncTable.java  | 1 +
 .../main/java/org/apache/hadoop/hbase/client/AsyncTableBase.java  | 1 +
 .../java/org/apache/hadoop/hbase/client/AsyncTableBuilder.java| 1 +
 .../org/apache/hadoop/hbase/client/AsyncTableRegionLocator.java   | 1 +
 .../java/org/apache/hadoop/hbase/client/BatchScanResultCache.java | 1 +
 .../org/apache/hadoop/hbase/client/ColumnFamilyDescriptor.java| 1 +
 .../apache/hadoop/hbase/client/ColumnFamilyDescriptorBuilder.java | 3 +++
 .../java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java   | 1 +
 .../main/java/org/apache/hadoop/hbase/client/RawAsyncTable.java   | 1 +
 .../org/apache/hadoop/hbase/client/RawScanResultConsumer.java | 1 +
 .../org/apache/hadoop/hbase/client/TableDescriptorBuilder.java| 3 +++
 .../src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClient.java | 1 +
 .../org/apache/hadoop/hbase/ipc/NettyRpcClientConfigHelper.java   | 1 +
 .../main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java | 1 +
 .../java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java   | 1 +
 .../hbase/security/NettyHBaseRpcConnectionHeaderHandler.java  | 1 +
 .../org/apache/hadoop/hbase/security/NettyHBaseSaslRpcClient.java | 1 +
 .../hadoop/hbase/security/NettyHBaseSaslRpcClientHandler.java | 1 +
 .../src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java | 1 +
 .../apache/hadoop/hbase/ipc/NettyRpcServerPreambleHandler.java| 1 +
 .../org/apache/hadoop/hbase/ipc/NettyRpcServerRequestDecoder.java | 1 +
 .../apache/hadoop/hbase/ipc/NettyRpcServerResponseEncoder.java| 1 +
 .../main/java/org/apache/hadoop/hbase/ipc/NettyServerCall.java| 1 +
 .../org/apache/hadoop/hbase/ipc/NettyServerRpcConnection.java | 1 +
 .../org/apache/hadoop/hbase/wal/NettyAsyncFSWALConfigHelper.java  | 1 +
 34 files changed, 41 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/3e1c598d/hbase-client/src/main/java/org/apache/hadoop/hbase/AsyncMetaTableAccessor.java
--
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/AsyncMetaTableAccessor.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/AsyncMetaTableAccessor.java
index 1018d7a..232212a 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/AsyncMetaTableAccessor.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/AsyncMetaTableAccessor.java
@@ -56,6 +56,7 @@ import org.apache.hadoop.hbase.util.Pair;
 /**
  * The asynchronous meta table accessor. Used to read/write region and 
assignment information store
  * in hbase:meta.
+ * @since 2.0.0
  */
 @InterfaceAudience.Private
 public class AsyncMetaTableAccessor {

http://git-wip-us.apache.org/repos/asf/hbase/blob/3e1c598d/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 4b33812..bdd047d 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
@@ -52,6 +52,7 @@ import com.google.protobuf.RpcChannel;
  * 
  * This feature is still under development, so marked as IA.Private. Will 
change to public when
  * done. Use it with caution.
+ * @since 2.0.0
  */
 @InterfaceAudience.Public
 

[2/2] hbase git commit: So far -- fix this message

2017-08-25 Thread stack
So far -- fix this message


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

Branch: refs/heads/master
Commit: 3bc64dac951a8bb40e8687dc2e60049ee75856f5
Parents: 368591d
Author: Michael Stack 
Authored: Thu Aug 24 20:58:59 2017 -0700
Committer: Michael Stack 
Committed: Fri Aug 25 13:12:30 2017 -0700

--
 .../java/org/apache/hadoop/hbase/backup/BackupClientFactory.java  | 1 +
 .../java/org/apache/hadoop/hbase/filter/BinaryComparator.java | 1 +
 .../main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java  | 1 +
 .../apache/hadoop/hbase/security/AbstractHBaseSaslRpcClient.java  | 1 +
 .../apache/hadoop/hbase/security/access/AccessControlUtil.java| 3 +++
 5 files changed, 7 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/3bc64dac/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java
--
diff --git 
a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java
 
b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java
index 21d73cc..6db39f8 100644
--- 
a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java
+++ 
b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java
@@ -25,6 +25,7 @@ import 
org.apache.hadoop.hbase.backup.impl.IncrementalTableBackupClient;
 import org.apache.hadoop.hbase.backup.impl.TableBackupClient;
 import org.apache.hadoop.hbase.client.Connection;
 
+@InterfaceAudience.Private
 public class BackupClientFactory {
 
   public static TableBackupClient create (Connection conn, String backupId, 
BackupRequest request)

http://git-wip-us.apache.org/repos/asf/hbase/blob/3bc64dac/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryComparator.java
--
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryComparator.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryComparator.java
index 87b622c..8a4aa34 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryComparator.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryComparator.java
@@ -33,6 +33,7 @@ import 
org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferE
 /**
  * A binary comparator which lexicographically compares against the specified
  * byte array using {@link 
org.apache.hadoop.hbase.util.Bytes#compareTo(byte[], byte[])}.
+ * @since 2.0.0
  */
 @InterfaceAudience.Public
 public class BinaryComparator extends 
org.apache.hadoop.hbase.filter.ByteArrayComparable {

http://git-wip-us.apache.org/repos/asf/hbase/blob/3bc64dac/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 7925505..12c829e 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
@@ -89,6 +89,7 @@ import org.apache.hadoop.security.token.TokenSelector;
  * outside the lock in {@link Call} and {@link HBaseRpcController} which means 
the implementations
  * of the callbacks are free to hold any lock.
  * 
+ * @since 2.0.0
  */
 @InterfaceAudience.Private
 public abstract class AbstractRpcClient implements 
RpcClient {

http://git-wip-us.apache.org/repos/asf/hbase/blob/3bc64dac/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AbstractHBaseSaslRpcClient.java
--
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AbstractHBaseSaslRpcClient.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AbstractHBaseSaslRpcClient.java
index cd2f4cd..de2c96e 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AbstractHBaseSaslRpcClient.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AbstractHBaseSaslRpcClient.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.security.token.TokenIdentifier;
 /**
  * A utility class that encapsulates SASL logic for RPC client. Copied from
  * org.apache.hadoop.security
+ * @since 2.0.0
  */
 @InterfaceAudience.Private
 public abstract class AbstractHBaseSaslRpcClient {


[1/2] hbase git commit: HBASE-18687 Add @since 2.0.0 to new classes

2017-08-25 Thread stack
Repository: hbase
Updated Branches:
  refs/heads/master 368591dfc -> 20d272bce


HBASE-18687 Add @since 2.0.0 to new classes


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

Branch: refs/heads/master
Commit: 20d272bce56ba8e586feceb710da159f288aa1e8
Parents: 3bc64da
Author: Michael Stack 
Authored: Fri Aug 25 13:12:20 2017 -0700
Committer: Michael Stack 
Committed: Fri Aug 25 13:12:30 2017 -0700

--
 .../main/java/org/apache/hadoop/hbase/AsyncMetaTableAccessor.java | 1 +
 .../src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java  | 1 +
 .../java/org/apache/hadoop/hbase/client/AsyncAdminBuilder.java| 1 +
 .../hadoop/hbase/client/AsyncAdminRequestRetryingCaller.java  | 3 +++
 .../main/java/org/apache/hadoop/hbase/client/AsyncConnection.java | 1 +
 .../main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java | 1 +
 .../hadoop/hbase/client/AsyncMasterRequestRpcRetryingCaller.java  | 1 +
 .../java/org/apache/hadoop/hbase/client/AsyncRequestFuture.java   | 3 ++-
 .../apache/hadoop/hbase/client/AsyncRpcRetryingCallerFactory.java | 1 +
 .../src/main/java/org/apache/hadoop/hbase/client/AsyncTable.java  | 1 +
 .../main/java/org/apache/hadoop/hbase/client/AsyncTableBase.java  | 1 +
 .../java/org/apache/hadoop/hbase/client/AsyncTableBuilder.java| 1 +
 .../org/apache/hadoop/hbase/client/AsyncTableRegionLocator.java   | 1 +
 .../java/org/apache/hadoop/hbase/client/BatchScanResultCache.java | 1 +
 .../org/apache/hadoop/hbase/client/ColumnFamilyDescriptor.java| 1 +
 .../apache/hadoop/hbase/client/ColumnFamilyDescriptorBuilder.java | 3 +++
 .../java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java   | 1 +
 .../main/java/org/apache/hadoop/hbase/client/RawAsyncTable.java   | 1 +
 .../org/apache/hadoop/hbase/client/RawScanResultConsumer.java | 1 +
 .../org/apache/hadoop/hbase/client/TableDescriptorBuilder.java| 3 +++
 .../src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClient.java | 1 +
 .../org/apache/hadoop/hbase/ipc/NettyRpcClientConfigHelper.java   | 1 +
 .../main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java | 1 +
 .../java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java   | 1 +
 .../hbase/security/NettyHBaseRpcConnectionHeaderHandler.java  | 1 +
 .../org/apache/hadoop/hbase/security/NettyHBaseSaslRpcClient.java | 1 +
 .../hadoop/hbase/security/NettyHBaseSaslRpcClientHandler.java | 1 +
 .../src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java | 1 +
 .../apache/hadoop/hbase/ipc/NettyRpcServerPreambleHandler.java| 1 +
 .../org/apache/hadoop/hbase/ipc/NettyRpcServerRequestDecoder.java | 1 +
 .../apache/hadoop/hbase/ipc/NettyRpcServerResponseEncoder.java| 1 +
 .../main/java/org/apache/hadoop/hbase/ipc/NettyServerCall.java| 1 +
 .../org/apache/hadoop/hbase/ipc/NettyServerRpcConnection.java | 1 +
 .../org/apache/hadoop/hbase/wal/NettyAsyncFSWALConfigHelper.java  | 1 +
 34 files changed, 41 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/20d272bc/hbase-client/src/main/java/org/apache/hadoop/hbase/AsyncMetaTableAccessor.java
--
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/AsyncMetaTableAccessor.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/AsyncMetaTableAccessor.java
index 1018d7a..232212a 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/AsyncMetaTableAccessor.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/AsyncMetaTableAccessor.java
@@ -56,6 +56,7 @@ import org.apache.hadoop.hbase.util.Pair;
 /**
  * The asynchronous meta table accessor. Used to read/write region and 
assignment information store
  * in hbase:meta.
+ * @since 2.0.0
  */
 @InterfaceAudience.Private
 public class AsyncMetaTableAccessor {

http://git-wip-us.apache.org/repos/asf/hbase/blob/20d272bc/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 4b33812..bdd047d 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
@@ -52,6 +52,7 @@ import com.google.protobuf.RpcChannel;
  * 
  * This feature is still under development, so marked as IA.Private. Will 
change to public when
  * done. Use it with caution.
+ * @since 2.0.0
  */
 @InterfaceAudience.Public
 

hbase git commit: HBASE-18635 Fixed Asciidoc warning

2017-08-25 Thread misty
Repository: hbase
Updated Branches:
  refs/heads/master 2e8739623 -> 368591dfc


HBASE-18635 Fixed Asciidoc warning

Signed-off-by: Misty Stanley-Jones 


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

Branch: refs/heads/master
Commit: 368591dfcd1fb4ec2be480b0970466386b059b82
Parents: 2e87396
Author: Jan Hentschel 
Authored: Sun Aug 20 23:47:11 2017 +0200
Committer: Misty Stanley-Jones 
Committed: Fri Aug 25 13:09:57 2017 -0700

--
 src/main/asciidoc/_chapters/external_apis.adoc | 15 +++
 src/main/asciidoc/_chapters/schema_design.adoc | 12 ++--
 2 files changed, 13 insertions(+), 14 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/368591df/src/main/asciidoc/_chapters/external_apis.adoc
--
diff --git a/src/main/asciidoc/_chapters/external_apis.adoc 
b/src/main/asciidoc/_chapters/external_apis.adoc
index 2f85461..c0e4a5f 100644
--- a/src/main/asciidoc/_chapters/external_apis.adoc
+++ b/src/main/asciidoc/_chapters/external_apis.adoc
@@ -288,18 +288,17 @@ your filter to the file. For example, to return only rows 
for
 which keys start with u123 and use a batch size
 of 100, the filter file would look like this:
 
-+++
-
-Scanner batch="100"
-  filter
+[source,xml]
+
+
+  
 {
   "type": "PrefixFilter",
   "value": "u123"
 }
-  /filter
-/Scanner
-
-+++
+  
+
+
 
 Pass the file to the `-d` argument of the `curl` request.
 |curl -vi -X PUT \

http://git-wip-us.apache.org/repos/asf/hbase/blob/368591df/src/main/asciidoc/_chapters/schema_design.adoc
--
diff --git a/src/main/asciidoc/_chapters/schema_design.adoc 
b/src/main/asciidoc/_chapters/schema_design.adoc
index cef05f2..d17f06b 100644
--- a/src/main/asciidoc/_chapters/schema_design.adoc
+++ b/src/main/asciidoc/_chapters/schema_design.adoc
@@ -1113,7 +1113,7 @@ If you don't have time to build it both ways and compare, 
my advice would be to
 [[schema.ops]]
 == Operational and Performance Configuration Options
 
-  Tune HBase Server RPC Handling
+===  Tune HBase Server RPC Handling
 
 * Set `hbase.regionserver.handler.count` (in `hbase-site.xml`) to cores x 
spindles for concurrency.
 * Optionally, split the call queues into separate read and write queues for 
differentiated service. The parameter 
`hbase.ipc.server.callqueue.handler.factor` specifies the number of call queues:
@@ -1129,7 +1129,7 @@ If you don't have time to build it both ways and compare, 
my advice would be to
 - `< 0.5` for more short-read
 - `> 0.5` for more long-read
 
-  Disable Nagle for RPC
+===  Disable Nagle for RPC
 
 Disable Nagle’s algorithm. Delayed ACKs can add up to ~200ms to RPC round 
trip time. Set the following parameters:
 
@@ -1140,7 +1140,7 @@ Disable Nagle’s algorithm. Delayed ACKs can add up to 
~200ms to RPC round trip
 - `hbase.ipc.client.tcpnodelay = true`
 - `hbase.ipc.server.tcpnodelay = true`
 
-  Limit Server Failure Impact
+===  Limit Server Failure Impact
 
 Detect regionserver failure as fast as reasonable. Set the following 
parameters:
 
@@ -1149,7 +1149,7 @@ Detect regionserver failure as fast as reasonable. Set 
the following parameters:
 - `dfs.namenode.avoid.read.stale.datanode = true`
 - `dfs.namenode.avoid.write.stale.datanode = true`
 
-  Optimize on the Server Side for Low Latency
+===  Optimize on the Server Side for Low Latency
 
 * Skip the network for local blocks. In `hbase-site.xml`, set the following 
parameters:
 - `dfs.client.read.shortcircuit = true`
@@ -1187,7 +1187,7 @@ Detect regionserver failure as fast as reasonable. Set 
the following parameters:
 
 ==  Special Cases
 
-  For applications where failing quickly is better than waiting
+===  For applications where failing quickly is better than waiting
 
 *  In `hbase-site.xml` on the client side, set the following parameters:
 - Set `hbase.client.pause = 1000`
@@ -1196,7 +1196,7 @@ Detect regionserver failure as fast as reasonable. Set 
the following parameters:
 - Set the RecoverableZookeeper retry count: `zookeeper.recovery.retry = 1` (no 
retry)
 * In `hbase-site.xml` on the server side, set the Zookeeper session timeout 
for detecting server failures: `zookeeper.session.timeout` <= 30 seconds (20-30 
is good).
 
-  For applications that can tolerate slightly out of date information
+===  For applications that can tolerate slightly out of date information
 
 **HBase timeline consistency (HBASE-10070) **
 With read replicas enabled, read-only copies of 

hbase git commit: Revert "HBASE-17249 Get/Scan's setTimeRange/setColumnFamilyTimeRange can take the TimeRange reference as the parameter instead of creating a new setColumnFamilyTimeRange instance. (h

2017-08-25 Thread huaxiangsun
Repository: hbase
Updated Branches:
  refs/heads/branch-2 e58670683 -> f74cf679e


Revert "HBASE-17249 Get/Scan's setTimeRange/setColumnFamilyTimeRange can take 
the TimeRange reference as the parameter instead of creating a new 
setColumnFamilyTimeRange instance. (huaxiang sun)"

This reverts commit 61220e4d7c8d7e5fb8ed3bbe2469bc86632c48de.


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

Branch: refs/heads/branch-2
Commit: f74cf679ecebed2ab5c95ccd678f48482644b829
Parents: e586706
Author: Huaxiang Sun 
Authored: Thu Aug 24 16:28:16 2017 -0700
Committer: Huaxiang Sun 
Committed: Fri Aug 25 11:19:46 2017 -0700

--
 .../org/apache/hadoop/hbase/client/Get.java | 33 +++--
 .../org/apache/hadoop/hbase/client/Query.java   | 38 +---
 .../org/apache/hadoop/hbase/client/Scan.java| 36 ---
 .../hadoop/hbase/protobuf/ProtobufUtil.java | 12 ---
 .../hbase/shaded/protobuf/ProtobufUtil.java | 12 ---
 5 files changed, 43 insertions(+), 88 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/f74cf679/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 226ca85..d40e0f2 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
@@ -73,6 +73,7 @@ public class Get extends Query
   private boolean cacheBlocks = true;
   private int storeLimit = -1;
   private int storeOffset = 0;
+  private TimeRange tr = new TimeRange();
   private boolean checkExistenceOnly = false;
   private boolean closestRowBefore = false;
   private Map> familyMap = new 
TreeMap<>(Bytes.BYTES_COMPARATOR);
@@ -222,19 +223,9 @@ public class Get extends Query
* @throws IOException
* @return this for invocation chaining
*/
-  @Override
   public Get setTimeRange(long minStamp, long maxStamp) throws IOException {
-return (Get) super.setTimeRange(minStamp, maxStamp);
-  }
-
-  /**
-   * Get versions of columns only within the specified timestamp range,
-   * @param tr Input TimeRange
-   * @return this for invocation chaining
-   */
-  @Override
-  public Get setTimeRange(TimeRange tr) {
-return (Get) super.setTimeRange(tr);
+tr = new TimeRange(minStamp, maxStamp);
+return this;
   }
 
   /**
@@ -245,7 +236,7 @@ public class Get extends Query
   public Get setTimeStamp(long timestamp)
   throws IOException {
 try {
-  super.setTimeRange(timestamp, timestamp + 1);
+  tr = new TimeRange(timestamp, timestamp+1);
 } catch(Exception e) {
   // This should never happen, unless integer overflow or something 
extremely wrong...
   LOG.error("TimeRange failed, likely caused by integer overflow. ", e);
@@ -254,16 +245,10 @@ public class Get extends Query
 return this;
   }
 
-  @Override
-  public Get setColumnFamilyTimeRange(byte[] cf, long minStamp, long maxStamp) 
{
+  @Override public Get setColumnFamilyTimeRange(byte[] cf, long minStamp, long 
maxStamp) {
 return (Get) super.setColumnFamilyTimeRange(cf, minStamp, maxStamp);
   }
 
-  @Override
-  public Get setColumnFamilyTimeRange(byte[] cf, TimeRange tr) {
-return (Get) super.setColumnFamilyTimeRange(cf, tr);
-  }
-
   /**
* Get all available versions.
* @return this for invocation chaining
@@ -404,6 +389,14 @@ public class Get extends Query
   }
 
   /**
+   * Method for retrieving the get's TimeRange
+   * @return timeRange
+   */
+  public TimeRange getTimeRange() {
+return this.tr;
+  }
+
+  /**
* Method for retrieving the keys in the familyMap
* @return keys in the current familyMap
*/

http://git-wip-us.apache.org/repos/asf/hbase/blob/f74cf679/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 cc9e9d4..a738c84 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
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import java.io.IOException;
 import java.util.Map;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
@@ -44,7 +43,6 @@ public 

hbase git commit: Revert "HBASE-17249 Get/Scan's setTimeRange/setColumnFamilyTimeRange can take the TimeRange reference as the parameter instead of creating a new setColumnFamilyTimeRange instance. (h

2017-08-25 Thread huaxiangsun
Repository: hbase
Updated Branches:
  refs/heads/master ac1fe07f4 -> 2e8739623


Revert "HBASE-17249 Get/Scan's setTimeRange/setColumnFamilyTimeRange can take 
the TimeRange reference as the parameter instead of creating a new 
setColumnFamilyTimeRange instance. (huaxiang sun)"

This reverts commit 61220e4d7c8d7e5fb8ed3bbe2469bc86632c48de.


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

Branch: refs/heads/master
Commit: 2e87396236b4d142940b19250bacbef2c15af3d5
Parents: ac1fe07
Author: Huaxiang Sun 
Authored: Thu Aug 24 16:28:16 2017 -0700
Committer: Huaxiang Sun 
Committed: Fri Aug 25 10:53:18 2017 -0700

--
 .../org/apache/hadoop/hbase/client/Get.java | 33 +++--
 .../org/apache/hadoop/hbase/client/Query.java   | 38 +---
 .../org/apache/hadoop/hbase/client/Scan.java| 36 ---
 .../hadoop/hbase/protobuf/ProtobufUtil.java | 12 ---
 .../hbase/shaded/protobuf/ProtobufUtil.java | 12 ---
 5 files changed, 43 insertions(+), 88 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/2e873962/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 226ca85..d40e0f2 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
@@ -73,6 +73,7 @@ public class Get extends Query
   private boolean cacheBlocks = true;
   private int storeLimit = -1;
   private int storeOffset = 0;
+  private TimeRange tr = new TimeRange();
   private boolean checkExistenceOnly = false;
   private boolean closestRowBefore = false;
   private Map> familyMap = new 
TreeMap<>(Bytes.BYTES_COMPARATOR);
@@ -222,19 +223,9 @@ public class Get extends Query
* @throws IOException
* @return this for invocation chaining
*/
-  @Override
   public Get setTimeRange(long minStamp, long maxStamp) throws IOException {
-return (Get) super.setTimeRange(minStamp, maxStamp);
-  }
-
-  /**
-   * Get versions of columns only within the specified timestamp range,
-   * @param tr Input TimeRange
-   * @return this for invocation chaining
-   */
-  @Override
-  public Get setTimeRange(TimeRange tr) {
-return (Get) super.setTimeRange(tr);
+tr = new TimeRange(minStamp, maxStamp);
+return this;
   }
 
   /**
@@ -245,7 +236,7 @@ public class Get extends Query
   public Get setTimeStamp(long timestamp)
   throws IOException {
 try {
-  super.setTimeRange(timestamp, timestamp + 1);
+  tr = new TimeRange(timestamp, timestamp+1);
 } catch(Exception e) {
   // This should never happen, unless integer overflow or something 
extremely wrong...
   LOG.error("TimeRange failed, likely caused by integer overflow. ", e);
@@ -254,16 +245,10 @@ public class Get extends Query
 return this;
   }
 
-  @Override
-  public Get setColumnFamilyTimeRange(byte[] cf, long minStamp, long maxStamp) 
{
+  @Override public Get setColumnFamilyTimeRange(byte[] cf, long minStamp, long 
maxStamp) {
 return (Get) super.setColumnFamilyTimeRange(cf, minStamp, maxStamp);
   }
 
-  @Override
-  public Get setColumnFamilyTimeRange(byte[] cf, TimeRange tr) {
-return (Get) super.setColumnFamilyTimeRange(cf, tr);
-  }
-
   /**
* Get all available versions.
* @return this for invocation chaining
@@ -404,6 +389,14 @@ public class Get extends Query
   }
 
   /**
+   * Method for retrieving the get's TimeRange
+   * @return timeRange
+   */
+  public TimeRange getTimeRange() {
+return this.tr;
+  }
+
+  /**
* Method for retrieving the keys in the familyMap
* @return keys in the current familyMap
*/

http://git-wip-us.apache.org/repos/asf/hbase/blob/2e873962/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 cc9e9d4..a738c84 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
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import java.io.IOException;
 import java.util.Map;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
@@ -44,7 +43,6 @@ public 

[29/51] [partial] hbase-site git commit: Published site at .

2017-08-25 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/devapidocs/src-html/org/apache/hadoop/hbase/master/HMaster.RedirectServlet.html
--
diff --git 
a/devapidocs/src-html/org/apache/hadoop/hbase/master/HMaster.RedirectServlet.html
 
b/devapidocs/src-html/org/apache/hadoop/hbase/master/HMaster.RedirectServlet.html
index a04b2f2..fa9b7a2 100644
--- 
a/devapidocs/src-html/org/apache/hadoop/hbase/master/HMaster.RedirectServlet.html
+++ 
b/devapidocs/src-html/org/apache/hadoop/hbase/master/HMaster.RedirectServlet.html
@@ -47,3453 +47,3471 @@
 039import 
java.util.concurrent.ExecutionException;
 040import java.util.concurrent.Future;
 041import java.util.concurrent.TimeUnit;
-042import 
java.util.concurrent.atomic.AtomicInteger;
-043import 
java.util.concurrent.atomic.AtomicReference;
-044import java.util.function.Function;
-045import java.util.regex.Pattern;
-046
-047import javax.servlet.ServletException;
-048import javax.servlet.http.HttpServlet;
-049import 
javax.servlet.http.HttpServletRequest;
-050import 
javax.servlet.http.HttpServletResponse;
-051
-052import org.apache.commons.logging.Log;
-053import 
org.apache.commons.logging.LogFactory;
-054import 
org.apache.hadoop.conf.Configuration;
-055import org.apache.hadoop.fs.FileSystem;
-056import org.apache.hadoop.fs.Path;
-057import 
org.apache.hadoop.hbase.ClusterStatus;
-058import 
org.apache.hadoop.hbase.ClusterStatus.Options;
-059import 
org.apache.hadoop.hbase.CoordinatedStateException;
-060import 
org.apache.hadoop.hbase.CoordinatedStateManager;
-061import 
org.apache.hadoop.hbase.DoNotRetryIOException;
-062import 
org.apache.hadoop.hbase.HBaseIOException;
-063import 
org.apache.hadoop.hbase.HBaseInterfaceAudience;
-064import 
org.apache.hadoop.hbase.HConstants;
-065import 
org.apache.hadoop.hbase.HRegionInfo;
-066import 
org.apache.hadoop.hbase.MasterNotRunningException;
-067import 
org.apache.hadoop.hbase.MetaTableAccessor;
-068import 
org.apache.hadoop.hbase.NamespaceDescriptor;
-069import 
org.apache.hadoop.hbase.PleaseHoldException;
-070import 
org.apache.hadoop.hbase.ProcedureInfo;
-071import 
org.apache.hadoop.hbase.ServerLoad;
-072import 
org.apache.hadoop.hbase.ServerName;
-073import 
org.apache.hadoop.hbase.TableDescriptors;
-074import 
org.apache.hadoop.hbase.TableName;
-075import 
org.apache.hadoop.hbase.TableNotDisabledException;
-076import 
org.apache.hadoop.hbase.TableNotFoundException;
-077import 
org.apache.hadoop.hbase.UnknownRegionException;
-078import 
org.apache.hadoop.hbase.classification.InterfaceAudience;
-079import 
org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
-080import 
org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
-081import 
org.apache.hadoop.hbase.client.MasterSwitchType;
-082import 
org.apache.hadoop.hbase.client.Result;
-083import 
org.apache.hadoop.hbase.client.TableDescriptor;
-084import 
org.apache.hadoop.hbase.client.TableDescriptorBuilder;
-085import 
org.apache.hadoop.hbase.client.TableState;
-086import 
org.apache.hadoop.hbase.coprocessor.BypassCoprocessorException;
-087import 
org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
-088import 
org.apache.hadoop.hbase.exceptions.DeserializationException;
-089import 
org.apache.hadoop.hbase.exceptions.MergeRegionException;
-090import 
org.apache.hadoop.hbase.executor.ExecutorType;
-091import 
org.apache.hadoop.hbase.favored.FavoredNodesManager;
-092import 
org.apache.hadoop.hbase.favored.FavoredNodesPromoter;
-093import 
org.apache.hadoop.hbase.http.InfoServer;
-094import 
org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
-095import 
org.apache.hadoop.hbase.ipc.RpcServer;
-096import 
org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
-097import 
org.apache.hadoop.hbase.master.MasterRpcServices.BalanceSwitchMode;
-098import 
org.apache.hadoop.hbase.master.assignment.AssignmentManager;
-099import 
org.apache.hadoop.hbase.master.assignment.MergeTableRegionsProcedure;
-100import 
org.apache.hadoop.hbase.master.assignment.RegionStates;
-101import 
org.apache.hadoop.hbase.master.assignment.RegionStates.RegionStateNode;
-102import 
org.apache.hadoop.hbase.master.balancer.BalancerChore;
-103import 
org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer;
-104import 
org.apache.hadoop.hbase.master.balancer.ClusterStatusChore;
-105import 
org.apache.hadoop.hbase.master.balancer.LoadBalancerFactory;
-106import 
org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
-107import 
org.apache.hadoop.hbase.master.cleaner.LogCleaner;
-108import 
org.apache.hadoop.hbase.master.cleaner.ReplicationMetaCleaner;
-109import 
org.apache.hadoop.hbase.master.cleaner.ReplicationZKNodeCleaner;
-110import 
org.apache.hadoop.hbase.master.cleaner.ReplicationZKNodeCleanerChore;
-111import 
org.apache.hadoop.hbase.master.locking.LockManager;
-112import 
org.apache.hadoop.hbase.master.normalizer.NormalizationPlan;
-113import 
org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;

[05/51] [partial] hbase-site git commit: Published site at .

2017-08-25 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.WorkItemHdfsDir.html
--
diff --git 
a/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.WorkItemHdfsDir.html
 
b/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.WorkItemHdfsDir.html
index f5186b5..43afb13 100644
--- 
a/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.WorkItemHdfsDir.html
+++ 
b/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.WorkItemHdfsDir.html
@@ -36,5085 +36,5084 @@
 028import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.Multimap;
 029import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.Ordering;
 030import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.TreeMultimap;
-031import 
com.google.protobuf.ServiceException;
-032
-033import java.io.Closeable;
-034import java.io.FileNotFoundException;
-035import java.io.IOException;
-036import java.io.InterruptedIOException;
-037import java.io.PrintWriter;
-038import java.io.StringWriter;
-039import java.net.InetAddress;
-040import java.net.URI;
-041import java.util.ArrayList;
-042import java.util.Arrays;
-043import java.util.Collection;
-044import java.util.Collections;
-045import java.util.Comparator;
-046import java.util.HashMap;
-047import java.util.HashSet;
-048import java.util.Iterator;
-049import java.util.List;
-050import java.util.Locale;
-051import java.util.Map;
-052import java.util.Map.Entry;
-053import java.util.Set;
-054import java.util.SortedMap;
-055import java.util.SortedSet;
-056import java.util.TreeMap;
-057import java.util.TreeSet;
-058import java.util.Vector;
-059import java.util.concurrent.Callable;
-060import 
java.util.concurrent.ConcurrentSkipListMap;
-061import 
java.util.concurrent.ExecutionException;
-062import 
java.util.concurrent.ExecutorService;
-063import java.util.concurrent.Executors;
-064import java.util.concurrent.Future;
-065import java.util.concurrent.FutureTask;
-066import 
java.util.concurrent.ScheduledThreadPoolExecutor;
-067import java.util.concurrent.TimeUnit;
-068import 
java.util.concurrent.TimeoutException;
-069import 
java.util.concurrent.atomic.AtomicBoolean;
-070import 
java.util.concurrent.atomic.AtomicInteger;
-071
-072import org.apache.commons.io.IOUtils;
-073import 
org.apache.commons.lang.RandomStringUtils;
-074import 
org.apache.commons.lang.StringUtils;
-075import org.apache.commons.logging.Log;
-076import 
org.apache.commons.logging.LogFactory;
-077import 
org.apache.hadoop.conf.Configuration;
-078import 
org.apache.hadoop.conf.Configured;
-079import 
org.apache.hadoop.fs.FSDataOutputStream;
-080import org.apache.hadoop.fs.FileStatus;
-081import org.apache.hadoop.fs.FileSystem;
-082import org.apache.hadoop.fs.Path;
-083import 
org.apache.hadoop.fs.permission.FsAction;
-084import 
org.apache.hadoop.fs.permission.FsPermission;
-085import 
org.apache.hadoop.hbase.Abortable;
-086import org.apache.hadoop.hbase.Cell;
-087import 
org.apache.hadoop.hbase.CellUtil;
-088import 
org.apache.hadoop.hbase.ClusterStatus;
-089import 
org.apache.hadoop.hbase.HBaseConfiguration;
-090import 
org.apache.hadoop.hbase.HBaseInterfaceAudience;
-091import 
org.apache.hadoop.hbase.HConstants;
-092import 
org.apache.hadoop.hbase.HRegionInfo;
-093import 
org.apache.hadoop.hbase.HRegionLocation;
-094import 
org.apache.hadoop.hbase.KeyValue;
-095import 
org.apache.hadoop.hbase.MasterNotRunningException;
-096import 
org.apache.hadoop.hbase.MetaTableAccessor;
-097import 
org.apache.hadoop.hbase.RegionLocations;
-098import 
org.apache.hadoop.hbase.ServerName;
-099import 
org.apache.hadoop.hbase.TableName;
-100import 
org.apache.hadoop.hbase.ZooKeeperConnectionException;
-101import 
org.apache.hadoop.hbase.classification.InterfaceAudience;
-102import 
org.apache.hadoop.hbase.classification.InterfaceStability;
-103import 
org.apache.hadoop.hbase.client.Admin;
-104import 
org.apache.hadoop.hbase.client.ClusterConnection;
-105import 
org.apache.hadoop.hbase.client.Connection;
-106import 
org.apache.hadoop.hbase.client.ConnectionFactory;
-107import 
org.apache.hadoop.hbase.client.Delete;
-108import 
org.apache.hadoop.hbase.client.Get;
-109import 
org.apache.hadoop.hbase.client.Put;
-110import 
org.apache.hadoop.hbase.client.RegionReplicaUtil;
-111import 
org.apache.hadoop.hbase.client.Result;
-112import 
org.apache.hadoop.hbase.client.RowMutations;
-113import 
org.apache.hadoop.hbase.client.Table;
-114import 
org.apache.hadoop.hbase.client.TableState;
-115import 
org.apache.hadoop.hbase.io.FileLink;
-116import 
org.apache.hadoop.hbase.io.HFileLink;
-117import 
org.apache.hadoop.hbase.io.hfile.CacheConfig;
-118import 
org.apache.hadoop.hbase.io.hfile.HFile;
-119import 
org.apache.hadoop.hbase.master.MasterFileSystem;
-120import 
org.apache.hadoop.hbase.master.RegionState;
-121import 
org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-122import 

[35/51] [partial] hbase-site git commit: Published site at .

2017-08-25 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.CheckRegionConsistencyWorkItem.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.CheckRegionConsistencyWorkItem.html
 
b/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.CheckRegionConsistencyWorkItem.html
index 201e6c0..873d245 100644
--- 
a/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.CheckRegionConsistencyWorkItem.html
+++ 
b/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.CheckRegionConsistencyWorkItem.html
@@ -117,7 +117,7 @@ var activeTableTab = "activeTableTab";
 
 
 
-class HBaseFsck.CheckRegionConsistencyWorkItem
+class HBaseFsck.CheckRegionConsistencyWorkItem
 extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true;
 title="class or interface in java.lang">Object
 implements http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/Callable.html?is-external=true;
 title="class or interface in java.util.concurrent">Callablehttp://docs.oracle.com/javase/8/docs/api/java/lang/Void.html?is-external=true;
 title="class or interface in java.lang">Void
 
@@ -211,7 +211,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/util/concurren
 
 
 key
-private finalhttp://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true;
 title="class or interface in java.lang">String key
+private finalhttp://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true;
 title="class or interface in java.lang">String key
 
 
 
@@ -220,7 +220,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/util/concurren
 
 
 hbi
-private finalHBaseFsck.HbckInfo hbi
+private finalHBaseFsck.HbckInfo hbi
 
 
 
@@ -237,7 +237,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/util/concurren
 
 
 CheckRegionConsistencyWorkItem
-CheckRegionConsistencyWorkItem(http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true;
 title="class or interface in java.lang">Stringkey,
+CheckRegionConsistencyWorkItem(http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true;
 title="class or interface in java.lang">Stringkey,
HBaseFsck.HbckInfohbi)
 
 
@@ -255,7 +255,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/util/concurren
 
 
 call
-publichttp://docs.oracle.com/javase/8/docs/api/java/lang/Void.html?is-external=true;
 title="class or interface in java.lang">Voidcall()
+publichttp://docs.oracle.com/javase/8/docs/api/java/lang/Void.html?is-external=true;
 title="class or interface in java.lang">Voidcall()
   throws http://docs.oracle.com/javase/8/docs/api/java/lang/Exception.html?is-external=true;
 title="class or interface in java.lang">Exception
 
 Specified by:

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.ErrorReporter.ERROR_CODE.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.ErrorReporter.ERROR_CODE.html
 
b/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.ErrorReporter.ERROR_CODE.html
index 41ac3fe..1f08848 100644
--- 
a/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.ErrorReporter.ERROR_CODE.html
+++ 
b/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.ErrorReporter.ERROR_CODE.html
@@ -122,7 +122,7 @@ var activeTableTab = "activeTableTab";
 
 
 
-public static enum HBaseFsck.ErrorReporter.ERROR_CODE
+public static enum HBaseFsck.ErrorReporter.ERROR_CODE
 extends http://docs.oracle.com/javase/8/docs/api/java/lang/Enum.html?is-external=true;
 title="class or interface in java.lang">EnumHBaseFsck.ErrorReporter.ERROR_CODE
 
 
@@ -309,7 +309,7 @@ the order they are declared.
 
 
 UNKNOWN
-public static finalHBaseFsck.ErrorReporter.ERROR_CODE UNKNOWN
+public static finalHBaseFsck.ErrorReporter.ERROR_CODE UNKNOWN
 
 
 
@@ -318,7 +318,7 @@ the order they are declared.
 
 
 NO_META_REGION
-public static finalHBaseFsck.ErrorReporter.ERROR_CODE NO_META_REGION
+public static finalHBaseFsck.ErrorReporter.ERROR_CODE NO_META_REGION
 
 
 
@@ -327,7 +327,7 @@ the order they are declared.
 
 
 NULL_META_REGION
-public static finalHBaseFsck.ErrorReporter.ERROR_CODE NULL_META_REGION
+public static finalHBaseFsck.ErrorReporter.ERROR_CODE NULL_META_REGION
 
 
 
@@ -336,7 +336,7 @@ the order they are declared.
 
 
 NO_VERSION_FILE
-public static finalHBaseFsck.ErrorReporter.ERROR_CODE NO_VERSION_FILE
+public static finalHBaseFsck.ErrorReporter.ERROR_CODE NO_VERSION_FILE
 
 
 
@@ -345,7 +345,7 @@ the order they are declared.
 
 
 NOT_IN_META_HDFS
-public static finalHBaseFsck.ErrorReporter.ERROR_CODE NOT_IN_META_HDFS
+public static finalHBaseFsck.ErrorReporter.ERROR_CODE NOT_IN_META_HDFS
 
 
 
@@ -354,7 +354,7 @@ the order they are declared.
 
 
 NOT_IN_META
-public static 

[02/51] [partial] hbase-site git commit: Published site at .

2017-08-25 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.WorkItemRegion.html
--
diff --git 
a/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.WorkItemRegion.html
 
b/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.WorkItemRegion.html
index f5186b5..43afb13 100644
--- 
a/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.WorkItemRegion.html
+++ 
b/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.WorkItemRegion.html
@@ -36,5085 +36,5084 @@
 028import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.Multimap;
 029import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.Ordering;
 030import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.TreeMultimap;
-031import 
com.google.protobuf.ServiceException;
-032
-033import java.io.Closeable;
-034import java.io.FileNotFoundException;
-035import java.io.IOException;
-036import java.io.InterruptedIOException;
-037import java.io.PrintWriter;
-038import java.io.StringWriter;
-039import java.net.InetAddress;
-040import java.net.URI;
-041import java.util.ArrayList;
-042import java.util.Arrays;
-043import java.util.Collection;
-044import java.util.Collections;
-045import java.util.Comparator;
-046import java.util.HashMap;
-047import java.util.HashSet;
-048import java.util.Iterator;
-049import java.util.List;
-050import java.util.Locale;
-051import java.util.Map;
-052import java.util.Map.Entry;
-053import java.util.Set;
-054import java.util.SortedMap;
-055import java.util.SortedSet;
-056import java.util.TreeMap;
-057import java.util.TreeSet;
-058import java.util.Vector;
-059import java.util.concurrent.Callable;
-060import 
java.util.concurrent.ConcurrentSkipListMap;
-061import 
java.util.concurrent.ExecutionException;
-062import 
java.util.concurrent.ExecutorService;
-063import java.util.concurrent.Executors;
-064import java.util.concurrent.Future;
-065import java.util.concurrent.FutureTask;
-066import 
java.util.concurrent.ScheduledThreadPoolExecutor;
-067import java.util.concurrent.TimeUnit;
-068import 
java.util.concurrent.TimeoutException;
-069import 
java.util.concurrent.atomic.AtomicBoolean;
-070import 
java.util.concurrent.atomic.AtomicInteger;
-071
-072import org.apache.commons.io.IOUtils;
-073import 
org.apache.commons.lang.RandomStringUtils;
-074import 
org.apache.commons.lang.StringUtils;
-075import org.apache.commons.logging.Log;
-076import 
org.apache.commons.logging.LogFactory;
-077import 
org.apache.hadoop.conf.Configuration;
-078import 
org.apache.hadoop.conf.Configured;
-079import 
org.apache.hadoop.fs.FSDataOutputStream;
-080import org.apache.hadoop.fs.FileStatus;
-081import org.apache.hadoop.fs.FileSystem;
-082import org.apache.hadoop.fs.Path;
-083import 
org.apache.hadoop.fs.permission.FsAction;
-084import 
org.apache.hadoop.fs.permission.FsPermission;
-085import 
org.apache.hadoop.hbase.Abortable;
-086import org.apache.hadoop.hbase.Cell;
-087import 
org.apache.hadoop.hbase.CellUtil;
-088import 
org.apache.hadoop.hbase.ClusterStatus;
-089import 
org.apache.hadoop.hbase.HBaseConfiguration;
-090import 
org.apache.hadoop.hbase.HBaseInterfaceAudience;
-091import 
org.apache.hadoop.hbase.HConstants;
-092import 
org.apache.hadoop.hbase.HRegionInfo;
-093import 
org.apache.hadoop.hbase.HRegionLocation;
-094import 
org.apache.hadoop.hbase.KeyValue;
-095import 
org.apache.hadoop.hbase.MasterNotRunningException;
-096import 
org.apache.hadoop.hbase.MetaTableAccessor;
-097import 
org.apache.hadoop.hbase.RegionLocations;
-098import 
org.apache.hadoop.hbase.ServerName;
-099import 
org.apache.hadoop.hbase.TableName;
-100import 
org.apache.hadoop.hbase.ZooKeeperConnectionException;
-101import 
org.apache.hadoop.hbase.classification.InterfaceAudience;
-102import 
org.apache.hadoop.hbase.classification.InterfaceStability;
-103import 
org.apache.hadoop.hbase.client.Admin;
-104import 
org.apache.hadoop.hbase.client.ClusterConnection;
-105import 
org.apache.hadoop.hbase.client.Connection;
-106import 
org.apache.hadoop.hbase.client.ConnectionFactory;
-107import 
org.apache.hadoop.hbase.client.Delete;
-108import 
org.apache.hadoop.hbase.client.Get;
-109import 
org.apache.hadoop.hbase.client.Put;
-110import 
org.apache.hadoop.hbase.client.RegionReplicaUtil;
-111import 
org.apache.hadoop.hbase.client.Result;
-112import 
org.apache.hadoop.hbase.client.RowMutations;
-113import 
org.apache.hadoop.hbase.client.Table;
-114import 
org.apache.hadoop.hbase.client.TableState;
-115import 
org.apache.hadoop.hbase.io.FileLink;
-116import 
org.apache.hadoop.hbase.io.HFileLink;
-117import 
org.apache.hadoop.hbase.io.hfile.CacheConfig;
-118import 
org.apache.hadoop.hbase.io.hfile.HFile;
-119import 
org.apache.hadoop.hbase.master.MasterFileSystem;
-120import 
org.apache.hadoop.hbase.master.RegionState;
-121import 
org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-122import 

[09/51] [partial] hbase-site git commit: Published site at .

2017-08-25 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.RegionRepairException.html
--
diff --git 
a/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.RegionRepairException.html
 
b/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.RegionRepairException.html
index f5186b5..43afb13 100644
--- 
a/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.RegionRepairException.html
+++ 
b/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.RegionRepairException.html
@@ -36,5085 +36,5084 @@
 028import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.Multimap;
 029import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.Ordering;
 030import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.TreeMultimap;
-031import 
com.google.protobuf.ServiceException;
-032
-033import java.io.Closeable;
-034import java.io.FileNotFoundException;
-035import java.io.IOException;
-036import java.io.InterruptedIOException;
-037import java.io.PrintWriter;
-038import java.io.StringWriter;
-039import java.net.InetAddress;
-040import java.net.URI;
-041import java.util.ArrayList;
-042import java.util.Arrays;
-043import java.util.Collection;
-044import java.util.Collections;
-045import java.util.Comparator;
-046import java.util.HashMap;
-047import java.util.HashSet;
-048import java.util.Iterator;
-049import java.util.List;
-050import java.util.Locale;
-051import java.util.Map;
-052import java.util.Map.Entry;
-053import java.util.Set;
-054import java.util.SortedMap;
-055import java.util.SortedSet;
-056import java.util.TreeMap;
-057import java.util.TreeSet;
-058import java.util.Vector;
-059import java.util.concurrent.Callable;
-060import 
java.util.concurrent.ConcurrentSkipListMap;
-061import 
java.util.concurrent.ExecutionException;
-062import 
java.util.concurrent.ExecutorService;
-063import java.util.concurrent.Executors;
-064import java.util.concurrent.Future;
-065import java.util.concurrent.FutureTask;
-066import 
java.util.concurrent.ScheduledThreadPoolExecutor;
-067import java.util.concurrent.TimeUnit;
-068import 
java.util.concurrent.TimeoutException;
-069import 
java.util.concurrent.atomic.AtomicBoolean;
-070import 
java.util.concurrent.atomic.AtomicInteger;
-071
-072import org.apache.commons.io.IOUtils;
-073import 
org.apache.commons.lang.RandomStringUtils;
-074import 
org.apache.commons.lang.StringUtils;
-075import org.apache.commons.logging.Log;
-076import 
org.apache.commons.logging.LogFactory;
-077import 
org.apache.hadoop.conf.Configuration;
-078import 
org.apache.hadoop.conf.Configured;
-079import 
org.apache.hadoop.fs.FSDataOutputStream;
-080import org.apache.hadoop.fs.FileStatus;
-081import org.apache.hadoop.fs.FileSystem;
-082import org.apache.hadoop.fs.Path;
-083import 
org.apache.hadoop.fs.permission.FsAction;
-084import 
org.apache.hadoop.fs.permission.FsPermission;
-085import 
org.apache.hadoop.hbase.Abortable;
-086import org.apache.hadoop.hbase.Cell;
-087import 
org.apache.hadoop.hbase.CellUtil;
-088import 
org.apache.hadoop.hbase.ClusterStatus;
-089import 
org.apache.hadoop.hbase.HBaseConfiguration;
-090import 
org.apache.hadoop.hbase.HBaseInterfaceAudience;
-091import 
org.apache.hadoop.hbase.HConstants;
-092import 
org.apache.hadoop.hbase.HRegionInfo;
-093import 
org.apache.hadoop.hbase.HRegionLocation;
-094import 
org.apache.hadoop.hbase.KeyValue;
-095import 
org.apache.hadoop.hbase.MasterNotRunningException;
-096import 
org.apache.hadoop.hbase.MetaTableAccessor;
-097import 
org.apache.hadoop.hbase.RegionLocations;
-098import 
org.apache.hadoop.hbase.ServerName;
-099import 
org.apache.hadoop.hbase.TableName;
-100import 
org.apache.hadoop.hbase.ZooKeeperConnectionException;
-101import 
org.apache.hadoop.hbase.classification.InterfaceAudience;
-102import 
org.apache.hadoop.hbase.classification.InterfaceStability;
-103import 
org.apache.hadoop.hbase.client.Admin;
-104import 
org.apache.hadoop.hbase.client.ClusterConnection;
-105import 
org.apache.hadoop.hbase.client.Connection;
-106import 
org.apache.hadoop.hbase.client.ConnectionFactory;
-107import 
org.apache.hadoop.hbase.client.Delete;
-108import 
org.apache.hadoop.hbase.client.Get;
-109import 
org.apache.hadoop.hbase.client.Put;
-110import 
org.apache.hadoop.hbase.client.RegionReplicaUtil;
-111import 
org.apache.hadoop.hbase.client.Result;
-112import 
org.apache.hadoop.hbase.client.RowMutations;
-113import 
org.apache.hadoop.hbase.client.Table;
-114import 
org.apache.hadoop.hbase.client.TableState;
-115import 
org.apache.hadoop.hbase.io.FileLink;
-116import 
org.apache.hadoop.hbase.io.HFileLink;
-117import 
org.apache.hadoop.hbase.io.hfile.CacheConfig;
-118import 
org.apache.hadoop.hbase.io.hfile.HFile;
-119import 
org.apache.hadoop.hbase.master.MasterFileSystem;
-120import 
org.apache.hadoop.hbase.master.RegionState;
-121import 

[46/51] [partial] hbase-site git commit: Published site at .

2017-08-25 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/devapidocs/org/apache/hadoop/hbase/client/example/RefreshHFilesClient.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/client/example/RefreshHFilesClient.html 
b/devapidocs/org/apache/hadoop/hbase/client/example/RefreshHFilesClient.html
new file mode 100644
index 000..9e65e50
--- /dev/null
+++ b/devapidocs/org/apache/hadoop/hbase/client/example/RefreshHFilesClient.html
@@ -0,0 +1,380 @@
+http://www.w3.org/TR/html4/loose.dtd;>
+
+
+
+
+
+RefreshHFilesClient (Apache HBase 3.0.0-SNAPSHOT API)
+
+
+
+
+
+var methods = {"i0":10,"i1":10,"i2":10};
+var tabs = {65535:["t0","All Methods"],2:["t2","Instance 
Methods"],8:["t4","Concrete Methods"]};
+var altColor = "altColor";
+var rowColor = "rowColor";
+var tableTab = "tableTab";
+var activeTableTab = "activeTableTab";
+
+
+JavaScript is disabled on your browser.
+
+
+
+
+
+Skip navigation links
+
+
+
+
+Overview
+Package
+Class
+Use
+Tree
+Deprecated
+Index
+Help
+
+
+
+
+PrevClass
+NextClass
+
+
+Frames
+NoFrames
+
+
+AllClasses
+
+
+
+
+
+
+
+Summary:
+Nested|
+Field|
+Constr|
+Method
+
+
+Detail:
+Field|
+Constr|
+Method
+
+
+
+
+
+
+
+
+org.apache.hadoop.hbase.client.example
+Class 
RefreshHFilesClient
+
+
+
+http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true;
 title="class or interface in java.lang">java.lang.Object
+
+
+org.apache.hadoop.hbase.client.example.RefreshHFilesClient
+
+
+
+
+
+
+
+All Implemented Interfaces:
+http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.html?is-external=true;
 title="class or interface in java.io">Closeable, http://docs.oracle.com/javase/8/docs/api/java/lang/AutoCloseable.html?is-external=true;
 title="class or interface in java.lang">AutoCloseable
+
+
+
+public class RefreshHFilesClient
+extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true;
 title="class or interface in java.lang">Object
+implements http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.html?is-external=true;
 title="class or interface in java.io">Closeable
+This client class is for invoking the refresh HFile 
function deployed on the
+ Region Server side via the RefreshHFilesService.
+
+
+
+
+
+
+
+
+
+
+
+Field Summary
+
+Fields
+
+Modifier and Type
+Field and Description
+
+
+private Connection
+connection
+
+
+private static 
org.apache.commons.logging.Log
+LOG
+
+
+
+
+
+
+
+
+
+Constructor Summary
+
+Constructors
+
+Constructor and Description
+
+
+RefreshHFilesClient(org.apache.hadoop.conf.Configurationcfg)
+Constructor with Conf object
+
+
+
+
+
+
+
+
+
+
+Method Summary
+
+All MethodsInstance MethodsConcrete Methods
+
+Modifier and Type
+Method and Description
+
+
+void
+close()
+
+
+void
+refreshHFiles(Tabletable)
+
+
+void
+refreshHFiles(TableNametableName)
+
+
+
+
+
+
+Methods inherited from classjava.lang.http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true;
 title="class or interface in java.lang">Object
+http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true#clone--;
 title="class or interface in java.lang">clone, http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true#equals-java.lang.Object-;
 title="class or interface in java.lang">equals, http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true#finalize--;
 title="class or interface in java.lang">finalize, http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true#getClass--;
 title="class or interface in java.lang">getClass, http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true#hashCode--;
 title="class or interface in java.lang">hashCode, http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true#notify--;
 title="class or interface in java.lang">notify, http://docs.oracle.com/javase/8/docs/api/java/lang
 /Object.html?is-external=true#notifyAll--" title="class or interface in 
java.lang">notifyAll, http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true#toString--;
 title="class or interface in java.lang">toString, http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true#wait--;
 title="class or interface in java.lang">wait, http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true#wait-long-;
 title="class or interface in java.lang">wait, 

[15/51] [partial] hbase-site git commit: Published site at .

2017-08-25 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.HbckInfo.html
--
diff --git 
a/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.HbckInfo.html 
b/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.HbckInfo.html
index f5186b5..43afb13 100644
--- a/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.HbckInfo.html
+++ b/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.HbckInfo.html
@@ -36,5085 +36,5084 @@
 028import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.Multimap;
 029import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.Ordering;
 030import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.TreeMultimap;
-031import 
com.google.protobuf.ServiceException;
-032
-033import java.io.Closeable;
-034import java.io.FileNotFoundException;
-035import java.io.IOException;
-036import java.io.InterruptedIOException;
-037import java.io.PrintWriter;
-038import java.io.StringWriter;
-039import java.net.InetAddress;
-040import java.net.URI;
-041import java.util.ArrayList;
-042import java.util.Arrays;
-043import java.util.Collection;
-044import java.util.Collections;
-045import java.util.Comparator;
-046import java.util.HashMap;
-047import java.util.HashSet;
-048import java.util.Iterator;
-049import java.util.List;
-050import java.util.Locale;
-051import java.util.Map;
-052import java.util.Map.Entry;
-053import java.util.Set;
-054import java.util.SortedMap;
-055import java.util.SortedSet;
-056import java.util.TreeMap;
-057import java.util.TreeSet;
-058import java.util.Vector;
-059import java.util.concurrent.Callable;
-060import 
java.util.concurrent.ConcurrentSkipListMap;
-061import 
java.util.concurrent.ExecutionException;
-062import 
java.util.concurrent.ExecutorService;
-063import java.util.concurrent.Executors;
-064import java.util.concurrent.Future;
-065import java.util.concurrent.FutureTask;
-066import 
java.util.concurrent.ScheduledThreadPoolExecutor;
-067import java.util.concurrent.TimeUnit;
-068import 
java.util.concurrent.TimeoutException;
-069import 
java.util.concurrent.atomic.AtomicBoolean;
-070import 
java.util.concurrent.atomic.AtomicInteger;
-071
-072import org.apache.commons.io.IOUtils;
-073import 
org.apache.commons.lang.RandomStringUtils;
-074import 
org.apache.commons.lang.StringUtils;
-075import org.apache.commons.logging.Log;
-076import 
org.apache.commons.logging.LogFactory;
-077import 
org.apache.hadoop.conf.Configuration;
-078import 
org.apache.hadoop.conf.Configured;
-079import 
org.apache.hadoop.fs.FSDataOutputStream;
-080import org.apache.hadoop.fs.FileStatus;
-081import org.apache.hadoop.fs.FileSystem;
-082import org.apache.hadoop.fs.Path;
-083import 
org.apache.hadoop.fs.permission.FsAction;
-084import 
org.apache.hadoop.fs.permission.FsPermission;
-085import 
org.apache.hadoop.hbase.Abortable;
-086import org.apache.hadoop.hbase.Cell;
-087import 
org.apache.hadoop.hbase.CellUtil;
-088import 
org.apache.hadoop.hbase.ClusterStatus;
-089import 
org.apache.hadoop.hbase.HBaseConfiguration;
-090import 
org.apache.hadoop.hbase.HBaseInterfaceAudience;
-091import 
org.apache.hadoop.hbase.HConstants;
-092import 
org.apache.hadoop.hbase.HRegionInfo;
-093import 
org.apache.hadoop.hbase.HRegionLocation;
-094import 
org.apache.hadoop.hbase.KeyValue;
-095import 
org.apache.hadoop.hbase.MasterNotRunningException;
-096import 
org.apache.hadoop.hbase.MetaTableAccessor;
-097import 
org.apache.hadoop.hbase.RegionLocations;
-098import 
org.apache.hadoop.hbase.ServerName;
-099import 
org.apache.hadoop.hbase.TableName;
-100import 
org.apache.hadoop.hbase.ZooKeeperConnectionException;
-101import 
org.apache.hadoop.hbase.classification.InterfaceAudience;
-102import 
org.apache.hadoop.hbase.classification.InterfaceStability;
-103import 
org.apache.hadoop.hbase.client.Admin;
-104import 
org.apache.hadoop.hbase.client.ClusterConnection;
-105import 
org.apache.hadoop.hbase.client.Connection;
-106import 
org.apache.hadoop.hbase.client.ConnectionFactory;
-107import 
org.apache.hadoop.hbase.client.Delete;
-108import 
org.apache.hadoop.hbase.client.Get;
-109import 
org.apache.hadoop.hbase.client.Put;
-110import 
org.apache.hadoop.hbase.client.RegionReplicaUtil;
-111import 
org.apache.hadoop.hbase.client.Result;
-112import 
org.apache.hadoop.hbase.client.RowMutations;
-113import 
org.apache.hadoop.hbase.client.Table;
-114import 
org.apache.hadoop.hbase.client.TableState;
-115import 
org.apache.hadoop.hbase.io.FileLink;
-116import 
org.apache.hadoop.hbase.io.HFileLink;
-117import 
org.apache.hadoop.hbase.io.hfile.CacheConfig;
-118import 
org.apache.hadoop.hbase.io.hfile.HFile;
-119import 
org.apache.hadoop.hbase.master.MasterFileSystem;
-120import 
org.apache.hadoop.hbase.master.RegionState;
-121import 
org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-122import 

[18/51] [partial] hbase-site git commit: Published site at .

2017-08-25 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.ErrorReporter.html
--
diff --git 
a/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.ErrorReporter.html 
b/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.ErrorReporter.html
index f5186b5..43afb13 100644
--- 
a/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.ErrorReporter.html
+++ 
b/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.ErrorReporter.html
@@ -36,5085 +36,5084 @@
 028import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.Multimap;
 029import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.Ordering;
 030import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.TreeMultimap;
-031import 
com.google.protobuf.ServiceException;
-032
-033import java.io.Closeable;
-034import java.io.FileNotFoundException;
-035import java.io.IOException;
-036import java.io.InterruptedIOException;
-037import java.io.PrintWriter;
-038import java.io.StringWriter;
-039import java.net.InetAddress;
-040import java.net.URI;
-041import java.util.ArrayList;
-042import java.util.Arrays;
-043import java.util.Collection;
-044import java.util.Collections;
-045import java.util.Comparator;
-046import java.util.HashMap;
-047import java.util.HashSet;
-048import java.util.Iterator;
-049import java.util.List;
-050import java.util.Locale;
-051import java.util.Map;
-052import java.util.Map.Entry;
-053import java.util.Set;
-054import java.util.SortedMap;
-055import java.util.SortedSet;
-056import java.util.TreeMap;
-057import java.util.TreeSet;
-058import java.util.Vector;
-059import java.util.concurrent.Callable;
-060import 
java.util.concurrent.ConcurrentSkipListMap;
-061import 
java.util.concurrent.ExecutionException;
-062import 
java.util.concurrent.ExecutorService;
-063import java.util.concurrent.Executors;
-064import java.util.concurrent.Future;
-065import java.util.concurrent.FutureTask;
-066import 
java.util.concurrent.ScheduledThreadPoolExecutor;
-067import java.util.concurrent.TimeUnit;
-068import 
java.util.concurrent.TimeoutException;
-069import 
java.util.concurrent.atomic.AtomicBoolean;
-070import 
java.util.concurrent.atomic.AtomicInteger;
-071
-072import org.apache.commons.io.IOUtils;
-073import 
org.apache.commons.lang.RandomStringUtils;
-074import 
org.apache.commons.lang.StringUtils;
-075import org.apache.commons.logging.Log;
-076import 
org.apache.commons.logging.LogFactory;
-077import 
org.apache.hadoop.conf.Configuration;
-078import 
org.apache.hadoop.conf.Configured;
-079import 
org.apache.hadoop.fs.FSDataOutputStream;
-080import org.apache.hadoop.fs.FileStatus;
-081import org.apache.hadoop.fs.FileSystem;
-082import org.apache.hadoop.fs.Path;
-083import 
org.apache.hadoop.fs.permission.FsAction;
-084import 
org.apache.hadoop.fs.permission.FsPermission;
-085import 
org.apache.hadoop.hbase.Abortable;
-086import org.apache.hadoop.hbase.Cell;
-087import 
org.apache.hadoop.hbase.CellUtil;
-088import 
org.apache.hadoop.hbase.ClusterStatus;
-089import 
org.apache.hadoop.hbase.HBaseConfiguration;
-090import 
org.apache.hadoop.hbase.HBaseInterfaceAudience;
-091import 
org.apache.hadoop.hbase.HConstants;
-092import 
org.apache.hadoop.hbase.HRegionInfo;
-093import 
org.apache.hadoop.hbase.HRegionLocation;
-094import 
org.apache.hadoop.hbase.KeyValue;
-095import 
org.apache.hadoop.hbase.MasterNotRunningException;
-096import 
org.apache.hadoop.hbase.MetaTableAccessor;
-097import 
org.apache.hadoop.hbase.RegionLocations;
-098import 
org.apache.hadoop.hbase.ServerName;
-099import 
org.apache.hadoop.hbase.TableName;
-100import 
org.apache.hadoop.hbase.ZooKeeperConnectionException;
-101import 
org.apache.hadoop.hbase.classification.InterfaceAudience;
-102import 
org.apache.hadoop.hbase.classification.InterfaceStability;
-103import 
org.apache.hadoop.hbase.client.Admin;
-104import 
org.apache.hadoop.hbase.client.ClusterConnection;
-105import 
org.apache.hadoop.hbase.client.Connection;
-106import 
org.apache.hadoop.hbase.client.ConnectionFactory;
-107import 
org.apache.hadoop.hbase.client.Delete;
-108import 
org.apache.hadoop.hbase.client.Get;
-109import 
org.apache.hadoop.hbase.client.Put;
-110import 
org.apache.hadoop.hbase.client.RegionReplicaUtil;
-111import 
org.apache.hadoop.hbase.client.Result;
-112import 
org.apache.hadoop.hbase.client.RowMutations;
-113import 
org.apache.hadoop.hbase.client.Table;
-114import 
org.apache.hadoop.hbase.client.TableState;
-115import 
org.apache.hadoop.hbase.io.FileLink;
-116import 
org.apache.hadoop.hbase.io.HFileLink;
-117import 
org.apache.hadoop.hbase.io.hfile.CacheConfig;
-118import 
org.apache.hadoop.hbase.io.hfile.HFile;
-119import 
org.apache.hadoop.hbase.master.MasterFileSystem;
-120import 
org.apache.hadoop.hbase.master.RegionState;
-121import 
org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-122import 

[16/51] [partial] hbase-site git commit: Published site at .

2017-08-25 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.HBaseFsckTool.html
--
diff --git 
a/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.HBaseFsckTool.html 
b/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.HBaseFsckTool.html
index f5186b5..43afb13 100644
--- 
a/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.HBaseFsckTool.html
+++ 
b/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.HBaseFsckTool.html
@@ -36,5085 +36,5084 @@
 028import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.Multimap;
 029import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.Ordering;
 030import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.TreeMultimap;
-031import 
com.google.protobuf.ServiceException;
-032
-033import java.io.Closeable;
-034import java.io.FileNotFoundException;
-035import java.io.IOException;
-036import java.io.InterruptedIOException;
-037import java.io.PrintWriter;
-038import java.io.StringWriter;
-039import java.net.InetAddress;
-040import java.net.URI;
-041import java.util.ArrayList;
-042import java.util.Arrays;
-043import java.util.Collection;
-044import java.util.Collections;
-045import java.util.Comparator;
-046import java.util.HashMap;
-047import java.util.HashSet;
-048import java.util.Iterator;
-049import java.util.List;
-050import java.util.Locale;
-051import java.util.Map;
-052import java.util.Map.Entry;
-053import java.util.Set;
-054import java.util.SortedMap;
-055import java.util.SortedSet;
-056import java.util.TreeMap;
-057import java.util.TreeSet;
-058import java.util.Vector;
-059import java.util.concurrent.Callable;
-060import 
java.util.concurrent.ConcurrentSkipListMap;
-061import 
java.util.concurrent.ExecutionException;
-062import 
java.util.concurrent.ExecutorService;
-063import java.util.concurrent.Executors;
-064import java.util.concurrent.Future;
-065import java.util.concurrent.FutureTask;
-066import 
java.util.concurrent.ScheduledThreadPoolExecutor;
-067import java.util.concurrent.TimeUnit;
-068import 
java.util.concurrent.TimeoutException;
-069import 
java.util.concurrent.atomic.AtomicBoolean;
-070import 
java.util.concurrent.atomic.AtomicInteger;
-071
-072import org.apache.commons.io.IOUtils;
-073import 
org.apache.commons.lang.RandomStringUtils;
-074import 
org.apache.commons.lang.StringUtils;
-075import org.apache.commons.logging.Log;
-076import 
org.apache.commons.logging.LogFactory;
-077import 
org.apache.hadoop.conf.Configuration;
-078import 
org.apache.hadoop.conf.Configured;
-079import 
org.apache.hadoop.fs.FSDataOutputStream;
-080import org.apache.hadoop.fs.FileStatus;
-081import org.apache.hadoop.fs.FileSystem;
-082import org.apache.hadoop.fs.Path;
-083import 
org.apache.hadoop.fs.permission.FsAction;
-084import 
org.apache.hadoop.fs.permission.FsPermission;
-085import 
org.apache.hadoop.hbase.Abortable;
-086import org.apache.hadoop.hbase.Cell;
-087import 
org.apache.hadoop.hbase.CellUtil;
-088import 
org.apache.hadoop.hbase.ClusterStatus;
-089import 
org.apache.hadoop.hbase.HBaseConfiguration;
-090import 
org.apache.hadoop.hbase.HBaseInterfaceAudience;
-091import 
org.apache.hadoop.hbase.HConstants;
-092import 
org.apache.hadoop.hbase.HRegionInfo;
-093import 
org.apache.hadoop.hbase.HRegionLocation;
-094import 
org.apache.hadoop.hbase.KeyValue;
-095import 
org.apache.hadoop.hbase.MasterNotRunningException;
-096import 
org.apache.hadoop.hbase.MetaTableAccessor;
-097import 
org.apache.hadoop.hbase.RegionLocations;
-098import 
org.apache.hadoop.hbase.ServerName;
-099import 
org.apache.hadoop.hbase.TableName;
-100import 
org.apache.hadoop.hbase.ZooKeeperConnectionException;
-101import 
org.apache.hadoop.hbase.classification.InterfaceAudience;
-102import 
org.apache.hadoop.hbase.classification.InterfaceStability;
-103import 
org.apache.hadoop.hbase.client.Admin;
-104import 
org.apache.hadoop.hbase.client.ClusterConnection;
-105import 
org.apache.hadoop.hbase.client.Connection;
-106import 
org.apache.hadoop.hbase.client.ConnectionFactory;
-107import 
org.apache.hadoop.hbase.client.Delete;
-108import 
org.apache.hadoop.hbase.client.Get;
-109import 
org.apache.hadoop.hbase.client.Put;
-110import 
org.apache.hadoop.hbase.client.RegionReplicaUtil;
-111import 
org.apache.hadoop.hbase.client.Result;
-112import 
org.apache.hadoop.hbase.client.RowMutations;
-113import 
org.apache.hadoop.hbase.client.Table;
-114import 
org.apache.hadoop.hbase.client.TableState;
-115import 
org.apache.hadoop.hbase.io.FileLink;
-116import 
org.apache.hadoop.hbase.io.HFileLink;
-117import 
org.apache.hadoop.hbase.io.hfile.CacheConfig;
-118import 
org.apache.hadoop.hbase.io.hfile.HFile;
-119import 
org.apache.hadoop.hbase.master.MasterFileSystem;
-120import 
org.apache.hadoop.hbase.master.RegionState;
-121import 
org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-122import 

[08/51] [partial] hbase-site git commit: Published site at .

2017-08-25 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.TableInfo.HDFSIntegrityFixer.html
--
diff --git 
a/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.TableInfo.HDFSIntegrityFixer.html
 
b/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.TableInfo.HDFSIntegrityFixer.html
index f5186b5..43afb13 100644
--- 
a/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.TableInfo.HDFSIntegrityFixer.html
+++ 
b/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.TableInfo.HDFSIntegrityFixer.html
@@ -36,5085 +36,5084 @@
 028import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.Multimap;
 029import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.Ordering;
 030import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.TreeMultimap;
-031import 
com.google.protobuf.ServiceException;
-032
-033import java.io.Closeable;
-034import java.io.FileNotFoundException;
-035import java.io.IOException;
-036import java.io.InterruptedIOException;
-037import java.io.PrintWriter;
-038import java.io.StringWriter;
-039import java.net.InetAddress;
-040import java.net.URI;
-041import java.util.ArrayList;
-042import java.util.Arrays;
-043import java.util.Collection;
-044import java.util.Collections;
-045import java.util.Comparator;
-046import java.util.HashMap;
-047import java.util.HashSet;
-048import java.util.Iterator;
-049import java.util.List;
-050import java.util.Locale;
-051import java.util.Map;
-052import java.util.Map.Entry;
-053import java.util.Set;
-054import java.util.SortedMap;
-055import java.util.SortedSet;
-056import java.util.TreeMap;
-057import java.util.TreeSet;
-058import java.util.Vector;
-059import java.util.concurrent.Callable;
-060import 
java.util.concurrent.ConcurrentSkipListMap;
-061import 
java.util.concurrent.ExecutionException;
-062import 
java.util.concurrent.ExecutorService;
-063import java.util.concurrent.Executors;
-064import java.util.concurrent.Future;
-065import java.util.concurrent.FutureTask;
-066import 
java.util.concurrent.ScheduledThreadPoolExecutor;
-067import java.util.concurrent.TimeUnit;
-068import 
java.util.concurrent.TimeoutException;
-069import 
java.util.concurrent.atomic.AtomicBoolean;
-070import 
java.util.concurrent.atomic.AtomicInteger;
-071
-072import org.apache.commons.io.IOUtils;
-073import 
org.apache.commons.lang.RandomStringUtils;
-074import 
org.apache.commons.lang.StringUtils;
-075import org.apache.commons.logging.Log;
-076import 
org.apache.commons.logging.LogFactory;
-077import 
org.apache.hadoop.conf.Configuration;
-078import 
org.apache.hadoop.conf.Configured;
-079import 
org.apache.hadoop.fs.FSDataOutputStream;
-080import org.apache.hadoop.fs.FileStatus;
-081import org.apache.hadoop.fs.FileSystem;
-082import org.apache.hadoop.fs.Path;
-083import 
org.apache.hadoop.fs.permission.FsAction;
-084import 
org.apache.hadoop.fs.permission.FsPermission;
-085import 
org.apache.hadoop.hbase.Abortable;
-086import org.apache.hadoop.hbase.Cell;
-087import 
org.apache.hadoop.hbase.CellUtil;
-088import 
org.apache.hadoop.hbase.ClusterStatus;
-089import 
org.apache.hadoop.hbase.HBaseConfiguration;
-090import 
org.apache.hadoop.hbase.HBaseInterfaceAudience;
-091import 
org.apache.hadoop.hbase.HConstants;
-092import 
org.apache.hadoop.hbase.HRegionInfo;
-093import 
org.apache.hadoop.hbase.HRegionLocation;
-094import 
org.apache.hadoop.hbase.KeyValue;
-095import 
org.apache.hadoop.hbase.MasterNotRunningException;
-096import 
org.apache.hadoop.hbase.MetaTableAccessor;
-097import 
org.apache.hadoop.hbase.RegionLocations;
-098import 
org.apache.hadoop.hbase.ServerName;
-099import 
org.apache.hadoop.hbase.TableName;
-100import 
org.apache.hadoop.hbase.ZooKeeperConnectionException;
-101import 
org.apache.hadoop.hbase.classification.InterfaceAudience;
-102import 
org.apache.hadoop.hbase.classification.InterfaceStability;
-103import 
org.apache.hadoop.hbase.client.Admin;
-104import 
org.apache.hadoop.hbase.client.ClusterConnection;
-105import 
org.apache.hadoop.hbase.client.Connection;
-106import 
org.apache.hadoop.hbase.client.ConnectionFactory;
-107import 
org.apache.hadoop.hbase.client.Delete;
-108import 
org.apache.hadoop.hbase.client.Get;
-109import 
org.apache.hadoop.hbase.client.Put;
-110import 
org.apache.hadoop.hbase.client.RegionReplicaUtil;
-111import 
org.apache.hadoop.hbase.client.Result;
-112import 
org.apache.hadoop.hbase.client.RowMutations;
-113import 
org.apache.hadoop.hbase.client.Table;
-114import 
org.apache.hadoop.hbase.client.TableState;
-115import 
org.apache.hadoop.hbase.io.FileLink;
-116import 
org.apache.hadoop.hbase.io.HFileLink;
-117import 
org.apache.hadoop.hbase.io.hfile.CacheConfig;
-118import 
org.apache.hadoop.hbase.io.hfile.HFile;
-119import 
org.apache.hadoop.hbase.master.MasterFileSystem;
-120import 
org.apache.hadoop.hbase.master.RegionState;
-121import 

[36/51] [partial] hbase-site git commit: Published site at .

2017-08-25 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/devapidocs/org/apache/hadoop/hbase/security/visibility/VisibilityReplicationEndpoint.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/security/visibility/VisibilityReplicationEndpoint.html
 
b/devapidocs/org/apache/hadoop/hbase/security/visibility/VisibilityReplicationEndpoint.html
index 883bea2..4e16bd0 100644
--- 
a/devapidocs/org/apache/hadoop/hbase/security/visibility/VisibilityReplicationEndpoint.html
+++ 
b/devapidocs/org/apache/hadoop/hbase/security/visibility/VisibilityReplicationEndpoint.html
@@ -18,7 +18,7 @@
 catch(err) {
 }
 //-->
-var methods = 
{"i0":10,"i1":10,"i2":10,"i3":10,"i4":10,"i5":10,"i6":10,"i7":10,"i8":10,"i9":10,"i10":10,"i11":10,"i12":10,"i13":10,"i14":10,"i15":10};
+var methods = 
{"i0":10,"i1":10,"i2":10,"i3":10,"i4":10,"i5":10,"i6":10,"i7":10,"i8":10,"i9":10,"i10":10,"i11":10,"i12":10,"i13":10,"i14":10};
 var tabs = {65535:["t0","All Methods"],2:["t2","Instance 
Methods"],8:["t4","Concrete Methods"]};
 var altColor = "altColor";
 var rowColor = "rowColor";
@@ -109,7 +109,7 @@ var activeTableTab = "activeTableTab";
 
 
 All Implemented Interfaces:
-ReplicationEndpoint, ReplicationPeerConfigListener, 
org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.Service
+ReplicationEndpoint, ReplicationPeerConfigListener
 
 
 
@@ -136,13 +136,6 @@ implements ReplicationEndpoint
 ReplicationEndpoint.Context, ReplicationEndpoint.ReplicateContext
 
-
-
-
-
-Nested classes/interfaces inherited from 
interfaceorg.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.Service
-org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.Service.Listener,
 
org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.Service.State
-
 
 
 
@@ -204,59 +197,72 @@ implements 
 void
-addListener(org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.Service.Listenerlistener,
-   http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/Executor.html?is-external=true;
 title="class or interface in 
java.util.concurrent">Executorexecutor)
+awaitRunning()
+Waits for the ReplicationEndpoint to be 
up and running.
+
 
 
 void
-awaitRunning()
+awaitRunning(longtimeout,
+http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/TimeUnit.html?is-external=true;
 title="class or interface in 
java.util.concurrent">TimeUnitunit)
+Waits for the ReplicationEndpoint to to 
be up and running for no more
+ than the given time.
+
 
 
 void
-awaitRunning(longl,
-http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/TimeUnit.html?is-external=true;
 title="class or interface in 
java.util.concurrent">TimeUnittimeUnit)
+awaitTerminated()
+Waits for the ReplicationEndpoint to 
reach the terminated (internal) state.
+
 
 
 void
-awaitTerminated()
+awaitTerminated(longtimeout,
+   http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/TimeUnit.html?is-external=true;
 title="class or interface in 
java.util.concurrent">TimeUnitunit)
+Waits for the ReplicationEndpoint to 
reach a terminal state for no
+ more than the given time.
+
 
 
-void
-awaitTerminated(longl,
-   http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/TimeUnit.html?is-external=true;
 title="class or interface in 
java.util.concurrent">TimeUnittimeUnit)
-
-
 boolean
 canReplicateToSameCluster()
 Whether or not, the replication endpoint can replicate to 
it's source cluster with the same
  UUID
 
 
-
+
 http://docs.oracle.com/javase/8/docs/api/java/lang/Throwable.html?is-external=true;
 title="class or interface in java.lang">Throwable
-failureCause()
+failureCause()
+Returns the http://docs.oracle.com/javase/8/docs/api/java/lang/Throwable.html?is-external=true;
 title="class or interface in java.lang">Throwable that caused 
this service to fail.
+
 
-
+
 http://docs.oracle.com/javase/8/docs/api/java/util/UUID.html?is-external=true;
 title="class or interface in java.util">UUID
 getPeerUUID()
 Returns a UUID of the provided peer id.
 
 
-
+
 WALEntryFilter
 getWALEntryfilter()
 Returns a WALEntryFilter to use for filtering out 
WALEntries from the log.
 
 
-
+
 void
 init(ReplicationEndpoint.Contextcontext)
 Initialize the replication endpoint with the given 
context.
 
 
+
+boolean
+isRunning()
+Returns true if this service is RUNNING.
+
+
 
 boolean
-isRunning()
+isStarting()
 
 
 void
@@ -271,16 +277,16 @@ implements 
-org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.Service
-startAsync()
+void
+start()
+Initiates service startup and returns immediately.
+
 
 
-org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.Service.State
-state()
-
-
-org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.Service
-stopAsync()
+void
+stop()
+If the service is starting or running, this initiates 
service shutdown and returns immediately.
+
 
 
 
@@ -319,7 +325,7 

[22/51] [partial] hbase-site git commit: Published site at .

2017-08-25 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/devapidocs/src-html/org/apache/hadoop/hbase/rest/Constants.html
--
diff --git a/devapidocs/src-html/org/apache/hadoop/hbase/rest/Constants.html 
b/devapidocs/src-html/org/apache/hadoop/hbase/rest/Constants.html
index dbdd25a..4391c4c 100644
--- a/devapidocs/src-html/org/apache/hadoop/hbase/rest/Constants.html
+++ b/devapidocs/src-html/org/apache/hadoop/hbase/rest/Constants.html
@@ -86,13 +86,14 @@
 078  String SCAN_FETCH_SIZE = 
"hbase.rest.scan.fetchsize";
 079  String SCAN_FILTER = "filter";
 080  String SCAN_REVERSED = "reversed";
-081  String CUSTOM_FILTERS = 
"hbase.rest.custom.filters"; 
-082
-083  String ROW_KEYS_PARAM_NAME = "row";
-084  /** If this query parameter is present 
when processing row or scanner resources,
-085  it disables server side block 
caching */
-086  String NOCACHE_PARAM_NAME = 
"nocache";
-087}
+081  String SCAN_CACHE_BLOCKS = 
"cacheblocks";
+082  String CUSTOM_FILTERS = 
"hbase.rest.custom.filters"; 
+083
+084  String ROW_KEYS_PARAM_NAME = "row";
+085  /** If this query parameter is present 
when processing row or scanner resources,
+086  it disables server side block 
caching */
+087  String NOCACHE_PARAM_NAME = 
"nocache";
+088}
 
 
 

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/devapidocs/src-html/org/apache/hadoop/hbase/rest/TableResource.html
--
diff --git 
a/devapidocs/src-html/org/apache/hadoop/hbase/rest/TableResource.html 
b/devapidocs/src-html/org/apache/hadoop/hbase/rest/TableResource.html
index b1ff31d..e8a555d 100644
--- a/devapidocs/src-html/org/apache/hadoop/hbase/rest/TableResource.html
+++ b/devapidocs/src-html/org/apache/hadoop/hbase/rest/TableResource.html
@@ -129,95 +129,94 @@
 121
 122  @Path("{scanspec: .*[*]$}")
 123  public TableScanResource  
getScanResource(
-124  final @Context UriInfo uriInfo,
-125  final @PathParam("scanspec") String 
scanSpec,
-126  final @HeaderParam("Accept") String 
contentType,
-127  @DefaultValue(Integer.MAX_VALUE + 
"")
-128  @QueryParam(Constants.SCAN_LIMIT) 
int userRequestedLimit,
-129  @DefaultValue("") 
@QueryParam(Constants.SCAN_START_ROW) String startRow,
-130  @DefaultValue("") 
@QueryParam(Constants.SCAN_END_ROW) String endRow,
-131  @DefaultValue("") 
@QueryParam(Constants.SCAN_COLUMN) ListString column,
-132  @DefaultValue("1") 
@QueryParam(Constants.SCAN_MAX_VERSIONS) int maxVersions,
-133  @DefaultValue("-1") 
@QueryParam(Constants.SCAN_BATCH_SIZE) int batchSize,
-134  @DefaultValue("0") 
@QueryParam(Constants.SCAN_START_TIME) long startTime,
-135  @DefaultValue(Long.MAX_VALUE + "") 
@QueryParam(Constants.SCAN_END_TIME) long endTime,
-136  @DefaultValue("true") 
@QueryParam(Constants.SCAN_BATCH_SIZE) boolean cacheBlocks,
-137  @DefaultValue("false") 
@QueryParam(Constants.SCAN_REVERSED) boolean reversed,
-138  @DefaultValue("") 
@QueryParam(Constants.SCAN_FILTER) String paramFilter) {
-139try {
-140  Filter prefixFilter = null;
-141  Scan tableScan = new Scan();
-142  if (scanSpec.indexOf('*')  0) 
{
-143String prefix = 
scanSpec.substring(0, scanSpec.indexOf('*'));
-144byte[] prefixBytes = 
Bytes.toBytes(prefix);
-145prefixFilter = new 
PrefixFilter(Bytes.toBytes(prefix));
-146if (startRow.isEmpty()) {
-147  
tableScan.setStartRow(prefixBytes);
-148}
-149  }
-150  if (LOG.isTraceEnabled()) {
-151LOG.trace("Query parameters  : 
Table Name =  " + this.table + " Start Row = " + startRow
-152+ " End Row = " + endRow 
+ " Columns = " + column + " Start Time = " + startTime
-153+ " End Time = " + 
endTime + " Cache Blocks = " + cacheBlocks + " Max Versions = "
-154+ maxVersions + " Batch Size 
= " + batchSize);
-155  }
-156  Table hTable = 
RESTServlet.getInstance().getTable(this.table);
-157  tableScan.setBatch(batchSize);
-158  
tableScan.setMaxVersions(maxVersions);
-159  tableScan.setTimeRange(startTime, 
endTime);
-160  if (!startRow.isEmpty()) {
-161
tableScan.setStartRow(Bytes.toBytes(startRow));
-162  }
-163  
tableScan.setStopRow(Bytes.toBytes(endRow));
-164  for (String csplit : column) {
-165String[] familysplit = 
csplit.trim().split(":");
-166if (familysplit.length == 2) {
-167  if (familysplit[1].length() 
 0) {
-168if (LOG.isTraceEnabled()) {
-169  LOG.trace("Scan family and 
column : " + familysplit[0] + "  " + familysplit[1]);
-170}
-171
tableScan.addColumn(Bytes.toBytes(familysplit[0]), 
Bytes.toBytes(familysplit[1]));
-172  } else {
-173
tableScan.addFamily(Bytes.toBytes(familysplit[0]));
-174if (LOG.isTraceEnabled()) {
-175  LOG.trace("Scan 

[25/51] [partial] hbase-site git commit: Published site at .

2017-08-25 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/devapidocs/src-html/org/apache/hadoop/hbase/replication/ReplicationEndpoint.html
--
diff --git 
a/devapidocs/src-html/org/apache/hadoop/hbase/replication/ReplicationEndpoint.html
 
b/devapidocs/src-html/org/apache/hadoop/hbase/replication/ReplicationEndpoint.html
index 9038982..efa528d 100644
--- 
a/devapidocs/src-html/org/apache/hadoop/hbase/replication/ReplicationEndpoint.html
+++ 
b/devapidocs/src-html/org/apache/hadoop/hbase/replication/ReplicationEndpoint.html
@@ -29,17 +29,17 @@
 021import java.io.IOException;
 022import java.util.List;
 023import java.util.UUID;
-024
-025import 
org.apache.hadoop.hbase.Abortable;
-026import 
org.apache.hadoop.hbase.classification.InterfaceAudience;
-027import 
org.apache.hadoop.conf.Configuration;
-028import org.apache.hadoop.fs.FileSystem;
-029import 
org.apache.hadoop.hbase.HBaseInterfaceAudience;
-030import 
org.apache.hadoop.hbase.TableDescriptors;
-031import 
org.apache.hadoop.hbase.wal.WAL.Entry;
-032import 
org.apache.hadoop.hbase.replication.regionserver.MetricsSource;
-033
-034import 
org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.Service;
+024import java.util.concurrent.TimeUnit;
+025import 
java.util.concurrent.TimeoutException;
+026
+027import 
org.apache.hadoop.hbase.Abortable;
+028import 
org.apache.hadoop.hbase.classification.InterfaceAudience;
+029import 
org.apache.hadoop.conf.Configuration;
+030import org.apache.hadoop.fs.FileSystem;
+031import 
org.apache.hadoop.hbase.HBaseInterfaceAudience;
+032import 
org.apache.hadoop.hbase.TableDescriptors;
+033import 
org.apache.hadoop.hbase.wal.WAL.Entry;
+034import 
org.apache.hadoop.hbase.replication.regionserver.MetricsSource;
 035
 036/**
 037 * ReplicationEndpoint is a plugin which 
implements replication
@@ -55,136 +55,216 @@
 047 * and persisting of the WAL entries in 
the other cluster.
 048 */
 049@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.REPLICATION)
-050public interface ReplicationEndpoint 
extends Service, ReplicationPeerConfigListener {
-051
-052  
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.REPLICATION)
-053  class Context {
-054private final Configuration conf;
-055private final FileSystem fs;
-056private final TableDescriptors 
tableDescriptors;
-057private final ReplicationPeer 
replicationPeer;
-058private final String peerId;
-059private final UUID clusterId;
-060private final MetricsSource 
metrics;
-061private final Abortable abortable;
-062
-063@InterfaceAudience.Private
-064public Context(
-065final Configuration conf,
-066final FileSystem fs,
-067final String peerId,
-068final UUID clusterId,
-069final ReplicationPeer 
replicationPeer,
-070final MetricsSource metrics,
-071final TableDescriptors 
tableDescriptors,
-072final Abortable abortable) {
-073  this.conf = conf;
-074  this.fs = fs;
-075  this.clusterId = clusterId;
-076  this.peerId = peerId;
-077  this.replicationPeer = 
replicationPeer;
-078  this.metrics = metrics;
-079  this.tableDescriptors = 
tableDescriptors;
-080  this.abortable = abortable;
-081}
-082public Configuration 
getConfiguration() {
-083  return conf;
-084}
-085public FileSystem getFilesystem() {
-086  return fs;
-087}
-088public UUID getClusterId() {
-089  return clusterId;
-090}
-091public String getPeerId() {
-092  return peerId;
-093}
-094public ReplicationPeerConfig 
getPeerConfig() {
-095  return 
replicationPeer.getPeerConfig();
-096}
-097public ReplicationPeer 
getReplicationPeer() {
-098  return replicationPeer;
-099}
-100public MetricsSource getMetrics() {
-101  return metrics;
-102}
-103public TableDescriptors 
getTableDescriptors() {
-104  return tableDescriptors;
-105}
-106public Abortable getAbortable() { 
return abortable; }
-107  }
-108
-109  /**
-110   * Initialize the replication endpoint 
with the given context.
-111   * @param context replication context
-112   * @throws IOException
-113   */
-114  void init(Context context) throws 
IOException;
-115
-116  /** Whether or not, the replication 
endpoint can replicate to it's source cluster with the same
-117   * UUID */
-118  boolean canReplicateToSameCluster();
-119
-120  /**
-121   * Returns a UUID of the provided peer 
id. Every HBase cluster instance has a persisted
-122   * associated UUID. If the replication 
is not performed to an actual HBase cluster (but
-123   * some other system), the UUID 
returned has to uniquely identify the connected target system.
-124   * @return a UUID or null if the peer 
cluster does not exist or is not connected.
-125   */
-126  UUID getPeerUUID();
-127
-128  /**
-129   * Returns a WALEntryFilter to use for 
filtering out WALEntries from the 

[01/51] [partial] hbase-site git commit: Published site at .

2017-08-25 Thread git-site-role
Repository: hbase-site
Updated Branches:
  refs/heads/asf-site 0c22829dd -> 6b5143ed1


http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.html
--
diff --git a/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.html 
b/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.html
index f5186b5..43afb13 100644
--- a/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.html
+++ b/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.html
@@ -36,5085 +36,5084 @@
 028import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.Multimap;
 029import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.Ordering;
 030import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.TreeMultimap;
-031import 
com.google.protobuf.ServiceException;
-032
-033import java.io.Closeable;
-034import java.io.FileNotFoundException;
-035import java.io.IOException;
-036import java.io.InterruptedIOException;
-037import java.io.PrintWriter;
-038import java.io.StringWriter;
-039import java.net.InetAddress;
-040import java.net.URI;
-041import java.util.ArrayList;
-042import java.util.Arrays;
-043import java.util.Collection;
-044import java.util.Collections;
-045import java.util.Comparator;
-046import java.util.HashMap;
-047import java.util.HashSet;
-048import java.util.Iterator;
-049import java.util.List;
-050import java.util.Locale;
-051import java.util.Map;
-052import java.util.Map.Entry;
-053import java.util.Set;
-054import java.util.SortedMap;
-055import java.util.SortedSet;
-056import java.util.TreeMap;
-057import java.util.TreeSet;
-058import java.util.Vector;
-059import java.util.concurrent.Callable;
-060import 
java.util.concurrent.ConcurrentSkipListMap;
-061import 
java.util.concurrent.ExecutionException;
-062import 
java.util.concurrent.ExecutorService;
-063import java.util.concurrent.Executors;
-064import java.util.concurrent.Future;
-065import java.util.concurrent.FutureTask;
-066import 
java.util.concurrent.ScheduledThreadPoolExecutor;
-067import java.util.concurrent.TimeUnit;
-068import 
java.util.concurrent.TimeoutException;
-069import 
java.util.concurrent.atomic.AtomicBoolean;
-070import 
java.util.concurrent.atomic.AtomicInteger;
-071
-072import org.apache.commons.io.IOUtils;
-073import 
org.apache.commons.lang.RandomStringUtils;
-074import 
org.apache.commons.lang.StringUtils;
-075import org.apache.commons.logging.Log;
-076import 
org.apache.commons.logging.LogFactory;
-077import 
org.apache.hadoop.conf.Configuration;
-078import 
org.apache.hadoop.conf.Configured;
-079import 
org.apache.hadoop.fs.FSDataOutputStream;
-080import org.apache.hadoop.fs.FileStatus;
-081import org.apache.hadoop.fs.FileSystem;
-082import org.apache.hadoop.fs.Path;
-083import 
org.apache.hadoop.fs.permission.FsAction;
-084import 
org.apache.hadoop.fs.permission.FsPermission;
-085import 
org.apache.hadoop.hbase.Abortable;
-086import org.apache.hadoop.hbase.Cell;
-087import 
org.apache.hadoop.hbase.CellUtil;
-088import 
org.apache.hadoop.hbase.ClusterStatus;
-089import 
org.apache.hadoop.hbase.HBaseConfiguration;
-090import 
org.apache.hadoop.hbase.HBaseInterfaceAudience;
-091import 
org.apache.hadoop.hbase.HConstants;
-092import 
org.apache.hadoop.hbase.HRegionInfo;
-093import 
org.apache.hadoop.hbase.HRegionLocation;
-094import 
org.apache.hadoop.hbase.KeyValue;
-095import 
org.apache.hadoop.hbase.MasterNotRunningException;
-096import 
org.apache.hadoop.hbase.MetaTableAccessor;
-097import 
org.apache.hadoop.hbase.RegionLocations;
-098import 
org.apache.hadoop.hbase.ServerName;
-099import 
org.apache.hadoop.hbase.TableName;
-100import 
org.apache.hadoop.hbase.ZooKeeperConnectionException;
-101import 
org.apache.hadoop.hbase.classification.InterfaceAudience;
-102import 
org.apache.hadoop.hbase.classification.InterfaceStability;
-103import 
org.apache.hadoop.hbase.client.Admin;
-104import 
org.apache.hadoop.hbase.client.ClusterConnection;
-105import 
org.apache.hadoop.hbase.client.Connection;
-106import 
org.apache.hadoop.hbase.client.ConnectionFactory;
-107import 
org.apache.hadoop.hbase.client.Delete;
-108import 
org.apache.hadoop.hbase.client.Get;
-109import 
org.apache.hadoop.hbase.client.Put;
-110import 
org.apache.hadoop.hbase.client.RegionReplicaUtil;
-111import 
org.apache.hadoop.hbase.client.Result;
-112import 
org.apache.hadoop.hbase.client.RowMutations;
-113import 
org.apache.hadoop.hbase.client.Table;
-114import 
org.apache.hadoop.hbase.client.TableState;
-115import 
org.apache.hadoop.hbase.io.FileLink;
-116import 
org.apache.hadoop.hbase.io.HFileLink;
-117import 
org.apache.hadoop.hbase.io.hfile.CacheConfig;
-118import 
org.apache.hadoop.hbase.io.hfile.HFile;
-119import 
org.apache.hadoop.hbase.master.MasterFileSystem;
-120import 
org.apache.hadoop.hbase.master.RegionState;
-121import 
org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-122import 

[50/51] [partial] hbase-site git commit: Published site at .

2017-08-25 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/checkstyle-aggregate.html
--
diff --git a/checkstyle-aggregate.html b/checkstyle-aggregate.html
index fe5256b..922278c 100644
--- a/checkstyle-aggregate.html
+++ b/checkstyle-aggregate.html
@@ -7,7 +7,7 @@
   
 
 
-
+
 
 Apache HBase  Checkstyle Results
 
@@ -286,10 +286,10 @@
 Warnings
 Errors
 
-2030
+2031
 0
 0
-12845
+12867
 
 Files
 
@@ -1374,305 +1374,315 @@
 0
 2
 
+org/apache/hadoop/hbase/client/example/RefreshHFilesClient.java
+0
+0
+16
+
 org/apache/hadoop/hbase/client/locking/EntityLock.java
 0
 0
 5
-
+
 org/apache/hadoop/hbase/client/locking/LockServiceClient.java
 0
 0
 2
-
+
 org/apache/hadoop/hbase/client/metrics/ScanMetrics.java
 0
 0
 12
-
+
 org/apache/hadoop/hbase/client/metrics/ServerSideScanMetrics.java
 0
 0
 10
-
+
 org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
 0
 0
 37
-
+
 org/apache/hadoop/hbase/client/replication/ReplicationSerDeHelper.java
 0
 0
 11
-
+
 org/apache/hadoop/hbase/client/replication/TableCFs.java
 0
 0
 1
-
+
 org/apache/hadoop/hbase/codec/BaseDecoder.java
 0
 0
 5
-
+
 org/apache/hadoop/hbase/codec/BaseEncoder.java
 0
 0
 2
-
+
 org/apache/hadoop/hbase/codec/CellCodec.java
 0
 0
 5
-
+
 org/apache/hadoop/hbase/codec/CellCodecWithTags.java
 0
 0
 5
-
+
 org/apache/hadoop/hbase/codec/KeyValueCodecWithTags.java
 0
 0
 1
-
+
 org/apache/hadoop/hbase/codec/MessageCodec.java
 0
 0
 1
-
+
 org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeBlockMeta.java
 0
 0
 103
-
+
 org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeCodec.java
 0
 0
 2
-
+
 org/apache/hadoop/hbase/codec/prefixtree/decode/DecoderFactory.java
 0
 0
 1
-
+
 org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeArrayReversibleScanner.java
 0
 0
 1
-
+
 org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeArrayScanner.java
 0
 0
 1
-
+
 org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeArraySearcher.java
 0
 0
 4
-
+
 org/apache/hadoop/hbase/codec/prefixtree/decode/row/RowNodeReader.java
 0
 0
 3
-
+
 org/apache/hadoop/hbase/codec/prefixtree/encode/EncoderFactory.java
 0
 0
 1
-
+
 org/apache/hadoop/hbase/codec/prefixtree/encode/PrefixTreeEncoder.java
 0
 0
 2
-
+
 org/apache/hadoop/hbase/codec/prefixtree/encode/column/ColumnSectionWriter.java
 0
 0
 2
-
+
 org/apache/hadoop/hbase/codec/prefixtree/encode/other/LongEncoder.java
 0
 0
 1
-
+
 org/apache/hadoop/hbase/codec/prefixtree/encode/row/RowSectionWriter.java
 0
 0
 1
-
+
 org/apache/hadoop/hbase/codec/prefixtree/encode/tokenize/Tokenizer.java
 0
 0
 1
-
+
 org/apache/hadoop/hbase/codec/prefixtree/encode/tokenize/TokenizerNode.java
 0
 0
 1
-
+
 org/apache/hadoop/hbase/codec/prefixtree/scanner/CellSearcher.java
 0
 0
 3
-
+
 org/apache/hadoop/hbase/codec/prefixtree/scanner/ReversibleCellScanner.java
 0
 0
 3
-
+
 org/apache/hadoop/hbase/conf/ConfigurationManager.java
 0
 0
 3
-
+
 org/apache/hadoop/hbase/constraint/BaseConstraint.java
 0
 0
 1
-
+
 org/apache/hadoop/hbase/constraint/Constraint.java
 0
 0
 1
-
+
 org/apache/hadoop/hbase/constraint/ConstraintException.java
 0
 0
 1
-
+
 org/apache/hadoop/hbase/constraint/ConstraintProcessor.java
 0
 0
 3
-
+
 org/apache/hadoop/hbase/constraint/Constraints.java
 0
 0
 3
-
+
 org/apache/hadoop/hbase/constraint/package-info.java
 0
 0
 61
-
+
 org/apache/hadoop/hbase/coordination/SplitLogManagerCoordination.java
 0
 0
 6
-
+
 org/apache/hadoop/hbase/coordination/SplitLogWorkerCoordination.java
 0
 0
 1
-
+
 org/apache/hadoop/hbase/coordination/ZKSplitLogManagerCoordination.java
 0
 0
 12
-
+
 org/apache/hadoop/hbase/coordination/ZkSplitLogWorkerCoordination.java
 0
 0
 7
-
+
 org/apache/hadoop/hbase/coprocessor/AggregateImplementation.java
 0
 0
 10
-
+
 org/apache/hadoop/hbase/coprocessor/BaseRowProcessorEndpoint.java
 0
 0
 3
-
+
 org/apache/hadoop/hbase/coprocessor/BulkLoadObserver.java
 0
 0
 3
-
+
 org/apache/hadoop/hbase/coprocessor/ColumnInterpreter.java
 0
 0
 21
-
+
 org/apache/hadoop/hbase/coprocessor/CoprocessorException.java
 0
 0
 3
-
+
 org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
 0
 0
 12
-
+
 org/apache/hadoop/hbase/coprocessor/CoprocessorService.java
 0
 0
 1
-
+
 org/apache/hadoop/hbase/coprocessor/EndpointObserver.java
 0
 0
 2
-
+
 org/apache/hadoop/hbase/coprocessor/MasterCoprocessorEnvironment.java
 0
 0
 1
-
+
 org/apache/hadoop/hbase/coprocessor/MasterObserver.java
 0
 0
 27
-
+
 org/apache/hadoop/hbase/coprocessor/MetricsCoprocessor.java
 0
 0
 1
-
+
 org/apache/hadoop/hbase/coprocessor/MultiRowMutationEndpoint.java
 0
 0
 5
-
+
 org/apache/hadoop/hbase/coprocessor/ObserverContext.java
 0
 0
 2
-
+
 org/apache/hadoop/hbase/coprocessor/RegionCoprocessorEnvironment.java
 0
 0
 1
-
+
 org/apache/hadoop/hbase/coprocessor/RegionObserver.java
 0
 0
 48
-
+
 org/apache/hadoop/hbase/coprocessor/RegionServerObserver.java
 0
 0
 1
-
+
 

[19/51] [partial] hbase-site git commit: Published site at .

2017-08-25 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.ErrorReporter.ERROR_CODE.html
--
diff --git 
a/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.ErrorReporter.ERROR_CODE.html
 
b/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.ErrorReporter.ERROR_CODE.html
index f5186b5..43afb13 100644
--- 
a/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.ErrorReporter.ERROR_CODE.html
+++ 
b/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.ErrorReporter.ERROR_CODE.html
@@ -36,5085 +36,5084 @@
 028import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.Multimap;
 029import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.Ordering;
 030import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.TreeMultimap;
-031import 
com.google.protobuf.ServiceException;
-032
-033import java.io.Closeable;
-034import java.io.FileNotFoundException;
-035import java.io.IOException;
-036import java.io.InterruptedIOException;
-037import java.io.PrintWriter;
-038import java.io.StringWriter;
-039import java.net.InetAddress;
-040import java.net.URI;
-041import java.util.ArrayList;
-042import java.util.Arrays;
-043import java.util.Collection;
-044import java.util.Collections;
-045import java.util.Comparator;
-046import java.util.HashMap;
-047import java.util.HashSet;
-048import java.util.Iterator;
-049import java.util.List;
-050import java.util.Locale;
-051import java.util.Map;
-052import java.util.Map.Entry;
-053import java.util.Set;
-054import java.util.SortedMap;
-055import java.util.SortedSet;
-056import java.util.TreeMap;
-057import java.util.TreeSet;
-058import java.util.Vector;
-059import java.util.concurrent.Callable;
-060import 
java.util.concurrent.ConcurrentSkipListMap;
-061import 
java.util.concurrent.ExecutionException;
-062import 
java.util.concurrent.ExecutorService;
-063import java.util.concurrent.Executors;
-064import java.util.concurrent.Future;
-065import java.util.concurrent.FutureTask;
-066import 
java.util.concurrent.ScheduledThreadPoolExecutor;
-067import java.util.concurrent.TimeUnit;
-068import 
java.util.concurrent.TimeoutException;
-069import 
java.util.concurrent.atomic.AtomicBoolean;
-070import 
java.util.concurrent.atomic.AtomicInteger;
-071
-072import org.apache.commons.io.IOUtils;
-073import 
org.apache.commons.lang.RandomStringUtils;
-074import 
org.apache.commons.lang.StringUtils;
-075import org.apache.commons.logging.Log;
-076import 
org.apache.commons.logging.LogFactory;
-077import 
org.apache.hadoop.conf.Configuration;
-078import 
org.apache.hadoop.conf.Configured;
-079import 
org.apache.hadoop.fs.FSDataOutputStream;
-080import org.apache.hadoop.fs.FileStatus;
-081import org.apache.hadoop.fs.FileSystem;
-082import org.apache.hadoop.fs.Path;
-083import 
org.apache.hadoop.fs.permission.FsAction;
-084import 
org.apache.hadoop.fs.permission.FsPermission;
-085import 
org.apache.hadoop.hbase.Abortable;
-086import org.apache.hadoop.hbase.Cell;
-087import 
org.apache.hadoop.hbase.CellUtil;
-088import 
org.apache.hadoop.hbase.ClusterStatus;
-089import 
org.apache.hadoop.hbase.HBaseConfiguration;
-090import 
org.apache.hadoop.hbase.HBaseInterfaceAudience;
-091import 
org.apache.hadoop.hbase.HConstants;
-092import 
org.apache.hadoop.hbase.HRegionInfo;
-093import 
org.apache.hadoop.hbase.HRegionLocation;
-094import 
org.apache.hadoop.hbase.KeyValue;
-095import 
org.apache.hadoop.hbase.MasterNotRunningException;
-096import 
org.apache.hadoop.hbase.MetaTableAccessor;
-097import 
org.apache.hadoop.hbase.RegionLocations;
-098import 
org.apache.hadoop.hbase.ServerName;
-099import 
org.apache.hadoop.hbase.TableName;
-100import 
org.apache.hadoop.hbase.ZooKeeperConnectionException;
-101import 
org.apache.hadoop.hbase.classification.InterfaceAudience;
-102import 
org.apache.hadoop.hbase.classification.InterfaceStability;
-103import 
org.apache.hadoop.hbase.client.Admin;
-104import 
org.apache.hadoop.hbase.client.ClusterConnection;
-105import 
org.apache.hadoop.hbase.client.Connection;
-106import 
org.apache.hadoop.hbase.client.ConnectionFactory;
-107import 
org.apache.hadoop.hbase.client.Delete;
-108import 
org.apache.hadoop.hbase.client.Get;
-109import 
org.apache.hadoop.hbase.client.Put;
-110import 
org.apache.hadoop.hbase.client.RegionReplicaUtil;
-111import 
org.apache.hadoop.hbase.client.Result;
-112import 
org.apache.hadoop.hbase.client.RowMutations;
-113import 
org.apache.hadoop.hbase.client.Table;
-114import 
org.apache.hadoop.hbase.client.TableState;
-115import 
org.apache.hadoop.hbase.io.FileLink;
-116import 
org.apache.hadoop.hbase.io.HFileLink;
-117import 
org.apache.hadoop.hbase.io.hfile.CacheConfig;
-118import 
org.apache.hadoop.hbase.io.hfile.HFile;
-119import 
org.apache.hadoop.hbase.master.MasterFileSystem;
-120import 
org.apache.hadoop.hbase.master.RegionState;
-121import 

[34/51] [partial] hbase-site git commit: Published site at .

2017-08-25 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.MetaEntry.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.MetaEntry.html 
b/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.MetaEntry.html
index 18f8cca..93e2580 100644
--- a/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.MetaEntry.html
+++ b/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.MetaEntry.html
@@ -122,7 +122,7 @@ var activeTableTab = "activeTableTab";
 
 
 
-static class HBaseFsck.MetaEntry
+static class HBaseFsck.MetaEntry
 extends HRegionInfo
 Stores the regioninfo entries scanned from META
 
@@ -250,7 +250,7 @@ extends 
 
 regionServer
-ServerName regionServer
+ServerName regionServer
 
 
 
@@ -259,7 +259,7 @@ extends 
 
 modTime
-long modTime
+long modTime
 
 
 
@@ -268,7 +268,7 @@ extends 
 
 splitA
-HRegionInfo splitA
+HRegionInfo splitA
 
 
 
@@ -277,7 +277,7 @@ extends 
 
 splitB
-HRegionInfo splitB
+HRegionInfo splitB
 
 
 
@@ -294,7 +294,7 @@ extends 
 
 MetaEntry
-publicMetaEntry(HRegionInforinfo,
+publicMetaEntry(HRegionInforinfo,
  ServerNameregionServer,
  longmodTime)
 
@@ -305,7 +305,7 @@ extends 
 
 MetaEntry
-publicMetaEntry(HRegionInforinfo,
+publicMetaEntry(HRegionInforinfo,
  ServerNameregionServer,
  longmodTime,
  HRegionInfosplitA,
@@ -326,7 +326,7 @@ extends 
 
 equals
-publicbooleanequals(http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true;
 title="class or interface in java.lang">Objecto)
+publicbooleanequals(http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true;
 title="class or interface in java.lang">Objecto)
 
 Overrides:
 equalsin
 classHRegionInfo
@@ -341,7 +341,7 @@ extends 
 
 hashCode
-publicinthashCode()
+publicinthashCode()
 
 Overrides:
 hashCodein
 classHRegionInfo

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.OnlineEntry.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.OnlineEntry.html 
b/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.OnlineEntry.html
index 6863c63..e6c6a3d 100644
--- a/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.OnlineEntry.html
+++ b/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.OnlineEntry.html
@@ -113,7 +113,7 @@ var activeTableTab = "activeTableTab";
 
 
 
-static class HBaseFsck.OnlineEntry
+static class HBaseFsck.OnlineEntry
 extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true;
 title="class or interface in java.lang">Object
 Stores the regioninfo retrieved from Online region 
servers.
 
@@ -206,7 +206,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 hri
-HRegionInfo hri
+HRegionInfo hri
 
 
 
@@ -215,7 +215,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 hsa
-ServerName hsa
+ServerName hsa
 
 
 
@@ -232,7 +232,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 OnlineEntry
-OnlineEntry()
+OnlineEntry()
 
 
 
@@ -249,7 +249,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 toString
-publichttp://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true;
 title="class or interface in java.lang">StringtoString()
+publichttp://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true;
 title="class or interface in java.lang">StringtoString()
 
 Overrides:
 http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true#toString--;
 title="class or interface in java.lang">toStringin 
classhttp://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true;
 title="class or interface in java.lang">Object

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.PrintingErrorReporter.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.PrintingErrorReporter.html 
b/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.PrintingErrorReporter.html
index 4cfaf52..183a253 100644
--- 
a/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.PrintingErrorReporter.html
+++ 
b/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.PrintingErrorReporter.html
@@ -117,7 +117,7 @@ var activeTableTab = "activeTableTab";
 
 
 
-static class HBaseFsck.PrintingErrorReporter
+static class HBaseFsck.PrintingErrorReporter
 extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true;
 title="class or interface in java.lang">Object
 implements HBaseFsck.ErrorReporter
 
@@ -301,7 +301,7 @@ implements 
 
 errorCount
-publicint errorCount
+publicint errorCount
 
 
 
@@ -310,7 

[38/51] [partial] hbase-site git commit: Published site at .

2017-08-25 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/devapidocs/org/apache/hadoop/hbase/rest/ResourceBase.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/rest/ResourceBase.html 
b/devapidocs/org/apache/hadoop/hbase/rest/ResourceBase.html
index 0172094..a14380e 100644
--- a/devapidocs/org/apache/hadoop/hbase/rest/ResourceBase.html
+++ b/devapidocs/org/apache/hadoop/hbase/rest/ResourceBase.html
@@ -153,7 +153,7 @@ implements 
 
 Fields inherited from interfaceorg.apache.hadoop.hbase.rest.Constants
-CRLF,
 CUSTOM_FILTERS,
 DEFAULT_LISTEN_PORT,
 DEFAULT_MAX_AGE,
 FILTER_CLASSES,
 MIMETYPE_BINARY,
 MIMETYPE_HTML,
 MIMETYPE_JSON,
 MIMETYPE_PROTOBUF,
 MIMETYPE_PROTOBUF_IETF, MIMETYPE_TEXT,
 MIMETYPE_XML,
 NOCACHE_PARAM_NAME,
 REST_AUTHENTICATION_PRINCIPAL,
 REST_AUTHENTICATION_TYPE,
 REST_CONNECTOR_ACCEPT_QUEUE_SIZE,
 REST_DNS_INTERFACE,
 REST_DNS_NAMESERVER
 , REST_KERBEROS_PRINCIPAL,
 REST_KEYTAB_FILE,
 REST_SSL_ENABLED,
 REST_SSL_EXCLUDE_CIPHER_SUITES,
 REST_SSL_EXCLUDE_PROTOCOLS,
 REST_SSL_INCLUDE_CIPHER_SUITES,
 REST_SSL_INCLUDE_PROTOCOLS,
 REST_SSL_KEYSTORE_KEYPASSWORD<
 /a>, REST_SSL_KEYSTORE_PASSWORD,
 REST_SSL_KEYSTORE_STORE,
 REST_THREAD_POOL_TASK_QUEUE_SIZE,
 REST_THREAD_POOL_THREAD_IDLE_TIMEOUT,
 REST_THREAD_POOL_THREADS_MAX,
 REST_THREAD_POOL_THREADS_MIN,
 ROW_KEYS_PARAM_NAME,
 SCAN_BATCH_SIZE, SCAN_COLUMN,
 SCAN_END_ROW,
 SCAN_END_TIME,
 SCAN_FETCH_SIZE,
 SCAN_FILTER,
 SCAN_LIMIT,
 SCAN_MAX_VERSIONS,
 SCAN_REVERSED,
 SCAN_START_ROW,
 SCAN_START_TIME, VERSION_STRING
+CRLF,
 CUSTOM_FILTERS,
 DEFAULT_LISTEN_PORT,
 DEFAULT_MAX_AGE,
 FILTER_CLASSES,
 MIMETYPE_BINARY,
 MIMETYPE_HTML,
 MIMETYPE_JSON,
 MIMETYPE_PROTOBUF,
 MIMETYPE_PROTOBUF_IETF, MIMETYPE_TEXT,
 MIMETYPE_XML,
 NOCACHE_PARAM_NAME,
 REST_AUTHENTICATION_PRINCIPAL,
 REST_AUTHENTICATION_TYPE,
 REST_CONNECTOR_ACCEPT_QUEUE_SIZE,
 REST_DNS_INTERFACE,
 REST_DNS_NAMESERVER
 , REST_KERBEROS_PRINCIPAL,
 REST_KEYTAB_FILE,
 REST_SSL_ENABLED,
 REST_SSL_EXCLUDE_CIPHER_SUITES,
 REST_SSL_EXCLUDE_PROTOCOLS,
 REST_SSL_INCLUDE_CIPHER_SUITES,
 REST_SSL_INCLUDE_PROTOCOLS,
 REST_SSL_KEYSTORE_KEYPASSWORD<
 /a>, REST_SSL_KEYSTORE_PASSWORD,
 REST_SSL_KEYSTORE_STORE,
 REST_THREAD_POOL_TASK_QUEUE_SIZE,
 REST_THREAD_POOL_THREAD_IDLE_TIMEOUT,
 REST_THREAD_POOL_THREADS_MAX,
 REST_THREAD_POOL_THREADS_MIN,
 ROW_KEYS_PARAM_NAME,
 SCAN_BATCH_SIZE, SCAN_CACHE_BLOCKS,
 SCAN_COLUMN,
 SCAN_END_ROW,
 SCAN_END_TIME,
 SCAN_FETCH_SIZE,
 SCAN_FILTER,
 SCAN_LIMIT,
 SCAN_MAX_VERSIONS,
 SCAN_REVERSED,
 SCAN_START_ROW, SCAN_START_TIME,
 VERSION_STRING
 
 
 

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/devapidocs/org/apache/hadoop/hbase/rest/RootResource.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/rest/RootResource.html 
b/devapidocs/org/apache/hadoop/hbase/rest/RootResource.html
index bbbe523..cfe1ad4 100644
--- a/devapidocs/org/apache/hadoop/hbase/rest/RootResource.html
+++ b/devapidocs/org/apache/hadoop/hbase/rest/RootResource.html
@@ -161,7 +161,7 @@ extends 
 
 Fields inherited from interfaceorg.apache.hadoop.hbase.rest.Constants
-CRLF,
 CUSTOM_FILTERS,
 DEFAULT_LISTEN_PORT,
 DEFAULT_MAX_AGE,
 FILTER_CLASSES,
 MIMETYPE_BINARY,
 MIMETYPE_HTML,
 MIMETYPE_JSON,
 MIMETYPE_PROTOBUF,
 MIMETYPE_PROTOBUF_IETF, MIMETYPE_TEXT,
 MIMETYPE_XML,
 NOCACHE_PARAM_NAME,
 REST_AUTHENTICATION_PRINCIPAL,
 REST_AUTHENTICATION_TYPE,
 REST_CONNECTOR_ACCEPT_QUEUE_SIZE,
 REST_DNS_INTERFACE,
 REST_DNS_NAMESERVER
 , REST_KERBEROS_PRINCIPAL,
 REST_KEYTAB_FILE,
 REST_SSL_ENABLED,
 REST_SSL_EXCLUDE_CIPHER_SUITES,
 REST_SSL_EXCLUDE_PROTOCOLS,
 REST_SSL_INCLUDE_CIPHER_SUITES,
 REST_SSL_INCLUDE_PROTOCOLS,
 REST_SSL_KEYSTORE_KEYPASSWORD<
 /a>, REST_SSL_KEYSTORE_PASSWORD,
 REST_SSL_KEYSTORE_STORE,
 REST_THREAD_POOL_TASK_QUEUE_SIZE,
 REST_THREAD_POOL_THREAD_IDLE_TIMEOUT,
 REST_THREAD_POOL_THREADS_MAX,
 REST_THREAD_POOL_THREADS_MIN,
 ROW_KEYS_PARAM_NAME,
 SCAN_BATCH_SIZE, SCAN_COLUMN,
 SCAN_END_ROW,
 SCAN_END_TIME,
 SCAN_FETCH_SIZE,
 SCAN_FILTER,
 SCAN_LIMIT,
 SCAN_MAX_VERSIONS,
 SCAN_REVERSED,
 SCAN_START_ROW,
 SCAN_START_TIME, VERSION_STRING
+CRLF,
 CUSTOM_FILTERS,
 DEFAULT_LISTEN_PORT,
 DEFAULT_MAX_AGE,
 FILTER_CLASSES,
 MIMETYPE_BINARY,
 MIMETYPE_HTML,
 MIMETYPE_JSON,
 MIMETYPE_PROTOBUF,
 MIMETYPE_PROTOBUF_IETF, MIMETYPE_TEXT,
 MIMETYPE_XML,
 NOCACHE_PARAM_NAME,
 REST_AUTHENTICATION_PRINCIPAL,
 REST_AUTHENTICATION_TYPE,
 REST_CONNECTOR_ACCEPT_QUEUE_SIZE,
 REST_DNS_INTERFACE,
 REST_DNS_NAMESERVER
 , REST_KERBEROS_PRINCIPAL,
 REST_KEYTAB_FILE,
 REST_SSL_ENABLED,
 REST_SSL_EXCLUDE_CIPHER_SUITES,
 REST_SSL_EXCLUDE_PROTOCOLS,
 REST_SSL_INCLUDE_CIPHER_SUITES,
 REST_SSL_INCLUDE_PROTOCOLS,
 REST_SSL_KEYSTORE_KEYPASSWORD<
 /a>, REST_SSL_KEYSTORE_PASSWORD,
 REST_SSL_KEYSTORE_STORE,
 REST_THREAD_POOL_TASK_QUEUE_SIZE,
 REST_THREAD_POOL_THREAD_IDLE_TIMEOUT,
 REST_THREAD_POOL_THREADS_MAX,
 REST_THREAD_POOL_THREADS_MIN,
 ROW_KEYS_PARAM_NAME,
 

[48/51] [partial] hbase-site git commit: Published site at .

2017-08-25 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/devapidocs/index-all.html
--
diff --git a/devapidocs/index-all.html b/devapidocs/index-all.html
index 1739525..607c302 100644
--- a/devapidocs/index-all.html
+++ b/devapidocs/index-all.html
@@ -1166,8 +1166,6 @@
 
 add(byte[][])
 - Static method in class org.apache.hadoop.hbase.util.Bytes
 
-add(T)
 - Method in class org.apache.hadoop.hbase.util.ConcatenatedLists
-
 add(long) 
- Method in class org.apache.hadoop.hbase.util.Counter
 
 Deprecated.
@@ -1231,8 +1229,6 @@
 
 addAll(Collection?
 extends V) - Method in class org.apache.hadoop.hbase.types.CopyOnWriteArrayMap.ArrayValueCollection
 
-addAll(Collection?
 extends T) - Method in class org.apache.hadoop.hbase.util.ConcatenatedLists
-
 addAll(Collection?
 extends E) - Method in class org.apache.hadoop.hbase.util.SortedList
 
 addAll(int,
 Collection? extends E) - Method in class 
org.apache.hadoop.hbase.util.SortedList
@@ -1893,8 +1889,6 @@
 
 Listen for failures to a given process.
 
-addListener(Service.Listener,
 Executor) - Method in class 
org.apache.hadoop.hbase.security.visibility.VisibilityReplicationEndpoint
-
 addLiveNode(String,
 long, int, int) - Method in class 
org.apache.hadoop.hbase.rest.model.StorageClusterStatusModel
 
 Add a live node to the cluster representation.
@@ -4608,10 +4602,28 @@
 
 Wait until the read point catches up to the write point; 
i.e.
 
+awaitRunning()
 - Method in interface org.apache.hadoop.hbase.replication.ReplicationEndpoint
+
+Waits for the ReplicationEndpoint to be 
up and running.
+
+awaitRunning(long,
 TimeUnit) - Method in interface 
org.apache.hadoop.hbase.replication.ReplicationEndpoint
+
+Waits for the ReplicationEndpoint to to 
be up and running for no more
+ than the given time.
+
 awaitRunning()
 - Method in class org.apache.hadoop.hbase.security.visibility.VisibilityReplicationEndpoint
 
 awaitRunning(long,
 TimeUnit) - Method in class 
org.apache.hadoop.hbase.security.visibility.VisibilityReplicationEndpoint
 
+awaitTerminated()
 - Method in interface org.apache.hadoop.hbase.replication.ReplicationEndpoint
+
+Waits for the ReplicationEndpoint to 
reach the terminated (internal) state.
+
+awaitTerminated(long,
 TimeUnit) - Method in interface 
org.apache.hadoop.hbase.replication.ReplicationEndpoint
+
+Waits for the ReplicationEndpoint to 
reach a terminal state for no
+ more than the given time.
+
 awaitTerminated()
 - Method in class org.apache.hadoop.hbase.security.visibility.VisibilityReplicationEndpoint
 
 awaitTerminated(long,
 TimeUnit) - Method in class 
org.apache.hadoop.hbase.security.visibility.VisibilityReplicationEndpoint
@@ -11519,8 +11531,6 @@
 
 Similar to http://docs.oracle.com/javase/8/docs/api/java/nio/Buffer.html?is-external=true#clear--;
 title="class or interface in java.nio">Buffer.clear().
 
-clear()
 - Method in class org.apache.hadoop.hbase.util.ConcatenatedLists
-
 clear()
 - Method in interface org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter
 
 clear()
 - Method in class org.apache.hadoop.hbase.util.HBaseFsck.PrintingErrorReporter
@@ -12098,6 +12108,8 @@
 
 close()
 - Method in class org.apache.hadoop.hbase.client.coprocessor.AggregationClient
 
+close()
 - Method in class org.apache.hadoop.hbase.client.example.RefreshHFilesClient
+
 close()
 - Method in class org.apache.hadoop.hbase.client.HBaseAdmin
 
 close()
 - Method in class org.apache.hadoop.hbase.client.HRegionLocator
@@ -16128,6 +16140,8 @@
 
 connection
 - Variable in class org.apache.hadoop.hbase.client.example.MultiThreadedClientExample.WriteExampleCallable
 
+connection
 - Variable in class org.apache.hadoop.hbase.client.example.RefreshHFilesClient
+
 connection
 - Variable in class org.apache.hadoop.hbase.client.HBaseAdmin
 
 connection
 - Variable in class org.apache.hadoop.hbase.client.HRegionLocator
@@ -16561,8 +16575,6 @@
 
 contains(byte[],
 byte[]) - Static method in class org.apache.hadoop.hbase.util.Bytes
 
-contains(Object)
 - Method in class org.apache.hadoop.hbase.util.ConcatenatedLists
-
 contains(String[],
 String) - Static method in class org.apache.hadoop.hbase.util.FSUtils
 
 contains(Object)
 - Method in class org.apache.hadoop.hbase.util.SortedList
@@ -16581,8 +16593,6 @@
 
 containsAll(Collection?)
 - Method in class org.apache.hadoop.hbase.types.CopyOnWriteArrayMap.ArrayValueCollection
 
-containsAll(Collection?)
 - Method in class org.apache.hadoop.hbase.util.ConcatenatedLists
-
 containsAll(Collection?)
 - Method in class org.apache.hadoop.hbase.util.SortedList
 
 containsBlock(BlockCacheKey)
 - Method in class org.apache.hadoop.hbase.io.hfile.LruBlockCache
@@ -21999,6 +22009,8 @@
 
 Default operation timeout in milliseconds.
 
+DEFAULT_HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS
 - Static variable in class org.apache.hadoop.hbase.master.HMaster
+
 DEFAULT_HBASE_META_BLOCK_SIZE
 - Static variable in class org.apache.hadoop.hbase.HConstants
 
 Default value 

[06/51] [partial] hbase-site git commit: Published site at .

2017-08-25 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.TableInfo.html
--
diff --git 
a/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.TableInfo.html 
b/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.TableInfo.html
index f5186b5..43afb13 100644
--- a/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.TableInfo.html
+++ b/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.TableInfo.html
@@ -36,5085 +36,5084 @@
 028import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.Multimap;
 029import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.Ordering;
 030import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.TreeMultimap;
-031import 
com.google.protobuf.ServiceException;
-032
-033import java.io.Closeable;
-034import java.io.FileNotFoundException;
-035import java.io.IOException;
-036import java.io.InterruptedIOException;
-037import java.io.PrintWriter;
-038import java.io.StringWriter;
-039import java.net.InetAddress;
-040import java.net.URI;
-041import java.util.ArrayList;
-042import java.util.Arrays;
-043import java.util.Collection;
-044import java.util.Collections;
-045import java.util.Comparator;
-046import java.util.HashMap;
-047import java.util.HashSet;
-048import java.util.Iterator;
-049import java.util.List;
-050import java.util.Locale;
-051import java.util.Map;
-052import java.util.Map.Entry;
-053import java.util.Set;
-054import java.util.SortedMap;
-055import java.util.SortedSet;
-056import java.util.TreeMap;
-057import java.util.TreeSet;
-058import java.util.Vector;
-059import java.util.concurrent.Callable;
-060import 
java.util.concurrent.ConcurrentSkipListMap;
-061import 
java.util.concurrent.ExecutionException;
-062import 
java.util.concurrent.ExecutorService;
-063import java.util.concurrent.Executors;
-064import java.util.concurrent.Future;
-065import java.util.concurrent.FutureTask;
-066import 
java.util.concurrent.ScheduledThreadPoolExecutor;
-067import java.util.concurrent.TimeUnit;
-068import 
java.util.concurrent.TimeoutException;
-069import 
java.util.concurrent.atomic.AtomicBoolean;
-070import 
java.util.concurrent.atomic.AtomicInteger;
-071
-072import org.apache.commons.io.IOUtils;
-073import 
org.apache.commons.lang.RandomStringUtils;
-074import 
org.apache.commons.lang.StringUtils;
-075import org.apache.commons.logging.Log;
-076import 
org.apache.commons.logging.LogFactory;
-077import 
org.apache.hadoop.conf.Configuration;
-078import 
org.apache.hadoop.conf.Configured;
-079import 
org.apache.hadoop.fs.FSDataOutputStream;
-080import org.apache.hadoop.fs.FileStatus;
-081import org.apache.hadoop.fs.FileSystem;
-082import org.apache.hadoop.fs.Path;
-083import 
org.apache.hadoop.fs.permission.FsAction;
-084import 
org.apache.hadoop.fs.permission.FsPermission;
-085import 
org.apache.hadoop.hbase.Abortable;
-086import org.apache.hadoop.hbase.Cell;
-087import 
org.apache.hadoop.hbase.CellUtil;
-088import 
org.apache.hadoop.hbase.ClusterStatus;
-089import 
org.apache.hadoop.hbase.HBaseConfiguration;
-090import 
org.apache.hadoop.hbase.HBaseInterfaceAudience;
-091import 
org.apache.hadoop.hbase.HConstants;
-092import 
org.apache.hadoop.hbase.HRegionInfo;
-093import 
org.apache.hadoop.hbase.HRegionLocation;
-094import 
org.apache.hadoop.hbase.KeyValue;
-095import 
org.apache.hadoop.hbase.MasterNotRunningException;
-096import 
org.apache.hadoop.hbase.MetaTableAccessor;
-097import 
org.apache.hadoop.hbase.RegionLocations;
-098import 
org.apache.hadoop.hbase.ServerName;
-099import 
org.apache.hadoop.hbase.TableName;
-100import 
org.apache.hadoop.hbase.ZooKeeperConnectionException;
-101import 
org.apache.hadoop.hbase.classification.InterfaceAudience;
-102import 
org.apache.hadoop.hbase.classification.InterfaceStability;
-103import 
org.apache.hadoop.hbase.client.Admin;
-104import 
org.apache.hadoop.hbase.client.ClusterConnection;
-105import 
org.apache.hadoop.hbase.client.Connection;
-106import 
org.apache.hadoop.hbase.client.ConnectionFactory;
-107import 
org.apache.hadoop.hbase.client.Delete;
-108import 
org.apache.hadoop.hbase.client.Get;
-109import 
org.apache.hadoop.hbase.client.Put;
-110import 
org.apache.hadoop.hbase.client.RegionReplicaUtil;
-111import 
org.apache.hadoop.hbase.client.Result;
-112import 
org.apache.hadoop.hbase.client.RowMutations;
-113import 
org.apache.hadoop.hbase.client.Table;
-114import 
org.apache.hadoop.hbase.client.TableState;
-115import 
org.apache.hadoop.hbase.io.FileLink;
-116import 
org.apache.hadoop.hbase.io.HFileLink;
-117import 
org.apache.hadoop.hbase.io.hfile.CacheConfig;
-118import 
org.apache.hadoop.hbase.io.hfile.HFile;
-119import 
org.apache.hadoop.hbase.master.MasterFileSystem;
-120import 
org.apache.hadoop.hbase.master.RegionState;
-121import 
org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-122import 

[07/51] [partial] hbase-site git commit: Published site at .

2017-08-25 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.TableInfo.IntegrityFixSuggester.html
--
diff --git 
a/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.TableInfo.IntegrityFixSuggester.html
 
b/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.TableInfo.IntegrityFixSuggester.html
index f5186b5..43afb13 100644
--- 
a/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.TableInfo.IntegrityFixSuggester.html
+++ 
b/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.TableInfo.IntegrityFixSuggester.html
@@ -36,5085 +36,5084 @@
 028import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.Multimap;
 029import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.Ordering;
 030import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.TreeMultimap;
-031import 
com.google.protobuf.ServiceException;
-032
-033import java.io.Closeable;
-034import java.io.FileNotFoundException;
-035import java.io.IOException;
-036import java.io.InterruptedIOException;
-037import java.io.PrintWriter;
-038import java.io.StringWriter;
-039import java.net.InetAddress;
-040import java.net.URI;
-041import java.util.ArrayList;
-042import java.util.Arrays;
-043import java.util.Collection;
-044import java.util.Collections;
-045import java.util.Comparator;
-046import java.util.HashMap;
-047import java.util.HashSet;
-048import java.util.Iterator;
-049import java.util.List;
-050import java.util.Locale;
-051import java.util.Map;
-052import java.util.Map.Entry;
-053import java.util.Set;
-054import java.util.SortedMap;
-055import java.util.SortedSet;
-056import java.util.TreeMap;
-057import java.util.TreeSet;
-058import java.util.Vector;
-059import java.util.concurrent.Callable;
-060import 
java.util.concurrent.ConcurrentSkipListMap;
-061import 
java.util.concurrent.ExecutionException;
-062import 
java.util.concurrent.ExecutorService;
-063import java.util.concurrent.Executors;
-064import java.util.concurrent.Future;
-065import java.util.concurrent.FutureTask;
-066import 
java.util.concurrent.ScheduledThreadPoolExecutor;
-067import java.util.concurrent.TimeUnit;
-068import 
java.util.concurrent.TimeoutException;
-069import 
java.util.concurrent.atomic.AtomicBoolean;
-070import 
java.util.concurrent.atomic.AtomicInteger;
-071
-072import org.apache.commons.io.IOUtils;
-073import 
org.apache.commons.lang.RandomStringUtils;
-074import 
org.apache.commons.lang.StringUtils;
-075import org.apache.commons.logging.Log;
-076import 
org.apache.commons.logging.LogFactory;
-077import 
org.apache.hadoop.conf.Configuration;
-078import 
org.apache.hadoop.conf.Configured;
-079import 
org.apache.hadoop.fs.FSDataOutputStream;
-080import org.apache.hadoop.fs.FileStatus;
-081import org.apache.hadoop.fs.FileSystem;
-082import org.apache.hadoop.fs.Path;
-083import 
org.apache.hadoop.fs.permission.FsAction;
-084import 
org.apache.hadoop.fs.permission.FsPermission;
-085import 
org.apache.hadoop.hbase.Abortable;
-086import org.apache.hadoop.hbase.Cell;
-087import 
org.apache.hadoop.hbase.CellUtil;
-088import 
org.apache.hadoop.hbase.ClusterStatus;
-089import 
org.apache.hadoop.hbase.HBaseConfiguration;
-090import 
org.apache.hadoop.hbase.HBaseInterfaceAudience;
-091import 
org.apache.hadoop.hbase.HConstants;
-092import 
org.apache.hadoop.hbase.HRegionInfo;
-093import 
org.apache.hadoop.hbase.HRegionLocation;
-094import 
org.apache.hadoop.hbase.KeyValue;
-095import 
org.apache.hadoop.hbase.MasterNotRunningException;
-096import 
org.apache.hadoop.hbase.MetaTableAccessor;
-097import 
org.apache.hadoop.hbase.RegionLocations;
-098import 
org.apache.hadoop.hbase.ServerName;
-099import 
org.apache.hadoop.hbase.TableName;
-100import 
org.apache.hadoop.hbase.ZooKeeperConnectionException;
-101import 
org.apache.hadoop.hbase.classification.InterfaceAudience;
-102import 
org.apache.hadoop.hbase.classification.InterfaceStability;
-103import 
org.apache.hadoop.hbase.client.Admin;
-104import 
org.apache.hadoop.hbase.client.ClusterConnection;
-105import 
org.apache.hadoop.hbase.client.Connection;
-106import 
org.apache.hadoop.hbase.client.ConnectionFactory;
-107import 
org.apache.hadoop.hbase.client.Delete;
-108import 
org.apache.hadoop.hbase.client.Get;
-109import 
org.apache.hadoop.hbase.client.Put;
-110import 
org.apache.hadoop.hbase.client.RegionReplicaUtil;
-111import 
org.apache.hadoop.hbase.client.Result;
-112import 
org.apache.hadoop.hbase.client.RowMutations;
-113import 
org.apache.hadoop.hbase.client.Table;
-114import 
org.apache.hadoop.hbase.client.TableState;
-115import 
org.apache.hadoop.hbase.io.FileLink;
-116import 
org.apache.hadoop.hbase.io.HFileLink;
-117import 
org.apache.hadoop.hbase.io.hfile.CacheConfig;
-118import 
org.apache.hadoop.hbase.io.hfile.HFile;
-119import 
org.apache.hadoop.hbase.master.MasterFileSystem;
-120import 
org.apache.hadoop.hbase.master.RegionState;
-121import 

[24/51] [partial] hbase-site git commit: Published site at .

2017-08-25 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/devapidocs/src-html/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.LogsComparator.html
--
diff --git 
a/devapidocs/src-html/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.LogsComparator.html
 
b/devapidocs/src-html/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.LogsComparator.html
index 2ef9b72..be1a1b0 100644
--- 
a/devapidocs/src-html/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.LogsComparator.html
+++ 
b/devapidocs/src-html/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.LogsComparator.html
@@ -27,552 +27,548 @@
 019package 
org.apache.hadoop.hbase.replication.regionserver;
 020
 021import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
-022import 
org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.ListenableFuture;
-023import 
org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.Service;
-024
-025import java.io.IOException;
-026import java.util.ArrayList;
-027import java.util.Collection;
-028import java.util.Comparator;
-029import java.util.HashMap;
-030import java.util.List;
-031import java.util.Map;
-032import java.util.UUID;
-033import 
java.util.concurrent.ConcurrentHashMap;
-034import 
java.util.concurrent.PriorityBlockingQueue;
-035import java.util.concurrent.TimeUnit;
-036import 
java.util.concurrent.TimeoutException;
-037import 
java.util.concurrent.atomic.AtomicLong;
-038
-039import 
org.apache.commons.lang.StringUtils;
-040import org.apache.commons.logging.Log;
-041import 
org.apache.commons.logging.LogFactory;
-042import 
org.apache.hadoop.conf.Configuration;
-043import org.apache.hadoop.fs.FileSystem;
-044import org.apache.hadoop.fs.Path;
-045import 
org.apache.hadoop.hbase.HBaseConfiguration;
-046import 
org.apache.hadoop.hbase.HConstants;
-047import 
org.apache.hadoop.hbase.Stoppable;
-048import 
org.apache.hadoop.hbase.TableName;
-049import 
org.apache.hadoop.hbase.classification.InterfaceAudience;
-050import 
org.apache.hadoop.hbase.regionserver.RSRpcServices;
-051import 
org.apache.hadoop.hbase.replication.ChainWALEntryFilter;
-052import 
org.apache.hadoop.hbase.replication.ClusterMarkingEntryFilter;
-053import 
org.apache.hadoop.hbase.replication.ReplicationEndpoint;
-054import 
org.apache.hadoop.hbase.replication.ReplicationException;
-055import 
org.apache.hadoop.hbase.replication.ReplicationPeer;
-056import 
org.apache.hadoop.hbase.replication.ReplicationPeers;
-057import 
org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
-058import 
org.apache.hadoop.hbase.replication.ReplicationQueues;
-059import 
org.apache.hadoop.hbase.replication.SystemTableWALEntryFilter;
-060import 
org.apache.hadoop.hbase.replication.WALEntryFilter;
-061import 
org.apache.hadoop.hbase.util.Bytes;
-062import 
org.apache.hadoop.hbase.util.Pair;
-063import 
org.apache.hadoop.hbase.util.Threads;
-064import 
org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
-065import 
org.apache.hadoop.hbase.wal.WAL.Entry;
-066
-067
-068/**
-069 * Class that handles the source of a 
replication stream.
-070 * Currently does not handle more than 1 
slave
-071 * For each slave cluster it selects a 
random number of peers
-072 * using a replication ratio. For 
example, if replication ration = 0.1
-073 * and slave cluster has 100 region 
servers, 10 will be selected.
-074 * p
-075 * A stream is considered down when we 
cannot contact a region server on the
-076 * peer cluster for more than 55 seconds 
by default.
-077 * /p
-078 *
-079 */
-080@InterfaceAudience.Private
-081public class ReplicationSource extends 
Thread implements ReplicationSourceInterface {
-082
-083  private static final Log LOG = 
LogFactory.getLog(ReplicationSource.class);
-084  // Queues of logs to process, entry in 
format of walGroupId-queue,
-085  // each presents a queue for one wal 
group
-086  private MapString, 
PriorityBlockingQueuePath queues = new HashMap();
-087  // per group queue size, keep no more 
than this number of logs in each wal group
-088  protected int queueSizePerGroup;
-089  protected ReplicationQueues 
replicationQueues;
-090  private ReplicationPeers 
replicationPeers;
-091
-092  protected Configuration conf;
-093  protected ReplicationQueueInfo 
replicationQueueInfo;
-094  // id of the peer cluster this source 
replicates to
-095  private String peerId;
-096
-097  // The manager of all sources to which 
we ping back our progress
-098  protected ReplicationSourceManager 
manager;
-099  // Should we stop everything?
-100  protected Stoppable stopper;
-101  // How long should we sleep for each 
retry
-102  private long sleepForRetries;
-103  protected FileSystem fs;
-104  // id of this cluster
-105  private UUID clusterId;
-106  // id of the other cluster
-107  private UUID peerClusterId;
-108  // total number of edits we 
replicated
-109  private 

[40/51] [partial] hbase-site git commit: Published site at .

2017-08-25 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/devapidocs/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.html
 
b/devapidocs/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.html
index 663f736..fc943c3 100644
--- 
a/devapidocs/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.html
+++ 
b/devapidocs/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.html
@@ -123,7 +123,7 @@ var activeTableTab = "activeTableTab";
 
 
 @InterfaceAudience.Private
-public class ReplicationSource
+public class ReplicationSource
 extends http://docs.oracle.com/javase/8/docs/api/java/lang/Thread.html?is-external=true;
 title="class or interface in java.lang">Thread
 implements ReplicationSourceInterface
 Class that handles the source of a replication stream.
@@ -194,98 +194,110 @@ implements currentBandwidth
 
 
+static int
+DEFAULT_WAIT_ON_ENDPOINT_SECONDS
+
+
 private long
 defaultBandwidth
 
-
+
 protected 
org.apache.hadoop.fs.FileSystem
 fs
 
-
+
 private static 
org.apache.commons.logging.Log
 LOG
 
-
+
 private int
 logQueueWarnThreshold
 
-
+
 protected ReplicationSourceManager
 manager
 
-
+
 private int
 maxRetriesMultiplier
 
-
+
 private MetricsSource
 metrics
 
-
+
 private http://docs.oracle.com/javase/8/docs/api/java/util/UUID.html?is-external=true;
 title="class or interface in java.util">UUID
 peerClusterId
 
-
+
 protected http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true;
 title="class or interface in java.lang">String
 peerClusterZnode
 
-
+
 private http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true;
 title="class or interface in java.lang">String
 peerId
 
-
+
 private http://docs.oracle.com/javase/8/docs/api/java/util/Map.html?is-external=true;
 title="class or interface in java.util">Maphttp://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true;
 title="class or interface in java.lang">String,http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/PriorityBlockingQueue.html?is-external=true;
 title="class or interface in 
java.util.concurrent">PriorityBlockingQueueorg.apache.hadoop.fs.Path
 queues
 
-
+
 protected int
 queueSizePerGroup
 
-
+
 private ReplicationEndpoint
 replicationEndpoint
 
-
+
 private ReplicationPeers
 replicationPeers
 
-
+
 protected ReplicationQueueInfo
 replicationQueueInfo
 
-
+
 protected ReplicationQueues
 replicationQueues
 
-
+
 private long
 sleepForRetries
 
-
+
 private boolean
 sourceRunning
 
-
+
 protected Stoppable
 stopper
 
-
+
 private ReplicationThrottler
 throttler
 
-
+
 private http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/atomic/AtomicLong.html?is-external=true;
 title="class or interface in 
java.util.concurrent.atomic">AtomicLong
 totalBufferUsed
 
-
+
 private http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/atomic/AtomicLong.html?is-external=true;
 title="class or interface in 
java.util.concurrent.atomic">AtomicLong
 totalReplicatedEdits
 
+
+static http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true;
 title="class or interface in java.lang">String
+WAIT_ON_ENDPOINT_SECONDS
+
 
+private int
+waitOnEndpointSeconds
+
+
 protected WALEntryFilter
 walEntryFilter
 
-
+
 protected http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/ConcurrentHashMap.html?is-external=true;
 title="class or interface in java.util.concurrent">ConcurrentHashMaphttp://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true;
 title="class or interface in java.lang">String,ReplicationSourceShipper
 workerThreads
 
@@ -524,7 +536,7 @@ implements 
 
 LOG
-private static finalorg.apache.commons.logging.Log LOG
+private static finalorg.apache.commons.logging.Log LOG
 
 
 
@@ -533,7 +545,7 @@ implements 
 
 queues
-privatehttp://docs.oracle.com/javase/8/docs/api/java/util/Map.html?is-external=true;
 title="class or interface in java.util">Maphttp://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true;
 title="class or interface in java.lang">String,http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/PriorityBlockingQueue.html?is-external=true;
 title="class or interface in 
java.util.concurrent">PriorityBlockingQueueorg.apache.hadoop.fs.Path
 queues
+privatehttp://docs.oracle.com/javase/8/docs/api/java/util/Map.html?is-external=true;
 title="class or interface in java.util">Maphttp://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true;
 title="class or interface in java.lang">String,http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/PriorityBlockingQueue.html?is-external=true;
 title="class or interface in 
java.util.concurrent">PriorityBlockingQueueorg.apache.hadoop.fs.Path
 queues
 
 
 
@@ -542,7 +554,7 @@ 

[51/51] [partial] hbase-site git commit: Published site at .

2017-08-25 Thread git-site-role
Published site at .


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

Branch: refs/heads/asf-site
Commit: 6b5143ed1b274f013e0291d50519faeaaa66cb4c
Parents: 0c22829
Author: jenkins 
Authored: Fri Aug 25 15:09:34 2017 +
Committer: jenkins 
Committed: Fri Aug 25 15:09:34 2017 +

--
 acid-semantics.html | 4 +-
 apache_hbase_reference_guide.pdf| 6 +-
 apidocs/constant-values.html|27 +-
 apidocs/index-all.html  | 2 +
 .../org/apache/hadoop/hbase/rest/Constants.html |43 +-
 .../org/apache/hadoop/hbase/rest/Constants.html |15 +-
 book.html   | 2 +-
 bulk-loads.html | 4 +-
 checkstyle-aggregate.html   | 13212 +
 checkstyle.rss  |58 +-
 coc.html| 4 +-
 cygwin.html | 4 +-
 dependencies.html   | 4 +-
 dependency-convergence.html | 4 +-
 dependency-info.html| 4 +-
 dependency-management.html  | 4 +-
 devapidocs/allclasses-frame.html| 3 +-
 devapidocs/allclasses-noframe.html  | 3 +-
 devapidocs/constant-values.html |73 +-
 devapidocs/index-all.html   |   159 +-
 .../org/apache/hadoop/hbase/Coprocessor.html| 2 +-
 .../org/apache/hadoop/hbase/ServerName.html | 4 +-
 devapidocs/org/apache/hadoop/hbase/Service.html |   278 -
 .../hbase/ServiceNotRunningException.html   | 4 +-
 .../hadoop/hbase/backup/package-tree.html   | 2 +-
 .../hadoop/hbase/class-use/Coprocessor.html | 8 +-
 .../hbase/class-use/CoprocessorEnvironment.html |12 +-
 .../apache/hadoop/hbase/class-use/Service.html  |   180 -
 .../hadoop/hbase/class-use/TableName.html   | 4 +
 .../InterfaceAudience.LimitedPrivate.html   |26 +-
 .../class-use/InterfaceAudience.Private.html|80 +-
 .../hbase/classification/package-tree.html  | 4 +-
 .../hbase/client/class-use/Connection.html  | 8 +-
 .../hadoop/hbase/client/class-use/Table.html|56 +-
 ...eadedClientExample.WriteExampleCallable.html | 4 +-
 .../client/example/RefreshHFilesClient.html |   380 +
 .../example/class-use/RefreshHFilesClient.html  |   125 +
 .../hbase/client/example/package-frame.html | 1 +
 .../hbase/client/example/package-summary.html   | 7 +
 .../hbase/client/example/package-tree.html  | 1 +
 .../hadoop/hbase/client/package-tree.html   |26 +-
 .../apache/hadoop/hbase/client/package-use.html | 5 +
 .../hbase/coprocessor/CoprocessorService.html   | 2 +-
 .../class-use/CoprocessorService.html   | 6 +
 .../class-use/RegionCoprocessorEnvironment.html | 4 +
 .../ExampleRegionObserverWithMetrics.html   | 4 +-
 .../example/RefreshHFilesEndpoint.html  |   442 +
 .../coprocessor/example/RowCountEndpoint.html   | 4 +-
 .../class-use/RefreshHFilesEndpoint.html|   125 +
 .../coprocessor/example/package-frame.html  | 1 +
 .../coprocessor/example/package-summary.html|10 +-
 .../hbase/coprocessor/example/package-tree.html | 5 +
 .../hadoop/hbase/executor/package-tree.html | 2 +-
 .../hadoop/hbase/filter/package-tree.html   | 8 +-
 .../hadoop/hbase/io/hfile/package-tree.html | 4 +-
 .../apache/hadoop/hbase/ipc/package-tree.html   | 4 +-
 .../hadoop/hbase/mapreduce/package-tree.html| 2 +-
 .../hbase/master/ClusterSchemaService.html  |25 +-
 .../hbase/master/ClusterSchemaServiceImpl.html  |   144 +-
 .../master/HMaster.InitializationMonitor.html   |20 +-
 .../hbase/master/HMaster.RedirectServlet.html   |12 +-
 .../org/apache/hadoop/hbase/master/HMaster.html |   484 +-
 .../master/HMasterCommandLine.LocalHMaster.html | 2 +-
 .../hadoop/hbase/master/package-tree.html   |14 +-
 .../hbase/master/procedure/package-tree.html| 4 +-
 .../compactions/PartitionedMobCompactor.html|28 +-
 .../org/apache/hadoop/hbase/package-frame.html  | 1 -
 .../apache/hadoop/hbase/package-summary.html|14 +-
 .../org/apache/hadoop/hbase/package-tree.html   |11 +-
 .../org/apache/hadoop/hbase/package-use.html|19 +-
 .../hadoop/hbase/procedure2/package-tree.html   | 4 +-
 .../hadoop/hbase/quotas/package-tree.html   |10 +-
 ...eFileManager.KeyBeforeConcatenatedLists.html |

[13/51] [partial] hbase-site git commit: Published site at .

2017-08-25 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.MetaEntry.html
--
diff --git 
a/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.MetaEntry.html 
b/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.MetaEntry.html
index f5186b5..43afb13 100644
--- a/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.MetaEntry.html
+++ b/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.MetaEntry.html
@@ -36,5085 +36,5084 @@
 028import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.Multimap;
 029import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.Ordering;
 030import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.TreeMultimap;
-031import 
com.google.protobuf.ServiceException;
-032
-033import java.io.Closeable;
-034import java.io.FileNotFoundException;
-035import java.io.IOException;
-036import java.io.InterruptedIOException;
-037import java.io.PrintWriter;
-038import java.io.StringWriter;
-039import java.net.InetAddress;
-040import java.net.URI;
-041import java.util.ArrayList;
-042import java.util.Arrays;
-043import java.util.Collection;
-044import java.util.Collections;
-045import java.util.Comparator;
-046import java.util.HashMap;
-047import java.util.HashSet;
-048import java.util.Iterator;
-049import java.util.List;
-050import java.util.Locale;
-051import java.util.Map;
-052import java.util.Map.Entry;
-053import java.util.Set;
-054import java.util.SortedMap;
-055import java.util.SortedSet;
-056import java.util.TreeMap;
-057import java.util.TreeSet;
-058import java.util.Vector;
-059import java.util.concurrent.Callable;
-060import 
java.util.concurrent.ConcurrentSkipListMap;
-061import 
java.util.concurrent.ExecutionException;
-062import 
java.util.concurrent.ExecutorService;
-063import java.util.concurrent.Executors;
-064import java.util.concurrent.Future;
-065import java.util.concurrent.FutureTask;
-066import 
java.util.concurrent.ScheduledThreadPoolExecutor;
-067import java.util.concurrent.TimeUnit;
-068import 
java.util.concurrent.TimeoutException;
-069import 
java.util.concurrent.atomic.AtomicBoolean;
-070import 
java.util.concurrent.atomic.AtomicInteger;
-071
-072import org.apache.commons.io.IOUtils;
-073import 
org.apache.commons.lang.RandomStringUtils;
-074import 
org.apache.commons.lang.StringUtils;
-075import org.apache.commons.logging.Log;
-076import 
org.apache.commons.logging.LogFactory;
-077import 
org.apache.hadoop.conf.Configuration;
-078import 
org.apache.hadoop.conf.Configured;
-079import 
org.apache.hadoop.fs.FSDataOutputStream;
-080import org.apache.hadoop.fs.FileStatus;
-081import org.apache.hadoop.fs.FileSystem;
-082import org.apache.hadoop.fs.Path;
-083import 
org.apache.hadoop.fs.permission.FsAction;
-084import 
org.apache.hadoop.fs.permission.FsPermission;
-085import 
org.apache.hadoop.hbase.Abortable;
-086import org.apache.hadoop.hbase.Cell;
-087import 
org.apache.hadoop.hbase.CellUtil;
-088import 
org.apache.hadoop.hbase.ClusterStatus;
-089import 
org.apache.hadoop.hbase.HBaseConfiguration;
-090import 
org.apache.hadoop.hbase.HBaseInterfaceAudience;
-091import 
org.apache.hadoop.hbase.HConstants;
-092import 
org.apache.hadoop.hbase.HRegionInfo;
-093import 
org.apache.hadoop.hbase.HRegionLocation;
-094import 
org.apache.hadoop.hbase.KeyValue;
-095import 
org.apache.hadoop.hbase.MasterNotRunningException;
-096import 
org.apache.hadoop.hbase.MetaTableAccessor;
-097import 
org.apache.hadoop.hbase.RegionLocations;
-098import 
org.apache.hadoop.hbase.ServerName;
-099import 
org.apache.hadoop.hbase.TableName;
-100import 
org.apache.hadoop.hbase.ZooKeeperConnectionException;
-101import 
org.apache.hadoop.hbase.classification.InterfaceAudience;
-102import 
org.apache.hadoop.hbase.classification.InterfaceStability;
-103import 
org.apache.hadoop.hbase.client.Admin;
-104import 
org.apache.hadoop.hbase.client.ClusterConnection;
-105import 
org.apache.hadoop.hbase.client.Connection;
-106import 
org.apache.hadoop.hbase.client.ConnectionFactory;
-107import 
org.apache.hadoop.hbase.client.Delete;
-108import 
org.apache.hadoop.hbase.client.Get;
-109import 
org.apache.hadoop.hbase.client.Put;
-110import 
org.apache.hadoop.hbase.client.RegionReplicaUtil;
-111import 
org.apache.hadoop.hbase.client.Result;
-112import 
org.apache.hadoop.hbase.client.RowMutations;
-113import 
org.apache.hadoop.hbase.client.Table;
-114import 
org.apache.hadoop.hbase.client.TableState;
-115import 
org.apache.hadoop.hbase.io.FileLink;
-116import 
org.apache.hadoop.hbase.io.HFileLink;
-117import 
org.apache.hadoop.hbase.io.hfile.CacheConfig;
-118import 
org.apache.hadoop.hbase.io.hfile.HFile;
-119import 
org.apache.hadoop.hbase.master.MasterFileSystem;
-120import 
org.apache.hadoop.hbase.master.RegionState;
-121import 
org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-122import 

[20/51] [partial] hbase-site git commit: Published site at .

2017-08-25 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.CheckRegionConsistencyWorkItem.html
--
diff --git 
a/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.CheckRegionConsistencyWorkItem.html
 
b/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.CheckRegionConsistencyWorkItem.html
index f5186b5..43afb13 100644
--- 
a/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.CheckRegionConsistencyWorkItem.html
+++ 
b/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.CheckRegionConsistencyWorkItem.html
@@ -36,5085 +36,5084 @@
 028import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.Multimap;
 029import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.Ordering;
 030import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.TreeMultimap;
-031import 
com.google.protobuf.ServiceException;
-032
-033import java.io.Closeable;
-034import java.io.FileNotFoundException;
-035import java.io.IOException;
-036import java.io.InterruptedIOException;
-037import java.io.PrintWriter;
-038import java.io.StringWriter;
-039import java.net.InetAddress;
-040import java.net.URI;
-041import java.util.ArrayList;
-042import java.util.Arrays;
-043import java.util.Collection;
-044import java.util.Collections;
-045import java.util.Comparator;
-046import java.util.HashMap;
-047import java.util.HashSet;
-048import java.util.Iterator;
-049import java.util.List;
-050import java.util.Locale;
-051import java.util.Map;
-052import java.util.Map.Entry;
-053import java.util.Set;
-054import java.util.SortedMap;
-055import java.util.SortedSet;
-056import java.util.TreeMap;
-057import java.util.TreeSet;
-058import java.util.Vector;
-059import java.util.concurrent.Callable;
-060import 
java.util.concurrent.ConcurrentSkipListMap;
-061import 
java.util.concurrent.ExecutionException;
-062import 
java.util.concurrent.ExecutorService;
-063import java.util.concurrent.Executors;
-064import java.util.concurrent.Future;
-065import java.util.concurrent.FutureTask;
-066import 
java.util.concurrent.ScheduledThreadPoolExecutor;
-067import java.util.concurrent.TimeUnit;
-068import 
java.util.concurrent.TimeoutException;
-069import 
java.util.concurrent.atomic.AtomicBoolean;
-070import 
java.util.concurrent.atomic.AtomicInteger;
-071
-072import org.apache.commons.io.IOUtils;
-073import 
org.apache.commons.lang.RandomStringUtils;
-074import 
org.apache.commons.lang.StringUtils;
-075import org.apache.commons.logging.Log;
-076import 
org.apache.commons.logging.LogFactory;
-077import 
org.apache.hadoop.conf.Configuration;
-078import 
org.apache.hadoop.conf.Configured;
-079import 
org.apache.hadoop.fs.FSDataOutputStream;
-080import org.apache.hadoop.fs.FileStatus;
-081import org.apache.hadoop.fs.FileSystem;
-082import org.apache.hadoop.fs.Path;
-083import 
org.apache.hadoop.fs.permission.FsAction;
-084import 
org.apache.hadoop.fs.permission.FsPermission;
-085import 
org.apache.hadoop.hbase.Abortable;
-086import org.apache.hadoop.hbase.Cell;
-087import 
org.apache.hadoop.hbase.CellUtil;
-088import 
org.apache.hadoop.hbase.ClusterStatus;
-089import 
org.apache.hadoop.hbase.HBaseConfiguration;
-090import 
org.apache.hadoop.hbase.HBaseInterfaceAudience;
-091import 
org.apache.hadoop.hbase.HConstants;
-092import 
org.apache.hadoop.hbase.HRegionInfo;
-093import 
org.apache.hadoop.hbase.HRegionLocation;
-094import 
org.apache.hadoop.hbase.KeyValue;
-095import 
org.apache.hadoop.hbase.MasterNotRunningException;
-096import 
org.apache.hadoop.hbase.MetaTableAccessor;
-097import 
org.apache.hadoop.hbase.RegionLocations;
-098import 
org.apache.hadoop.hbase.ServerName;
-099import 
org.apache.hadoop.hbase.TableName;
-100import 
org.apache.hadoop.hbase.ZooKeeperConnectionException;
-101import 
org.apache.hadoop.hbase.classification.InterfaceAudience;
-102import 
org.apache.hadoop.hbase.classification.InterfaceStability;
-103import 
org.apache.hadoop.hbase.client.Admin;
-104import 
org.apache.hadoop.hbase.client.ClusterConnection;
-105import 
org.apache.hadoop.hbase.client.Connection;
-106import 
org.apache.hadoop.hbase.client.ConnectionFactory;
-107import 
org.apache.hadoop.hbase.client.Delete;
-108import 
org.apache.hadoop.hbase.client.Get;
-109import 
org.apache.hadoop.hbase.client.Put;
-110import 
org.apache.hadoop.hbase.client.RegionReplicaUtil;
-111import 
org.apache.hadoop.hbase.client.Result;
-112import 
org.apache.hadoop.hbase.client.RowMutations;
-113import 
org.apache.hadoop.hbase.client.Table;
-114import 
org.apache.hadoop.hbase.client.TableState;
-115import 
org.apache.hadoop.hbase.io.FileLink;
-116import 
org.apache.hadoop.hbase.io.HFileLink;
-117import 
org.apache.hadoop.hbase.io.hfile.CacheConfig;
-118import 
org.apache.hadoop.hbase.io.hfile.HFile;
-119import 
org.apache.hadoop.hbase.master.MasterFileSystem;
-120import 
org.apache.hadoop.hbase.master.RegionState;
-121import 

hbase-site git commit: INFRA-10751 Empty commit

2017-08-25 Thread git-site-role
Repository: hbase-site
Updated Branches:
  refs/heads/asf-site 6b5143ed1 -> e8ae19713


INFRA-10751 Empty commit


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

Branch: refs/heads/asf-site
Commit: e8ae19713adf6681cebd9863001e952eb0c40c0c
Parents: 6b5143e
Author: jenkins 
Authored: Fri Aug 25 15:09:57 2017 +
Committer: jenkins 
Committed: Fri Aug 25 15:09:57 2017 +

--

--




[49/51] [partial] hbase-site git commit: Published site at .

2017-08-25 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/checkstyle.rss
--
diff --git a/checkstyle.rss b/checkstyle.rss
index 0122a8b..18266d6 100644
--- a/checkstyle.rss
+++ b/checkstyle.rss
@@ -25,8 +25,8 @@ under the License.
 en-us
 2007 - 2017 The Apache Software Foundation
 
-  File: 2030,
- Errors: 12845,
+  File: 2031,
+ Errors: 12867,
  Warnings: 0,
  Infos: 0
   
@@ -391,7 +391,7 @@ under the License.
   0
 
 
-  2
+  4
 
   
   
@@ -5123,7 +5123,7 @@ under the License.
   0
 
 
-  4
+  3
 
   
   
@@ -14256,6 +14256,20 @@ under the License.
   
   
 
+  http://hbase.apache.org/checkstyle.html#org.apache.hadoop.hbase.coprocessor.example.RefreshHFilesEndpoint.java;>org/apache/hadoop/hbase/coprocessor/example/RefreshHFilesEndpoint.java
+
+
+  0
+
+
+  0
+
+
+  2
+
+  
+  
+
   http://hbase.apache.org/checkstyle.html#org.apache.hadoop.hbase.ipc.RemoteWithExtrasException.java;>org/apache/hadoop/hbase/ipc/RemoteWithExtrasException.java
 
 
@@ -14993,7 +15007,7 @@ under the License.
   0
 
 
-  3
+  1
 
   
   
@@ -19996,6 +20010,20 @@ under the License.
   
   
 
+  http://hbase.apache.org/checkstyle.html#org.apache.hadoop.hbase.client.example.RefreshHFilesClient.java;>org/apache/hadoop/hbase/client/example/RefreshHFilesClient.java
+
+
+  0
+
+
+  0
+
+
+  16
+
+  
+  
+
   http://hbase.apache.org/checkstyle.html#org.apache.hadoop.hbase.metrics.MBeanSource.java;>org/apache/hadoop/hbase/metrics/MBeanSource.java
 
 
@@ -2,7 +21139,7 @@ under the License.
   0
 
 
-  4
+  3
 
   
   
@@ -21867,7 +21895,7 @@ under the License.
   0
 
 
-  3
+  6
 
   
   
@@ -22357,7 +22385,7 @@ under the License.
   0
 
 
-  2
+  5
 
   
   
@@ -24462,20 +24490,6 @@ under the License.
   
   
 
-  http://hbase.apache.org/checkstyle.html#org.apache.hadoop.hbase.Service.java;>org/apache/hadoop/hbase/Service.java
-
-
-  0
-
-
-  0
-
-
-  0
-
-  
-  
-
   http://hbase.apache.org/checkstyle.html#org.apache.hadoop.hbase.ipc.FifoRpcScheduler.java;>org/apache/hadoop/hbase/ipc/FifoRpcScheduler.java
 
 

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/coc.html
--
diff --git a/coc.html b/coc.html
index 35bd2e7..975e743 100644
--- a/coc.html
+++ b/coc.html
@@ -7,7 +7,7 @@
   
 
 
-
+
 
 Apache HBase  
   Code of Conduct Policy
@@ -380,7 +380,7 @@ email to mailto:priv...@hbase.apache.org;>the priv
 https://www.apache.org/;>The Apache Software 
Foundation.
 All rights reserved.  
 
-  Last Published: 
2017-08-24
+  Last Published: 
2017-08-25
 
 
 

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/cygwin.html
--
diff --git a/cygwin.html b/cygwin.html
index 5d7f18b..2861b35 100644
--- a/cygwin.html
+++ b/cygwin.html
@@ -7,7 +7,7 @@
   
 
 
-
+
 
 Apache HBase  Installing Apache HBase (TM) on Windows using 
Cygwin

[44/51] [partial] hbase-site git commit: Published site at .

2017-08-25 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/devapidocs/org/apache/hadoop/hbase/master/HMaster.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/master/HMaster.html 
b/devapidocs/org/apache/hadoop/hbase/master/HMaster.html
index 06e851c..0735bc1 100644
--- a/devapidocs/org/apache/hadoop/hbase/master/HMaster.html
+++ b/devapidocs/org/apache/hadoop/hbase/master/HMaster.html
@@ -128,7 +128,7 @@ var activeTableTab = "activeTableTab";
 
 
 @InterfaceAudience.LimitedPrivate(value="Tools")
-public class HMaster
+public class HMaster
 extends HRegionServer
 implements MasterServices
 HMaster is the "master server" for HBase. An HBase cluster 
has one active
@@ -248,21 +248,29 @@ implements cpHost
 
 
+static int
+DEFAULT_HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS
+
+
 private DrainingServerTracker
 drainingServerTracker
 
-
+
 private ExpiredMobFileCleanerChore
 expiredMobFileCleanerChore
 
-
+
 private FavoredNodesManager
 favoredNodesManager
 
-
+
 private MasterFileSystem
 fileSystemManager
 
+
+static http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true;
 title="class or interface in java.lang">String
+HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS
+
 
 private HFileCleaner
 hfileCleaner
@@ -1410,7 +1418,7 @@ implements 
 
 LOG
-private static finalorg.apache.commons.logging.Log LOG
+private static finalorg.apache.commons.logging.Log LOG
 
 
 
@@ -1419,7 +1427,7 @@ implements 
 
 MASTER
-public static finalhttp://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true;
 title="class or interface in java.lang">String MASTER
+public static finalhttp://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true;
 title="class or interface in java.lang">String MASTER
 
 See Also:
 Constant
 Field Values
@@ -1432,7 +1440,7 @@ implements 
 
 activeMasterManager
-private finalActiveMasterManager activeMasterManager
+private finalActiveMasterManager activeMasterManager
 
 
 
@@ -1441,7 +1449,7 @@ implements 
 
 regionServerTracker
-RegionServerTracker regionServerTracker
+RegionServerTracker regionServerTracker
 
 
 
@@ -1450,7 +1458,7 @@ implements 
 
 drainingServerTracker
-privateDrainingServerTracker 
drainingServerTracker
+privateDrainingServerTracker 
drainingServerTracker
 
 
 
@@ -1459,7 +1467,7 @@ implements 
 
 loadBalancerTracker
-LoadBalancerTracker loadBalancerTracker
+LoadBalancerTracker loadBalancerTracker
 
 
 
@@ -1468,7 +1476,7 @@ implements 
 
 splitOrMergeTracker
-privateSplitOrMergeTracker splitOrMergeTracker
+privateSplitOrMergeTracker splitOrMergeTracker
 
 
 
@@ -1477,7 +1485,7 @@ implements 
 
 regionNormalizerTracker
-privateRegionNormalizerTracker 
regionNormalizerTracker
+privateRegionNormalizerTracker 
regionNormalizerTracker
 
 
 
@@ -1486,7 +1494,7 @@ implements 
 
 maintenanceModeTracker
-privateMasterMaintenanceModeTracker maintenanceModeTracker
+privateMasterMaintenanceModeTracker maintenanceModeTracker
 
 
 
@@ -1495,7 +1503,33 @@ implements 
 
 clusterSchemaService
-privateClusterSchemaService clusterSchemaService
+privateClusterSchemaService clusterSchemaService
+
+
+
+
+
+
+
+HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS
+public static finalhttp://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true;
 title="class or interface in java.lang">String HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS
+
+See Also:
+Constant
 Field Values
+
+
+
+
+
+
+
+
+DEFAULT_HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS
+public static finalint DEFAULT_HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS
+
+See Also:
+Constant
 Field Values
+
 
 
 
@@ -1504,7 +1538,7 @@ implements 
 
 metricsMaster
-finalMetricsMaster metricsMaster
+finalMetricsMaster metricsMaster
 
 
 
@@ -1513,7 +1547,7 @@ implements 
 
 fileSystemManager
-privateMasterFileSystem fileSystemManager
+privateMasterFileSystem fileSystemManager
 
 
 
@@ -1522,7 +1556,7 @@ implements 
 
 walManager
-privateMasterWalManager walManager
+privateMasterWalManager walManager
 
 
 
@@ -1531,7 +1565,7 @@ implements 
 
 serverManager
-private volatileServerManager serverManager
+private volatileServerManager serverManager
 
 
 
@@ -1540,7 +1574,7 @@ implements 
 
 assignmentManager
-privateAssignmentManager assignmentManager
+privateAssignmentManager assignmentManager
 
 
 
@@ -1549,7 +1583,7 @@ implements 
 
 replicationManager
-privateReplicationManager replicationManager
+privateReplicationManager replicationManager
 
 
 
@@ -1558,7 +1592,7 @@ implements 
 
 rsFatals
-MemoryBoundedLogMessageBuffer rsFatals
+MemoryBoundedLogMessageBuffer rsFatals
 
 
 
@@ -1567,7 +1601,7 @@ implements 
 
 activeMaster
-private volatileboolean activeMaster
+private volatileboolean activeMaster
 
 
 
@@ -1576,7 +1610,7 @@ implements 
 
 initialized
-private finalProcedureEvent initialized
+private finalProcedureEvent initialized
 
 
 
@@ -1585,7 +1619,7 @@ implements 
 
 serviceStarted
-volatileboolean serviceStarted
+volatileboolean 

[30/51] [partial] hbase-site git commit: Published site at .

2017-08-25 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/devapidocs/src-html/org/apache/hadoop/hbase/master/HMaster.InitializationMonitor.html
--
diff --git 
a/devapidocs/src-html/org/apache/hadoop/hbase/master/HMaster.InitializationMonitor.html
 
b/devapidocs/src-html/org/apache/hadoop/hbase/master/HMaster.InitializationMonitor.html
index a04b2f2..fa9b7a2 100644
--- 
a/devapidocs/src-html/org/apache/hadoop/hbase/master/HMaster.InitializationMonitor.html
+++ 
b/devapidocs/src-html/org/apache/hadoop/hbase/master/HMaster.InitializationMonitor.html
@@ -47,3453 +47,3471 @@
 039import 
java.util.concurrent.ExecutionException;
 040import java.util.concurrent.Future;
 041import java.util.concurrent.TimeUnit;
-042import 
java.util.concurrent.atomic.AtomicInteger;
-043import 
java.util.concurrent.atomic.AtomicReference;
-044import java.util.function.Function;
-045import java.util.regex.Pattern;
-046
-047import javax.servlet.ServletException;
-048import javax.servlet.http.HttpServlet;
-049import 
javax.servlet.http.HttpServletRequest;
-050import 
javax.servlet.http.HttpServletResponse;
-051
-052import org.apache.commons.logging.Log;
-053import 
org.apache.commons.logging.LogFactory;
-054import 
org.apache.hadoop.conf.Configuration;
-055import org.apache.hadoop.fs.FileSystem;
-056import org.apache.hadoop.fs.Path;
-057import 
org.apache.hadoop.hbase.ClusterStatus;
-058import 
org.apache.hadoop.hbase.ClusterStatus.Options;
-059import 
org.apache.hadoop.hbase.CoordinatedStateException;
-060import 
org.apache.hadoop.hbase.CoordinatedStateManager;
-061import 
org.apache.hadoop.hbase.DoNotRetryIOException;
-062import 
org.apache.hadoop.hbase.HBaseIOException;
-063import 
org.apache.hadoop.hbase.HBaseInterfaceAudience;
-064import 
org.apache.hadoop.hbase.HConstants;
-065import 
org.apache.hadoop.hbase.HRegionInfo;
-066import 
org.apache.hadoop.hbase.MasterNotRunningException;
-067import 
org.apache.hadoop.hbase.MetaTableAccessor;
-068import 
org.apache.hadoop.hbase.NamespaceDescriptor;
-069import 
org.apache.hadoop.hbase.PleaseHoldException;
-070import 
org.apache.hadoop.hbase.ProcedureInfo;
-071import 
org.apache.hadoop.hbase.ServerLoad;
-072import 
org.apache.hadoop.hbase.ServerName;
-073import 
org.apache.hadoop.hbase.TableDescriptors;
-074import 
org.apache.hadoop.hbase.TableName;
-075import 
org.apache.hadoop.hbase.TableNotDisabledException;
-076import 
org.apache.hadoop.hbase.TableNotFoundException;
-077import 
org.apache.hadoop.hbase.UnknownRegionException;
-078import 
org.apache.hadoop.hbase.classification.InterfaceAudience;
-079import 
org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
-080import 
org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
-081import 
org.apache.hadoop.hbase.client.MasterSwitchType;
-082import 
org.apache.hadoop.hbase.client.Result;
-083import 
org.apache.hadoop.hbase.client.TableDescriptor;
-084import 
org.apache.hadoop.hbase.client.TableDescriptorBuilder;
-085import 
org.apache.hadoop.hbase.client.TableState;
-086import 
org.apache.hadoop.hbase.coprocessor.BypassCoprocessorException;
-087import 
org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
-088import 
org.apache.hadoop.hbase.exceptions.DeserializationException;
-089import 
org.apache.hadoop.hbase.exceptions.MergeRegionException;
-090import 
org.apache.hadoop.hbase.executor.ExecutorType;
-091import 
org.apache.hadoop.hbase.favored.FavoredNodesManager;
-092import 
org.apache.hadoop.hbase.favored.FavoredNodesPromoter;
-093import 
org.apache.hadoop.hbase.http.InfoServer;
-094import 
org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
-095import 
org.apache.hadoop.hbase.ipc.RpcServer;
-096import 
org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
-097import 
org.apache.hadoop.hbase.master.MasterRpcServices.BalanceSwitchMode;
-098import 
org.apache.hadoop.hbase.master.assignment.AssignmentManager;
-099import 
org.apache.hadoop.hbase.master.assignment.MergeTableRegionsProcedure;
-100import 
org.apache.hadoop.hbase.master.assignment.RegionStates;
-101import 
org.apache.hadoop.hbase.master.assignment.RegionStates.RegionStateNode;
-102import 
org.apache.hadoop.hbase.master.balancer.BalancerChore;
-103import 
org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer;
-104import 
org.apache.hadoop.hbase.master.balancer.ClusterStatusChore;
-105import 
org.apache.hadoop.hbase.master.balancer.LoadBalancerFactory;
-106import 
org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
-107import 
org.apache.hadoop.hbase.master.cleaner.LogCleaner;
-108import 
org.apache.hadoop.hbase.master.cleaner.ReplicationMetaCleaner;
-109import 
org.apache.hadoop.hbase.master.cleaner.ReplicationZKNodeCleaner;
-110import 
org.apache.hadoop.hbase.master.cleaner.ReplicationZKNodeCleanerChore;
-111import 
org.apache.hadoop.hbase.master.locking.LockManager;
-112import 
org.apache.hadoop.hbase.master.normalizer.NormalizationPlan;
-113import 

[28/51] [partial] hbase-site git commit: Published site at .

2017-08-25 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/devapidocs/src-html/org/apache/hadoop/hbase/master/HMaster.html
--
diff --git a/devapidocs/src-html/org/apache/hadoop/hbase/master/HMaster.html 
b/devapidocs/src-html/org/apache/hadoop/hbase/master/HMaster.html
index a04b2f2..fa9b7a2 100644
--- a/devapidocs/src-html/org/apache/hadoop/hbase/master/HMaster.html
+++ b/devapidocs/src-html/org/apache/hadoop/hbase/master/HMaster.html
@@ -47,3453 +47,3471 @@
 039import 
java.util.concurrent.ExecutionException;
 040import java.util.concurrent.Future;
 041import java.util.concurrent.TimeUnit;
-042import 
java.util.concurrent.atomic.AtomicInteger;
-043import 
java.util.concurrent.atomic.AtomicReference;
-044import java.util.function.Function;
-045import java.util.regex.Pattern;
-046
-047import javax.servlet.ServletException;
-048import javax.servlet.http.HttpServlet;
-049import 
javax.servlet.http.HttpServletRequest;
-050import 
javax.servlet.http.HttpServletResponse;
-051
-052import org.apache.commons.logging.Log;
-053import 
org.apache.commons.logging.LogFactory;
-054import 
org.apache.hadoop.conf.Configuration;
-055import org.apache.hadoop.fs.FileSystem;
-056import org.apache.hadoop.fs.Path;
-057import 
org.apache.hadoop.hbase.ClusterStatus;
-058import 
org.apache.hadoop.hbase.ClusterStatus.Options;
-059import 
org.apache.hadoop.hbase.CoordinatedStateException;
-060import 
org.apache.hadoop.hbase.CoordinatedStateManager;
-061import 
org.apache.hadoop.hbase.DoNotRetryIOException;
-062import 
org.apache.hadoop.hbase.HBaseIOException;
-063import 
org.apache.hadoop.hbase.HBaseInterfaceAudience;
-064import 
org.apache.hadoop.hbase.HConstants;
-065import 
org.apache.hadoop.hbase.HRegionInfo;
-066import 
org.apache.hadoop.hbase.MasterNotRunningException;
-067import 
org.apache.hadoop.hbase.MetaTableAccessor;
-068import 
org.apache.hadoop.hbase.NamespaceDescriptor;
-069import 
org.apache.hadoop.hbase.PleaseHoldException;
-070import 
org.apache.hadoop.hbase.ProcedureInfo;
-071import 
org.apache.hadoop.hbase.ServerLoad;
-072import 
org.apache.hadoop.hbase.ServerName;
-073import 
org.apache.hadoop.hbase.TableDescriptors;
-074import 
org.apache.hadoop.hbase.TableName;
-075import 
org.apache.hadoop.hbase.TableNotDisabledException;
-076import 
org.apache.hadoop.hbase.TableNotFoundException;
-077import 
org.apache.hadoop.hbase.UnknownRegionException;
-078import 
org.apache.hadoop.hbase.classification.InterfaceAudience;
-079import 
org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
-080import 
org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
-081import 
org.apache.hadoop.hbase.client.MasterSwitchType;
-082import 
org.apache.hadoop.hbase.client.Result;
-083import 
org.apache.hadoop.hbase.client.TableDescriptor;
-084import 
org.apache.hadoop.hbase.client.TableDescriptorBuilder;
-085import 
org.apache.hadoop.hbase.client.TableState;
-086import 
org.apache.hadoop.hbase.coprocessor.BypassCoprocessorException;
-087import 
org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
-088import 
org.apache.hadoop.hbase.exceptions.DeserializationException;
-089import 
org.apache.hadoop.hbase.exceptions.MergeRegionException;
-090import 
org.apache.hadoop.hbase.executor.ExecutorType;
-091import 
org.apache.hadoop.hbase.favored.FavoredNodesManager;
-092import 
org.apache.hadoop.hbase.favored.FavoredNodesPromoter;
-093import 
org.apache.hadoop.hbase.http.InfoServer;
-094import 
org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
-095import 
org.apache.hadoop.hbase.ipc.RpcServer;
-096import 
org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
-097import 
org.apache.hadoop.hbase.master.MasterRpcServices.BalanceSwitchMode;
-098import 
org.apache.hadoop.hbase.master.assignment.AssignmentManager;
-099import 
org.apache.hadoop.hbase.master.assignment.MergeTableRegionsProcedure;
-100import 
org.apache.hadoop.hbase.master.assignment.RegionStates;
-101import 
org.apache.hadoop.hbase.master.assignment.RegionStates.RegionStateNode;
-102import 
org.apache.hadoop.hbase.master.balancer.BalancerChore;
-103import 
org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer;
-104import 
org.apache.hadoop.hbase.master.balancer.ClusterStatusChore;
-105import 
org.apache.hadoop.hbase.master.balancer.LoadBalancerFactory;
-106import 
org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
-107import 
org.apache.hadoop.hbase.master.cleaner.LogCleaner;
-108import 
org.apache.hadoop.hbase.master.cleaner.ReplicationMetaCleaner;
-109import 
org.apache.hadoop.hbase.master.cleaner.ReplicationZKNodeCleaner;
-110import 
org.apache.hadoop.hbase.master.cleaner.ReplicationZKNodeCleanerChore;
-111import 
org.apache.hadoop.hbase.master.locking.LockManager;
-112import 
org.apache.hadoop.hbase.master.normalizer.NormalizationPlan;
-113import 
org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
-114import 
org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
-115import 

[45/51] [partial] hbase-site git commit: Published site at .

2017-08-25 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/devapidocs/org/apache/hadoop/hbase/coprocessor/example/package-tree.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/coprocessor/example/package-tree.html 
b/devapidocs/org/apache/hadoop/hbase/coprocessor/example/package-tree.html
index 6bd803e..9d97aef 100644
--- a/devapidocs/org/apache/hadoop/hbase/coprocessor/example/package-tree.html
+++ b/devapidocs/org/apache/hadoop/hbase/coprocessor/example/package-tree.html
@@ -94,6 +94,11 @@
 
 
 org.apache.hadoop.hbase.coprocessor.example.ExampleRegionObserverWithMetrics (implements 
org.apache.hadoop.hbase.coprocessor.RegionObserver)
+org.apache.hadoop.hbase.protobuf.generated.RefreshHFilesProtos.RefreshHFilesService
 (implements com.google.protobuf.Service)
+
+org.apache.hadoop.hbase.coprocessor.example.RefreshHFilesEndpoint (implements 
org.apache.hadoop.hbase.Coprocessor, 
org.apache.hadoop.hbase.coprocessor.CoprocessorService)
+
+
 org.apache.hadoop.hbase.coprocessor.example.ZooKeeperScanPolicyObserver (implements 
org.apache.hadoop.hbase.coprocessor.RegionObserver)
 org.apache.hadoop.hbase.coprocessor.example.ZooKeeperScanPolicyObserver.ZKWatcher 
(implements org.apache.zookeeper.Watcher)
 

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/devapidocs/org/apache/hadoop/hbase/executor/package-tree.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/executor/package-tree.html 
b/devapidocs/org/apache/hadoop/hbase/executor/package-tree.html
index 5bdab95..bbf68db 100644
--- a/devapidocs/org/apache/hadoop/hbase/executor/package-tree.html
+++ b/devapidocs/org/apache/hadoop/hbase/executor/package-tree.html
@@ -104,8 +104,8 @@
 
 java.lang.http://docs.oracle.com/javase/8/docs/api/java/lang/Enum.html?is-external=true;
 title="class or interface in java.lang">EnumE (implements java.lang.http://docs.oracle.com/javase/8/docs/api/java/lang/Comparable.html?is-external=true;
 title="class or interface in java.lang">ComparableT, java.io.http://docs.oracle.com/javase/8/docs/api/java/io/Serializable.html?is-external=true;
 title="class or interface in java.io">Serializable)
 
-org.apache.hadoop.hbase.executor.ExecutorType
 org.apache.hadoop.hbase.executor.EventType
+org.apache.hadoop.hbase.executor.ExecutorType
 
 
 

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/devapidocs/org/apache/hadoop/hbase/filter/package-tree.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/filter/package-tree.html 
b/devapidocs/org/apache/hadoop/hbase/filter/package-tree.html
index 4ef4e0f..e254b98 100644
--- a/devapidocs/org/apache/hadoop/hbase/filter/package-tree.html
+++ b/devapidocs/org/apache/hadoop/hbase/filter/package-tree.html
@@ -175,14 +175,14 @@
 
 java.lang.http://docs.oracle.com/javase/8/docs/api/java/lang/Enum.html?is-external=true;
 title="class or interface in java.lang">EnumE (implements java.lang.http://docs.oracle.com/javase/8/docs/api/java/lang/Comparable.html?is-external=true;
 title="class or interface in java.lang">ComparableT, java.io.http://docs.oracle.com/javase/8/docs/api/java/io/Serializable.html?is-external=true;
 title="class or interface in java.io">Serializable)
 
-org.apache.hadoop.hbase.filter.FilterWrapper.FilterRowRetCode
-org.apache.hadoop.hbase.filter.FuzzyRowFilter.Order
-org.apache.hadoop.hbase.filter.FuzzyRowFilter.SatisfiesCode
-org.apache.hadoop.hbase.filter.RegexStringComparator.EngineType
 org.apache.hadoop.hbase.filter.Filter.ReturnCode
 org.apache.hadoop.hbase.filter.BitComparator.BitwiseOp
+org.apache.hadoop.hbase.filter.RegexStringComparator.EngineType
+org.apache.hadoop.hbase.filter.FuzzyRowFilter.Order
+org.apache.hadoop.hbase.filter.FuzzyRowFilter.SatisfiesCode
 org.apache.hadoop.hbase.filter.CompareFilter.CompareOp
 org.apache.hadoop.hbase.filter.FilterList.Operator
+org.apache.hadoop.hbase.filter.FilterWrapper.FilterRowRetCode
 
 
 

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/devapidocs/org/apache/hadoop/hbase/io/hfile/package-tree.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/io/hfile/package-tree.html 
b/devapidocs/org/apache/hadoop/hbase/io/hfile/package-tree.html
index 2c49400..74c7952 100644
--- a/devapidocs/org/apache/hadoop/hbase/io/hfile/package-tree.html
+++ b/devapidocs/org/apache/hadoop/hbase/io/hfile/package-tree.html
@@ -274,11 +274,11 @@
 java.lang.http://docs.oracle.com/javase/8/docs/api/java/lang/Enum.html?is-external=true;
 title="class or interface in java.lang">EnumE (implements java.lang.http://docs.oracle.com/javase/8/docs/api/java/lang/Comparable.html?is-external=true;
 title="class or interface in java.lang">ComparableT, java.io.http://docs.oracle.com/javase/8/docs/api/java/io/Serializable.html?is-external=true;

[10/51] [partial] hbase-site git commit: Published site at .

2017-08-25 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.RegionBoundariesInformation.html
--
diff --git 
a/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.RegionBoundariesInformation.html
 
b/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.RegionBoundariesInformation.html
index f5186b5..43afb13 100644
--- 
a/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.RegionBoundariesInformation.html
+++ 
b/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.RegionBoundariesInformation.html
@@ -36,5085 +36,5084 @@
 028import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.Multimap;
 029import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.Ordering;
 030import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.TreeMultimap;
-031import 
com.google.protobuf.ServiceException;
-032
-033import java.io.Closeable;
-034import java.io.FileNotFoundException;
-035import java.io.IOException;
-036import java.io.InterruptedIOException;
-037import java.io.PrintWriter;
-038import java.io.StringWriter;
-039import java.net.InetAddress;
-040import java.net.URI;
-041import java.util.ArrayList;
-042import java.util.Arrays;
-043import java.util.Collection;
-044import java.util.Collections;
-045import java.util.Comparator;
-046import java.util.HashMap;
-047import java.util.HashSet;
-048import java.util.Iterator;
-049import java.util.List;
-050import java.util.Locale;
-051import java.util.Map;
-052import java.util.Map.Entry;
-053import java.util.Set;
-054import java.util.SortedMap;
-055import java.util.SortedSet;
-056import java.util.TreeMap;
-057import java.util.TreeSet;
-058import java.util.Vector;
-059import java.util.concurrent.Callable;
-060import 
java.util.concurrent.ConcurrentSkipListMap;
-061import 
java.util.concurrent.ExecutionException;
-062import 
java.util.concurrent.ExecutorService;
-063import java.util.concurrent.Executors;
-064import java.util.concurrent.Future;
-065import java.util.concurrent.FutureTask;
-066import 
java.util.concurrent.ScheduledThreadPoolExecutor;
-067import java.util.concurrent.TimeUnit;
-068import 
java.util.concurrent.TimeoutException;
-069import 
java.util.concurrent.atomic.AtomicBoolean;
-070import 
java.util.concurrent.atomic.AtomicInteger;
-071
-072import org.apache.commons.io.IOUtils;
-073import 
org.apache.commons.lang.RandomStringUtils;
-074import 
org.apache.commons.lang.StringUtils;
-075import org.apache.commons.logging.Log;
-076import 
org.apache.commons.logging.LogFactory;
-077import 
org.apache.hadoop.conf.Configuration;
-078import 
org.apache.hadoop.conf.Configured;
-079import 
org.apache.hadoop.fs.FSDataOutputStream;
-080import org.apache.hadoop.fs.FileStatus;
-081import org.apache.hadoop.fs.FileSystem;
-082import org.apache.hadoop.fs.Path;
-083import 
org.apache.hadoop.fs.permission.FsAction;
-084import 
org.apache.hadoop.fs.permission.FsPermission;
-085import 
org.apache.hadoop.hbase.Abortable;
-086import org.apache.hadoop.hbase.Cell;
-087import 
org.apache.hadoop.hbase.CellUtil;
-088import 
org.apache.hadoop.hbase.ClusterStatus;
-089import 
org.apache.hadoop.hbase.HBaseConfiguration;
-090import 
org.apache.hadoop.hbase.HBaseInterfaceAudience;
-091import 
org.apache.hadoop.hbase.HConstants;
-092import 
org.apache.hadoop.hbase.HRegionInfo;
-093import 
org.apache.hadoop.hbase.HRegionLocation;
-094import 
org.apache.hadoop.hbase.KeyValue;
-095import 
org.apache.hadoop.hbase.MasterNotRunningException;
-096import 
org.apache.hadoop.hbase.MetaTableAccessor;
-097import 
org.apache.hadoop.hbase.RegionLocations;
-098import 
org.apache.hadoop.hbase.ServerName;
-099import 
org.apache.hadoop.hbase.TableName;
-100import 
org.apache.hadoop.hbase.ZooKeeperConnectionException;
-101import 
org.apache.hadoop.hbase.classification.InterfaceAudience;
-102import 
org.apache.hadoop.hbase.classification.InterfaceStability;
-103import 
org.apache.hadoop.hbase.client.Admin;
-104import 
org.apache.hadoop.hbase.client.ClusterConnection;
-105import 
org.apache.hadoop.hbase.client.Connection;
-106import 
org.apache.hadoop.hbase.client.ConnectionFactory;
-107import 
org.apache.hadoop.hbase.client.Delete;
-108import 
org.apache.hadoop.hbase.client.Get;
-109import 
org.apache.hadoop.hbase.client.Put;
-110import 
org.apache.hadoop.hbase.client.RegionReplicaUtil;
-111import 
org.apache.hadoop.hbase.client.Result;
-112import 
org.apache.hadoop.hbase.client.RowMutations;
-113import 
org.apache.hadoop.hbase.client.Table;
-114import 
org.apache.hadoop.hbase.client.TableState;
-115import 
org.apache.hadoop.hbase.io.FileLink;
-116import 
org.apache.hadoop.hbase.io.HFileLink;
-117import 
org.apache.hadoop.hbase.io.hfile.CacheConfig;
-118import 
org.apache.hadoop.hbase.io.hfile.HFile;
-119import 
org.apache.hadoop.hbase.master.MasterFileSystem;
-120import 
org.apache.hadoop.hbase.master.RegionState;
-121import 

[43/51] [partial] hbase-site git commit: Published site at .

2017-08-25 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/devapidocs/org/apache/hadoop/hbase/master/HMasterCommandLine.LocalHMaster.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/master/HMasterCommandLine.LocalHMaster.html
 
b/devapidocs/org/apache/hadoop/hbase/master/HMasterCommandLine.LocalHMaster.html
index cd5c15e..7819c90 100644
--- 
a/devapidocs/org/apache/hadoop/hbase/master/HMasterCommandLine.LocalHMaster.html
+++ 
b/devapidocs/org/apache/hadoop/hbase/master/HMasterCommandLine.LocalHMaster.html
@@ -191,7 +191,7 @@ extends 
 
 Fields inherited from classorg.apache.hadoop.hbase.master.HMaster
-catalogJanitorChore,
 coprocessorServiceHandlers,
 cpHost,
 loadBalancerTracker,
 MASTER,
 metricsMaster,
 regionServerTracker,
 rsFatals,
 serviceStarted,
 snapshotManager
+catalogJanitorChore,
 coprocessorServiceHandlers,
 cpHost,
 DEFAULT_HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS,
 HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS,
 loadBalancerTracker,
 MASTER,
 metricsMaster,
 regionServerTracker,
 rsFatals,
 serviceStarted,
 snapshotManager
 
 
 

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/devapidocs/org/apache/hadoop/hbase/master/package-tree.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/master/package-tree.html 
b/devapidocs/org/apache/hadoop/hbase/master/package-tree.html
index 3116abb..a509686 100644
--- a/devapidocs/org/apache/hadoop/hbase/master/package-tree.html
+++ b/devapidocs/org/apache/hadoop/hbase/master/package-tree.html
@@ -81,6 +81,11 @@
 
 java.lang.http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true;
 title="class or interface in java.lang">Object
 
+org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.AbstractService
 (implements 
org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.Service)
+
+org.apache.hadoop.hbase.master.ClusterSchemaServiceImpl (implements 
org.apache.hadoop.hbase.master.ClusterSchemaService)
+
+
 org.apache.hadoop.hbase.regionserver.AnnotationReadingPriorityFunction (implements 
org.apache.hadoop.hbase.ipc.PriorityFunction)
 
 org.apache.hadoop.hbase.master.MasterAnnotationReadingPriorityFunction
@@ -111,7 +116,6 @@
 
 
 
-org.apache.hadoop.hbase.master.ClusterSchemaServiceImpl (implements 
org.apache.hadoop.hbase.master.ClusterSchemaService)
 org.apache.hadoop.hbase.master.ClusterStatusPublisher.MulticastPublisher 
(implements org.apache.hadoop.hbase.master.ClusterStatusPublisher.Publisher)
 org.apache.hadoop.hbase.master.ClusterStatusPublisher.MulticastPublisher.HBaseDatagramChannelFactoryT
 (implements 
org.apache.hadoop.hbase.shaded.io.netty.bootstrap.ChannelFactoryT)
 org.apache.hadoop.conf.Configured (implements 
org.apache.hadoop.conf.Configurable)
@@ -271,7 +275,7 @@
 
 org.apache.hadoop.hbase.master.ClusterSchema
 
-org.apache.hadoop.hbase.master.ClusterSchemaService (also extends 
org.apache.hadoop.hbase.Service)
+org.apache.hadoop.hbase.master.ClusterSchemaService (also extends 
org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.Service)
 
 
 org.apache.hadoop.conf.Configurable
@@ -289,7 +293,7 @@
 org.apache.hadoop.hbase.master.MetricsMasterSourceFactory
 org.apache.hadoop.hbase.master.MetricsMasterWrapper
 org.apache.hadoop.hbase.master.ServerListener
-org.apache.hadoop.hbase.Service
+org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.Service
 
 org.apache.hadoop.hbase.master.ClusterSchemaService (also extends 
org.apache.hadoop.hbase.master.ClusterSchema)
 
@@ -312,11 +316,11 @@
 
 java.lang.http://docs.oracle.com/javase/8/docs/api/java/lang/Enum.html?is-external=true;
 title="class or interface in java.lang">EnumE (implements java.lang.http://docs.oracle.com/javase/8/docs/api/java/lang/Comparable.html?is-external=true;
 title="class or interface in java.lang">ComparableT, java.io.http://docs.oracle.com/javase/8/docs/api/java/io/Serializable.html?is-external=true;
 title="class or interface in java.io">Serializable)
 
-org.apache.hadoop.hbase.master.SplitLogManager.ResubmitDirective
 org.apache.hadoop.hbase.master.RegionState.State
+org.apache.hadoop.hbase.master.SplitLogManager.ResubmitDirective
 org.apache.hadoop.hbase.master.MetricsMasterSourceFactoryImpl.FactoryStorage
-org.apache.hadoop.hbase.master.MasterRpcServices.BalanceSwitchMode
 org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus
+org.apache.hadoop.hbase.master.MasterRpcServices.BalanceSwitchMode
 
 
 

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/devapidocs/org/apache/hadoop/hbase/master/procedure/package-tree.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/master/procedure/package-tree.html 
b/devapidocs/org/apache/hadoop/hbase/master/procedure/package-tree.html
index 9f03eb0..0377dbe 100644
--- 

[12/51] [partial] hbase-site git commit: Published site at .

2017-08-25 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.OnlineEntry.html
--
diff --git 
a/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.OnlineEntry.html 
b/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.OnlineEntry.html
index f5186b5..43afb13 100644
--- 
a/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.OnlineEntry.html
+++ 
b/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.OnlineEntry.html
@@ -36,5085 +36,5084 @@
 028import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.Multimap;
 029import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.Ordering;
 030import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.TreeMultimap;
-031import 
com.google.protobuf.ServiceException;
-032
-033import java.io.Closeable;
-034import java.io.FileNotFoundException;
-035import java.io.IOException;
-036import java.io.InterruptedIOException;
-037import java.io.PrintWriter;
-038import java.io.StringWriter;
-039import java.net.InetAddress;
-040import java.net.URI;
-041import java.util.ArrayList;
-042import java.util.Arrays;
-043import java.util.Collection;
-044import java.util.Collections;
-045import java.util.Comparator;
-046import java.util.HashMap;
-047import java.util.HashSet;
-048import java.util.Iterator;
-049import java.util.List;
-050import java.util.Locale;
-051import java.util.Map;
-052import java.util.Map.Entry;
-053import java.util.Set;
-054import java.util.SortedMap;
-055import java.util.SortedSet;
-056import java.util.TreeMap;
-057import java.util.TreeSet;
-058import java.util.Vector;
-059import java.util.concurrent.Callable;
-060import 
java.util.concurrent.ConcurrentSkipListMap;
-061import 
java.util.concurrent.ExecutionException;
-062import 
java.util.concurrent.ExecutorService;
-063import java.util.concurrent.Executors;
-064import java.util.concurrent.Future;
-065import java.util.concurrent.FutureTask;
-066import 
java.util.concurrent.ScheduledThreadPoolExecutor;
-067import java.util.concurrent.TimeUnit;
-068import 
java.util.concurrent.TimeoutException;
-069import 
java.util.concurrent.atomic.AtomicBoolean;
-070import 
java.util.concurrent.atomic.AtomicInteger;
-071
-072import org.apache.commons.io.IOUtils;
-073import 
org.apache.commons.lang.RandomStringUtils;
-074import 
org.apache.commons.lang.StringUtils;
-075import org.apache.commons.logging.Log;
-076import 
org.apache.commons.logging.LogFactory;
-077import 
org.apache.hadoop.conf.Configuration;
-078import 
org.apache.hadoop.conf.Configured;
-079import 
org.apache.hadoop.fs.FSDataOutputStream;
-080import org.apache.hadoop.fs.FileStatus;
-081import org.apache.hadoop.fs.FileSystem;
-082import org.apache.hadoop.fs.Path;
-083import 
org.apache.hadoop.fs.permission.FsAction;
-084import 
org.apache.hadoop.fs.permission.FsPermission;
-085import 
org.apache.hadoop.hbase.Abortable;
-086import org.apache.hadoop.hbase.Cell;
-087import 
org.apache.hadoop.hbase.CellUtil;
-088import 
org.apache.hadoop.hbase.ClusterStatus;
-089import 
org.apache.hadoop.hbase.HBaseConfiguration;
-090import 
org.apache.hadoop.hbase.HBaseInterfaceAudience;
-091import 
org.apache.hadoop.hbase.HConstants;
-092import 
org.apache.hadoop.hbase.HRegionInfo;
-093import 
org.apache.hadoop.hbase.HRegionLocation;
-094import 
org.apache.hadoop.hbase.KeyValue;
-095import 
org.apache.hadoop.hbase.MasterNotRunningException;
-096import 
org.apache.hadoop.hbase.MetaTableAccessor;
-097import 
org.apache.hadoop.hbase.RegionLocations;
-098import 
org.apache.hadoop.hbase.ServerName;
-099import 
org.apache.hadoop.hbase.TableName;
-100import 
org.apache.hadoop.hbase.ZooKeeperConnectionException;
-101import 
org.apache.hadoop.hbase.classification.InterfaceAudience;
-102import 
org.apache.hadoop.hbase.classification.InterfaceStability;
-103import 
org.apache.hadoop.hbase.client.Admin;
-104import 
org.apache.hadoop.hbase.client.ClusterConnection;
-105import 
org.apache.hadoop.hbase.client.Connection;
-106import 
org.apache.hadoop.hbase.client.ConnectionFactory;
-107import 
org.apache.hadoop.hbase.client.Delete;
-108import 
org.apache.hadoop.hbase.client.Get;
-109import 
org.apache.hadoop.hbase.client.Put;
-110import 
org.apache.hadoop.hbase.client.RegionReplicaUtil;
-111import 
org.apache.hadoop.hbase.client.Result;
-112import 
org.apache.hadoop.hbase.client.RowMutations;
-113import 
org.apache.hadoop.hbase.client.Table;
-114import 
org.apache.hadoop.hbase.client.TableState;
-115import 
org.apache.hadoop.hbase.io.FileLink;
-116import 
org.apache.hadoop.hbase.io.HFileLink;
-117import 
org.apache.hadoop.hbase.io.hfile.CacheConfig;
-118import 
org.apache.hadoop.hbase.io.hfile.HFile;
-119import 
org.apache.hadoop.hbase.master.MasterFileSystem;
-120import 
org.apache.hadoop.hbase.master.RegionState;
-121import 
org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-122import 

[11/51] [partial] hbase-site git commit: Published site at .

2017-08-25 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.PrintingErrorReporter.html
--
diff --git 
a/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.PrintingErrorReporter.html
 
b/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.PrintingErrorReporter.html
index f5186b5..43afb13 100644
--- 
a/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.PrintingErrorReporter.html
+++ 
b/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.PrintingErrorReporter.html
@@ -36,5085 +36,5084 @@
 028import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.Multimap;
 029import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.Ordering;
 030import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.TreeMultimap;
-031import 
com.google.protobuf.ServiceException;
-032
-033import java.io.Closeable;
-034import java.io.FileNotFoundException;
-035import java.io.IOException;
-036import java.io.InterruptedIOException;
-037import java.io.PrintWriter;
-038import java.io.StringWriter;
-039import java.net.InetAddress;
-040import java.net.URI;
-041import java.util.ArrayList;
-042import java.util.Arrays;
-043import java.util.Collection;
-044import java.util.Collections;
-045import java.util.Comparator;
-046import java.util.HashMap;
-047import java.util.HashSet;
-048import java.util.Iterator;
-049import java.util.List;
-050import java.util.Locale;
-051import java.util.Map;
-052import java.util.Map.Entry;
-053import java.util.Set;
-054import java.util.SortedMap;
-055import java.util.SortedSet;
-056import java.util.TreeMap;
-057import java.util.TreeSet;
-058import java.util.Vector;
-059import java.util.concurrent.Callable;
-060import 
java.util.concurrent.ConcurrentSkipListMap;
-061import 
java.util.concurrent.ExecutionException;
-062import 
java.util.concurrent.ExecutorService;
-063import java.util.concurrent.Executors;
-064import java.util.concurrent.Future;
-065import java.util.concurrent.FutureTask;
-066import 
java.util.concurrent.ScheduledThreadPoolExecutor;
-067import java.util.concurrent.TimeUnit;
-068import 
java.util.concurrent.TimeoutException;
-069import 
java.util.concurrent.atomic.AtomicBoolean;
-070import 
java.util.concurrent.atomic.AtomicInteger;
-071
-072import org.apache.commons.io.IOUtils;
-073import 
org.apache.commons.lang.RandomStringUtils;
-074import 
org.apache.commons.lang.StringUtils;
-075import org.apache.commons.logging.Log;
-076import 
org.apache.commons.logging.LogFactory;
-077import 
org.apache.hadoop.conf.Configuration;
-078import 
org.apache.hadoop.conf.Configured;
-079import 
org.apache.hadoop.fs.FSDataOutputStream;
-080import org.apache.hadoop.fs.FileStatus;
-081import org.apache.hadoop.fs.FileSystem;
-082import org.apache.hadoop.fs.Path;
-083import 
org.apache.hadoop.fs.permission.FsAction;
-084import 
org.apache.hadoop.fs.permission.FsPermission;
-085import 
org.apache.hadoop.hbase.Abortable;
-086import org.apache.hadoop.hbase.Cell;
-087import 
org.apache.hadoop.hbase.CellUtil;
-088import 
org.apache.hadoop.hbase.ClusterStatus;
-089import 
org.apache.hadoop.hbase.HBaseConfiguration;
-090import 
org.apache.hadoop.hbase.HBaseInterfaceAudience;
-091import 
org.apache.hadoop.hbase.HConstants;
-092import 
org.apache.hadoop.hbase.HRegionInfo;
-093import 
org.apache.hadoop.hbase.HRegionLocation;
-094import 
org.apache.hadoop.hbase.KeyValue;
-095import 
org.apache.hadoop.hbase.MasterNotRunningException;
-096import 
org.apache.hadoop.hbase.MetaTableAccessor;
-097import 
org.apache.hadoop.hbase.RegionLocations;
-098import 
org.apache.hadoop.hbase.ServerName;
-099import 
org.apache.hadoop.hbase.TableName;
-100import 
org.apache.hadoop.hbase.ZooKeeperConnectionException;
-101import 
org.apache.hadoop.hbase.classification.InterfaceAudience;
-102import 
org.apache.hadoop.hbase.classification.InterfaceStability;
-103import 
org.apache.hadoop.hbase.client.Admin;
-104import 
org.apache.hadoop.hbase.client.ClusterConnection;
-105import 
org.apache.hadoop.hbase.client.Connection;
-106import 
org.apache.hadoop.hbase.client.ConnectionFactory;
-107import 
org.apache.hadoop.hbase.client.Delete;
-108import 
org.apache.hadoop.hbase.client.Get;
-109import 
org.apache.hadoop.hbase.client.Put;
-110import 
org.apache.hadoop.hbase.client.RegionReplicaUtil;
-111import 
org.apache.hadoop.hbase.client.Result;
-112import 
org.apache.hadoop.hbase.client.RowMutations;
-113import 
org.apache.hadoop.hbase.client.Table;
-114import 
org.apache.hadoop.hbase.client.TableState;
-115import 
org.apache.hadoop.hbase.io.FileLink;
-116import 
org.apache.hadoop.hbase.io.HFileLink;
-117import 
org.apache.hadoop.hbase.io.hfile.CacheConfig;
-118import 
org.apache.hadoop.hbase.io.hfile.HFile;
-119import 
org.apache.hadoop.hbase.master.MasterFileSystem;
-120import 
org.apache.hadoop.hbase.master.RegionState;
-121import 

[03/51] [partial] hbase-site git commit: Published site at .

2017-08-25 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/6b5143ed/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.WorkItemOverlapMerge.html
--
diff --git 
a/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.WorkItemOverlapMerge.html
 
b/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.WorkItemOverlapMerge.html
index f5186b5..43afb13 100644
--- 
a/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.WorkItemOverlapMerge.html
+++ 
b/devapidocs/src-html/org/apache/hadoop/hbase/util/HBaseFsck.WorkItemOverlapMerge.html
@@ -36,5085 +36,5084 @@
 028import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.Multimap;
 029import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.Ordering;
 030import 
org.apache.hadoop.hbase.shaded.com.google.common.collect.TreeMultimap;
-031import 
com.google.protobuf.ServiceException;
-032
-033import java.io.Closeable;
-034import java.io.FileNotFoundException;
-035import java.io.IOException;
-036import java.io.InterruptedIOException;
-037import java.io.PrintWriter;
-038import java.io.StringWriter;
-039import java.net.InetAddress;
-040import java.net.URI;
-041import java.util.ArrayList;
-042import java.util.Arrays;
-043import java.util.Collection;
-044import java.util.Collections;
-045import java.util.Comparator;
-046import java.util.HashMap;
-047import java.util.HashSet;
-048import java.util.Iterator;
-049import java.util.List;
-050import java.util.Locale;
-051import java.util.Map;
-052import java.util.Map.Entry;
-053import java.util.Set;
-054import java.util.SortedMap;
-055import java.util.SortedSet;
-056import java.util.TreeMap;
-057import java.util.TreeSet;
-058import java.util.Vector;
-059import java.util.concurrent.Callable;
-060import 
java.util.concurrent.ConcurrentSkipListMap;
-061import 
java.util.concurrent.ExecutionException;
-062import 
java.util.concurrent.ExecutorService;
-063import java.util.concurrent.Executors;
-064import java.util.concurrent.Future;
-065import java.util.concurrent.FutureTask;
-066import 
java.util.concurrent.ScheduledThreadPoolExecutor;
-067import java.util.concurrent.TimeUnit;
-068import 
java.util.concurrent.TimeoutException;
-069import 
java.util.concurrent.atomic.AtomicBoolean;
-070import 
java.util.concurrent.atomic.AtomicInteger;
-071
-072import org.apache.commons.io.IOUtils;
-073import 
org.apache.commons.lang.RandomStringUtils;
-074import 
org.apache.commons.lang.StringUtils;
-075import org.apache.commons.logging.Log;
-076import 
org.apache.commons.logging.LogFactory;
-077import 
org.apache.hadoop.conf.Configuration;
-078import 
org.apache.hadoop.conf.Configured;
-079import 
org.apache.hadoop.fs.FSDataOutputStream;
-080import org.apache.hadoop.fs.FileStatus;
-081import org.apache.hadoop.fs.FileSystem;
-082import org.apache.hadoop.fs.Path;
-083import 
org.apache.hadoop.fs.permission.FsAction;
-084import 
org.apache.hadoop.fs.permission.FsPermission;
-085import 
org.apache.hadoop.hbase.Abortable;
-086import org.apache.hadoop.hbase.Cell;
-087import 
org.apache.hadoop.hbase.CellUtil;
-088import 
org.apache.hadoop.hbase.ClusterStatus;
-089import 
org.apache.hadoop.hbase.HBaseConfiguration;
-090import 
org.apache.hadoop.hbase.HBaseInterfaceAudience;
-091import 
org.apache.hadoop.hbase.HConstants;
-092import 
org.apache.hadoop.hbase.HRegionInfo;
-093import 
org.apache.hadoop.hbase.HRegionLocation;
-094import 
org.apache.hadoop.hbase.KeyValue;
-095import 
org.apache.hadoop.hbase.MasterNotRunningException;
-096import 
org.apache.hadoop.hbase.MetaTableAccessor;
-097import 
org.apache.hadoop.hbase.RegionLocations;
-098import 
org.apache.hadoop.hbase.ServerName;
-099import 
org.apache.hadoop.hbase.TableName;
-100import 
org.apache.hadoop.hbase.ZooKeeperConnectionException;
-101import 
org.apache.hadoop.hbase.classification.InterfaceAudience;
-102import 
org.apache.hadoop.hbase.classification.InterfaceStability;
-103import 
org.apache.hadoop.hbase.client.Admin;
-104import 
org.apache.hadoop.hbase.client.ClusterConnection;
-105import 
org.apache.hadoop.hbase.client.Connection;
-106import 
org.apache.hadoop.hbase.client.ConnectionFactory;
-107import 
org.apache.hadoop.hbase.client.Delete;
-108import 
org.apache.hadoop.hbase.client.Get;
-109import 
org.apache.hadoop.hbase.client.Put;
-110import 
org.apache.hadoop.hbase.client.RegionReplicaUtil;
-111import 
org.apache.hadoop.hbase.client.Result;
-112import 
org.apache.hadoop.hbase.client.RowMutations;
-113import 
org.apache.hadoop.hbase.client.Table;
-114import 
org.apache.hadoop.hbase.client.TableState;
-115import 
org.apache.hadoop.hbase.io.FileLink;
-116import 
org.apache.hadoop.hbase.io.HFileLink;
-117import 
org.apache.hadoop.hbase.io.hfile.CacheConfig;
-118import 
org.apache.hadoop.hbase.io.hfile.HFile;
-119import 
org.apache.hadoop.hbase.master.MasterFileSystem;
-120import 
org.apache.hadoop.hbase.master.RegionState;
-121import 
org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;

  1   2   >