[13/50] [abbrv] hbase git commit: HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi) Move to a new AssignmentManager, one that describes Assignment using a State Machine built on top

2017-05-23 Thread stack
http://git-wip-us.apache.org/repos/asf/hbase/blob/f56592fd/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
new file mode 100644
index 000..f1c1a40
--- /dev/null
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
@@ -0,0 +1,1792 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.master.assignment;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseIOException;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.PleaseHoldException;
+import org.apache.hadoop.hbase.RegionException;
+import org.apache.hadoop.hbase.RegionStateListener;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
+import org.apache.hadoop.hbase.favored.FavoredNodeLoadBalancer;
+import org.apache.hadoop.hbase.favored.FavoredNodesManager;
+import org.apache.hadoop.hbase.favored.FavoredNodesPromoter;
+import org.apache.hadoop.hbase.master.AssignmentListener;
+import org.apache.hadoop.hbase.master.LoadBalancer;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.master.MetricsAssignmentManager;
+import org.apache.hadoop.hbase.master.NoSuchProcedureException;
+import org.apache.hadoop.hbase.master.RegionPlan;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.master.RegionState.State;
+import org.apache.hadoop.hbase.master.ServerListener;
+import org.apache.hadoop.hbase.master.TableStateManager;
+import org.apache.hadoop.hbase.master.assignment.RegionStates.RegionStateNode;
+import org.apache.hadoop.hbase.master.assignment.RegionStates.ServerState;
+import org.apache.hadoop.hbase.master.assignment.RegionStates.ServerStateNode;
+// TODO: why are they here?
+import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureScheduler;
+import org.apache.hadoop.hbase.master.procedure.ProcedureSyncWait;
+import org.apache.hadoop.hbase.master.procedure.ServerCrashProcedure;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.procedure2.ProcedureInMemoryChore;
+import org.apache.hadoop.hbase.procedure2.util.StringUtils;
+import org.apache.hadoop.hbase.quotas.QuotaExceededException;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionState

[46/50] [abbrv] hbase git commit: Fix archiving of pv2 WAL files

2017-05-23 Thread stack
Fix archiving of pv2 WAL files


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

Branch: refs/heads/HBASE-14614
Commit: a8dfa377caedcd81a55fed7169e18fa062b5936f
Parents: 95c6180
Author: Michael Stack 
Authored: Fri May 12 13:02:32 2017 -0700
Committer: Michael Stack 
Committed: Tue May 23 00:33:03 2017 -0700

--
 .../procedure2/store/wal/ProcedureWALFile.java  | 28 ++--
 .../procedure2/store/wal/WALProcedureStore.java | 28 +---
 .../org/apache/hadoop/hbase/master/HMaster.java |  4 ++-
 .../assignment/TestAssignmentManager.java   |  2 +-
 4 files changed, 36 insertions(+), 26 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/a8dfa377/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java
--
diff --git 
a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java
 
b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java
index 2221cfc..42abe8f 100644
--- 
a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java
+++ 
b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java
@@ -26,7 +26,6 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.procedure2.store.ProcedureStoreTracker;
@@ -156,22 +155,23 @@ public class ProcedureWALFile implements 
Comparable {
 this.logSize += size;
   }
 
-  public void removeFile() throws IOException {
+  public void removeFile(final Path walArchiveDir) throws IOException {
 close();
-// TODO: FIX THIS. MAKE THIS ARCHIVE FORMAL.
-Path archiveDir =
-new Path(logFile.getParent().getParent(), 
HConstants.HFILE_ARCHIVE_DIRECTORY);
-try {
-  fs.mkdirs(archiveDir);
-} catch (IOException ioe) {
-  LOG.warn("Making " + archiveDir, ioe);
+boolean archived = false;
+if (walArchiveDir != null) {
+  Path archivedFile = new Path(walArchiveDir, logFile.getName());
+  LOG.info("ARCHIVED (TODO: FILES ARE NOT PURGED FROM ARCHIVE!) " + 
logFile + " to " + walArchiveDir);
+  if (!fs.rename(logFile, archivedFile)) {
+LOG.warn("Failed archive of " + logFile + ", deleting");
+  } else {
+archived = true;
+  }
 }
-Path archivedFile = new Path(archiveDir, logFile.getName());
-LOG.info("ARCHIVED WAL (TODO: FILES ARE NOT PURGED FROM ARCHIVE!) " + 
logFile + " to " + archivedFile);
-if (!fs.rename(logFile, archivedFile)) {
-  LOG.warn("Failed archive of " + logFile);
+if (!archived) {
+  if (!fs.delete(logFile, false)) {
+LOG.warn("Failed delete of " + logFile);
+  }
 }
-// fs.delete(logFile, false);
   }
 
   public void setProcIds(long minId, long maxId) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a8dfa377/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
--
diff --git 
a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
 
b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
index 300e023..df818fe 100644
--- 
a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
+++ 
b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
@@ -124,6 +124,7 @@ public class WALProcedureStore extends ProcedureStoreBase {
   private final Configuration conf;
   private final FileSystem fs;
   private final Path walDir;
+  private final Path walArchiveDir;
 
   private final AtomicReference syncException = new 
AtomicReference<>();
   private final AtomicBoolean loading = new AtomicBoolean(true);
@@ -185,9 +186,15 @@ public class WALProcedureStore extends ProcedureStoreBase {
 
   public WALProcedureStore(final Configuration conf, final FileSystem fs, 
final Path walDir,
   final LeaseRecovery leaseRecovery) {
+this(conf, fs, walDir, null, leaseRecovery);
+  }
+
+  public WALProcedureStore(final Configuration conf, final FileSystem fs, 
final Path walDir,
+  final Path walArchiveDir, final LeaseRecovery

[16/50] [abbrv] hbase git commit: HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi) Move to a new AssignmentManager, one that describes Assignment using a State Machine built on top

2017-05-23 Thread stack
http://git-wip-us.apache.org/repos/asf/hbase/blob/f56592fd/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
deleted file mode 100644
index 69ebd97..000
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
+++ /dev/null
@@ -1,3053 +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.master;
-
-import com.google.common.annotations.VisibleForTesting;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.NavigableMap;
-import java.util.Random;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.CoordinatedStateException;
-import org.apache.hadoop.hbase.HBaseIOException;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HRegionLocation;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.MetaTableAccessor;
-import org.apache.hadoop.hbase.NotServingRegionException;
-import org.apache.hadoop.hbase.RegionLocations;
-import org.apache.hadoop.hbase.RegionStateListener;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.client.MasterSwitchType;
-import org.apache.hadoop.hbase.client.RegionReplicaUtil;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.TableState;
-import org.apache.hadoop.hbase.executor.EventHandler;
-import org.apache.hadoop.hbase.executor.EventType;
-import org.apache.hadoop.hbase.executor.ExecutorService;
-import org.apache.hadoop.hbase.favored.FavoredNodesManager;
-import org.apache.hadoop.hbase.favored.FavoredNodesPromoter;
-import org.apache.hadoop.hbase.ipc.FailedServerException;
-import org.apache.hadoop.hbase.ipc.RpcClient;
-import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
-import org.apache.hadoop.hbase.master.RegionState.State;
-import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
-import 
org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
-import 
org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
-import org.apache.hadoop.hbase.quotas.QuotaExceededException;
-import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
-import org.apache.hadoop.hbase.regionserver.RegionServerAbortedException;
-import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.util.KeyLocker;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.util.PairOfSameType;
-import org.apache.hadoop.hbase.util.RetryCounter;
-import org.apache.hadoop.hbase.util.Threads;
-import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.hadoop.ip

[10/50] [abbrv] hbase git commit: HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi) Move to a new AssignmentManager, one that describes Assignment using a State Machine built on top

2017-05-23 Thread stack
http://git-wip-us.apache.org/repos/asf/hbase/blob/f56592fd/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java
index b53ce45..4d45af3 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java
@@ -21,12 +21,9 @@ package org.apache.hadoop.hbase.master.procedure;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
-import java.util.List;
-import java.util.concurrent.ExecutorService;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotEnabledException;
@@ -34,17 +31,11 @@ import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.constraint.ConstraintException;
-import org.apache.hadoop.hbase.master.AssignmentManager;
-import org.apache.hadoop.hbase.master.BulkAssigner;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
-import org.apache.hadoop.hbase.master.RegionState;
-import org.apache.hadoop.hbase.master.RegionStates;
 import org.apache.hadoop.hbase.master.TableStateManager;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
 import 
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DisableTableState;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.htrace.Trace;
 
 @InterfaceAudience.Private
 public class DisableTableProcedure
@@ -116,12 +107,8 @@ public class DisableTableProcedure
 setNextState(DisableTableState.DISABLE_TABLE_MARK_REGIONS_OFFLINE);
 break;
   case DISABLE_TABLE_MARK_REGIONS_OFFLINE:
-if (markRegionsOffline(env, tableName, true) ==
-MarkRegionOfflineOpResult.MARK_ALL_REGIONS_OFFLINE_SUCCESSFUL) {
-  
setNextState(DisableTableState.DISABLE_TABLE_SET_DISABLED_TABLE_STATE);
-} else {
-  LOG.trace("Retrying later to disable the missing regions");
-}
+
addChildProcedure(env.getAssignmentManager().createUnassignProcedures(tableName));
+setNextState(DisableTableState.DISABLE_TABLE_SET_DISABLED_TABLE_STATE);
 break;
   case DISABLE_TABLE_SET_DISABLED_TABLE_STATE:
 setTableStateToDisabled(env, tableName);
@@ -290,83 +277,6 @@ public class DisableTableProcedure
   }
 
   /**
-   * Mark regions of the table offline with retries
-   * @param env MasterProcedureEnv
-   * @param tableName the target table
-   * @param retryRequired whether to retry if the first run failed
-   * @return whether the operation is fully completed or being interrupted.
-   * @throws IOException
-   */
-  protected static MarkRegionOfflineOpResult markRegionsOffline(
-  final MasterProcedureEnv env,
-  final TableName tableName,
-  final Boolean retryRequired) throws IOException {
-// Dev consideration: add a config to control max number of retry. For 
now, it is hard coded.
-int maxTry = (retryRequired ? 10 : 1);
-MarkRegionOfflineOpResult operationResult =
-MarkRegionOfflineOpResult.BULK_ASSIGN_REGIONS_FAILED;
-do {
-  try {
-operationResult = markRegionsOffline(env, tableName);
-if (operationResult == 
MarkRegionOfflineOpResult.MARK_ALL_REGIONS_OFFLINE_SUCCESSFUL) {
-  break;
-}
-maxTry--;
-  } catch (Exception e) {
-LOG.warn("Received exception while marking regions online. tries left: 
" + maxTry, e);
-maxTry--;
-if (maxTry > 0) {
-  continue; // we still have some retry left, try again.
-}
-throw e;
-  }
-} while (maxTry > 0);
-
-if (operationResult != 
MarkRegionOfflineOpResult.MARK_ALL_REGIONS_OFFLINE_SUCCESSFUL) {
-  LOG.warn("Some or all regions of the Table '" + tableName + "' were 
still online");
-}
-
-return operationResult;
-  }
-
-  /**
-   * Mark regions of the table offline
-   * @param env MasterProcedureEnv
-   * @param tableName the target table
-   * @return whether the operation is fully completed or being interrupted.
-   * @throws IOException
-   */
-  private static MarkRegionOfflineOpResult markRegionsOffline(
-  final MasterProcedureEnv env,
-  final TableName tableName) throws IOException {
-// Get list of online reg

[15/50] [abbrv] hbase git commit: HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi) Move to a new AssignmentManager, one that describes Assignment using a State Machine built on top

2017-05-23 Thread stack
http://git-wip-us.apache.org/repos/asf/hbase/blob/f56592fd/hbase-server/src/main/java/org/apache/hadoop/hbase/master/BulkAssigner.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/BulkAssigner.java 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/BulkAssigner.java
deleted file mode 100644
index 929cd4e..000
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/BulkAssigner.java
+++ /dev/null
@@ -1,122 +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.master;
-
-import java.io.IOException;
-import java.lang.Thread.UncaughtExceptionHandler;
-import java.util.concurrent.Executors;
-
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.Server;
-
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-
-/**
- * Base class used bulk assigning and unassigning regions.
- * Encapsulates a fixed size thread pool of executors to run 
assignment/unassignment.
- * Implement {@link #populatePool(java.util.concurrent.ExecutorService)} and
- * {@link #waitUntilDone(long)}.  The default implementation of
- * the {@link #getUncaughtExceptionHandler()} is to abort the hosting
- * Server.
- */
-@InterfaceAudience.Private
-public abstract class BulkAssigner {
-  protected final Server server;
-
-  /**
-   * @param server An instance of Server
-   */
-  public BulkAssigner(final Server server) {
-this.server = server;
-  }
-
-  /**
-   * @return What to use for a thread prefix when executor runs.
-   */
-  protected String getThreadNamePrefix() {
-return this.server.getServerName() + "-" + this.getClass().getName(); 
-  }
-
-  protected UncaughtExceptionHandler getUncaughtExceptionHandler() {
-return new UncaughtExceptionHandler() {
-  @Override
-  public void uncaughtException(Thread t, Throwable e) {
-// Abort if exception of any kind.
-server.abort("Uncaught exception in " + t.getName(), e);
-  }
-};
-  }
-
-  protected int getThreadCount() {
-return this.server.getConfiguration().
-  getInt("hbase.bulk.assignment.threadpool.size", 20);
-  }
-
-  protected long getTimeoutOnRIT() {
-return this.server.getConfiguration().
-  getLong("hbase.bulk.assignment.waiton.empty.rit", 5 * 60 * 1000);
-  }
-
-  protected abstract void populatePool(
-  final java.util.concurrent.ExecutorService pool) throws IOException;
-
-  public boolean bulkAssign() throws InterruptedException, IOException {
-return bulkAssign(true);
-  }
-
-  /**
-   * Run the bulk assign.
-   * 
-   * @param sync
-   *  Whether to assign synchronously.
-   * @throws InterruptedException
-   * @return True if done.
-   * @throws IOException
-   */
-  public boolean bulkAssign(boolean sync) throws InterruptedException,
-  IOException {
-boolean result = false;
-ThreadFactoryBuilder builder = new ThreadFactoryBuilder();
-builder.setDaemon(true);
-builder.setNameFormat(getThreadNamePrefix() + "-%1$d");
-builder.setUncaughtExceptionHandler(getUncaughtExceptionHandler());
-int threadCount = getThreadCount();
-java.util.concurrent.ExecutorService pool =
-  Executors.newFixedThreadPool(threadCount, builder.build());
-try {
-  populatePool(pool);
-  // How long to wait on empty regions-in-transition.  If we timeout, the
-  // RIT monitor should do fixup.
-  if (sync) result = waitUntilDone(getTimeoutOnRIT());
-} finally {
-  // We're done with the pool.  It'll exit when its done all in queue.
-  pool.shutdown();
-}
-return result;
-  }
-
-  /**
-   * Wait until bulk assign is done.
-   * @param timeout How long to wait.
-   * @throws InterruptedException
-   * @return True if the condition we were waiting on happened.
-   */
-  protected abstract boolean waitUntilDone(final long timeout)
-  throws InterruptedException;
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/f56592fd/hbase-server/src/main/java/org/apache/hadoop/hbase/master/BulkReOpen.java
--
diff --git 
a/hbase-server

[27/50] [abbrv] hbase git commit: Fix failing hbase-procedure tests

2017-05-23 Thread stack
Fix failing hbase-procedure tests


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

Branch: refs/heads/HBASE-14614
Commit: dd7b565b768e064425c653a27050b5132f7b95fb
Parents: 150d60d
Author: Michael Stack 
Authored: Sat May 6 10:45:58 2017 -0700
Committer: Michael Stack 
Committed: Tue May 23 00:33:02 2017 -0700

--
 .../java/org/apache/hadoop/hbase/procedure2/Procedure.java | 2 +-
 .../org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java  | 6 --
 2 files changed, 5 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/dd7b565b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
--
diff --git 
a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
 
b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
index 5527076..9c47957 100644
--- 
a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
+++ 
b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
@@ -351,7 +351,7 @@ public abstract class Procedure implements 
Comparable {
* @param builder the string builder to use to append the proc specific 
information
*/
   protected void toStringClassDetails(StringBuilder builder) {
-builder.append(getClass().getSimpleName());
+builder.append(getClass().getName());
   }
 
   // ==

http://git-wip-us.apache.org/repos/asf/hbase/blob/dd7b565b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
--
diff --git 
a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
 
b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
index d36be64..e819ae8 100644
--- 
a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
+++ 
b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
@@ -1403,11 +1403,13 @@ public class ProcedureExecutor {
 reExecute = true;
 LOG.info("Short-circuit to rexecute for pid=" + 
procedure.getProcId());
   } else {
-// yield the current procedure, and make the subprocedure runnable
+// Yield the current procedure, and make the subprocedure runnable
+// subprocs may come back 'null'.
 subprocs = initializeChildren(procStack, procedure, subprocs);
 LOG.info("Initialized subprocedures=" +
+  (subprocs == null? null:
 Stream.of(subprocs).map(e -> "{" + e.toString() + "}").
-collect(Collectors.toList()).toString());
+collect(Collectors.toList()).toString()));
   }
 } else if (procedure.getState() == ProcedureState.WAITING_TIMEOUT) {
   if (LOG.isTraceEnabled()) {



[34/50] [abbrv] hbase git commit: Wait a second before killing a regionserver if state is not what is expected. Also, stop active expire of a RS from setting state on regions to offline.... let Server

2017-05-23 Thread stack
Wait a second before killing a regionserver if state is not what is expected. 
Also, stop active expire of a RS from setting state on regions to offline 
let ServerCrashProcedure do this Its messing us up when a Region is set 
OFFLINE of a sudden


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

Branch: refs/heads/HBASE-14614
Commit: ed59f58a2328bb083a7de9046e56e29dd1b00ccd
Parents: 36368fe
Author: Michael Stack 
Authored: Mon May 8 19:28:06 2017 -0700
Committer: Michael Stack 
Committed: Tue May 23 00:33:03 2017 -0700

--
 .../master/assignment/AssignmentManager.java| 21 
 .../hbase/master/assignment/RegionStates.java   |  9 -
 .../master/procedure/DisableTableProcedure.java |  2 +-
 .../hadoop/hbase/HBaseTestingUtility.java   |  6 +-
 .../hbase/client/TestAsyncRegionAdminApi.java   |  7 ++-
 .../TestFavoredStochasticLoadBalancer.java  |  7 ---
 .../TestSimpleRegionNormalizerOnCluster.java|  3 ++-
 7 files changed, 39 insertions(+), 16 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/ed59f58a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
index e13a052..e567d2d 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
@@ -83,6 +83,7 @@ import 
org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProto
 import 
org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
 import 
org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdge;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Threads;
@@ -831,6 +832,7 @@ public class AssignmentManager implements ServerListener {
 }
   }
 
+  // FYI: regionNode is sometimes synchronized by the caller but not always.
   private boolean reportTransition(final RegionStateNode regionNode,
   final ServerStateNode serverNode, final TransitionCode state, final long 
seqId)
   throws UnexpectedStateException {
@@ -988,18 +990,15 @@ public class AssignmentManager implements ServerListener {
 }
   }
 
-  public void checkOnlineRegionsReport(final ServerStateNode serverNode,
-  final Set regionNames) {
+  void checkOnlineRegionsReport(final ServerStateNode serverNode, final 
Set regionNames) {
 final ServerName serverName = serverNode.getServerName();
 try {
   for (byte[] regionName: regionNames) {
 if (!isRunning()) return;
-
 final RegionStateNode regionNode = 
regionStates.getRegionNodeFromName(regionName);
 if (regionNode == null) {
   throw new UnexpectedStateException("Not online: " + 
Bytes.toStringBinary(regionName));
 }
-
 synchronized (regionNode) {
   if (regionNode.isInState(State.OPENING, State.OPEN)) {
 if (!regionNode.getRegionLocation().equals(serverName)) {
@@ -1017,9 +1016,14 @@ public class AssignmentManager implements ServerListener 
{
   }
 }
   } else if (!regionNode.isInState(State.CLOSING, State.SPLITTING)) {
-// TODO: We end up killing the RS if we get a report while we 
already
-// transitioned to close or split. we should have a 
timeout/timestamp to compare
-throw new UnexpectedStateException(regionNode.toString() + " 
reported unexpected OPEN");
+long diff = regionNode.getLastUpdate() - 
EnvironmentEdgeManager.currentTime();
+if (diff > 1000/*One Second... make configurable if an issue*/) {
+  // So, we can get report that a region is CLOSED or SPLIT 
because a heartbeat
+  // came in at about same time as a region transition. Make sure 
there is some
+  // elapsed time between killing remote server.
+  throw new UnexpectedStateException(regionNode.toString() +
+" reported an unexpected OPEN; time since last update=" + 
diff);
+}
   }
 }
   }
@@ -1804,9 +1808,10 @@ public class Assi

[28/50] [abbrv] hbase git commit: Exceptions are not logging identifying details like region; makes it hard to debug who is problem

2017-05-23 Thread stack
Exceptions are not logging identifying details like region; makes it hard to 
debug who is problem


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

Branch: refs/heads/HBASE-14614
Commit: 6a7d518364128c106d8d0738cac6adec344fddb0
Parents: dd7b565
Author: Michael Stack 
Authored: Sat May 6 11:22:57 2017 -0700
Committer: Michael Stack 
Committed: Tue May 23 00:33:02 2017 -0700

--
 .../master/assignment/AssignmentManager.java  | 18 --
 .../hbase/master/assignment/RegionStates.java |  2 +-
 2 files changed, 9 insertions(+), 11 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/6a7d5183/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
index 2986868..ed55235 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
@@ -983,7 +983,7 @@ public class AssignmentManager implements ServerListener {
   }
 } catch (UnexpectedStateException e) {
   final ServerName serverName = serverNode.getServerName();
-  LOG.warn("Killing " + serverName + ": " + e.getMessage());
+  LOG.warn("KILLING " + serverName + ": " + e.getMessage());
   killRegionServer(serverNode);
 }
   }
@@ -1003,25 +1003,23 @@ public class AssignmentManager implements 
ServerListener {
 synchronized (regionNode) {
   if (regionNode.isInState(State.OPENING, State.OPEN)) {
 if (!regionNode.getRegionLocation().equals(serverName)) {
-  throw new UnexpectedStateException(
-"Reported OPEN on server=" + serverName +
-" but state found says server=" + 
regionNode.getRegionLocation());
+  throw new UnexpectedStateException(regionNode.toString() +
+"reported OPEN on server=" + serverName +
+" but state has otherwise.");
 } else if (regionNode.isInState(State.OPENING)) {
   try {
 if (!reportTransition(regionNode, serverNode, 
TransitionCode.OPENED, 0)) {
-  LOG.warn("Reported OPEN on server=" + serverName +
-" but state found says " + regionNode + " and NO procedure 
is running");
+  LOG.warn(regionNode.toString() + " reported OPEN on server=" 
+ serverName +
+" but state has otherwise AND NO procedure is running");
 }
   } catch (UnexpectedStateException e) {
-LOG.warn("Unexpected exception while trying to report " + 
regionNode +
-  " as open: " + e.getMessage(), e);
+LOG.warn(regionNode.toString() + " reported unexpteced OPEN: " 
+ e.getMessage(), e);
   }
 }
   } else if (!regionNode.isInState(State.CLOSING, State.SPLITTING)) {
 // TODO: We end up killing the RS if we get a report while we 
already
 // transitioned to close or split. we should have a 
timeout/timestamp to compare
-throw new UnexpectedStateException(
-"Reported OPEN but state found says " + regionNode.getState());
+throw new UnexpectedStateException(regionNode.toString() + " 
reported unexpected OPEN");
   }
 }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6a7d5183/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
index aa2627c..1c852c9 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
@@ -173,7 +173,7 @@ public class RegionStates {
 public ServerName setRegionLocation(final ServerName serverName) {
   ServerName lastRegionLocation = this.regionLocation;
   if (serverName == null) {
-LOG.debug("REMOVE tracking when we are set to null", new 
Throwable("DEBUG"));
+LOG.debug("REMOVE tracking when we are set to null " + this, new 
Throwable("DEBUG"));
  

[01/50] [abbrv] hbase git commit: HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi) Move to a new AssignmentManager, one that describes Assignment using a State Machine built on top

2017-05-23 Thread stack
Repository: hbase
Updated Branches:
  refs/heads/HBASE-14614 fc58a5345 -> 052fab32a (forced update)


http://git-wip-us.apache.org/repos/asf/hbase/blob/f56592fd/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
--
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
index 81846df..a64d102 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
@@ -39,7 +39,7 @@ 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.Abortable;
+import org.apache.hadoop.hbase.CategoryBasedTimeout;
 import org.apache.hadoop.hbase.CoordinatedStateManager;
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
@@ -66,23 +66,26 @@ 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.TestReplicasClient.SlowMeCopro;
-import org.apache.hadoop.hbase.coprocessor.MasterObserver;
 import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.MasterObserver;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
-import org.apache.hadoop.hbase.master.AssignmentManager;
+import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.MasterRpcServices;
 import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.master.RegionState.State;
-import org.apache.hadoop.hbase.master.RegionStates;
-import 
org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import 
org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
-import 
org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
+import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
+import org.apache.hadoop.hbase.master.assignment.RegionStates;
+import org.apache.hadoop.hbase.master.NoSuchProcedureException;
+import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
 import 
org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -98,11 +101,11 @@ import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
+import org.junit.rules.TestRule;
 
 /**
  * The below tests are testing split region against a running cluster
@@ -110,8 +113,9 @@ import org.junit.rules.TestName;
 @Category({RegionServerTests.class, LargeTests.class})
 @SuppressWarnings("deprecation")
 public class TestSplitTransactionOnCluster {
-  private static final Log LOG =
-LogFactory.getLog(TestSplitTransactionOnCluster.class);
+  private static final Log LOG = 
LogFactory.getLog(TestSplitTransactionOnCluster.class);
+  @Rule public final TestRule timeout = 
CategoryBasedTimeout.builder().withTimeout(this.getClass()).
+  withLookingForStuckThread(true).build();
   private Admin admin = null;
   private MiniHBaseCluster cluster = null;
   private static final int NB_SERVERS = 3;
@@ -150,8 +154,11 @@ public class TestSplitTransactionOnCluster {
   throws IOException, InterruptedException {
 assertEquals(1, regions.size());
 HRegionInfo hri = regions.get(0).getRegionInfo();
-cluster.getMast

[42/50] [abbrv] hbase git commit: Two SPLIT requests came in on top of each other; the second failed because it saw parent region was SPLIT. I 'fixed' this before but my fix was in the wrong place

2017-05-23 Thread stack
Two SPLIT requests came in on top of each other; the second failed because it 
saw parent region was SPLIT. I 'fixed' this before but my fix was in the wrong 
place


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

Branch: refs/heads/HBASE-14614
Commit: c613f613b442a1fe155adfd7bd998d79987d6a06
Parents: af2abbd
Author: Michael Stack 
Authored: Mon May 15 09:49:30 2017 -0700
Committer: Michael Stack 
Committed: Tue May 23 00:33:03 2017 -0700

--
 .../hbase/procedure2/ProcedureExecutor.java |  2 +-
 .../assignment/SplitTableRegionProcedure.java   | 20 
 .../hadoop/hbase/regionserver/HRegion.java  |  5 +
 3 files changed, 18 insertions(+), 9 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/c613f613/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
--
diff --git 
a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
 
b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
index 258e268..bc73453 100644
--- 
a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
+++ 
b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
@@ -1147,7 +1147,7 @@ public class ProcedureExecutor {
   scheduler.yield(proc);
   break;
 case LOCK_EVENT_WAIT:
-  // someone will wake us up when the lock is available
+  // Someone will wake us up when the lock is available
   LOG.debug(lockState + " " + proc);
   break;
 default:

http://git-wip-us.apache.org/repos/asf/hbase/blob/c613f613/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
index 6815e9f..7ebe769 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
@@ -341,6 +341,15 @@ public class SplitTableRegionProcedure
 if (node != null) {
   parentHRI = node.getRegionInfo();
 
+  // Lookup the parent HRI state from the AM, which has the latest updated 
info.
+  // Protect against the case where concurrent SPLIT requests came in. 
Check a SPLIT
+  // did not just run.
+  if (parentHRI.isSplit() || parentHRI.isOffline()) {
+setFailure(new IOException("Split " + 
parentHRI.getRegionNameAsString() + " FAILED because " +
+"offline/split already."));
+return false;
+  }
+
   // expected parent to be online or closed
   if (!node.isInState(EXPECTED_SPLIT_STATES)) {
 // We may have SPLIT already?
@@ -350,13 +359,6 @@ public class SplitTableRegionProcedure
 return false;
   }
 
-  // lookup the parent HRI state from the AM, which has the latest updated 
info.
-  if (parentHRI.isSplit() || parentHRI.isOffline()) {
-setFailure(new IOException("Split " + 
parentHRI.getRegionNameAsString() + " FAILED because " +
-"offline/split already."));
-return false;
-  }
-
   // Ask the remote regionserver if this region is splittable. If we get 
an IOE, report it
   // along w/ the failure so can see why we are not splittable at this 
time.
   IOException splittableCheckIOE = null;
@@ -365,7 +367,9 @@ public class SplitTableRegionProcedure
 GetRegionInfoResponse response =
 Util.getRegionInfoResponse(env, node.getRegionLocation(), 
node.getRegionInfo());
 splittable = response.hasSplittable() && response.getSplittable();
-LOG.info("REMOVE splittable " + splittable + " " + this + " " + node);
+if (LOG.isDebugEnabled()) {
+  LOG.debug("Splittable=" + splittable + " " + this + " " + 
node.toShortString());
+}
   } catch (IOException e) {
 splittableCheckIOE = e;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c613f613/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/

[26/50] [abbrv] hbase git commit: Rebase with master fixup

2017-05-23 Thread stack
Rebase with master fixup


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

Branch: refs/heads/HBASE-14614
Commit: 150d60d24f0e086a1d6be8eff0c845b638cfb920
Parents: 11aef64
Author: Michael Stack 
Authored: Fri May 5 21:58:22 2017 -0700
Committer: Michael Stack 
Committed: Tue May 23 00:33:02 2017 -0700

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/150d60d2/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java
--
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java
index 79eb7d2..a1b33f7 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java
@@ -49,6 +49,7 @@ import java.util.concurrent.LinkedBlockingQueue;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CategoryBasedTimeout;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.client.Put;
@@ -66,8 +67,10 @@ import org.apache.hadoop.hbase.util.EnvironmentEdge;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.Threads;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.rules.TestRule;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 



[33/50] [abbrv] hbase git commit: WALSplitter was spewing log about skipped edits messing up my long-running test

2017-05-23 Thread stack
WALSplitter was spewing log about skipped edits messing up my long-running test


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

Branch: refs/heads/HBASE-14614
Commit: 0f81116fc029dc1515978999706730cad24d394a
Parents: d7c4793
Author: Michael Stack 
Authored: Fri May 12 10:00:31 2017 -0700
Committer: Michael Stack 
Committed: Tue May 23 00:33:03 2017 -0700

--
 .../hbase/procedure2/ProcedureExecutor.java | 16 ++-
 .../hadoop/hbase/master/CatalogJanitor.java | 21 +---
 .../org/apache/hadoop/hbase/master/HMaster.java | 14 +
 .../apache/hadoop/hbase/wal/WALSplitter.java|  6 --
 4 files changed, 29 insertions(+), 28 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/0f81116f/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
--
diff --git 
a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
 
b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
index fa3df04..258e268 100644
--- 
a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
+++ 
b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
@@ -1526,17 +1526,13 @@ public class ProcedureExecutor {
 }
 
 // If this procedure is the last child awake the parent procedure
-if (LOG.isDebugEnabled()) {
-  LOG.debug("Finish suprocedure " + procedure);
-}
+LOG.info("Finish suprocedure " + procedure);
 if (parent.tryRunnable()) {
   // If we succeeded in making the parent runnable -- i.e. all of its
   // children have completed, move parent to front of the queue.
   store.update(parent);
   scheduler.addFront(parent);
-  if (LOG.isDebugEnabled()) {
-LOG.debug("Finished subprocedure(s) of " + parent + "; resume parent 
processing.");
-  }
+  LOG.info("Finished subprocedure(s) of " + parent + "; resume parent 
processing.");
   return;
 }
   }
@@ -1646,8 +1642,8 @@ public class ProcedureExecutor {
   if (this.activeProcedure == null) continue;
   int activeCount = activeExecutorCount.incrementAndGet();
   int runningCount = store.setRunningProcedureCount(activeCount);
-  if (LOG.isDebugEnabled()) {
-LOG.debug("Execute pid=" + this.activeProcedure.getProcId() +
+  if (LOG.isTraceEnabled()) {
+LOG.trace("Execute pid=" + this.activeProcedure.getProcId() +
 " runningCount=" + runningCount + ", activeCount=" + 
activeCount);
   }
   executionStartTime.set(EnvironmentEdgeManager.currentTime());
@@ -1659,8 +1655,8 @@ public class ProcedureExecutor {
   } finally {
 activeCount = activeExecutorCount.decrementAndGet();
 runningCount = store.setRunningProcedureCount(activeCount);
-if (LOG.isDebugEnabled()) {
-  LOG.debug("Halt pid=" + this.activeProcedure.getProcId() +
+if (LOG.isTraceEnabled()) {
+  LOG.trace("Halt pid=" + this.activeProcedure.getProcId() +
   " runningCount=" + runningCount + ", activeCount=" + 
activeCount);
 }
 this.activeProcedure = null;

http://git-wip-us.apache.org/repos/asf/hbase/blob/0f81116f/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
index edd163c..2331d8f 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
@@ -211,14 +211,12 @@ public class CatalogJanitor extends ScheduledChore {
   LOG.warn("Merged region does not exist: " + 
mergedRegion.getEncodedName());
 }
 if (regionFs == null || !regionFs.hasReferences(htd)) {
-  LOG.debug("Deleting region " + regionA.getRegionNameAsString() + " and "
-  + regionB.getRegionNameAsString()
+  LOG.debug("Deleting region " + regionA.getShortNameToLog() + " and "
+  + regionB.getShortNameToLog()
   + " from fs because merged region no longer holds references");
   ProcedureExecutor pe = 
this.services.getMasterProcedureExecutor();
-  GCMergedRegionsProcedure proc =
-  new GCMergedRegionsProcedure(pe.getEnvironment(),merge

[44/50] [abbrv] hbase git commit: LOG, dont throw exception, if already SPLIT

2017-05-23 Thread stack
LOG, dont throw exception, if already SPLIT


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

Branch: refs/heads/HBASE-14614
Commit: 7b5e5cd66a9f2dc9388c73b682cf150ca4cbed60
Parents: c613f61
Author: Michael Stack 
Authored: Tue May 16 22:14:06 2017 -0700
Committer: Michael Stack 
Committed: Tue May 23 00:33:03 2017 -0700

--
 .../hbase/master/assignment/SplitTableRegionProcedure.java   | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/7b5e5cd6/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
index 7ebe769..2b5f46b 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
@@ -345,8 +345,8 @@ public class SplitTableRegionProcedure
   // Protect against the case where concurrent SPLIT requests came in. 
Check a SPLIT
   // did not just run.
   if (parentHRI.isSplit() || parentHRI.isOffline()) {
-setFailure(new IOException("Split " + 
parentHRI.getRegionNameAsString() + " FAILED because " +
-"offline/split already."));
+LOG.info("Split of " + parentHRI.getShortNameToLog() +
+" skipped because already offline/split.");
 return false;
   }
 



hbase git commit: HBASE-18093 Overloading the meaning of 'enabled' in Quota Manager to indicate either quota disabled or quota manager not ready is not good (Stephen Yuan Jiang)

2017-05-23 Thread syuanjiang
Repository: hbase
Updated Branches:
  refs/heads/master 3f75ba195 -> 1d0295f4e


HBASE-18093 Overloading the meaning of 'enabled' in Quota Manager to indicate 
either quota disabled or quota manager not ready is not good (Stephen Yuan 
Jiang)


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

Branch: refs/heads/master
Commit: 1d0295f4e290ce9f0bcc30df9398cd81d75c4d50
Parents: 3f75ba1
Author: Stephen Yuan Jiang 
Authored: Tue May 23 06:40:33 2017 -0700
Committer: Stephen Yuan Jiang 
Committed: Tue May 23 06:40:33 2017 -0700

--
 .../org/apache/hadoop/hbase/master/HMaster.java |  2 +-
 .../hadoop/hbase/quotas/MasterQuotaManager.java | 45 ++--
 .../TestSimpleRegionNormalizerOnCluster.java|  2 +-
 .../hbase/namespace/TestNamespaceAuditor.java   | 12 +++---
 4 files changed, 39 insertions(+), 22 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/1d0295f4/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index cd96552..4dd6353 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -2247,7 +2247,7 @@ public class HMaster extends HRegionServer implements 
MasterServices {
 // coprocessor.
 MasterQuotaManager quotaManager = getMasterQuotaManager();
 if (quotaManager != null) {
-  if (quotaManager.isQuotaEnabled()) {
+  if (quotaManager.isQuotaInitialized()) {
 Quotas quotaForTable = QuotaUtil.getTableQuota(getConnection(), 
tableName);
 if (quotaForTable != null && quotaForTable.hasSpace()) {
   SpaceViolationPolicy policy = 
quotaForTable.getSpace().getViolationPolicy();

http://git-wip-us.apache.org/repos/asf/hbase/blob/1d0295f4/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
index 1fb8cf4..9d24e6c 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
@@ -49,6 +49,7 @@ import 
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle;
 import 
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.ThrottleRequest;
 import 
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -71,7 +72,7 @@ public class MasterQuotaManager implements 
RegionStateListener {
   private NamedLock namespaceLocks;
   private NamedLock tableLocks;
   private NamedLock userLocks;
-  private boolean enabled = false;
+  private boolean initialized = false;
   private NamespaceAuditor namespaceQuotaManager;
   private ConcurrentHashMap 
regionSizes;
 
@@ -101,14 +102,14 @@ public class MasterQuotaManager implements 
RegionStateListener {
 
 namespaceQuotaManager = new NamespaceAuditor(masterServices);
 namespaceQuotaManager.start();
-enabled = true;
+initialized = true;
   }
 
   public void stop() {
   }
 
-  public boolean isQuotaEnabled() {
-return enabled && namespaceQuotaManager.isInitialized();
+  public boolean isQuotaInitialized() {
+return initialized && namespaceQuotaManager.isInitialized();
   }
 
   /* ==
@@ -284,13 +285,13 @@ public class MasterQuotaManager implements 
RegionStateListener {
   }
 
   public void setNamespaceQuota(NamespaceDescriptor desc) throws IOException {
-if (enabled) {
+if (initialized) {
   this.namespaceQuotaManager.addNamespace(desc);
 }
   }
 
   public void removeNamespaceQuota(String namespace) throws IOException {
-if (enabled) {
+if (initialized) {
   this.namespaceQuotaManager.deleteNamespace(namespace);
 }
   }
@@ -325,13 +326,13 @@ public class MasterQuotaManager implements 
RegionStateListener {
   }
 
   public void checkNamespaceTableAndRegionQuota(TableName tName, int regions) 
throws IOException {
-if (ena

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

2017-05-23 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/quotas/QuotaObserverChore.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/quotas/QuotaObserverChore.html 
b/devapidocs/org/apache/hadoop/hbase/quotas/QuotaObserverChore.html
new file mode 100644
index 000..c89bd7c
--- /dev/null
+++ b/devapidocs/org/apache/hadoop/hbase/quotas/QuotaObserverChore.html
@@ -0,0 +1,1120 @@
+http://www.w3.org/TR/html4/loose.dtd";>
+
+
+
+
+
+QuotaObserverChore (Apache HBase 2.0.0-SNAPSHOT API)
+
+
+
+
+
+var methods = 
{"i0":10,"i1":10,"i2":10,"i3":9,"i4":10,"i5":10,"i6":10,"i7":9,"i8":9,"i9":10,"i10":10,"i11":10,"i12":9,"i13":10,"i14":10,"i15":10,"i16":10,"i17":10,"i18":10,"i19":10,"i20":10};
+var tabs = {65535:["t0","All Methods"],1:["t1","Static 
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
+
+
+
+
+Prev Class
+Next Class
+
+
+Frames
+No Frames
+
+
+All Classes
+
+
+
+
+
+
+
+Summary: 
+Nested | 
+Field | 
+Constr | 
+Method
+
+
+Detail: 
+Field | 
+Constr | 
+Method
+
+
+
+
+
+
+
+
+org.apache.hadoop.hbase.quotas
+Class 
QuotaObserverChore
+
+
+
+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.ScheduledChore
+
+
+org.apache.hadoop.hbase.quotas.QuotaObserverChore
+
+
+
+
+
+
+
+
+
+All Implemented Interfaces:
+http://docs.oracle.com/javase/8/docs/api/java/lang/Runnable.html?is-external=true";
 title="class or interface in java.lang">Runnable
+
+
+
+@InterfaceAudience.Private
+public class QuotaObserverChore
+extends ScheduledChore
+Reads the currently received Region filesystem-space use 
reports and acts on those which
+ violate a defined quota.
+
+
+
+
+
+
+
+
+
+
+
+Nested Class Summary
+
+Nested Classes 
+
+Modifier and Type
+Class and Description
+
+
+(package private) static class 
+QuotaObserverChore.TablesWithQuotas
+A container which encapsulates the tables that have either 
a table quota or are contained in a
+ namespace which have a namespace quota.
+
+
+
+
+
+
+
+
+
+
+Field Summary
+
+Fields 
+
+Modifier and Type
+Field and Description
+
+
+private 
org.apache.hadoop.conf.Configuration
+conf 
+
+
+private Connection
+conn 
+
+
+private static 
org.apache.commons.logging.Log
+LOG 
+
+
+private MetricsMaster
+metrics 
+
+
+private http://docs.oracle.com/javase/8/docs/api/java/util/Map.html?is-external=true";
 title="class or interface in java.util">MapString,SpaceQuotaSnapshot>
+namespaceQuotaSnapshots 
+
+
+private QuotaSnapshotStoreString>
+namespaceSnapshotStore 
+
+
+(package private) static long
+QUOTA_OBSERVER_CHORE_DELAY_DEFAULT 
+
+
+(package private) static http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String
+QUOTA_OBSERVER_CHORE_DELAY_KEY 
+
+
+(package private) static int
+QUOTA_OBSERVER_CHORE_PERIOD_DEFAULT 
+
+
+(package private) static http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String
+QUOTA_OBSERVER_CHORE_PERIOD_KEY 
+
+
+(package private) static double
+QUOTA_OBSERVER_CHORE_REPORT_PERCENT_DEFAULT 
+
+
+(package private) static http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String
+QUOTA_OBSERVER_CHORE_REPORT_PERCENT_KEY 
+
+
+(package private) static http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String
+QUOTA_OBSERVER_CHORE_TIMEUNIT_DEFAULT 
+
+
+(package private) static http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String
+QUOTA_OBSERVER_CHORE_TIMEUNIT_KEY 
+
+
+private MasterQuotaManager
+quotaManager 
+
+
+private http://docs.oracle.com/javase/8/docs/api/java/util/Map.html?is-external=true";
 title="class or interf

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

2017-05-23 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/quotas/NamespaceQuotaSnapshotStore.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/quotas/NamespaceQuotaSnapshotStore.html 
b/devapidocs/org/apache/hadoop/hbase/quotas/NamespaceQuotaSnapshotStore.html
new file mode 100644
index 000..b687106
--- /dev/null
+++ b/devapidocs/org/apache/hadoop/hbase/quotas/NamespaceQuotaSnapshotStore.html
@@ -0,0 +1,565 @@
+http://www.w3.org/TR/html4/loose.dtd";>
+
+
+
+
+
+NamespaceQuotaSnapshotStore (Apache HBase 2.0.0-SNAPSHOT API)
+
+
+
+
+
+var methods = {"i0":10,"i1":10,"i2":10,"i3":10,"i4":10,"i5":10,"i6":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
+
+
+
+
+Prev Class
+Next Class
+
+
+Frames
+No Frames
+
+
+All Classes
+
+
+
+
+
+
+
+Summary: 
+Nested | 
+Field | 
+Constr | 
+Method
+
+
+Detail: 
+Field | 
+Constr | 
+Method
+
+
+
+
+
+
+
+
+org.apache.hadoop.hbase.quotas
+Class 
NamespaceQuotaSnapshotStore
+
+
+
+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.quotas.NamespaceQuotaSnapshotStore
+
+
+
+
+
+
+
+All Implemented Interfaces:
+QuotaSnapshotStoreString>
+
+
+
+@InterfaceAudience.Private
+public class NamespaceQuotaSnapshotStore
+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 QuotaSnapshotStoreString>
+QuotaSnapshotStore 
implementation for namespaces.
+
+
+
+
+
+
+
+
+
+
+
+Nested Class Summary
+
+
+
+
+Nested classes/interfaces inherited from 
interface org.apache.hadoop.hbase.quotas.QuotaSnapshotStore
+QuotaSnapshotStore.ViolationState
+
+
+
+
+
+
+
+
+Field Summary
+
+Fields 
+
+Modifier and Type
+Field and Description
+
+
+private QuotaObserverChore
+chore 
+
+
+private Connection
+conn 
+
+
+private http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/locks/ReentrantReadWriteLock.html?is-external=true";
 title="class or interface in 
java.util.concurrent.locks">ReentrantReadWriteLock
+lock 
+
+
+private http://docs.oracle.com/javase/8/docs/api/java/util/Map.html?is-external=true";
 title="class or interface in java.util">MapLong>
+regionUsage 
+
+
+private http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/locks/ReentrantReadWriteLock.ReadLock.html?is-external=true";
 title="class or interface in 
java.util.concurrent.locks">ReentrantReadWriteLock.ReadLock
+rlock 
+
+
+private http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/locks/ReentrantReadWriteLock.WriteLock.html?is-external=true";
 title="class or interface in 
java.util.concurrent.locks">ReentrantReadWriteLock.WriteLock
+wlock 
+
+
+
+
+
+
+Fields inherited from interface org.apache.hadoop.hbase.quotas.QuotaSnapshotStore
+NO_QUOTA
+
+
+
+
+
+
+
+
+Constructor Summary
+
+Constructors 
+
+Constructor and Description
+
+
+NamespaceQuotaSnapshotStore(Connection conn,
+   QuotaObserverChore chore,
+   http://docs.oracle.com/javase/8/docs/api/java/util/Map.html?is-external=true";
 title="class or interface in java.util">MapLong> regionUsage) 
+
+
+
+
+
+
+
+
+
+Method Summary
+
+All Methods Instance Methods Concrete Methods 
+
+Modifier and Type
+Method and Description
+
+
+http://docs.oracle.com/javase/8/docs/api/java/lang/Iterable.html?is-external=true";
 title="class or interface in java.lang">IterableMap.Entry

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

2017-05-23 Thread git-site-role
Repository: hbase-site
Updated Branches:
  refs/heads/asf-site 64facdcf5 -> dab57116f


http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.html 
b/devapidocs/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.html
index f088abe..ca1d939 100644
--- a/devapidocs/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.html
+++ b/devapidocs/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.html
@@ -18,7 +18,7 @@
 catch(err) {
 }
 //-->
-var methods = 
{"i0":9,"i1":9,"i2":9,"i3":9,"i4":9,"i5":9,"i6":9,"i7":9,"i8":9,"i9":9,"i10":9,"i11":9,"i12":9,"i13":9,"i14":9,"i15":9,"i16":9,"i17":9,"i18":9};
+var methods = 
{"i0":9,"i1":9,"i2":9,"i3":9,"i4":9,"i5":9,"i6":9,"i7":9,"i8":9,"i9":9,"i10":9,"i11":9,"i12":9,"i13":9,"i14":9,"i15":9,"i16":9,"i17":9,"i18":9,"i19":9,"i20":9,"i21":9,"i22":9,"i23":9};
 var tabs = {65535:["t0","All Methods"],1:["t1","Static 
Methods"],8:["t4","Concrete Methods"]};
 var altColor = "altColor";
 var rowColor = "rowColor";
@@ -110,7 +110,7 @@ var activeTableTab = "activeTableTab";
 
 
 @InterfaceAudience.Public
-public class QuotaSettingsFactory
+public class QuotaSettingsFactory
 extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true";
 title="class or interface in java.lang">Object
 
 
@@ -186,35 +186,73 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
   
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas quotas) 
 
 
+(package private) static QuotaSettings
+fromSpace(TableName table,
+ http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String namespace,
+ 
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota protoQuota) 
+
+
 (package private) static http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">List
 fromTableQuotas(TableName tableName,

org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas quotas) 
 
-
+
 private static http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">List
 fromThrottle(http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String userName,
 TableName tableName,
 http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String namespace,
 
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle throttle) 
 
-
+
 (package private) static http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">List
 fromUserQuotas(http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String userName,
   
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas quotas) 
 
-
+
 (package private) static http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">List
 fromUserQuotas(http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String userName,
   http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String namespace,
   
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas quotas) 
 
-
+
 (package private) static http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">List
 fromUserQuotas(http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String userName,
   TableName tableName,
   
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas quotas) 
 
-
+
+static QuotaSettings
+limitNamespaceSpace(http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String namespace,
+   long sizeLimit,
+   SpaceViolationPolicy violationPolicy)
+Creates a QuotaSettings 
object to limit the FileSystem space usage for the given
+ namespace to the given size in bytes.
+
+
+
+static QuotaSettings
+limitTableSpace(TableName tableName,
+   long sizeLimit,
+   SpaceViolationPolicy violationPolicy)
+Creates a QuotaSettings 
object to limit the FileSystem space usage for the giv

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

2017-05-23 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/master/MetricsMasterQuotaSourceImpl.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/master/MetricsMasterQuotaSourceImpl.html 
b/devapidocs/org/apache/hadoop/hbase/master/MetricsMasterQuotaSourceImpl.html
new file mode 100644
index 000..f9454f3
--- /dev/null
+++ 
b/devapidocs/org/apache/hadoop/hbase/master/MetricsMasterQuotaSourceImpl.html
@@ -0,0 +1,591 @@
+http://www.w3.org/TR/html4/loose.dtd";>
+
+
+
+
+
+MetricsMasterQuotaSourceImpl (Apache HBase 2.0.0-SNAPSHOT API)
+
+
+
+
+
+var methods = {"i0":10,"i1":10,"i2":10,"i3":10,"i4":10,"i5":10,"i6":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
+
+
+
+
+Prev Class
+Next Class
+
+
+Frames
+No Frames
+
+
+All Classes
+
+
+
+
+
+
+
+Summary: 
+Nested | 
+Field | 
+Constr | 
+Method
+
+
+Detail: 
+Field | 
+Constr | 
+Method
+
+
+
+
+
+
+
+
+org.apache.hadoop.hbase.master
+Class 
MetricsMasterQuotaSourceImpl
+
+
+
+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.metrics.BaseSourceImpl
+
+
+org.apache.hadoop.hbase.master.MetricsMasterQuotaSourceImpl
+
+
+
+
+
+
+
+
+
+All Implemented Interfaces:
+MetricsMasterQuotaSource, BaseSource, 
org.apache.hadoop.metrics2.MetricsSource
+
+
+
+@InterfaceAudience.Private
+public class MetricsMasterQuotaSourceImpl
+extends BaseSourceImpl
+implements MetricsMasterQuotaSource
+Implementation of MetricsMasterQuotaSource which 
writes the values passed in via the
+ interface to the metrics backend.
+
+
+
+
+
+
+
+
+
+
+
+Field Summary
+
+Fields 
+
+Modifier and Type
+Field and Description
+
+
+private 
org.apache.hadoop.metrics2.lib.MutableGaugeLong
+namespacesViolatingQuotasGauge 
+
+
+private MetricHistogram
+quotaObserverTimeHisto 
+
+
+private 
org.apache.hadoop.metrics2.lib.MutableGaugeLong
+regionSpaceReportsGauge 
+
+
+private 
org.apache.hadoop.metrics2.lib.MutableGaugeLong
+spaceQuotasGauge 
+
+
+private 
org.apache.hadoop.metrics2.lib.MutableGaugeLong
+tablesViolatingQuotasGauge 
+
+
+private MetricsMasterWrapper
+wrapper 
+
+
+
+
+
+
+Fields inherited from class org.apache.hadoop.hbase.metrics.BaseSourceImpl
+metricsAdapter,
 metricsContext,
 metricsDescription,
 metricsJmxContext,
 metricsName,
 metricsRegistry,
 registry
+
+
+
+
+
+Fields inherited from interface org.apache.hadoop.hbase.master.MetricsMasterQuotaSource
+METRICS_CONTEXT,
 METRICS_DESCRIPTION,
 METRICS_JMX_CONTEXT,
 METRICS_NAME,
 NS_QUOTA_USAGE_DESC,
 NS_QUOTA_USAGE_NAME,
 NUM_NS_QUOTA_VIOLATIONS_DESC,
 NUM_NS_QUOTA_VIOLATIONS_NAME, NUM_REGION_SIZE_REPORTS_DESC,
 NUM_REGION_SIZE_REPORTS_NAME,
 NUM_SPACE_QUOTAS_DESC,
 NUM_SPACE_QUOTAS_NAME,
 NUM_TABLES_QUOTA_VIOLATIONS_DESC,
 NUM_TABLES_QUOTA_VIOLATIONS_NAME,
 QUOTA_OBSERVER_CHORE_TIME_DESC,
 QUOTA_OBSERVER_CHORE_TIME_NAME,
 TABLE_QUOTA_USAGE_DESC,
 TABLE_QUOTA_USAGE_NAME
+
+
+
+
+
+Fields inherited from interface org.apache.hadoop.hbase.metrics.BaseSource
+HBASE_METRICS_SYSTEM_NAME
+
+
+
+
+
+
+
+
+Constructor Summary
+
+Constructors 
+
+Constructor and Description
+
+
+MetricsMasterQuotaSourceImpl(MetricsMasterWrapper wrapper) 
+
+
+MetricsMasterQuotaSourceImpl(http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String metricsName,
+http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String metricsDescription,
+http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String metricsContext,
+http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String metricsJmxContext,
+MetricsMasterWrapper wrapper) 
+
+
+
+
+
+
+
+
+
+Method Summary
+
+All Methods Instance Methods Concrete Methods 
+
+Modifier and Type
+Method and Description
+
+
+private http://docs.oracle.com/javas

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

2017-05-23 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/quotas/QuotaRetriever.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/quotas/QuotaRetriever.html 
b/devapidocs/org/apache/hadoop/hbase/quotas/QuotaRetriever.html
index 74ba973..dd1e4ba 100644
--- a/devapidocs/org/apache/hadoop/hbase/quotas/QuotaRetriever.html
+++ b/devapidocs/org/apache/hadoop/hbase/quotas/QuotaRetriever.html
@@ -18,7 +18,7 @@
 catch(err) {
 }
 //-->
-var methods = {"i0":10,"i1":10,"i2":10,"i3":10,"i4":9,"i5":9};
+var methods = {"i0":10,"i1":10,"i2":10,"i3":10,"i4":10,"i5":9,"i6":9};
 var tabs = {65535:["t0","All Methods"],1:["t1","Static 
Methods"],2:["t2","Instance Methods"],8:["t4","Concrete Methods"]};
 var altColor = "altColor";
 var rowColor = "rowColor";
@@ -49,7 +49,7 @@ var activeTableTab = "activeTableTab";
 
 
 
-Prev Class
+Prev Class
 Next Class
 
 
@@ -114,7 +114,7 @@ var activeTableTab = "activeTableTab";
 
 
 @InterfaceAudience.Public
-public class QuotaRetriever
+public class QuotaRetriever
 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, http://docs.oracle.com/javase/8/docs/api/java/lang/Iterable.html?is-external=true";
 title="class or interface in java.lang">Iterable
 Scanner to iterate over the quota settings.
@@ -166,14 +166,20 @@ implements http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.h
 
 
 
+private boolean
+isManagedConnection
+Should QutoaRetriever manage the state of the connection, 
or leave it be.
+
+
+
 private static 
org.apache.commons.logging.Log
 LOG 
 
-
+
 private ResultScanner
 scanner 
 
-
+
 private Table
 table 
 
@@ -189,12 +195,10 @@ implements http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.h
 
 Constructors 
 
-Modifier
-Constructor and Description
+Constructor and Description
 
 
-private 
-QuotaRetriever() 
+QuotaRetriever() 
 
 
 
@@ -221,20 +225,25 @@ implements http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.h
 Scan scan) 
 
 
+(package private) void
+init(Connection conn,
+Scan scan) 
+
+
 http://docs.oracle.com/javase/8/docs/api/java/util/Iterator.html?is-external=true";
 title="class or interface in java.util">Iterator
 iterator() 
 
-
+
 QuotaSettings
 next() 
 
-
+
 static QuotaRetriever
 open(org.apache.hadoop.conf.Configuration conf)
 Open a QuotaRetriever with no filter, all the quota 
settings will be returned.
 
 
-
+
 static QuotaRetriever
 open(org.apache.hadoop.conf.Configuration conf,
 QuotaFilter filter)
@@ -276,7 +285,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.h
 
 
 LOG
-private static final org.apache.commons.logging.Log LOG
+private static final org.apache.commons.logging.Log LOG
 
 
 
@@ -285,7 +294,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.h
 
 
 cache
-private final http://docs.oracle.com/javase/8/docs/api/java/util/Queue.html?is-external=true";
 title="class or interface in java.util">Queue cache
+private final http://docs.oracle.com/javase/8/docs/api/java/util/Queue.html?is-external=true";
 title="class or interface in java.util">Queue cache
 
 
 
@@ -294,7 +303,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.h
 
 
 scanner
-private ResultScanner scanner
+private ResultScanner scanner
 
 
 
@@ -303,7 +312,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.h
 
 
 connection
-private Connection connection
+private Connection connection
 Connection to use.
  Could pass one in and have this class use it but this class wants to be 
standalone.
 
@@ -311,10 +320,20 @@ implements http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.h
 
 
 
-
+
 
 table
-private Table table
+private Table table
+
+
+
+
+
+
+
+isManagedConnection
+private boolean isManagedConnection
+Should QutoaRetriever manage the state of the connection, 
or leave it be.
 
 
 
@@ -331,7 +350,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.h
 
 
 QuotaRetriever
-private QuotaRetriever()
+QuotaRetriever()
 
 
 
@@ -348,7 +367,22 @@ implements http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.h
 
 
 init
-void init(org.apache.hadoop.conf.Configuration conf,
+void init(org.apache.hadoop.conf.Configuration conf,
+  Scan scan)
+   throws http://docs.oracle.com/javase/8/docs/api/java/io/IOException.html?is-external=true";
 title="class or interface in java.io">IOException
+
+Throws:
+http://docs.oracle.com/javase/8/docs/api/java/io/IOException.html?is-external=true";
 title="class or interface in java.io">IOException
+
+
+
+
+
+
+
+
+init
+void init(Connection conn,
   Scan sc

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

2017-05-23 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/quotas/MasterSpaceQuotaObserver.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/quotas/MasterSpaceQuotaObserver.html 
b/devapidocs/org/apache/hadoop/hbase/quotas/MasterSpaceQuotaObserver.html
new file mode 100644
index 000..2937981
--- /dev/null
+++ b/devapidocs/org/apache/hadoop/hbase/quotas/MasterSpaceQuotaObserver.html
@@ -0,0 +1,478 @@
+http://www.w3.org/TR/html4/loose.dtd";>
+
+
+
+
+
+MasterSpaceQuotaObserver (Apache HBase 2.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
+
+
+
+
+Prev Class
+Next Class
+
+
+Frames
+No Frames
+
+
+All Classes
+
+
+
+
+
+
+
+Summary: 
+Nested | 
+Field | 
+Constr | 
+Method
+
+
+Detail: 
+Field | 
+Constr | 
+Method
+
+
+
+
+
+
+
+
+org.apache.hadoop.hbase.quotas
+Class 
MasterSpaceQuotaObserver
+
+
+
+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.quotas.MasterSpaceQuotaObserver
+
+
+
+
+
+
+
+All Implemented Interfaces:
+Coprocessor, MasterObserver
+
+
+
+@InterfaceAudience.Private
+public class MasterSpaceQuotaObserver
+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 MasterObserver
+An observer to automatically delete space quotas when a 
table/namespace
+ are deleted.
+
+
+
+
+
+
+
+
+
+
+
+Nested Class Summary
+
+
+
+
+Nested classes/interfaces inherited from 
interface org.apache.hadoop.hbase.Coprocessor
+Coprocessor.State
+
+
+
+
+
+
+
+
+Field Summary
+
+Fields 
+
+Modifier and Type
+Field and Description
+
+
+private 
org.apache.hadoop.conf.Configuration
+conf 
+
+
+private CoprocessorEnvironment
+cpEnv 
+
+
+private boolean
+quotasEnabled 
+
+
+static http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String
+REMOVE_QUOTA_ON_TABLE_DELETE 
+
+
+static boolean
+REMOVE_QUOTA_ON_TABLE_DELETE_DEFAULT 
+
+
+
+
+
+
+Fields inherited from interface org.apache.hadoop.hbase.Coprocessor
+PRIORITY_HIGHEST,
 PRIORITY_LOWEST,
 PRIORITY_SYSTEM,
 PRIORITY_USER,
 VERSION
+
+
+
+
+
+
+
+
+Constructor Summary
+
+Constructors 
+
+Constructor and Description
+
+
+MasterSpaceQuotaObserver() 
+
+
+
+
+
+
+
+
+
+Method Summary
+
+All Methods Instance Methods Concrete Methods 
+
+Modifier and Type
+Method and Description
+
+
+void
+postDeleteNamespace(ObserverContext ctx,
+   http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String namespace)
+Called after the deleteNamespace operation has been 
requested.
+
+
+
+void
+postDeleteTable(ObserverContext ctx,
+   TableName tableName)
+Called after the deleteTable operation has been 
requested.
+
+
+
+void
+start(CoprocessorEnvironment ctx) 
+
+
+
+
+
+
+Methods inherited from class 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
+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/jav

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

2017-05-23 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/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 e634b62..d42fd08 100644
--- a/devapidocs/org/apache/hadoop/hbase/master/package-tree.html
+++ b/devapidocs/org/apache/hadoop/hbase/master/package-tree.html
@@ -95,6 +95,7 @@
 org.apache.hadoop.hbase.master.MetricsAssignmentManagerSourceImpl (implements 
org.apache.hadoop.hbase.master.MetricsAssignmentManagerSource)
 org.apache.hadoop.hbase.master.MetricsMasterFilesystemSourceImpl (implements 
org.apache.hadoop.hbase.master.MetricsMasterFileSystemSource)
 org.apache.hadoop.hbase.master.MetricsMasterProcSourceImpl (implements 
org.apache.hadoop.hbase.master.MetricsMasterProcSource)
+org.apache.hadoop.hbase.master.MetricsMasterQuotaSourceImpl (implements 
org.apache.hadoop.hbase.master.MetricsMasterQuotaSource)
 org.apache.hadoop.hbase.master.MetricsMasterSourceImpl (implements 
org.apache.hadoop.hbase.master.MetricsMasterSource)
 org.apache.hadoop.hbase.master.MetricsSnapshotSourceImpl (implements 
org.apache.hadoop.hbase.master.MetricsSnapshotSource)
 
@@ -181,6 +182,7 @@
 org.apache.hadoop.hbase.master.MetricsMaster
 org.apache.hadoop.hbase.master.MetricsMasterFileSystem
 org.apache.hadoop.hbase.master.MetricsMasterProcSourceFactoryImpl (implements 
org.apache.hadoop.hbase.master.MetricsMasterProcSourceFactory)
+org.apache.hadoop.hbase.master.MetricsMasterQuotaSourceFactoryImpl (implements 
org.apache.hadoop.hbase.master.MetricsMasterQuotaSourceFactory)
 org.apache.hadoop.hbase.master.MetricsMasterSourceFactoryImpl (implements 
org.apache.hadoop.hbase.master.MetricsMasterSourceFactory)
 org.apache.hadoop.hbase.master.MetricsMasterWrapperImpl (implements 
org.apache.hadoop.hbase.master.MetricsMasterWrapper)
 org.apache.hadoop.hbase.master.MetricsSnapshot
@@ -276,6 +278,7 @@
 org.apache.hadoop.hbase.master.MetricsAssignmentManagerSource
 org.apache.hadoop.hbase.master.MetricsMasterFileSystemSource
 org.apache.hadoop.hbase.master.MetricsMasterProcSource
+org.apache.hadoop.hbase.master.MetricsMasterQuotaSource
 org.apache.hadoop.hbase.master.MetricsMasterSource
 org.apache.hadoop.hbase.master.MetricsSnapshotSource
 
@@ -296,6 +299,7 @@
 
 
 org.apache.hadoop.hbase.master.MetricsMasterProcSourceFactory
+org.apache.hadoop.hbase.master.MetricsMasterQuotaSourceFactory
 org.apache.hadoop.hbase.master.MetricsMasterSourceFactory
 org.apache.hadoop.hbase.master.MetricsMasterWrapper
 org.apache.hadoop.hbase.master.ServerListener
@@ -325,8 +329,8 @@
 org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus
 org.apache.hadoop.hbase.master.MetricsMasterSourceFactoryImpl.FactoryStorage
 org.apache.hadoop.hbase.master.MasterRpcServices.BalanceSwitchMode
-org.apache.hadoop.hbase.master.RegionState.State
 org.apache.hadoop.hbase.master.SplitLogManager.ResubmitDirective
+org.apache.hadoop.hbase.master.RegionState.State
 
 
 

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/master/package-use.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/master/package-use.html 
b/devapidocs/org/apache/hadoop/hbase/master/package-use.html
index b15d13f..5ccb6f0 100644
--- a/devapidocs/org/apache/hadoop/hbase/master/package-use.html
+++ b/devapidocs/org/apache/hadoop/hbase/master/package-use.html
@@ -492,6 +492,16 @@
 
 
 
+MetricsMasterQuotaSource
+A collection of exposed metrics for space quotas from the 
HBase Master.
+
+
+
+MetricsMasterQuotaSourceFactory
+Interface of a factory to create MetricsMasterQuotaSource 
when given a MetricsMasterWrapper.
+
+
+
 MetricsMasterSource
 Interface that classes that expose metrics about the master 
will implement.
 
@@ -862,10 +872,21 @@
 
 
 
+HMaster
+HMaster is the "master server" for HBase.
+
+
+
 MasterServices
 Services Master supplies
 
 
+
+MetricsMaster
+This class is for maintaining the various master statistics
+ and publishing them through the metrics interfaces.
+
+
 
 
 

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/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 a3aadda..d3cc432 100644
--- a/devapidocs/org/apache/hadoop/hbase/master/procedure/package-tree.html
+++ b/devapidocs/org/apache/hadoop/hbase/master/procedure/package-tree.html
@@ -174,9 +174,9 @@
 
 java.lang.http://docs.oracle.com/javase/8/docs/api/java/lang/Enum.html?is-external=true";
 title="class or interface in java.lang">Enum (implements java.lang.http://docs.oracle.com/javase/8/docs/api

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

2017-05-23 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/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 96a43aa..37921ef 100644
--- 
a/devapidocs/org/apache/hadoop/hbase/master/HMasterCommandLine.LocalHMaster.html
+++ 
b/devapidocs/org/apache/hadoop/hbase/master/HMasterCommandLine.LocalHMaster.html
@@ -198,7 +198,7 @@ extends 
 
 Fields inherited from class org.apache.hadoop.hbase.regionserver.HRegionServer
-cacheConfig,
 cacheFlusher,
 CLOSE,
 clusterConnection,
 clusterStatusTracker,
 compactSplitThread,
 conf,
 configurationManager,
 csm, DEFAULT_REGION_LOCK_AWAIT_TIME_SEC,
 fs,
 fsOk,
 hMemManager,
 infoServer,
 initLatch,
 leases,
 lock,
 MASTER_HOST
 NAME_KEY, metaTableLocator,
 movedRegions,
 msgInterval,
 numRegionsToReport,
 onlineRegions,
 OPEN,
 recoveringRegions,
 REGION_LOCK_AWAIT_TIME_SEC,
 regionFavoredNodesMap,
 REGIONSERVER,
 regionsInTransitionInRS,
 replicationSinkHandler,
 replicationSourceHandler,
 rpcServices,
 secureBulkLoadManager,
 serverName,
 service, sleeper,
 startcode,
 tableDescriptors,
 TEST_SKIP_REPORTING_TRANSITION,
 threadWakeFrequency,
 useThisHostnameInstead,
 walFactory,
 walFs,
 walRoller,
 zooKeeper
+cacheConfig,
 cacheFlusher,
 CLOSE,
 clusterConnection,
 clusterStatusTracker,
 compactSplitThread,
 conf,
 configurationManager,
 csm, DEFAULT_REGION_LOCK_AWAIT_TIME_SEC,
 fs,
 fsOk,
 fsUtilizationChore,
 hMemManager,
 infoServer,
 initLatch,
 leases,
 lock, MASTER_HOSTNAME_KEY,
 metaTableLocator,
 movedRegions,
 msgInterval,
 numRegionsToReport,
 onlineRegions,
 OPEN,
 recoveringRegions,
 REGION_LOCK_AWAIT_TIME_SEC, 
regionFavoredNodesMap,
 REGIONSERVER,
 regionsInTransitionInRS,
 replicationSinkHandler,
 replicationSourceHandler,
 rpcServices,
 secureBulkLoadManager,
 serverName, service,
 sleeper,
 startcode,
 tableDescriptors,
 TEST_SKIP_REPORTING_TRANSITION,
 threadWakeFrequency,
 useThisHostnameInstead,
 walFactory, walFs,
 walRoller,
 zooKeeper
 
 
 
@@ -248,14 +248,14 @@ extends 
 
 Methods inherited from class org.apache.hadoop.hbase.master.HMaster
-abort,
 abortProcedure,
 addColumn,
 addReplicationPeer,
 balance,
 balance,
 balanceSwitch,
 canCreateBaseZNode, canUpdateTableDescriptor,
 checkInitialized,
 checkServiceStarted,
 checkTableModifiable,
 configureInfoServer,
 constructMaster,
 createMetaBootstrap, createNamespace,
 createRpcServices,
 createServerManager,
 createSystemTable,
 createTable,
 deleteColumn,
 deleteNamespace,
 deleteTable,
 disableReplicationPeer,
 disableTable,
 drainRegionServer,
 enableReplicationPeer,
 enableTable,
 getAssignmentManager, getAverageLoad,
 getCatalogJanitor,
 getClientIdAuditPrefix,
 getClusterSchema,
 getClusterStatus,
 getDumpServlet,
 getFavoredNodesManager,
 getFsTableDescriptors,
 getHFileCleaner,
 getInitializedEvent,
 getLastMajorCompactionTimestamp,
 getLastMajorCompactionTimestampForRegion,
 getLoadBalancer,
 getLoadBalancerClassName,
 getLoadedCoprocessors,
 getLockMan
 ager, getLogCleaner,
 getMasterActiveTime,
 getMasterCoprocessorHost,
 getMasterCoprocessors,
 getMasterFileSystem,
 getMasterMetrics,
 getMasterProcedureExecutor,
 getMasterProcedureManagerHost,
 getMasterQuotaManager,
 getMasterRpcServices,
 getMasterStartTime,
 getMasterWalManager,
 getMergePlanCount,
 getMobCompactionState,
 getNamespace,
 getNamespaces,
 getNumWALFiles, getProcessName,
 getRegionNormalizer,
 getRegionNormalizerTracker,
 getRegionServerFatalLogBuffer,
 getRegionServerInfoPort,
 getRegionServerVersion,
 getRemoteInetAddress,
 getReplicationPeerConfig,
 getServerCrashProcessingEnabledEvent,
 getServerManager,
 getServerName,
 getSnapshotManager,
 getSplitOrMergeTracker,
 getSplitPlanCount,
 getTableDescriptors,
 getTableRegionForRow,
 getTableStateManager,
 getWalProcedureStore,
 getZooKeeper,
 initClusterSchemaService,
 initializeZKBasedSystemTrackers,
 initQuotaManager,
 isActiveMaster,
 isBalancerOn, isCatalogJanitorEnabled,
 isCleanerChoreEnabled,
 isInitializationStartsMetaRegionAssignment,
 isInitialized,
 isInMaintenanceMode,
 isNormalizerOn,
 isServerCrashProcessingEnabled,
 isSplitOrMergeEnabled, listDrainingRegionServers,
 listLocks,
 listProcedures,
 listReplicationPeers,
 listTableDescriptors,
 listTableDescriptorsByNamespace,
 listTableNames
 , listTableNamesByNamespace,
 login,
 main,
 mergeRegions,
 modifyColumn,
 modifyNamespace,
 modifyTable,
 move,
 normalizeRegions,
 registerService,
 removeDrainFromRegionServer,
 removeReplicationPeer,
 reportMobCompactionEnd,
 reportMobCompactionStart, requestMobCompaction,
 restoreSnapshot,
 sendShutdownInterrupt,
 setCatalogJanitorEnabled,
 setInitialized,
 setServerCrashProcessingEnabled,
 shutdown, splitRegion,
 stopMaster,
 stopServiceThreads,
 truncateTable,
 updateReplicationP

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

2017-05-23 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/ipc/ServerRpcConnection.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/ipc/ServerRpcConnection.html 
b/devapidocs/org/apache/hadoop/hbase/ipc/ServerRpcConnection.html
index 12f89ea..51dce01 100644
--- a/devapidocs/org/apache/hadoop/hbase/ipc/ServerRpcConnection.html
+++ b/devapidocs/org/apache/hadoop/hbase/ipc/ServerRpcConnection.html
@@ -18,7 +18,7 @@
 catch(err) {
 }
 //-->
-var methods = 
{"i0":10,"i1":6,"i2":10,"i3":10,"i4":10,"i5":10,"i6":10,"i7":10,"i8":10,"i9":10,"i10":10,"i11":10,"i12":10,"i13":6,"i14":10,"i15":10,"i16":10,"i17":10,"i18":10,"i19":10,"i20":10,"i21":10,"i22":10};
+var methods = 
{"i0":10,"i1":6,"i2":10,"i3":10,"i4":10,"i5":10,"i6":10,"i7":6,"i8":10,"i9":10,"i10":10,"i11":10,"i12":10,"i13":10,"i14":10,"i15":10,"i16":6,"i17":10,"i18":10,"i19":10,"i20":10,"i21":10,"i22":10,"i23":10,"i24":10,"i25":10,"i26":10};
 var tabs = {65535:["t0","All Methods"],2:["t2","Instance 
Methods"],4:["t3","Abstract Methods"],8:["t4","Concrete Methods"]};
 var altColor = "altColor";
 var rowColor = "rowColor";
@@ -117,7 +117,7 @@ var activeTableTab = "activeTableTab";
 
 
 
-abstract class ServerRpcConnection
+abstract class ServerRpcConnection
 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
 Reads calls from a connection and queues them for 
handling.
@@ -171,41 +171,25 @@ implements http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.h
 authenticatedWithFallback 
 
 
-protected ServerCall
-authFailedCall 
-
-
-protected http://docs.oracle.com/javase/8/docs/api/java/io/ByteArrayOutputStream.html?is-external=true";
 title="class or interface in java.io">ByteArrayOutputStream
-authFailedResponse 
-
-
 protected AuthMethod
 authMethod 
 
 
-protected static int
-AUTHORIZATION_FAILED_CALLID 
-
-
 protected RpcServer.CallCleanup
 callCleanup 
 
-
+
 protected Codec
 codec
 Codec the client asked use.
 
 
-
+
 protected 
org.apache.hadoop.io.compress.CompressionCodec
 compressionCodec
 Compression codec the client asked us use.
 
 
-
-protected static int
-CONNECTION_HEADER_RESPONSE_CALLID 
-
 
 protected 
org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeader
 connectionHeader 
@@ -215,53 +199,37 @@ implements http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.h
 connectionHeaderRead 
 
 
-protected boolean
-connectionPreambleRead 
-
-
 protected CryptoAES
 cryptoAES 
 
-
+
 protected http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String
 hostAddress 
 
-
+
 protected int
 remotePort 
 
-
+
 protected boolean
 retryImmediatelySupported 
 
-
+
 protected RpcServer
 rpcServer 
 
-
-protected static int
-SASL_CALLID 
-
 
-protected ServerCall
-saslCall 
-
-
 protected boolean
 saslContextEstablished 
 
-
+
 protected http://docs.oracle.com/javase/8/docs/api/javax/security/sasl/SaslServer.html?is-external=true";
 title="class or interface in javax.security.sasl">SaslServer
 saslServer 
 
-
+
 protected 
org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingService
 service 
 
-
-protected ServerCall
-setConnectionHeaderResponseCall 
-
 
 protected boolean
 skipInitialSaslHandshake 
@@ -327,7 +295,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.h
 Method and Description
 
 
-protected boolean
+private boolean
 authorizeConnection() 
 
 
@@ -354,11 +322,14 @@ implements http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.h
 
 
 private void
-doConnectionHeaderResponse(byte[] wrappedCipherMetaData)
-Send the response for connection header
-
+doBadPreambleHandling(http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String msg) 
 
 
+private void
+doBadPreambleHandling(http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String msg,
+ http://docs.oracle.com/javase/8/docs/api/java/lang/Exception.html?is-external=true";
 title="class or interface in java.lang">Exception e) 
+
+
 protected void
 doRawSaslReply(SaslStatus status,
   org.apache.hadoop.io.Writable rv,
@@ -367,77 +338,92 @@ implements http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.h
 No protobuf encoding of raw sasl messages
 
 
-
+
+protected abstract void
+doRespond(RpcResponse resp) 
+
+
 protected 
org.apache.hadoop.security.UserGroupInformation
 getAuthorizedUgi(http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in 
java.lang">String authorizedId) 

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

2017-05-23 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/apache_hbase_reference_guide.pdfmarks
--
diff --git a/apache_hbase_reference_guide.pdfmarks 
b/apache_hbase_reference_guide.pdfmarks
index 855f2df..ec2a8e3 100644
--- a/apache_hbase_reference_guide.pdfmarks
+++ b/apache_hbase_reference_guide.pdfmarks
@@ -2,8 +2,8 @@
   /Author (Apache HBase Team)
   /Subject ()
   /Keywords ()
-  /ModDate (D:20170522173653)
-  /CreationDate (D:20170522173653)
+  /ModDate (D:20170523145027)
+  /CreationDate (D:20170523145027)
   /Creator (Asciidoctor PDF 1.5.0.alpha.6, based on Prawn 1.2.1)
   /Producer ()
   /DOCINFO pdfmark

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/apidocs/allclasses-frame.html
--
diff --git a/apidocs/allclasses-frame.html b/apidocs/allclasses-frame.html
index 7e2b65d..96e0d2a 100644
--- a/apidocs/allclasses-frame.html
+++ b/apidocs/allclasses-frame.html
@@ -323,6 +323,8 @@
 SnapshotExistsException
 SnapshotInfo
 SnapshotType
+SpaceLimitingException
+SpaceViolationPolicy
 Stoppable
 StoppedRpcClientException
 Struct

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/apidocs/allclasses-noframe.html
--
diff --git a/apidocs/allclasses-noframe.html b/apidocs/allclasses-noframe.html
index c1d8a61..b8a9673 100644
--- a/apidocs/allclasses-noframe.html
+++ b/apidocs/allclasses-noframe.html
@@ -323,6 +323,8 @@
 SnapshotExistsException
 SnapshotInfo
 SnapshotType
+SpaceLimitingException
+SpaceViolationPolicy
 Stoppable
 StoppedRpcClientException
 Struct

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/apidocs/index-all.html
--
diff --git a/apidocs/index-all.html b/apidocs/index-all.html
index 170daad..3ccce50 100644
--- a/apidocs/index-all.html
+++ b/apidocs/index-all.html
@@ -5817,6 +5817,8 @@
  
 getMemStoreSizeMB()
 - Method in class org.apache.hadoop.hbase.RegionLoad
  
+getMessage()
 - Method in exception org.apache.hadoop.hbase.quotas.SpaceLimitingException
+ 
 getMin()
 - Method in class org.apache.hadoop.hbase.filter.TimestampsFilter
 
 Gets the minimum timestamp requested by filter.
@@ -6198,6 +6200,8 @@
 
 Return a QuotaRetriever to list the quotas based on the 
filter.
 
+getQuotaStates(RpcController,
 QuotaProtos.GetQuotaStatesRequest) - Method in class 
org.apache.hadoop.hbase.client.ShortCircuitMasterConnection
+ 
 getQuotaType()
 - Method in class org.apache.hadoop.hbase.quotas.QuotaSettings
  
 getRandomKey()
 - Method in class org.apache.hadoop.hbase.io.crypto.Cipher
@@ -6640,6 +6644,8 @@
  
 getSourceName()
 - Method in exception org.apache.hadoop.hbase.errorhandling.TimeoutException
  
+getSpaceQuotaRegionSizes(RpcController,
 QuotaProtos.GetSpaceQuotaRegionSizesRequest) - Method in class 
org.apache.hadoop.hbase.client.ShortCircuitMasterConnection
+ 
 getSplitKey(byte[],
 byte[], boolean) - Static method in class 
org.apache.hadoop.hbase.mapreduce.TableInputFormatBase
 
 select a split point in the region.
@@ -7074,6 +7080,10 @@
 
 getVersionedBytes()
 - Method in class org.apache.hadoop.hbase.ServerName
  
+getViolationPolicy()
 - Method in exception org.apache.hadoop.hbase.quotas.SpaceLimitingException
+
+Returns the violation policy in effect.
+
 getVisibilityExpressionResolver()
 - Method in class org.apache.hadoop.hbase.mapreduce.CellCreator
  
 getVLong(int)
 - Method in interface org.apache.hadoop.hbase.util.ByteRange
@@ -8541,7 +8551,7 @@
  
 isLegalNamespaceName(byte[],
 int, int) - Static method in class org.apache.hadoop.hbase.TableName
 
-Valid namespace characters are [a-zA-Z_0-9]
+Valid namespace characters are alphabetic characters, 
numbers, and underscores.
 
 isLegalTableQualifierName(byte[])
 - Static method in class org.apache.hadoop.hbase.TableName
  
@@ -9093,6 +9103,11 @@
 The directory from which co-processor/custom filter jars 
can be loaded
  dynamically by the region servers.
 
+limitNamespaceSpace(String,
 long, SpaceViolationPolicy) - Static method in class 
org.apache.hadoop.hbase.quotas.QuotaSettingsFactory
+
+Creates a QuotaSettings object to limit 
the FileSystem space usage for the given
+ namespace to the given size in bytes.
+
 limitNumMapTasks(String,
 JobConf) - Static method in class org.apache.hadoop.hbase.mapred.TableMapReduceUtil
 
 Ensures that the given number of map tasks for the given job
@@ -9108,6 +9123,11 @@
 Ensures that the given number of reduce tasks for the given 
job
  configuration does not exceed the number of regions for the given table.
 
+limitTableSpace(TableName,
 long, SpaceViolationPolicy) - Static method in class 
org.apache.hadoop.hbase.quotas.QuotaSettingsFactory
+
+Creates a QuotaSettings object to limit 
the FileSystem space usage for the given table
+ to the given size in bytes.
+
 listCells()
 

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

2017-05-23 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/class-use/TableName.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/class-use/TableName.html 
b/devapidocs/org/apache/hadoop/hbase/class-use/TableName.html
index 14e189b..246e1a9 100644
--- a/devapidocs/org/apache/hadoop/hbase/class-use/TableName.html
+++ b/devapidocs/org/apache/hadoop/hbase/class-use/TableName.html
@@ -225,69 +225,73 @@ Input/OutputFormats, a table indexing MapReduce job, and 
utility methods.
  
 
 
-org.apache.hadoop.hbase.regionserver
+org.apache.hadoop.hbase.quotas.policies
  
 
 
-org.apache.hadoop.hbase.regionserver.wal
+org.apache.hadoop.hbase.regionserver
  
 
 
+org.apache.hadoop.hbase.regionserver.wal
+ 
+
+
 org.apache.hadoop.hbase.replication
 
 Multi Cluster Replication
 
 
-
+
 org.apache.hadoop.hbase.replication.regionserver
  
 
-
+
 org.apache.hadoop.hbase.rest
 
 HBase REST
 
 
-
+
 org.apache.hadoop.hbase.rest.client
  
 
-
+
 org.apache.hadoop.hbase.rsgroup
  
 
-
+
 org.apache.hadoop.hbase.security.access
  
 
-
+
 org.apache.hadoop.hbase.security.visibility
  
 
-
+
 org.apache.hadoop.hbase.snapshot
  
 
-
+
 org.apache.hadoop.hbase.thrift
 
 Provides an HBase http://incubator.apache.org/thrift/";>Thrift
 service.
 
 
-
+
 org.apache.hadoop.hbase.tool
  
 
-
+
 org.apache.hadoop.hbase.util
  
 
-
+
 org.apache.hadoop.hbase.wal
  
 
-
+
 org.apache.hbase.archetypes.exemplars.client
 
 This package provides fully-functional exemplar Java code 
demonstrating
@@ -295,7 +299,7 @@ service.
  archetype with hbase-client dependency.
 
 
-
+
 org.apache.hbase.archetypes.exemplars.shaded_client
 
 This package provides fully-functional exemplar Java code 
demonstrating
@@ -7673,6 +7677,30 @@ service.
 
 
 
+private http://docs.oracle.com/javase/8/docs/api/java/util/Map.html?is-external=true";
 title="class or interface in java.util">Map
+ActivePolicyEnforcement.activePolicies 
+
+
+private http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/atomic/AtomicReference.html?is-external=true";
 title="class or interface in 
java.util.concurrent.atomic">AtomicReferenceMap>
+RegionServerSpaceQuotaManager.currentQuotaSnapshots 
+
+
+private http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/ConcurrentHashMap.html?is-external=true";
 title="class or interface in java.util.concurrent">ConcurrentHashMap
+RegionServerSpaceQuotaManager.enforcedPolicies 
+
+
+private http://docs.oracle.com/javase/8/docs/api/java/util/Map.html?is-external=true";
 title="class or interface in java.util">Map
+ActivePolicyEnforcement.locallyCachedPolicies 
+
+
+private http://docs.oracle.com/javase/8/docs/api/java/util/Map.html?is-external=true";
 title="class or interface in java.util">Map
+QuotaObserverChore.readOnlyTableQuotaSnapshots 
+
+
+private http://docs.oracle.com/javase/8/docs/api/java/util/Map.html?is-external=true";
 title="class or interface in java.util">Map
+ActivePolicyEnforcement.snapshots 
+
+
 private http://docs.oracle.com/javase/8/docs/api/java/util/Map.html?is-external=true";
 title="class or interface in java.util">Map
 UserQuotaState.tableLimiters 
 
@@ -7684,6 +7712,22 @@ service.
 private http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/ConcurrentHashMap.html?is-external=true";
 title="class or interface in java.util.concurrent">ConcurrentHashMap
 QuotaCache.tableQuotaCache 
 
+
+private http://docs.oracle.com/javase/8/docs/api/java/util/Map.html?is-external=true";
 title="class or interface in java.util">Map
+QuotaObserverChore.tableQuotaSnapshots 
+
+
+private QuotaSnapshotStore
+QuotaObserverChore.tableSnapshotStore 
+
+
+private http://docs.oracle.com/javase/8/docs/api/java/util/Set.html?is-external=true";
 title="class or interface in java.util">Set
+QuotaObserverChore.TablesWithQuotas.tablesWithNamespaceQuotas 
+
+
+private http://docs.oracle.com/javase/8/docs/api/java/util/Set.html?is-external=true";
 title="class or interface in java.util">Set
+QuotaObserverChore.TablesWithQuotas.tablesWithTableQuotas 
+
 
 
 
@@ -7711,14 +7755,110 @@ service.
 
 
 
+(package private) http://docs.oracle.com/javase/8/docs/api/java/util/Map.html?is-external=true";
 title="class or interface in java.util">Map
+RegionServerSpaceQuotaManager.copyActiveEnforcements()
+Returns the collection of tables which have quota violation 
policies enforced on
+ this RegionServer.
+
+
+
+http://docs.oracle.com/javase/8/docs/a

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

2017-05-23 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/client/package-summary.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/client/package-summary.html 
b/devapidocs/org/apache/hadoop/hbase/client/package-summary.html
index e40fc54..313b6ba 100644
--- a/devapidocs/org/apache/hadoop/hbase/client/package-summary.html
+++ b/devapidocs/org/apache/hadoop/hbase/client/package-summary.html
@@ -1020,18 +1020,24 @@
  
 
 
+QuotaStatusCalls
+
+Client class to wrap RPCs to HBase servers for space quota 
status information.
+
+
+
 RawAsyncTableImpl
 
 The implementation of RawAsyncTable.
 
 
-
+
 RegionAdminServiceCallable
 
 Similar to RegionServerCallable but for the AdminService 
interface.
 
 
-
+
 RegionCoprocessorRpcChannel
 
 Provides clients with an RPC connection to call Coprocessor 
Endpoint
@@ -1039,87 +1045,87 @@
  against a given table region.
 
 
-
+
 RegionCoprocessorRpcChannelImpl
 
 The implementation of a region based coprocessor rpc 
channel.
 
 
-
+
 RegionCoprocessorServiceExec
 
 Represents a coprocessor service method execution against a 
single region.
 
 
-
+
 RegionLoadStats
 
 POJO representing region server load
 
 
-
+
 RegionReplicaUtil
 
 Utility methods which contain the logic for regions and 
replicas.
 
 
-
+
 RegionServerCallable
 
 Implementations make a RPC call against a RegionService via 
a protobuf Service.
 
 
-
+
 RegistryFactory
 
 Get instance of configured Registry.
 
 
-
+
 RequestControllerFactory
 
 A factory class that constructs an RequestController.
 
 
-
+
 Result
 
 Single row result of a Get or Scan query.
 
 
-
+
 ResultBoundedCompletionService
 
 A completion service for the RpcRetryingCallerFactory.
 
 
-
+
 ResultStatsUtil
 
 A Result with some statistics 
about the server/region status
 
 
-
+
 RetriesExhaustedException.ThrowableWithExtraContext
 
 Datastructure that allows adding more info around Throwable 
incident.
 
 
-
+
 RetryingCallerInterceptor
 
 This class is designed to fit into the RetryingCaller class 
which forms the
  central piece of intelligence for the client side retries for most 
calls.
 
 
-
+
 RetryingCallerInterceptorContext
 
 The context object used in the RpcRetryingCaller to enable
  RetryingCallerInterceptor to 
intercept calls.
 
 
-
+
 RetryingCallerInterceptorFactory
 
 Factory implementation to provide the ConnectionImplementation with
@@ -1127,190 +1133,190 @@
  to intercept the RpcRetryingCaller during the 
course of their calls.
 
 
-
+
 RetryingTimeTracker
 
 Tracks the amount of time remaining for an operation.
 
 
-
+
 ReversedClientScanner
 
 A reversed client scanner which support backward 
scanning
 
 
-
+
 ReversedScannerCallable
 
 A reversed ScannerCallable which supports backward 
scanning.
 
 
-
+
 RowMutations
 
 Performs multiple mutations atomically on a single 
row.
 
 
-
+
 RpcRetryingCallable
 
 A RetryingCallable for RPC connection operations.
 
 
-
+
 RpcRetryingCallerFactory
 
 Factory to create an RpcRetryingCaller
 
 
-
+
 RpcRetryingCallerImpl
 
 Runs an rpc'ing RetryingCallable.
 
 
-
+
 RpcRetryingCallerWithReadReplicas
 
 Caller that goes to replica if the primary region does no 
answer within a configurable
  timeout.
 
 
-
+
 Scan
 
 Used to perform Scan operations.
 
 
-
+
 ScannerCallable
 
 Scanner operations such as create, next, etc.
 
 
-
+
 ScannerCallableWithReplicas
 
 This class has the logic for handling scanners for regions 
with and without replicas.
 
 
-
+
 SecureBulkLoadClient
 
 Client proxy for SecureBulkLoadProtocol
 
 
-
+
 ServerStatisticTracker
 
 Tracks the statistics for multiple regions
 
 
-
+
 ShortCircuitMasterConnection
 
 A short-circuit connection that can bypass the RPC layer 
(serialization, deserialization,
  networking, etc..) when talking to a local master
 
 
-
+
 SimpleRequestController
 
 Holds back the requests if they reach any thresholds.
 
 
-
+
 SimpleRequestController.RequestHeapSizeChecker
 
 limit the heap size for each request.
 
 
-
+
 SimpleRequestController.RequestRowsChecker
 
 limit the number of rows for each request.
 
 
-
+
 SimpleRequestController.SubmittedSizeChecker
 
 limit the heapsize of total submitted data.
 
 
-
+
 SimpleRequestController.TaskCountChecker
 
 limit the max number of tasks in an AsyncProcess.
 
 
-
+
 SingleResponse
 
 Class for single action response
 
 
-
+
 SingleResponse.Entry
  
 
-
+
 SnapshotDescription
 
 The POJO equivalent of HBaseProtos.SnapshotDescription
 
 
-
+
 SyncCoprocessorRpcChannel
 
 Base class which provides clients with an RPC connection to
  call coprocessor endpoint Services.
 
 
-
+
 TableBuilderBase
 
 Base class for all table builders.
 
 
-
+
 TableDescriptorBuilder
  
 
-
+
 TableDescriptorBuilder.ModifyableTableDescriptor
 
 TODO: make this private after removing the 
HTableDescriptor
 
 
-
+
 TableSnapshotScanner
 
 A Scanner which performs a scan over

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

2017-05-23 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.html 
b/devapidocs/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.html
index d4542ff..025ba8d 100644
--- 
a/devapidocs/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.html
+++ 
b/devapidocs/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.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,"i16":10,"i17":10,"i18":10,"i19":10,"i20":10,"i21":10,"i22":10,"i23":10,"i24":10,"i25":10,"i26":10,"i27":10,"i28":10,"i29":10,"i30":10,"i31":10,"i32":10,"i33":10,"i34":10,"i35":10,"i36":10,"i37":10,"i38":10,"i39":10,"i40":10,"i41":10,"i42":10,"i43":10,"i44":10,"i45":10,"i46":10,"i47":10,"i48":10,"i49":10,"i50":10,"i51":10,"i52":10,"i53":10,"i54":10,"i55":10,"i56":10,"i57":10,"i58":10,"i59":10,"i60":10,"i61":10,"i62":10,"i63":10,"i64":10,"i65":10,"i66":10,"i67":10,"i68":10,"i69":10,"i70":10,"i71":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,"i15":10,"i16":10,"i17":10,"i18":10,"i19":10,"i20":10,"i21":10,"i22":10,"i23":10,"i24":10,"i25":10,"i26":10,"i27":10,"i28":10,"i29":10,"i30":10,"i31":10,"i32":10,"i33":10,"i34":10,"i35":10,"i36":10,"i37":10,"i38":10,"i39":10,"i40":10,"i41":10,"i42":10,"i43":10,"i44":10,"i45":10,"i46":10,"i47":10,"i48":10,"i49":10,"i50":10,"i51":10,"i52":10,"i53":10,"i54":10,"i55":10,"i56":10,"i57":10,"i58":10,"i59":10,"i60":10,"i61":10,"i62":10,"i63":10,"i64":10,"i65":10,"i66":10,"i67":10,"i68":10,"i69":10,"i70":10,"i71":10,"i72":10,"i73":10};
 var tabs = {65535:["t0","All Methods"],2:["t2","Instance 
Methods"],8:["t4","Concrete Methods"]};
 var altColor = "altColor";
 var rowColor = "rowColor";
@@ -114,7 +114,7 @@ var activeTableTab = "activeTableTab";
 
 
 @InterfaceAudience.Public
-public class ShortCircuitMasterConnection
+public class ShortCircuitMasterConnection
 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 MasterKeepAliveConnection
 A short-circuit connection that can bypass the RPC layer 
(serialization, deserialization,
@@ -308,226 +308,236 @@ implements 
+org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse
+getQuotaStates(org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+  
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest request) 
+
+
 org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigResponse
 getReplicationPeerConfig(org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
 
org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigRequest request) 
 
-
+
 org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse
 getSchemaAlterStatus(org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
 
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest request) 
 
-
+
 org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse
 getSecurityCapabilities(org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,

org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest request) 
 
-
+
+org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse
+getSpaceQuotaRegionSizes(org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest request) 
+
+
 org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse
 getTableDescriptors(org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,

org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest request) 
 
-
+
 org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesResponse
 getTableNames(org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
  
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest request) 
 
-
+
 org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse
 getTableState(org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controll

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

2017-05-23 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/class-use/ServerName.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/class-use/ServerName.html 
b/devapidocs/org/apache/hadoop/hbase/class-use/ServerName.html
index 6291620..4317828 100644
--- a/devapidocs/org/apache/hadoop/hbase/class-use/ServerName.html
+++ b/devapidocs/org/apache/hadoop/hbase/class-use/ServerName.html
@@ -137,48 +137,52 @@
  
 
 
-org.apache.hadoop.hbase.regionserver
+org.apache.hadoop.hbase.quotas
  
 
 
-org.apache.hadoop.hbase.regionserver.handler
+org.apache.hadoop.hbase.regionserver
  
 
 
+org.apache.hadoop.hbase.regionserver.handler
+ 
+
+
 org.apache.hadoop.hbase.replication
 
 Multi Cluster Replication
 
 
-
+
 org.apache.hadoop.hbase.replication.regionserver
  
 
-
+
 org.apache.hadoop.hbase.rsgroup
  
 
-
+
 org.apache.hadoop.hbase.security.access
  
 
-
+
 org.apache.hadoop.hbase.tmpl.master
  
 
-
+
 org.apache.hadoop.hbase.tool
  
 
-
+
 org.apache.hadoop.hbase.util
  
 
-
+
 org.apache.hadoop.hbase.wal
  
 
-
+
 org.apache.hadoop.hbase.zookeeper
  
 
@@ -1094,6 +1098,24 @@
  TableName tableName) 
 
 
+static 
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse
+QuotaStatusCalls.getRegionServerQuotaSnapshot(ClusterConnection clusterConn,
+int timeout,
+ServerName sn)
+See QuotaStatusCalls.getRegionServerQuotaSnapshot(ClusterConnection,
 RpcControllerFactory, int, ServerName)
+
+
+
+static 
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse
+QuotaStatusCalls.getRegionServerQuotaSnapshot(ClusterConnection conn,
+RpcControllerFactory factory,
+int timeout,
+ServerName sn)
+Executes an RPC to the RegionServer identified by the 
ServerName to fetch its view
+ on space quotas.
+
+
+
 (package private) 
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService.Interface
 AsyncConnectionImpl.getRegionServerStub(ServerName serverName) 
 
@@ -4439,6 +4461,27 @@
 
 
 
+
+
+
+Uses of ServerName in org.apache.hadoop.hbase.quotas
+
+Methods in org.apache.hadoop.hbase.quotas
 with parameters of type ServerName 
+
+Modifier and Type
+Method and Description
+
+
+
+static http://docs.oracle.com/javase/8/docs/api/java/util/Map.html?is-external=true";
 title="class or interface in java.util">Map
+QuotaTableUtil.getRegionServerQuotaSnapshots(Connection conn,
+ ServerName regionServer)
+Fetches the observed SpaceQuotaSnapshots observed 
by a RegionServer.
+
+
+
+
+
 
 
 

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/class-use/Stoppable.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/class-use/Stoppable.html 
b/devapidocs/org/apache/hadoop/hbase/class-use/Stoppable.html
index 20b8b69..4b1e830 100644
--- a/devapidocs/org/apache/hadoop/hbase/class-use/Stoppable.html
+++ b/devapidocs/org/apache/hadoop/hbase/class-use/Stoppable.html
@@ -667,6 +667,14 @@
 
 
 
+QuotaObserverChore(Connection conn,
+  org.apache.hadoop.conf.Configuration conf,
+  SpaceQuotaSnapshotNotifier snapshotNotifier,
+  MasterQuotaManager quotaManager,
+  Stoppable stopper,
+  MetricsMaster metrics) 
+
+
 QuotaRefresherChore(int period,
Stoppable stoppable) 
 



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

2017-05-23 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/ipc/class-use/NettyRpcServer.MessageDecoder.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/ipc/class-use/NettyRpcServer.MessageDecoder.html
 
b/devapidocs/org/apache/hadoop/hbase/ipc/class-use/NettyRpcServer.MessageDecoder.html
deleted file mode 100644
index 10a0873..000
--- 
a/devapidocs/org/apache/hadoop/hbase/ipc/class-use/NettyRpcServer.MessageDecoder.html
+++ /dev/null
@@ -1,125 +0,0 @@
-http://www.w3.org/TR/html4/loose.dtd";>
-
-
-
-
-
-Uses of Class org.apache.hadoop.hbase.ipc.NettyRpcServer.MessageDecoder 
(Apache HBase 2.0.0-SNAPSHOT API)
-
-
-
-
-
-
-
-JavaScript is disabled on your browser.
-
-
-
-
-
-Skip navigation links
-
-
-
-
-Overview
-Package
-Class
-Use
-Tree
-Deprecated
-Index
-Help
-
-
-
-
-Prev
-Next
-
-
-Frames
-No Frames
-
-
-All Classes
-
-
-
-
-
-
-
-
-
-
-Uses 
of Classorg.apache.hadoop.hbase.ipc.NettyRpcServer.MessageDecoder
-
-No usage of 
org.apache.hadoop.hbase.ipc.NettyRpcServer.MessageDecoder
-
-
-
-
-Skip navigation links
-
-
-
-
-Overview
-Package
-Class
-Use
-Tree
-Deprecated
-Index
-Help
-
-
-
-
-Prev
-Next
-
-
-Frames
-No Frames
-
-
-All Classes
-
-
-
-
-
-
-
-
-
-Copyright © 2007–2017 https://www.apache.org/";>The Apache Software Foundation. All rights 
reserved.
-
-

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/ipc/class-use/NettyRpcServer.MessageEncoder.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/ipc/class-use/NettyRpcServer.MessageEncoder.html
 
b/devapidocs/org/apache/hadoop/hbase/ipc/class-use/NettyRpcServer.MessageEncoder.html
deleted file mode 100644
index 7b4a91d..000
--- 
a/devapidocs/org/apache/hadoop/hbase/ipc/class-use/NettyRpcServer.MessageEncoder.html
+++ /dev/null
@@ -1,125 +0,0 @@
-http://www.w3.org/TR/html4/loose.dtd";>
-
-
-
-
-
-Uses of Class org.apache.hadoop.hbase.ipc.NettyRpcServer.MessageEncoder 
(Apache HBase 2.0.0-SNAPSHOT API)
-
-
-
-
-
-
-
-JavaScript is disabled on your browser.
-
-
-
-
-
-Skip navigation links
-
-
-
-
-Overview
-Package
-Class
-Use
-Tree
-Deprecated
-Index
-Help
-
-
-
-
-Prev
-Next
-
-
-Frames
-No Frames
-
-
-All Classes
-
-
-
-
-
-
-
-
-
-
-Uses 
of Classorg.apache.hadoop.hbase.ipc.NettyRpcServer.MessageEncoder
-
-No usage of 
org.apache.hadoop.hbase.ipc.NettyRpcServer.MessageEncoder
-
-
-
-
-Skip navigation links
-
-
-
-
-Overview
-Package
-Class
-Use
-Tree
-Deprecated
-Index
-Help
-
-
-
-
-Prev
-Next
-
-
-Frames
-No Frames
-
-
-All Classes
-
-
-
-
-
-
-
-
-
-Copyright © 2007–2017 https://www.apache.org/";>The Apache Software Foundation. All rights 
reserved.
-
-

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/ipc/class-use/NettyRpcServer.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/ipc/class-use/NettyRpcServer.html 
b/devapidocs/org/apache/hadoop/hbase/ipc/class-use/NettyRpcServer.html
index 7c8bddc..6c61471 100644
--- a/devapidocs/org/apache/hadoop/hbase/ipc/class-use/NettyRpcServer.html
+++ b/devapidocs/org/apache/hadoop/hbase/ipc/class-use/NettyRpcServer.html
@@ -97,6 +97,19 @@
 
 
 Uses of NettyRpcServer in org.apache.hadoop.hbase.ipc
+
+Fields in org.apache.hadoop.hbase.ipc
 declared as NettyRpcServer 
+
+Modifier and Type
+Field and Description
+
+
+
+private NettyRpcServer
+NettyRpcServerP

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

2017-05-23 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/quotas/MasterQuotaManager.SetQuotaOperations.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/quotas/MasterQuotaManager.SetQuotaOperations.html
 
b/devapidocs/org/apache/hadoop/hbase/quotas/MasterQuotaManager.SetQuotaOperations.html
index 11bbc8e..c5dc3a6 100644
--- 
a/devapidocs/org/apache/hadoop/hbase/quotas/MasterQuotaManager.SetQuotaOperations.html
+++ 
b/devapidocs/org/apache/hadoop/hbase/quotas/MasterQuotaManager.SetQuotaOperations.html
@@ -50,7 +50,7 @@ var activeTableTab = "activeTableTab";
 
 
 Prev Class
-Next Class
+Next Class
 
 
 Frames
@@ -105,7 +105,7 @@ var activeTableTab = "activeTableTab";
 
 
 
-private static interface MasterQuotaManager.SetQuotaOperations
+private static interface MasterQuotaManager.SetQuotaOperations
 
 
 
@@ -165,7 +165,7 @@ var activeTableTab = "activeTableTab";
 
 
 fetch
-org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas fetch()
+org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas fetch()
 throws http://docs.oracle.com/javase/8/docs/api/java/io/IOException.html?is-external=true";
 title="class or interface in java.io">IOException
 
 Throws:
@@ -179,7 +179,7 @@ var activeTableTab = "activeTableTab";
 
 
 delete
-void delete()
+void delete()
  throws http://docs.oracle.com/javase/8/docs/api/java/io/IOException.html?is-external=true";
 title="class or interface in java.io">IOException
 
 Throws:
@@ -193,7 +193,7 @@ var activeTableTab = "activeTableTab";
 
 
 update
-void update(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas quotas)
+void update(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas quotas)
  throws http://docs.oracle.com/javase/8/docs/api/java/io/IOException.html?is-external=true";
 title="class or interface in java.io">IOException
 
 Throws:
@@ -207,7 +207,7 @@ var activeTableTab = "activeTableTab";
 
 
 preApply
-void preApply(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas quotas)
+void preApply(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas quotas)
throws http://docs.oracle.com/javase/8/docs/api/java/io/IOException.html?is-external=true";
 title="class or interface in java.io">IOException
 
 Throws:
@@ -221,7 +221,7 @@ var activeTableTab = "activeTableTab";
 
 
 postApply
-void postApply(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas quotas)
+void postApply(org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas quotas)
 throws http://docs.oracle.com/javase/8/docs/api/java/io/IOException.html?is-external=true";
 title="class or interface in java.io">IOException
 
 Throws:
@@ -258,7 +258,7 @@ var activeTableTab = "activeTableTab";
 
 
 Prev Class
-Next Class
+Next Class
 
 
 Frames

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/quotas/MasterQuotaManager.SizeSnapshotWithTimestamp.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/quotas/MasterQuotaManager.SizeSnapshotWithTimestamp.html
 
b/devapidocs/org/apache/hadoop/hbase/quotas/MasterQuotaManager.SizeSnapshotWithTimestamp.html
new file mode 100644
index 000..bb06901
--- /dev/null
+++ 
b/devapidocs/org/apache/hadoop/hbase/quotas/MasterQuotaManager.SizeSnapshotWithTimestamp.html
@@ -0,0 +1,392 @@
+http://www.w3.org/TR/html4/loose.dtd";>
+
+
+
+
+
+MasterQuotaManager.SizeSnapshotWithTimestamp (Apache HBase 
2.0.0-SNAPSHOT API)
+
+
+
+
+
+var methods = {"i0":10,"i1":10,"i2":10,"i3":10,"i4":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
+
+
+
+
+Prev Class
+Next Class
+
+
+Frames
+No Frames
+
+
+All Classes
+
+
+
+
+
+
+
+Summary: 
+Nested | 
+Field | 
+Constr | 
+Method
+
+
+Detail: 
+Field | 
+Constr | 
+Method
+
+
+
+
+
+
+
+
+org.apache.hadoop.hbase.quotas
+Class MasterQuotaManager.SizeSnapshotWithTimestamp
+
+
+
+http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-exter

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

2017-05-23 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/checkstyle.rss
--
diff --git a/checkstyle.rss b/checkstyle.rss
index c8ddcf2..96adb9c 100644
--- a/checkstyle.rss
+++ b/checkstyle.rss
@@ -25,8 +25,8 @@ under the License.
 en-us
 ©2007 - 2017 The Apache Software Foundation
 
-  File: 2167,
- Errors: 14408,
+  File: 2202,
+ Errors: 14458,
  Warnings: 0,
  Infos: 0
   
@@ -676,6 +676,20 @@ under the License.
   
   
 
+  http://hbase.apache.org/checkstyle.html#org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager.java";>org/apache/hadoop/hbase/quotas/RegionServerRpcQuotaManager.java
+
+
+  0
+
+
+  0
+
+
+  4
+
+  
+  
+
   http://hbase.apache.org/checkstyle.html#org.apache.hadoop.hbase.regionserver.CompactionRequestor.java";>org/apache/hadoop/hbase/regionserver/CompactionRequestor.java
 
 
@@ -769,7 +783,7 @@ under the License.
   0
 
 
-  54
+  58
 
   
   
@@ -3140,6 +3154,20 @@ under the License.
   
   
 
+  http://hbase.apache.org/checkstyle.html#org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshotNotifier.java";>org/apache/hadoop/hbase/quotas/SpaceQuotaSnapshotNotifier.java
+
+
+  0
+
+
+  0
+
+
+  0
+
+  
+  
+
   http://hbase.apache.org/checkstyle.html#org.apache.hadoop.hbase.security.visibility.SimpleScanLabelGenerator.java";>org/apache/hadoop/hbase/security/visibility/SimpleScanLabelGenerator.java
 
 
@@ -3527,7 +3555,7 @@ under the License.
   0
 
 
-  1
+  0
 
   
   
@@ -3938,6 +3966,20 @@ under the License.
   
   
 
+  http://hbase.apache.org/checkstyle.html#org.apache.hadoop.hbase.quotas.QuotaSnapshotStore.java";>org/apache/hadoop/hbase/quotas/QuotaSnapshotStore.java
+
+
+  0
+
+
+  0
+
+
+  0
+
+  
+  
+
   http://hbase.apache.org/checkstyle.html#org.apache.hadoop.hbase.replication.regionserver.MetricsReplicationSinkSourceImpl.java";>org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSourceImpl.java
 
 
@@ -4470,6 +4512,20 @@ under the License.
   
   
 
+  http://hbase.apache.org/checkstyle.html#org.apache.hadoop.hbase.quotas.TableSpaceQuotaSnapshotNotifier.java";>org/apache/hadoop/hbase/quotas/TableSpaceQuotaSnapshotNotifier.java
+
+
+  0
+
+
+  0
+
+
+  0
+
+  
+  
+
   http://hbase.apache.org/checkstyle.html#org.apache.hadoop.hbase.http.jmx.JMXJsonServlet.java";>org/apache/hadoop/hbase/http/jmx/JMXJsonServlet.java
 
 
@@ -4932,6 +4988,20 @@ under the License.
   
   
 
+  http://hbase.apache.org/checkstyle.html#org.apache.hadoop.hbase.client.QuotaStatusCalls.java";>org/apache/hadoop/hbase/client/QuotaStatusCalls.java
+
+
+  0
+
+
+  0
+
+
+  4
+
+  
+  
+
   http://hbase.apache.org/checkstyle.html#org.apache.hadoop.hbase.coprocessor.CoprocessorService.java";>org/apache/hadoop/hbase/coprocessor/CoprocessorService.java
 
 
@@ -5039,7 +5109,7 @@ under the License.
   0
 
 
-  3
+  2
 
   
   
@@ -5100,6 +5170,20 @@ under

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

2017-05-23 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/client/MasterKeepAliveConnection.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/client/MasterKeepAliveConnection.html 
b/devapidocs/org/apache/hadoop/hbase/client/MasterKeepAliveConnection.html
index b53ee9d..061614e 100644
--- a/devapidocs/org/apache/hadoop/hbase/client/MasterKeepAliveConnection.html
+++ b/devapidocs/org/apache/hadoop/hbase/client/MasterKeepAliveConnection.html
@@ -148,7 +148,7 @@ extends 
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterSer
 
 
 Methods inherited from 
interface org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService.BlockingInterface
-abortProcedure, addColumn, addReplicationPeer, assignRegion, balance, 
createNamespace, createTable, deleteColumn, deleteNamespace, deleteSnapshot, 
deleteTable, disableReplicationPeer, disableTable, drainRegionServers, 
enableCatalogJanitor, enableReplicationPeer, enableTable, execMasterService, 
execProcedure, execProcedureWithRet, getClusterStatus, getCompletedSnapshots, 
getLastMajorCompactionTimestamp, getLastMajorCompactionTimestampForRegion, 
getNamespaceDescriptor, getProcedureResult, getReplicationPeerConfig, 
getSchemaAlterStatus, getSecurityCapabilities, getTableDescriptors, 
getTableNames, getTableState, isBalancerEnabled, isCatalogJanitorEnabled, 
isCleanerChoreEnabled, isMasterInMaintenanceMode, isMasterRunning, 
isNormalizerEnabled, isProcedureDone, isSnapshotDone, isSplitOrMergeEnabled, 
listDrainingRegionServers, listLocks, listNamespaceDescriptors, listProcedures, 
listReplicationPeers, listTableDescriptorsByNamespace, 
listTableNamesByNamespace, mergeTableRegions, modify
 Column, modifyNamespace, modifyTable, moveRegion, normalize, offlineRegion, 
removeDrainFromRegionServers, removeReplicationPeer, restoreSnapshot, 
runCatalogScan, runCleanerChore, setBalancerRunning, setCleanerChoreRunning, 
setNormalizerRunning, setQuota, setSplitOrMergeEnabled, shutdown, snapshot, 
stopMaster, truncateTable, unassignRegion, 
updateReplicationPeerConfig
+abortProcedure, addColumn, addReplicationPeer, assignRegion, balance, 
createNamespace, createTable, deleteColumn, deleteNamespace, deleteSnapshot, 
deleteTable, disableReplicationPeer, disableTable, drainRegionServers, 
enableCatalogJanitor, enableReplicationPeer, enableTable, execMasterService, 
execProcedure, execProcedureWithRet, getClusterStatus, getCompletedSnapshots, 
getLastMajorCompactionTimestamp, getLastMajorCompactionTimestampForRegion, 
getNamespaceDescriptor, getProcedureResult, getQuotaStates, 
getReplicationPeerConfig, getSchemaAlterStatus, getSecurityCapabilities, 
getSpaceQuotaRegionSizes, getTableDescriptors, getTableNames, getTableState, 
isBalancerEnabled, isCatalogJanitorEnabled, isCleanerChoreEnabled, 
isMasterInMaintenanceMode, isMasterRunning, isNormalizerEnabled, 
isProcedureDone, isSnapshotDone, isSplitOrMergeEnabled, 
listDrainingRegionServers, listLocks, listNamespaceDescriptors, listProcedures, 
listReplicationPeers, listTableDescriptorsByNamespace, listTableN
 amesByNamespace, mergeTableRegions, modifyColumn, modifyNamespace, 
modifyTable, moveRegion, normalize, offlineRegion, 
removeDrainFromRegionServers, removeReplicationPeer, restoreSnapshot, 
runCatalogScan, runCleanerChore, setBalancerRunning, setCleanerChoreRunning, 
setNormalizerRunning, setQuota, setSplitOrMergeEnabled, shutdown, snapshot, 
stopMaster, truncateTable, unassignRegion, 
updateReplicationPeerConfig
 
 
 

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/client/Query.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/client/Query.html 
b/devapidocs/org/apache/hadoop/hbase/client/Query.html
index 65752f6..6485695 100644
--- a/devapidocs/org/apache/hadoop/hbase/client/Query.html
+++ b/devapidocs/org/apache/hadoop/hbase/client/Query.html
@@ -50,7 +50,7 @@ var activeTableTab = "activeTableTab";
 
 
 Prev Class
-Next Class
+Next Class
 
 
 Frames
@@ -832,7 +832,7 @@ extends 
 
 Prev Class
-Next Class
+Next Class
 
 
 Frames

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/client/QuotaStatusCalls.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/client/QuotaStatusCalls.html 
b/devapidocs/org/apache/hadoop/hbase/client/QuotaStatusCalls.html
new file mode 100644
index 000..d0a444e
--- /dev/null
+++ b/devapidocs/org/apache/hadoop/hbase/client/QuotaStatusCalls.html
@@ -0,0 +1,415 @@
+http://www.w3.org/TR/html4/loose.dtd";>
+
+
+
+
+
+QuotaStatusCalls (Apache HBase 2.0.0-SNAPSHOT API)
+
+
+
+
+
-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,"i16":10,"i17":10,"i18":10,"i19":10,"i20":10,"i21":10,"i22":10,"i23":10,"i24":10,"i25":10,"i26":10,"i27":10,"i28":10,"i29":10,"i30":10,"i31":10,"i32":10,"i33":10,"i34":10,"i35":10,"i36":10,"i37":10,"i38":10,"i39":10,"i40":10,"i41":10,"i42":10,"i43":10,"i44":10,"i45":10,"i46":10,"i47":10,"i48":10,"i49":10,"i50":10,"i51":10,"i52":10,"i53":10,"i54":10,"i55":10,"i56":10,"i57":10,"i58":10,"i59":10,"i60":10,"i61":10,"i62":10,"i63":10,"i64":10,"i65":10,"i66":10,"i67":10,"i68":10,"i69":10,"i70":10,"i71":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,"i15":10,"i16":10,"i17":10,"i18":10,"i19":10,"i20":10,"i21":10,"i22":10,"i23":10,"i24":10,"i25":10,"i26":10,"i27":10,"i28":10,"i29":10,"i30":10,"i31":10,"i32":10,"i33":10,"i34":10,"i35":10,"i36":10,"i37":10,"i38":10,"i39":10,"i40":10,"i41":10,"i42":10,"i43":10,"i44":10,"i45":10,"i46":10,"i47":10,"i48":10,"i49":10,"i50":10,"i51":10,"i52":10,"i53":10,"i54":10,"i55":10,"i56":10,"i57":10,"i58":10,"i59":10,"i60":10,"i61":10,"i62":10,"i63":10,"i64":10,"i65":10,"i66":10,"i67":10,"i68":10,"i69":10,"i70":10,"i71":10,"i72":10,"i73":10};
 var tabs = {65535:["t0","All Methods"],2:["t2","Instance 
Methods"],8:["t4","Concrete Methods"]};
 var altColor = "altColor";
 var rowColor = "rowColor";
@@ -110,7 +110,7 @@ var activeTableTab = "activeTableTab";
 
 
 @InterfaceAudience.Public
-public class ShortCircuitMasterConnection
+public class ShortCircuitMasterConnection
 extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true";
 title="class or interface in java.lang">Object
 A short-circuit connection that can bypass the RPC layer 
(serialization, deserialization,
  networking, etc..) when talking to a local master
@@ -284,226 +284,236 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
   
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest request) 
 
 
+org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse
+getQuotaStates(org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+  
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest request) 
+
+
 org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigResponse
 getReplicationPeerConfig(org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
 
org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigRequest request) 
 
-
+
 org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse
 getSchemaAlterStatus(org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
 
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest request) 
 
-
+
 org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse
 getSecurityCapabilities(org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,

org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest request) 
 
-
+
+org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse
+getSpaceQuotaRegionSizes(org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest request) 
+
+
 org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse
 getTableDescriptors(org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,

org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest request) 
 
-
+
 org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesResponse
 getTableNames(org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
  
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest request) 

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

2017-05-23 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/backup/impl/TableBackupClient.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/backup/impl/TableBackupClient.html 
b/devapidocs/org/apache/hadoop/hbase/backup/impl/TableBackupClient.html
index c96491b..bb53d3d 100644
--- a/devapidocs/org/apache/hadoop/hbase/backup/impl/TableBackupClient.html
+++ b/devapidocs/org/apache/hadoop/hbase/backup/impl/TableBackupClient.html
@@ -18,7 +18,7 @@
 catch(err) {
 }
 //-->
-var methods = 
{"i0":10,"i1":10,"i2":9,"i3":10,"i4":9,"i5":9,"i6":10,"i7":9,"i8":9,"i9":6,"i10":10,"i11":10,"i12":10,"i13":10,"i14":9,"i15":10,"i16":9};
+var methods = 
{"i0":10,"i1":10,"i2":9,"i3":10,"i4":9,"i5":9,"i6":10,"i7":9,"i8":9,"i9":6,"i10":10,"i11":10,"i12":10,"i13":10,"i14":10,"i15":10,"i16":9,"i17":10,"i18":9};
 var tabs = {65535:["t0","All Methods"],1:["t1","Static 
Methods"],2:["t2","Instance Methods"],4:["t3","Abstract 
Methods"],8:["t4","Concrete Methods"]};
 var altColor = "altColor";
 var rowColor = "rowColor";
@@ -50,7 +50,7 @@ var activeTableTab = "activeTableTab";
 
 
 Prev Class
-Next Class
+Next Class
 
 
 Frames
@@ -74,7 +74,7 @@ var activeTableTab = "activeTableTab";
 
 
 Summary: 
-Nested | 
+Nested | 
 Field | 
 Constr | 
 Method
@@ -114,7 +114,7 @@ var activeTableTab = "activeTableTab";
 
 
 @InterfaceAudience.Private
-public abstract class TableBackupClient
+public abstract class TableBackupClient
 extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true";
 title="class or interface in java.lang">Object
 Base class for backup operation. Concrete implementation for
  full and incremental backup are delegated to corresponding sub-classes:
@@ -125,6 +125,25 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 
+
+
+
+
+
+Nested Class Summary
+
+Nested Classes 
+
+Modifier and Type
+Class and Description
+
+
+static class 
+TableBackupClient.Stage 
+
+
+
+
 
 
 
@@ -142,34 +161,38 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 BACKUP_CLIENT_IMPL_CLASS 
 
 
+static http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String
+BACKUP_TEST_MODE_STAGE 
+
+
 protected http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String
 backupId 
 
-
+
 protected BackupInfo
 backupInfo 
 
-
+
 protected BackupManager
 backupManager 
 
-
+
 protected 
org.apache.hadoop.conf.Configuration
 conf 
 
-
+
 protected Connection
 conn 
 
-
+
 private static 
org.apache.commons.logging.Log
 LOG 
 
-
+
 protected http://docs.oracle.com/javase/8/docs/api/java/util/HashMap.html?is-external=true";
 title="class or interface in java.util">HashMapString,http://docs.oracle.com/javase/8/docs/api/java/lang/Long.html?is-external=true";
 title="class or interface in java.lang">Long>
 newTimestamps 
 
-
+
 protected http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">List
 tableList 
 
@@ -297,31 +320,39 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 
+protected void
+failStageIf(TableBackupClient.Stage stage) 
+
+
 protected http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String
 getMessage(http://docs.oracle.com/javase/8/docs/api/java/lang/Exception.html?is-external=true";
 title="class or interface in java.lang">Exception e) 
 
-
+
+protected TableBackupClient.Stage
+getTestStage() 
+
+
 void
 init(Connection conn,
 http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String backupId,
 BackupRequest request) 
 
-
+
 protected http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String
 obtainBackupMetaDataStr(BackupInfo backupInfo)
 Get backup request meta data dir as string.
 
 
-
+
 protected static void
 restoreBackupTable(Connection conn,
   
org.apache.hadoop.conf.Configuration conf) 
 
-
+
 protected void
 snapshotBackupTable() 
 
-
+
 protected static boolean
 snapshotExists(Admin admin,
   http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in 
java.lang">String snapshotName) 
@@ -354,20 +385,33 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 BACKUP_CLIENT_IMPL_CLASS
-public static final http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String BACKUP_CLIENT_IMPL_

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

2017-05-23 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/ipc/ServerCall.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/ipc/ServerCall.html 
b/devapidocs/org/apache/hadoop/hbase/ipc/ServerCall.html
index ac50afc..e139abe 100644
--- a/devapidocs/org/apache/hadoop/hbase/ipc/ServerCall.html
+++ b/devapidocs/org/apache/hadoop/hbase/ipc/ServerCall.html
@@ -18,8 +18,8 @@
 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,"i16":10,"i17":10,"i18":10,"i19":10,"i20":10,"i21":10,"i22":10,"i23":10,"i24":10,"i25":10,"i26":10,"i27":10,"i28":10,"i29":10,"i30":10,"i31":10,"i32":10,"i33":10,"i34":10,"i35":10,"i36":10,"i37":10,"i38":10,"i39":10};
-var tabs = {65535:["t0","All Methods"],2:["t2","Instance 
Methods"],8:["t4","Concrete Methods"]};
+var methods = 
{"i0":10,"i1":9,"i2":9,"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,"i16":10,"i17":10,"i18":10,"i19":10,"i20":10,"i21":10,"i22":10,"i23":10,"i24":10,"i25":10,"i26":10,"i27":10,"i28":10,"i29":10,"i30":10,"i31":10,"i32":9,"i33":10,"i34":10,"i35":10,"i36":10,"i37":10,"i38":9};
+var tabs = {65535:["t0","All Methods"],1:["t1","Static 
Methods"],2:["t2","Instance Methods"],8:["t4","Concrete Methods"]};
 var altColor = "altColor";
 var rowColor = "rowColor";
 var tableTab = "tableTab";
@@ -109,7 +109,7 @@ var activeTableTab = "activeTableTab";
 
 
 All Implemented Interfaces:
-RpcCall, RpcCallContext
+RpcCall, RpcCallContext, RpcResponse
 
 
 Direct Known Subclasses:
@@ -120,7 +120,7 @@ var activeTableTab = "activeTableTab";
 @InterfaceAudience.Private
 abstract class ServerCall
 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 RpcCall
+implements RpcCall, RpcResponse
 Datastructure that holds all necessary to a method 
invocation and then afterward, carries
  the result.
 
@@ -288,7 +288,7 @@ implements 
 Method Summary
 
-All Methods Instance Methods Concrete Methods 
+All Methods Static Methods Instance Methods Concrete Methods 
 
 Modifier and Type
 Method and Description
@@ -300,14 +300,14 @@ implements 
 
 
-private http://docs.oracle.com/javase/8/docs/api/java/nio/ByteBuffer.html?is-external=true";
 title="class or interface in java.nio">ByteBuffer
+private static http://docs.oracle.com/javase/8/docs/api/java/nio/ByteBuffer.html?is-external=true";
 title="class or interface in java.nio">ByteBuffer
 createHeaderAndMessageBytes(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message result,

org.apache.hadoop.hbase.shaded.com.google.protobuf.Message header,
int totalSize,
int totalPBSize) 
 
 
-protected http://docs.oracle.com/javase/8/docs/api/java/nio/ByteBuffer.html?is-external=true";
 title="class or interface in java.nio">ByteBuffer
+(package private) static http://docs.oracle.com/javase/8/docs/api/java/nio/ByteBuffer.html?is-external=true";
 title="class or interface in java.nio">ByteBuffer
 createHeaderAndMessageBytes(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message result,

org.apache.hadoop.hbase.shaded.com.google.protobuf.Message header,
int cellBlockSize,
@@ -320,7 +320,7 @@ implements 
 
 
-(package private) void
+void
 done()
 Call is done.
 
@@ -379,78 +379,78 @@ implements getRequestUserName() 
 
 
+BufferChain
+getResponse() 
+
+
 long
 getResponseBlockSize() 
 
-
+
 long
 getResponseCellSize()
 The size of response cells that have been accumulated so 
far.
 
 
-
+
 long
 getResponseExceptionSize() 
 
-
+
 org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingService
 getService() 
 
-
+
 long
 getSize()
 Used to calculate the request call queue size.
 
 
-
+
 long
 getStartTime() 
 
-
+
 int
 getTimeout() 
 
-
+
 org.apache.htrace.TraceInfo
 getTraceInfo() 
 
-
+
 void
 incrementResponseBlockSize(long blockSize) 
 
-
+
 void
 incrementResponseCellSize(long cellSize)
 Add on the given amount to the retained cell size.
 
 
-
+
 void
 incrementResponseExceptionSize(long exSize) 
 
-
+
 boolean
 isClientCellBlockSupported()
 If the client connected and specified a codec to use, then 
we will use this codec making
  cellblocks to return.
 
 
-
+
 boolean
 isRetryImmediatelySupported() 
 
-
+
 void
 setCallBack(RpcCallback callback)
 Sets a callback which has to be executed at the end of this 
RPC call.
 
 
-
-protected void
-setConnectionHeaderResponse(http://docs.oracle.com/javase/8/docs/api/java/nio/ByteBuffer.html?is-external=true";
 title="class or interface in 
java.nio">ByteBuffer response) 
-
 
-protected void
+(package private) static void
 setExceptionResp

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

2017-05-23 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/apidocs/src-html/org/apache/hadoop/hbase/quotas/QuotaExceededException.html
--
diff --git 
a/apidocs/src-html/org/apache/hadoop/hbase/quotas/QuotaExceededException.html 
b/apidocs/src-html/org/apache/hadoop/hbase/quotas/QuotaExceededException.html
index 54f1e48..bc0857d 100644
--- 
a/apidocs/src-html/org/apache/hadoop/hbase/quotas/QuotaExceededException.html
+++ 
b/apidocs/src-html/org/apache/hadoop/hbase/quotas/QuotaExceededException.html
@@ -37,7 +37,11 @@
 029  public QuotaExceededException(String 
msg) {
 030super(msg);
 031  }
-032}
+032
+033  public QuotaExceededException(String 
msg, Throwable cause) {
+034super(msg, cause);
+035  }
+036}
 
 
 

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/apidocs/src-html/org/apache/hadoop/hbase/quotas/QuotaRetriever.html
--
diff --git 
a/apidocs/src-html/org/apache/hadoop/hbase/quotas/QuotaRetriever.html 
b/apidocs/src-html/org/apache/hadoop/hbase/quotas/QuotaRetriever.html
index 0da9bb4..5ed5ed0 100644
--- a/apidocs/src-html/org/apache/hadoop/hbase/quotas/QuotaRetriever.html
+++ b/apidocs/src-html/org/apache/hadoop/hbase/quotas/QuotaRetriever.html
@@ -30,141 +30,158 @@
 022import java.io.IOException;
 023import java.util.Iterator;
 024import java.util.LinkedList;
-025import java.util.Queue;
-026
-027import org.apache.commons.logging.Log;
-028import 
org.apache.commons.logging.LogFactory;
-029import 
org.apache.hadoop.conf.Configuration;
-030import 
org.apache.hadoop.hbase.TableName;
-031import 
org.apache.hadoop.hbase.classification.InterfaceAudience;
-032import 
org.apache.hadoop.hbase.client.Connection;
-033import 
org.apache.hadoop.hbase.client.ConnectionFactory;
-034import 
org.apache.hadoop.hbase.client.Result;
-035import 
org.apache.hadoop.hbase.client.ResultScanner;
-036import 
org.apache.hadoop.hbase.client.Scan;
-037import 
org.apache.hadoop.hbase.client.Table;
-038import 
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
-039import 
org.apache.hadoop.util.StringUtils;
-040
-041/**
-042 * Scanner to iterate over the quota 
settings.
-043 */
-044@InterfaceAudience.Public
-045public class QuotaRetriever implements 
Closeable, Iterable {
-046  private static final Log LOG = 
LogFactory.getLog(QuotaRetriever.class);
-047
-048  private final 
Queue cache = new LinkedList<>();
-049  private ResultScanner scanner;
-050  /**
-051   * Connection to use.
-052   * Could pass one in and have this 
class use it but this class wants to be standalone.
-053   */
-054  private Connection connection;
-055  private Table table;
-056
-057  private QuotaRetriever() {
-058  }
-059
-060  void init(final Configuration conf, 
final Scan scan) throws IOException {
-061this.connection = 
ConnectionFactory.createConnection(conf);
-062this.table = 
this.connection.getTable(QuotaTableUtil.QUOTA_TABLE_NAME);
-063try {
-064  scanner = table.getScanner(scan);
-065} catch (IOException e) {
-066  try {
-067close();
-068  } catch (IOException ioe) {
-069LOG.warn("Failed getting scanner 
and then failed close on cleanup", e);
-070  }
-071  throw e;
-072}
-073  }
-074
-075  public void close() throws IOException 
{
-076if (this.table != null) {
-077  this.table.close();
-078  this.table = null;
-079}
-080if (this.connection != null) {
-081  this.connection.close();
-082  this.connection = null;
-083}
-084  }
-085
-086  public QuotaSettings next() throws 
IOException {
-087if (cache.isEmpty()) {
-088  Result result = scanner.next();
-089  if (result == null) {
-090return null;
-091  }
-092  
QuotaTableUtil.parseResultToCollection(result, cache);
-093}
-094return cache.poll();
-095  }
-096
-097  @Override
-098  public Iterator 
iterator() {
-099return new Iter();
-100  }
-101
-102  private class Iter implements 
Iterator {
-103QuotaSettings cache;
-104
-105public Iter() {
-106  try {
-107cache = 
QuotaRetriever.this.next();
-108  } catch (IOException e) {
-109
LOG.warn(StringUtils.stringifyException(e));
-110  }
-111}
-112
-113@Override
-114public boolean hasNext() {
-115  return cache != null;
-116}
-117
-118@Override
-119public QuotaSettings next() {
-120  QuotaSettings result = cache;
-121  try {
-122cache = 
QuotaRetriever.this.next();
-123  } catch (IOException e) {
-124
LOG.warn(StringUtils.stringifyException(e));
-125  }
-126  return result;
-127}
-128
-129@Override
-130public void remove() {
-131  throw new 
RuntimeException("remove() not supported");
-132}
-133  }
+025import java.util.Objects;
+026import java.util.Queue;
+027
+028import org

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

2017-05-23 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/ipc/SimpleServerCall.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/ipc/SimpleServerCall.html 
b/devapidocs/org/apache/hadoop/hbase/ipc/SimpleServerCall.html
index 6667a7e..831a125 100644
--- a/devapidocs/org/apache/hadoop/hbase/ipc/SimpleServerCall.html
+++ b/devapidocs/org/apache/hadoop/hbase/ipc/SimpleServerCall.html
@@ -114,7 +114,7 @@ var activeTableTab = "activeTableTab";
 
 
 All Implemented Interfaces:
-RpcCall, RpcCallContext
+RpcCall, RpcCallContext, RpcResponse
 
 
 
@@ -142,10 +142,6 @@ extends Field and Description
 
 
-(package private) long
-lastSentTime 
-
-
 (package private) SimpleRpcServerResponder
 responder 
 
@@ -204,7 +200,7 @@ extends Method and Description
 
 
-(package private) void
+void
 done()
 Call is done.
 
@@ -225,7 +221,7 @@ extends 
 
 Methods inherited from class org.apache.hadoop.hbase.ipc.ServerCall
-cleanup,
 createHeaderAndMessageBytes,
 disconnectSince,
 getCellScanner,
 getClientVersionInfo,
 getDeadline,
 getHeader,
 getMethod,
 getParam,
 getPriority,
 getReceiveTime,
 getRemoteAddress,
 getRemotePort,
 getRequestUser,
 getRequestUserName,
 getResponseBlockSize,
 getResponseCellSize,
 getResponseExceptionSize,
 getService,
 getSize,
 getStartTime,
 getTimeout,
 getTraceInfo,
 incrementResponseBlockSize,
 incrementResponseCellSize,
 incrementResponseExceptionSize,
 isClientCellBlockSupported,
 isRetryImmediatelySupported,
 setCallBack,
 setConnectionHeaderResponse,
 setExceptionResponse,
 s
 etResponse, setSaslTokenResponse,
 setStartTime,
 toShortString,
 toString,
 wrapWithSasl
+cleanup,
 createHeaderAndMessageBytes,
 disconnectSince,
 getCellScanner,
 getClientVersionInfo,
 getDeadline,
 getHeader,
 getMethod,
 getParam,
 getPriority,
 getReceiveTime,
 getRemoteAddress,
 getRemotePort,
 getRequestUser,
 getRequestUserName,
 getResponse,
 getResponseBlockSize,
 getResponseCellSize, getResponseExceptionSize,
 getService,
 getSize,
 getStartTime,
 getTimeout,
 getTraceInfo,
 incrementResponseBlockSize,
 incrementResponseCellSize,
 incrementResponseExceptionSize,
 isClientCellBlockSupported,
 isRetryImmediatelySupported,
 setCallBack,
 setExceptionResponse,
 setResponse,
 setStartTime, toShortString,
 toString,
 wrapWithSasl
 
 
 
@@ -248,22 +244,13 @@ extends 
 
 Field Detail
-
-
-
-
-
-lastSentTime
-long lastSentTime
-
-
 
 
 
 
 
 responder
-final SimpleRpcServerResponder responder
+final SimpleRpcServerResponder responder
 
 
 
@@ -280,7 +267,7 @@ extends 
 
 SimpleServerCall
-SimpleServerCall(int id,
+SimpleServerCall(int id,
  
org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingService service,
  
org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.MethodDescriptor md,
  
org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader header,
@@ -312,9 +299,11 @@ extends 
 
 done
-void done()
+public void done()
 Call is done. Execution happened and we returned results to 
client. It is now safe to cleanup.
 
+Specified by:
+done in
 interface RpcResponse
 Overrides:
 done in
 class ServerCall
 
@@ -326,7 +315,7 @@ extends 
 
 sendResponseIfReady
-public void sendResponseIfReady()
+public void sendResponseIfReady()
  throws http://docs.oracle.com/javase/8/docs/api/java/io/IOException.html?is-external=true";
 title="class or interface in java.io">IOException
 Description copied from 
interface: RpcCall
 Send the response of this RPC call.
@@ -343,7 +332,7 @@ extends 
 
 getConnection
-SimpleServerRpcConnection getConnection()
+SimpleServerRpcConnection getConnection()
 
 
 

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/ipc/SimpleServerRpcConnection.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/ipc/SimpleServerRpcConnection.html 
b/devapidocs/org/apache/hadoop/hbase/ipc/SimpleServerRpcConnection.html
index 0cb7e29..c6d4fa1 100644
--- a/devapidocs/org/apache/hadoop/hbase/ipc/SimpleServerRpcConnection.html
+++ b/devapidocs/org/apache/hadoop/hbase/ipc/SimpleServerRpcConnection.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";
@@ -119,7 +119,7 @@ var activeTableTab = "activeTableTab";
 
 
 @InterfaceAudience.Private
-class SimpleServerRpcConnection
+class SimpleServerRpcConnection
 extend

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

2017-05-23 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/ipc/NettyRpcServerResponseEncoder.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/ipc/NettyRpcServerResponseEncoder.html 
b/devapidocs/org/apache/hadoop/hbase/ipc/NettyRpcServerResponseEncoder.html
new file mode 100644
index 000..35f7fd6
--- /dev/null
+++ b/devapidocs/org/apache/hadoop/hbase/ipc/NettyRpcServerResponseEncoder.html
@@ -0,0 +1,373 @@
+http://www.w3.org/TR/html4/loose.dtd";>
+
+
+
+
+
+NettyRpcServerResponseEncoder (Apache HBase 2.0.0-SNAPSHOT API)
+
+
+
+
+
+var methods = {"i0":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
+
+
+
+
+Prev Class
+Next Class
+
+
+Frames
+No Frames
+
+
+All Classes
+
+
+
+
+
+
+
+Summary: 
+Nested | 
+Field | 
+Constr | 
+Method
+
+
+Detail: 
+Field | 
+Constr | 
+Method
+
+
+
+
+
+
+
+
+org.apache.hadoop.hbase.ipc
+Class 
NettyRpcServerResponseEncoder
+
+
+
+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
+
+
+io.netty.channel.ChannelHandlerAdapter
+
+
+io.netty.channel.ChannelOutboundHandlerAdapter
+
+
+org.apache.hadoop.hbase.ipc.NettyRpcServerResponseEncoder
+
+
+
+
+
+
+
+
+
+
+
+All Implemented Interfaces:
+io.netty.channel.ChannelHandler, 
io.netty.channel.ChannelOutboundHandler
+
+
+
+@InterfaceAudience.Private
+class NettyRpcServerResponseEncoder
+extends io.netty.channel.ChannelOutboundHandlerAdapter
+Encoder for RpcResponse.
+
+
+
+
+
+
+
+
+
+
+
+Nested Class Summary
+
+
+
+
+Nested classes/interfaces inherited from 
interface io.netty.channel.ChannelHandler
+io.netty.channel.ChannelHandler.Sharable
+
+
+
+
+
+
+
+
+Field Summary
+
+Fields 
+
+Modifier and Type
+Field and Description
+
+
+private MetricsHBaseServer
+metrics 
+
+
+
+
+
+
+
+
+
+Constructor Summary
+
+Constructors 
+
+Constructor and Description
+
+
+NettyRpcServerResponseEncoder(MetricsHBaseServer metrics) 
+
+
+
+
+
+
+
+
+
+Method Summary
+
+All Methods Instance Methods Concrete Methods 
+
+Modifier and Type
+Method and Description
+
+
+void
+write(io.netty.channel.ChannelHandlerContext ctx,
+ http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true";
 title="class or interface in java.lang">Object msg,
+ io.netty.channel.ChannelPromise promise) 
+
+
+
+
+
+
+Methods inherited from 
class io.netty.channel.ChannelOutboundHandlerAdapter
+bind, close, connect, deregister, disconnect, flush, read
+
+
+
+
+
+Methods inherited from 
class io.netty.channel.ChannelHandlerAdapter
+exceptionCaught, handlerAdded, handlerRemoved, isSharable
+
+
+
+
+
+Methods inherited from class 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
+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/Obj

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

2017-05-23 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/classification/class-use/InterfaceAudience.Private.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/classification/class-use/InterfaceAudience.Private.html
 
b/devapidocs/org/apache/hadoop/hbase/classification/class-use/InterfaceAudience.Private.html
index ad1b5d9..9ffa364 100644
--- 
a/devapidocs/org/apache/hadoop/hbase/classification/class-use/InterfaceAudience.Private.html
+++ 
b/devapidocs/org/apache/hadoop/hbase/classification/class-use/InterfaceAudience.Private.html
@@ -390,124 +390,128 @@ Input/OutputFormats, a table indexing MapReduce job, 
and utility methods.
  
 
 
-org.apache.hadoop.hbase.regionserver
+org.apache.hadoop.hbase.quotas.policies
  
 
 
-org.apache.hadoop.hbase.regionserver.compactions
+org.apache.hadoop.hbase.regionserver
  
 
 
-org.apache.hadoop.hbase.regionserver.handler
+org.apache.hadoop.hbase.regionserver.compactions
  
 
 
-org.apache.hadoop.hbase.regionserver.querymatcher
+org.apache.hadoop.hbase.regionserver.handler
  
 
 
-org.apache.hadoop.hbase.regionserver.snapshot
+org.apache.hadoop.hbase.regionserver.querymatcher
  
 
 
-org.apache.hadoop.hbase.regionserver.throttle
+org.apache.hadoop.hbase.regionserver.snapshot
  
 
 
-org.apache.hadoop.hbase.regionserver.wal
+org.apache.hadoop.hbase.regionserver.throttle
  
 
 
+org.apache.hadoop.hbase.regionserver.wal
+ 
+
+
 org.apache.hadoop.hbase.replication
 
 Multi Cluster Replication
 
 
-
+
 org.apache.hadoop.hbase.replication.master
  
 
-
+
 org.apache.hadoop.hbase.replication.regionserver
  
 
-
+
 org.apache.hadoop.hbase.rest
 
 HBase REST
 
 
-
+
 org.apache.hadoop.hbase.rest.filter
  
 
-
+
 org.apache.hadoop.hbase.rest.model
  
 
-
+
 org.apache.hadoop.hbase.rest.provider
  
 
-
+
 org.apache.hadoop.hbase.rest.provider.consumer
  
 
-
+
 org.apache.hadoop.hbase.rest.provider.producer
  
 
-
+
 org.apache.hadoop.hbase.rsgroup
  
 
-
+
 org.apache.hadoop.hbase.security
  
 
-
+
 org.apache.hadoop.hbase.security.access
  
 
-
+
 org.apache.hadoop.hbase.security.token
  
 
-
+
 org.apache.hadoop.hbase.security.visibility
  
 
-
+
 org.apache.hadoop.hbase.security.visibility.expression
  
 
-
+
 org.apache.hadoop.hbase.snapshot
  
 
-
+
 org.apache.hadoop.hbase.thrift
 
 Provides an HBase http://incubator.apache.org/thrift/";>Thrift
 service.
 
 
-
+
 org.apache.hadoop.hbase.thrift2
 
 Provides an HBase http://thrift.apache.org/";>Thrift
 service.
 
 
-
+
 org.apache.hadoop.hbase.tool
  
 
-
+
 org.apache.hadoop.hbase.trace
  
 
-
+
 org.apache.hadoop.hbase.types
 
 
@@ -515,47 +519,47 @@ service.
  extensible data type API.
 
 
-
+
 org.apache.hadoop.hbase.util
  
 
-
+
 org.apache.hadoop.hbase.util.byterange
  
 
-
+
 org.apache.hadoop.hbase.util.byterange.impl
  
 
-
+
 org.apache.hadoop.hbase.util.hbck
  
 
-
+
 org.apache.hadoop.hbase.util.test
  
 
-
+
 org.apache.hadoop.hbase.util.vint
  
 
-
+
 org.apache.hadoop.hbase.wal
  
 
-
+
 org.apache.hadoop.hbase.zookeeper
  
 
-
+
 org.apache.hadoop.metrics2.impl
  
 
-
+
 org.apache.hadoop.metrics2.lib
  
 
-
+
 org.apache.hadoop.metrics2.util
  
 
@@ -1959,18 +1963,24 @@ service.
 
 
 
+class 
+QuotaStatusCalls
+Client class to wrap RPCs to HBase servers for space quota 
status information.
+
+
+
 (package private) class 
 RawAsyncTableImpl
 The implementation of RawAsyncTable.
 
 
-
+
 class 
 RegionAdminServiceCallable
 Similar to RegionServerCallable but for the AdminService 
interface.
 
 
-
+
 (package private) class 
 RegionCoprocessorRpcChannel
 Provides clients with an RPC connection to call Coprocessor 
Endpoint
@@ -1978,87 +1988,87 @@ service.
  against a given table region.
 
 
-
+
 (package private) class 
 RegionCoprocessorRpcChannelImpl
 The implementation of a region based coprocessor rpc 
channel.
 
 
-
+
 class 
 RegionCoprocessorServiceExec
 Represents a coprocessor service method execution against a 
single region.
 
 
-
+
 (package private) class 
 RegionLocateType
 Indicate which row you want to locate.
 
 
-
+
 class 
 RegionReplicaUtil
 Utility methods which contain the logic for regions and 
replicas.
 
 
-
+
 class 
 RegionServerCallable
 Implementations make a RPC call against a RegionService via 
a protobuf Service.
 
 
-
+
 (package private) interface 
 Registry
 Cluster registry.
 
 
-
+
 (package private) class 
 RegistryFactory
 Get instance of configured Registry.
 
 
-
+
 class 
 ResultBoundedCompletionService
 A completion service for the RpcRetryingCallerFactory.
 
 
-
+
 class 
 ResultStatsUtil
 A Result with 
some statistics about the server/region status
 
 
-
+
 static class 
 RetriesExhaustedException.ThrowableWithExtraContext
 Datastructure that allows adding more info around Throwable 
incident.
 
 
-
+
 interface 
 RetryingCallable
 A Callable that will be retried.
 
 
-
+
 (package private) class 
 RetryingCallerInterceptor
 This class i

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

2017-05-23 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/class-use/CellScanner.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/class-use/CellScanner.html 
b/devapidocs/org/apache/hadoop/hbase/class-use/CellScanner.html
index aa4c86f..5a2bb70 100644
--- a/devapidocs/org/apache/hadoop/hbase/class-use/CellScanner.html
+++ b/devapidocs/org/apache/hadoop/hbase/class-use/CellScanner.html
@@ -875,17 +875,18 @@
 
 
 private Result
-RSRpcServices.append(Region region,
+RSRpcServices.append(Region region,
   OperationQuota quota,
   
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto mutation,
   CellScanner cellScanner,
-  long nonceGroup)
+  long nonceGroup,
+  ActivePolicyEnforcement spaceQuota)
 Execute an append mutation.
 
 
 
 private boolean
-RSRpcServices.checkAndRowMutate(Region region,
+RSRpcServices.checkAndRowMutate(Region region,
  http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in 
java.util">List actions,
  CellScanner cellScanner,
  byte[] row,
@@ -893,23 +894,25 @@
  byte[] qualifier,
  CompareFilter.CompareOp compareOp,
  ByteArrayComparable comparator,
- 
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult.Builder builder)
+ 
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult.Builder builder,
+ ActivePolicyEnforcement spaceQuotaEnforcement)
 Mutate a list of rows atomically.
 
 
 
 private void
-RSRpcServices.doBatchOp(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult.Builder builder,
+RSRpcServices.doBatchOp(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult.Builder builder,
  Region region,
  OperationQuota quota,
  http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in 
java.util">List mutations,
- CellScanner cells)
+ CellScanner cells,
+ ActivePolicyEnforcement spaceQuotaEnforcement)
 Execute a list of Put/Delete mutations.
 
 
 
 private http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">List
-RSRpcServices.doNonAtomicRegionMutation(Region region,
+RSRpcServices.doNonAtomicRegionMutation(Region region,
  OperationQuota quota,
  
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction actions,
  CellScanner cellScanner,
@@ -917,18 +920,20 @@
  http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">List cellsToReturn,
  long nonceGroup,
  RSRpcServices.RegionScannersCloseCallBack closeCallBack,
- RpcCallContext context)
+ RpcCallContext context,
+ ActivePolicyEnforcement spaceQuotaEnforcement)
 Run through the regionMutation rm and per 
Mutation, do the work, and then when
  done, add an instance of a ClientProtos.ResultOrException that 
corresponds to each Mutation.
 
 
 
 private Result
-RSRpcServices.increment(Region region,
+RSRpcServices.increment(Region region,
  OperationQuota quota,
  
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto mutation,
  CellScanner cells,
- long nonceGroup)
+ long nonceGroup,
+ ActivePolicyEnforcement spaceQuota)
 Execute an increment mutation.
 
 

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/class-use/Coprocessor.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/class-use/Coprocessor.html 
b/devapidocs/org/apache/hadoop/hbase/class-use/Coprocessor.html
index 7c6d940..5c83eec 100644
--- a/devapidocs/org/apache/hadoop/hbase/class-use/Coprocessor.html
+++ b/devapidocs/org/apache/hadoop/hbase/class-use/Coprocessor.html
@@ -111,34 +111,38 @@
  
 
 
-org.apache.hadoop.hbase.regionserver
+org.apache.hadoop.hbase.quotas
  
 
 
-org.apache.hadoop.hbase.regionserver.wal
+org.apache.hadoop.hbase.regionserver
  
 
 
-org.apache.hadoop.hbase.replication.regionserver
+org.apache.hadoop.hbase.regionserver.wal
  
 
 
-org.apache.hadoop.hbase.rsgroup
+org.apache.hadoop.hbase.replication.regionserver
  
 
 
-org.apache.hadoop.hbase.security.access
+org.ap

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

2017-05-23 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/master/MasterRpcServices.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/master/MasterRpcServices.html 
b/devapidocs/org/apache/hadoop/hbase/master/MasterRpcServices.html
index 2287513..fc31f90 100644
--- a/devapidocs/org/apache/hadoop/hbase/master/MasterRpcServices.html
+++ b/devapidocs/org/apache/hadoop/hbase/master/MasterRpcServices.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,"i16":10,"i17":10,"i18":10,"i19":10,"i20":10,"i21":10,"i22":10,"i23":10,"i24":10,"i25":10,"i26":10,"i27":10,"i28":10,"i29":10,"i30":10,"i31":10,"i32":10,"i33":10,"i34":10,"i35":10,"i36":10,"i37":10,"i38":10,"i39":10,"i40":10,"i41":10,"i42":10,"i43":10,"i44":10,"i45":10,"i46":10,"i47":10,"i48":10,"i49":10,"i50":10,"i51":10,"i52":10,"i53":10,"i54":10,"i55":10,"i56":10,"i57":10,"i58":10,"i59":10,"i60":10,"i61":10,"i62":10,"i63":10,"i64":10,"i65":10,"i66":10,"i67":10,"i68":10,"i69":10,"i70":10,"i71":10,"i72":10,"i73":10,"i74":10,"i75":10,"i76":10,"i77":10,"i78":10,"i79":10,"i80":10,"i81":10,"i82":10,"i83":10,"i84":10,"i85":10,"i86":10,"i87":10,"i88":10,"i89":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,"i15":10,"i16":10,"i17":10,"i18":10,"i19":10,"i20":10,"i21":10,"i22":10,"i23":10,"i24":10,"i25":10,"i26":10,"i27":10,"i28":10,"i29":10,"i30":10,"i31":10,"i32":10,"i33":10,"i34":10,"i35":10,"i36":10,"i37":10,"i38":10,"i39":10,"i40":10,"i41":10,"i42":10,"i43":10,"i44":10,"i45":10,"i46":10,"i47":10,"i48":10,"i49":10,"i50":10,"i51":10,"i52":10,"i53":10,"i54":10,"i55":10,"i56":10,"i57":10,"i58":10,"i59":10,"i60":10,"i61":10,"i62":10,"i63":10,"i64":10,"i65":10,"i66":10,"i67":10,"i68":10,"i69":10,"i70":10,"i71":10,"i72":10,"i73":10,"i74":10,"i75":10,"i76":10,"i77":10,"i78":10,"i79":10,"i80":10,"i81":10,"i82":10,"i83":10,"i84":10,"i85":10,"i86":10,"i87":10,"i88":10,"i89":10,"i90":10,"i91":10,"i92":10};
 var tabs = {65535:["t0","All Methods"],2:["t2","Instance 
Methods"],8:["t4","Concrete Methods"]};
 var altColor = "altColor";
 var rowColor = "rowColor";
@@ -119,7 +119,7 @@ var activeTableTab = "activeTableTab";
 
 
 @InterfaceAudience.Private
-public class MasterRpcServices
+public class MasterRpcServices
 extends RSRpcServices
 implements 
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService.BlockingInterface,
 
org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStatusService.BlockingInterface,
 
org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockService.BlockingInterface
 Implements the master RPC services.
@@ -383,305 +383,320 @@ implements 
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.Master
   
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest request) 
 
 
+org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse
+getQuotaStates(org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+  
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest request) 
+
+
 org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse
 getRegionInfo(org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
  
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest request) 
 
-
+
 org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigResponse
 getReplicationPeerConfig(org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
 
org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigRequest request) 
 
-
+
 org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse
 getSchemaAlterStatus(org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
 
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest req)
 Get the number of regions of the table that have been 
updated by the alter.
 
 
-
+
 org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse
 getSecurityCapabilities(org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,

org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest request)
 Returns the security capabilities in effect on the 
cluster
 
 
-
+
 protected http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in

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

2017-05-23 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/allclasses-noframe.html
--
diff --git a/devapidocs/allclasses-noframe.html 
b/devapidocs/allclasses-noframe.html
index 8ac0440..900185a 100644
--- a/devapidocs/allclasses-noframe.html
+++ b/devapidocs/allclasses-noframe.html
@@ -40,6 +40,7 @@
 AbstractRpcClient.RpcChannelImplementation
 AbstractStateMachineNamespaceProcedure
 AbstractStateMachineTableProcedure
+AbstractViolationPolicyEnforcement
 AccessControlClient
 AccessControlConstants
 AccessControlFilter
@@ -51,6 +52,7 @@
 AccessDeniedException
 Action
 ActiveMasterManager
+ActivePolicyEnforcement
 AdaptiveLifoCoDelCallQueue
 AddColumnFamilyProcedure
 Address
@@ -176,6 +178,7 @@
 BackupCommands.HelpCommand
 BackupCommands.HistoryCommand
 BackupCommands.ProgressCommand
+BackupCommands.RepairCommand
 BackupCopyJob
 BackupDriver
 BackupException
@@ -667,6 +670,7 @@
 DefaultStoreEngine
 DefaultStoreFileManager
 DefaultStoreFlusher
+DefaultViolationPolicyEnforcement
 DefaultVisibilityExpressionResolver
 DefaultVisibilityLabelServiceImpl
 DefinedSetFilterScanLabelGenerator
@@ -702,6 +706,7 @@
 DisableTableProcedure
 DisableTableProcedure.BulkDisabler
 DisableTableProcedure.MarkRegionOfflineOpResult
+DisableTableViolationPolicyEnforcement
 DNS
 DoNotRetryIOException
 DoNotRetryRegionException
@@ -838,6 +843,7 @@
 FileLink.FileLinkInputStream
 FileMmapEngine
 FileStatusFilter
+FileSystemUtilizationChore
 FileSystemVersionException
 Filter
 Filter.ReturnCode
@@ -1342,10 +1348,12 @@
 MasterQuotaManager
 MasterQuotaManager.NamedLock
 MasterQuotaManager.SetQuotaOperations
+MasterQuotaManager.SizeSnapshotWithTimestamp
 MasterRpcServices
 MasterRpcServices.BalanceSwitchMode
 MasterServices
 MasterSnapshotVerifier
+MasterSpaceQuotaObserver
 MasterStatusServlet
 MasterStatusTmpl
 MasterStatusTmpl.ImplData
@@ -1449,6 +1457,10 @@
 MetricsMasterProcSourceFactory
 MetricsMasterProcSourceFactoryImpl
 MetricsMasterProcSourceImpl
+MetricsMasterQuotaSource
+MetricsMasterQuotaSourceFactory
+MetricsMasterQuotaSourceFactoryImpl
+MetricsMasterQuotaSourceImpl
 MetricsMasterSource
 MetricsMasterSourceFactory
 MetricsMasterSourceFactoryImpl
@@ -1460,6 +1472,7 @@
 MetricsRegionAggregateSource
 MetricsRegionAggregateSourceImpl
 MetricsRegionServer
+MetricsRegionServerQuotaSource
 MetricsRegionServerSource
 MetricsRegionServerSourceFactory
 MetricsRegionServerSourceFactoryImpl
@@ -1515,6 +1528,7 @@
 MiniBatchOperationInProgress
 MiniZooKeeperCluster
 MinorCompactionScanQueryMatcher
+MissingSnapshotViolationPolicyEnforcement
 MobCacheConfig
 MobCompactionChore
 MobCompactionRequest
@@ -1589,6 +1603,7 @@
 NamespaceExistException
 NamespaceGroupingStrategy
 NamespaceNotFoundException
+NamespaceQuotaSnapshotStore
 NamespacesInstanceModel
 NamespacesInstanceResource
 NamespacesModel
@@ -1604,9 +1619,13 @@
 NettyRpcConnection
 NettyRpcDuplexHandler
 NettyRpcServer
+NettyRpcServerPreambleHandler
+NettyRpcServerRequestDecoder
+NettyRpcServerResponseEncoder
 NettyServerCall
 NettyServerRpcConnection
 NoCacheFilter
+NoInsertsViolationPolicyEnforcement
 NoLimitScannerContext
 NoLimitThroughputController
 NoncedRegionServerCallable
@@ -1634,6 +1653,8 @@
 NoTagsKeyValue
 NotAllMetaRegionsOnlineException
 NotServingRegionException
+NoWritesCompactionsViolationPolicyEnforcement
+NoWritesViolationPolicyEnforcement
 NullComparator
 ObjectIntPair
 ObjectPool
@@ -1806,12 +1827,17 @@
 QuotaFilter
 QuotaLimiter
 QuotaLimiterFactory
+QuotaObserverChore
+QuotaObserverChore.TablesWithQuotas
 QuotaRetriever
 QuotaScope
 QuotaSettings
 QuotaSettingsFactory
 QuotaSettingsFactory.QuotaGlobalsSettingsBypass
+QuotaSnapshotStore
+QuotaSnapshotStore.ViolationState
 QuotaState
+QuotaStatusCalls
 QuotaTableUtil
 QuotaTableUtil.NamespaceQuotasVisitor
 QuotaTableUtil.QuotasVisitor
@@ -1943,13 +1969,14 @@
 RegionServerObserver
 RegionServerProcedureManager
 RegionServerProcedureManagerHost
-RegionServerQuotaManager
+RegionServerRpcQuotaManager
 RegionServerRunningException
 RegionServerServices
 RegionServerServices.PostOpenDeployContext
 RegionServerServices.RegionStateTransitionContext
 RegionServerSnapshotManager
 RegionServerSnapshotManager.SnapshotSubprocedurePool
+RegionServerSpaceQuotaManager
 RegionServerStoppedException
 RegionServerTracker
 RegionServicesForStores
@@ -2142,6 +2169,7 @@
 RpcExecutor.CallPriorityComparator
 RpcExecutor.QueueBalancer
 RpcExecutor.RandomQueueBalancer
+RpcResponse
 RpcRetryingCallable
 RpcRetryingCaller
 RpcRetryingCallerFactory
@@ -2346,6 +2374,16 @@
 SortedCompactionPolicy
 SortedList
 SourceFSConfigurationProvider
+SpaceLimitingException
+SpaceLimitSettings
+SpaceQuotaRefresherChore
+SpaceQuotaSnapshot
+SpaceQuotaSnapshot.SpaceQuotaStatus
+SpaceQuotaSnapshotNotifier
+SpaceQuotaSnapshotNotifierFactory
+SpaceViolationPolicy
+SpaceViolationPolicyEnforcement
+SpaceViolationPolicyEnforcementFactory
 SpanReceiverHost
 SpanReceiverHost.Singleton

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

2017-05-23 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/apidocs/org/apache/hadoop/hbase/quotas/QuotaExceededException.html
--
diff --git a/apidocs/org/apache/hadoop/hbase/quotas/QuotaExceededException.html 
b/apidocs/org/apache/hadoop/hbase/quotas/QuotaExceededException.html
index cf622d2..172b2da 100644
--- a/apidocs/org/apache/hadoop/hbase/quotas/QuotaExceededException.html
+++ b/apidocs/org/apache/hadoop/hbase/quotas/QuotaExceededException.html
@@ -132,7 +132,7 @@
 
 
 Direct Known Subclasses:
-ThrottlingException
+SpaceLimitingException, ThrottlingException
 
 
 
@@ -164,6 +164,10 @@ extends 
 QuotaExceededException(http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String msg) 
 
+
+QuotaExceededException(http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String msg,
+  http://docs.oracle.com/javase/8/docs/api/java/lang/Throwable.html?is-external=true";
 title="class or interface in 
java.lang">Throwable cause) 
+
 
 
 
@@ -204,12 +208,22 @@ extends 
 
 
-
+
 
 QuotaExceededException
 public QuotaExceededException(http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String msg)
 
 
+
+
+
+
+
+QuotaExceededException
+public QuotaExceededException(http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String msg,
+  http://docs.oracle.com/javase/8/docs/api/java/lang/Throwable.html?is-external=true";
 title="class or interface in java.lang">Throwable cause)
+
+
 
 
 

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/apidocs/org/apache/hadoop/hbase/quotas/QuotaRetriever.html
--
diff --git a/apidocs/org/apache/hadoop/hbase/quotas/QuotaRetriever.html 
b/apidocs/org/apache/hadoop/hbase/quotas/QuotaRetriever.html
index 86d6741..724ee6b 100644
--- a/apidocs/org/apache/hadoop/hbase/quotas/QuotaRetriever.html
+++ b/apidocs/org/apache/hadoop/hbase/quotas/QuotaRetriever.html
@@ -114,7 +114,7 @@ var activeTableTab = "activeTableTab";
 
 
 @InterfaceAudience.Public
-public class QuotaRetriever
+public class QuotaRetriever
 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, http://docs.oracle.com/javase/8/docs/api/java/lang/Iterable.html?is-external=true";
 title="class or interface in java.lang">Iterable
 Scanner to iterate over the quota settings.
@@ -196,7 +196,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.h
 
 
 close
-public void close()
+public void close()
throws http://docs.oracle.com/javase/8/docs/api/java/io/IOException.html?is-external=true";
 title="class or interface in java.io">IOException
 
 Specified by:
@@ -214,7 +214,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.h
 
 
 next
-public QuotaSettings next()
+public QuotaSettings next()
throws http://docs.oracle.com/javase/8/docs/api/java/io/IOException.html?is-external=true";
 title="class or interface in java.io">IOException
 
 Throws:
@@ -228,7 +228,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.h
 
 
 iterator
-public http://docs.oracle.com/javase/8/docs/api/java/util/Iterator.html?is-external=true";
 title="class or interface in java.util">Iterator iterator()
+public http://docs.oracle.com/javase/8/docs/api/java/util/Iterator.html?is-external=true";
 title="class or interface in java.util">Iterator iterator()
 
 Specified by:
 http://docs.oracle.com/javase/8/docs/api/java/lang/Iterable.html?is-external=true#iterator--";
 title="class or interface in java.lang">iterator in 
interface http://docs.oracle.com/javase/8/docs/api/java/lang/Iterable.html?is-external=true";
 title="class or interface in java.lang">Iterable
@@ -241,7 +241,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.h
 
 
 open
-public static QuotaRetriever open(org.apache.hadoop.conf.Configuration conf)
+public static QuotaRetriever open(org.apache.hadoop.conf.Configuration conf)
throws http://docs.oracle.com/javase/8/docs/api/java/io/IOException.html?is-external=true";
 title="class or interface in java.io">IOException
 Open a QuotaRetriever with no filter, all the quota 
settings will be returned.
 
@@ -260,7 +260,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.h
 
 
 open
-public static QuotaRetriever open(org.apache.hadoop.

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

2017-05-23 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/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 73d01b1..93ff559 100644
--- a/devapidocs/org/apache/hadoop/hbase/master/HMaster.html
+++ b/devapidocs/org/apache/hadoop/hbase/master/HMaster.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,"i16":10,"i17":10,"i18":10,"i19":10,"i20":10,"i21":9,"i22":10,"i23":10,"i24":10,"i25":10,"i26":10,"i27":10,"i28":10,"i29":10,"i30":10,"i31":10,"i32":10,"i33":10,"i34":10,"i35":10,"i36":10,"i37":9,"i38":10,"i39":10,"i40":10,"i41":10,"i42":10,"i43":10,"i44":10,"i45":10,"i46":10,"i47":10,"i48":10,"i49":10,"i50":10,"i51":10,"i52":10,"i53":10,"i54":9,"i55":10,"i56":10,"i57":10,"i58":10,"i59":10,"i60":10,"i61":10,"i62":10,"i63":10,"i64":10,"i65":10,"i66":10,"i67":10,"i68":10,"i69":10,"i70":10,"i71":10,"i72":10,"i73":10,"i74":10,"i75":10,"i76":10,"i77":10,"i78":10,"i79":10,"i80":10,"i81":10,"i82":10,"i83":10,"i84":10,"i85":10,"i86":10,"i87":10,"i88":10,"i89":10,"i90":10,"i91":10,"i92":10,"i93":10,"i94":10,"i95":10,"i96":10,"i97":10,"i98":10,"i99":10,"i100":10,"i101":10,"i102":10,"i103":9,"i104":10,"i105":10,"i106":10,"i107":10,"i108":10,"i109"
 
:10,"i110":10,"i111":10,"i112":10,"i113":10,"i114":10,"i115":10,"i116":10,"i117":10,"i118":10,"i119":10,"i120":9,"i121":10,"i122":10,"i123":10,"i124":10,"i125":10,"i126":10,"i127":10,"i128":10,"i129":10,"i130":10,"i131":10,"i132":10,"i133":10,"i134":10,"i135":10,"i136":10,"i137":10,"i138":10,"i139":10,"i140":10,"i141":10,"i142":10,"i143":10,"i144":10,"i145":10,"i146":10,"i147":10,"i148":10,"i149":10,"i150":10,"i151":10,"i152":10,"i153":9};
+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,"i16":10,"i17":10,"i18":10,"i19":10,"i20":10,"i21":9,"i22":10,"i23":10,"i24":10,"i25":10,"i26":10,"i27":10,"i28":10,"i29":10,"i30":10,"i31":10,"i32":10,"i33":10,"i34":10,"i35":10,"i36":10,"i37":10,"i38":9,"i39":10,"i40":10,"i41":10,"i42":10,"i43":10,"i44":10,"i45":10,"i46":10,"i47":10,"i48":10,"i49":10,"i50":10,"i51":10,"i52":10,"i53":10,"i54":10,"i55":9,"i56":10,"i57":10,"i58":10,"i59":10,"i60":10,"i61":10,"i62":10,"i63":10,"i64":10,"i65":10,"i66":10,"i67":10,"i68":10,"i69":10,"i70":10,"i71":10,"i72":10,"i73":10,"i74":10,"i75":10,"i76":10,"i77":10,"i78":10,"i79":10,"i80":10,"i81":10,"i82":10,"i83":10,"i84":10,"i85":10,"i86":10,"i87":10,"i88":10,"i89":10,"i90":10,"i91":10,"i92":10,"i93":10,"i94":10,"i95":10,"i96":10,"i97":10,"i98":10,"i99":10,"i100":10,"i101":10,"i102":10,"i103":10,"i104":10,"i105":10,"i106":9,"i107":10,"i108":10,"i109"
 
:10,"i110":10,"i111":10,"i112":10,"i113":10,"i114":10,"i115":10,"i116":10,"i117":10,"i118":10,"i119":10,"i120":10,"i121":10,"i122":10,"i123":9,"i124":10,"i125":10,"i126":10,"i127":10,"i128":10,"i129":10,"i130":10,"i131":10,"i132":10,"i133":10,"i134":10,"i135":10,"i136":10,"i137":10,"i138":10,"i139":10,"i140":10,"i141":10,"i142":10,"i143":10,"i144":10,"i145":10,"i146":10,"i147":10,"i148":10,"i149":10,"i150":10,"i151":10,"i152":10,"i153":10,"i154":10,"i155":10,"i156":10,"i157":9};
 var tabs = {65535:["t0","All Methods"],1:["t1","Static 
Methods"],2:["t2","Instance Methods"],8:["t4","Concrete Methods"]};
 var altColor = "altColor";
 var rowColor = "rowColor";
@@ -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
@@ -388,45 +388,53 @@ implements quotaManager 
 
 
+private QuotaObserverChore
+quotaObserverChore 
+
+
 private RegionNormalizerTracker
 regionNormalizerTracker 
 
-
+
 (package private) RegionServerTracker
 regionServerTracker 
 
-
+
 private ReplicationManager
 replicationManager 
 
-
+
 private ReplicationMetaCleaner
 replicationMetaCleaner 
 
-
+
 private ReplicationZKNodeCleanerChore
 replicationZKNodeCleanerChore 
 
-
+
 (package private) MemoryBoundedLogMessageBuffer
 rsFatals 
 
-
+
 private ProcedureEvent
 serverCrashProcessingEnabled 
 
-
+
 private ServerManager
 serverManager 
 
-
+
 (package private) boolean
 serviceStarted 
 
-
+
 (package private) SnapshotManager
 snapshotManager 
 
+
+private SpaceQuotaSnapshotNotifier
+spaceQuotaSnapshotNotifier 
+
 
 private SplitOrMergeTracker
 splitOrMergeTracker 
@@ -449,7 +457,7 @@ implements HRegionServer
-cacheConfig,
 cacheFlusher,
 CLOSE,
 clusterConnection,
 clusterStatusTracker,
 compactSplitThread,
 conf,
 configurationManager,
 csm, DEFAULT_REGION_LOCK_AWAIT_TIME_SEC,
 fs,
 fsOk,
 hMemManager,
 infoServer

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

2017-05-23 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/client/class-use/ClusterConnection.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/client/class-use/ClusterConnection.html 
b/devapidocs/org/apache/hadoop/hbase/client/class-use/ClusterConnection.html
index b78dcdf..9fdf96b 100644
--- a/devapidocs/org/apache/hadoop/hbase/client/class-use/ClusterConnection.html
+++ b/devapidocs/org/apache/hadoop/hbase/client/class-use/ClusterConnection.html
@@ -336,6 +336,20 @@
TableName tableName) 
 
 
+static 
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse
+QuotaStatusCalls.getMasterQuotaStates(ClusterConnection clusterConn,
+int timeout)
+See QuotaStatusCalls.getMasterQuotaStates(Connection,
 RpcControllerFactory, RpcRetryingCallerFactory, int)
+
+
+
+static 
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse
+QuotaStatusCalls.getMasterRegionSizes(ClusterConnection clusterConn,
+int timeout)
+See QuotaStatusCalls.getMasterRegionSizes(Connection,
 RpcControllerFactory, RpcRetryingCallerFactory, int)
+
+
+
 (package private) static RegionLocations
 RpcRetryingCallerWithReadReplicas.getRegionLocations(boolean useCache,
   int replicaId,
@@ -352,6 +366,24 @@
   int replicaId) 
 
 
+static 
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse
+QuotaStatusCalls.getRegionServerQuotaSnapshot(ClusterConnection clusterConn,
+int timeout,
+ServerName sn)
+See QuotaStatusCalls.getRegionServerQuotaSnapshot(ClusterConnection,
 RpcControllerFactory, int, ServerName)
+
+
+
+static 
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse
+QuotaStatusCalls.getRegionServerQuotaSnapshot(ClusterConnection conn,
+RpcControllerFactory factory,
+int timeout,
+ServerName sn)
+Executes an RPC to the RegionServer identified by the 
ServerName to fetch its view
+ on space quotas.
+
+
+
 (package private) static NonceGenerator
 ConnectionImplementation.injectNonceGeneratorForTesting(ClusterConnection conn,
   NonceGenerator cnm) 

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/client/class-use/Connection.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/client/class-use/Connection.html 
b/devapidocs/org/apache/hadoop/hbase/client/class-use/Connection.html
index 8863c9a..c2e9c5e 100644
--- a/devapidocs/org/apache/hadoop/hbase/client/class-use/Connection.html
+++ b/devapidocs/org/apache/hadoop/hbase/client/class-use/Connection.html
@@ -1175,6 +1175,24 @@ Input/OutputFormats, a table indexing MapReduce job, and 
utility methods.
http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in 
java.lang">String bulkToken) 
 
 
+static 
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse
+QuotaStatusCalls.getMasterQuotaStates(Connection conn,
+RpcControllerFactory factory,
+RpcRetryingCallerFactory rpcCaller,
+int timeout)
+Executes an RPC tot he HBase master to fetch its view on 
space quotas.
+
+
+
+static 
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse
+QuotaStatusCalls.getMasterRegionSizes(Connection conn,
+RpcControllerFactory factory,
+RpcRetryingCallerFactory rpcCaller,
+int timeout)
+Executes an RPC to the HBase master to fetch its view on 
the Region sizes.
+
+
+
 (package private) static Registry
 RegistryFactory.getRegistry(Connection connection) 
 
@@ -1664,6 +1682,30 @@ Input/OutputFormats, a table indexing MapReduce job, and 
utility methods.
 
 
 private Connection
+TableQuotaSnapshotStore.conn 
+
+
+private Connection
+SpaceQuotaRefresherChore.conn 
+
+
+private Connection
+NamespaceQuotaSnapshotStore.conn 
+
+
+private Connection
+QuotaObserverChore.conn 
+
+
+private Connection
+QuotaObserverChore.TablesWithQuotas.conn 
+
+
+private Connection
+TableSpaceQuotaSnapshotNotifier.conn 
+
+
+private Connection
 QuotaRetriever.connection
 Connection to use.
 
@@ -1671,6 +1713,23 @@ Input/OutputFormats, a table indexing MapReduce job, and 
utility methods.
 
 
 
+Methods in org.apache.hadoop.hbase.quotas
 that return Connection 
+
+Modifier and Type
+Method and Description
+
+
+
+(package private) Connection
+RegionServerSpaceQuotaManager.getConnection() 
+
+
+(package private) Connection
+SpaceQuotaRefresherChore.getConnecti

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

2017-05-23 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/ipc/package-tree.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/ipc/package-tree.html 
b/devapidocs/org/apache/hadoop/hbase/ipc/package-tree.html
index 2aeabf2..8efbcf5 100644
--- a/devapidocs/org/apache/hadoop/hbase/ipc/package-tree.html
+++ b/devapidocs/org/apache/hadoop/hbase/ipc/package-tree.html
@@ -121,28 +121,23 @@
 
 io.netty.channel.ChannelInboundHandlerAdapter (implements 
io.netty.channel.ChannelInboundHandler)
 
-io.netty.handler.codec.ByteToMessageDecoder
-
-org.apache.hadoop.hbase.ipc.NettyRpcServer.ConnectionHeaderHandler
-
-
 io.netty.channel.ChannelDuplexHandler (implements 
io.netty.channel.ChannelOutboundHandler)
 
 org.apache.hadoop.hbase.ipc.BufferCallBeforeInitHandler
 org.apache.hadoop.hbase.ipc.NettyRpcDuplexHandler
 
 
-io.netty.channel.ChannelInitializer
+org.apache.hadoop.hbase.ipc.NettyRpcServerRequestDecoder
+io.netty.channel.SimpleChannelInboundHandler
 
-org.apache.hadoop.hbase.ipc.NettyRpcServer.Initializer
+org.apache.hadoop.hbase.ipc.NettyRpcServerPreambleHandler
 
 
-org.apache.hadoop.hbase.ipc.NettyRpcServer.MessageDecoder
 
 
 io.netty.channel.ChannelOutboundHandlerAdapter (implements 
io.netty.channel.ChannelOutboundHandler)
 
-org.apache.hadoop.hbase.ipc.NettyRpcServer.MessageEncoder
+org.apache.hadoop.hbase.ipc.NettyRpcServerResponseEncoder
 
 
 
@@ -163,7 +158,6 @@
 
 org.apache.hadoop.hbase.ipc.MetricsHBaseServerWrapperImpl (implements 
org.apache.hadoop.hbase.ipc.MetricsHBaseServerWrapper)
 org.apache.hadoop.hbase.ipc.NettyRpcClientConfigHelper
-org.apache.hadoop.hbase.ipc.NettyRpcServer.CallWriteListener (implements 
io.netty.channel.ChannelFutureListener)
 org.apache.hadoop.hbase.ipc.RpcClientFactory
 org.apache.hadoop.hbase.ipc.RpcConnection
 
@@ -207,7 +201,7 @@
 
 org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface
 org.apache.hadoop.hbase.ipc.RpcServerFactory
-org.apache.hadoop.hbase.ipc.ServerCall (implements 
org.apache.hadoop.hbase.ipc.RpcCall)
+org.apache.hadoop.hbase.ipc.ServerCall (implements 
org.apache.hadoop.hbase.ipc.RpcCall, org.apache.hadoop.hbase.ipc.RpcResponse)
 
 org.apache.hadoop.hbase.ipc.NettyServerCall
 org.apache.hadoop.hbase.ipc.SimpleServerCall
@@ -333,6 +327,7 @@
 org.apache.hadoop.hbase.ipc.HBaseRpcController (also extends 
org.apache.hadoop.hbase.CellScannable)
 
 
+org.apache.hadoop.hbase.ipc.RpcResponse
 org.apache.hadoop.hbase.ipc.RpcServer.CallCleanup
 org.apache.hadoop.hbase.ipc.RpcServerInterface
 
@@ -346,8 +341,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">Enum (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">Comparable, 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.ipc.MetricsHBaseServerSourceFactoryImpl.SourceStorage
 org.apache.hadoop.hbase.ipc.CallEvent.Type
+org.apache.hadoop.hbase.ipc.MetricsHBaseServerSourceFactoryImpl.SourceStorage
 org.apache.hadoop.hbase.ipc.BufferCallBeforeInitHandler.BufferCallAction
 
 

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/ipc/package-use.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/ipc/package-use.html 
b/devapidocs/org/apache/hadoop/hbase/ipc/package-use.html
index dc4388d..6aa27d3 100644
--- a/devapidocs/org/apache/hadoop/hbase/ipc/package-use.html
+++ b/devapidocs/org/apache/hadoop/hbase/ipc/package-use.html
@@ -361,72 +361,77 @@ Input/OutputFormats, a table indexing MapReduce job, and 
utility methods.
 RpcExecutor.QueueBalancer 
 
 
+RpcResponse
+An interface represent the response of an rpc call.
+
+
+
 RpcScheduler
 An interface for RPC request scheduling algorithm.
 
 
-
+
 RpcScheduler.Context
 Exposes runtime information of a RpcServer 
that a RpcScheduler may need.
 
 
-
+
 RpcServer
 An RPC server that hosts protobuf described Services.
 
 
-
+
 RpcServer.BlockingServiceAndInterface
 Datastructure for passing a BlockingService 
and its associated class of
  protobuf service interface.
 
 
-
+
 RpcServer.CallCleanup 
 
-
+
 RpcServerInterface 
 
-
+
 ServerCall
 Datastructure that holds all necessary to a method 
invocation and then afterward, carries
  the result.
 
 
-
+
 ServerRpcConnection
 Reads calls from a connection and queues them for 
handling.
 
 
-
+
 SimpleRpcServer
 The RPC server with native java NIO implementation deriving 
from Hadoop to
  host protobuf described Services.
 
 
-
+
 SimpleRpcServer.ConnectionManager 
 
-
+
 SimpleRpcServer.Listener
 Listens on the socket.
 
 
-
+
 SimpleRpcServer.Listener.Read

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

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/ipc/RpcServer.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/ipc/RpcServer.html 
b/devapidocs/org/apache/hadoop/hbase/ipc/RpcServer.html
index fb6e673..c9d55e3 100644
--- a/devapidocs/org/apache/hadoop/hbase/ipc/RpcServer.html
+++ b/devapidocs/org/apache/hadoop/hbase/ipc/RpcServer.html
@@ -18,7 +18,7 @@
 catch(err) {
 }
 //-->
-var methods = 
{"i0":10,"i1":9,"i2":10,"i3":10,"i4":9,"i5":10,"i6":10,"i7":10,"i8":9,"i9":10,"i10":10,"i11":9,"i12":6,"i13":9,"i14":9,"i15":9,"i16":9,"i17":10,"i18":10,"i19":9,"i20":9,"i21":9,"i22":9,"i23":10,"i24":9,"i25":10,"i26":10,"i27":10,"i28":10,"i29":10,"i30":10,"i31":10,"i32":10};
+var methods = 
{"i0":10,"i1":9,"i2":10,"i3":10,"i4":9,"i5":10,"i6":10,"i7":10,"i8":9,"i9":10,"i10":10,"i11":9,"i12":6,"i13":9,"i14":9,"i15":9,"i16":9,"i17":10,"i18":10,"i19":9,"i20":9,"i21":9,"i22":9,"i23":10,"i24":9,"i25":10,"i26":10,"i27":10,"i28":10,"i29":10,"i30":10,"i31":10};
 var tabs = {65535:["t0","All Methods"],1:["t1","Static 
Methods"],2:["t2","Instance Methods"],4:["t3","Abstract 
Methods"],8:["t4","Concrete Methods"]};
 var altColor = "altColor";
 var rowColor = "rowColor";
@@ -119,7 +119,7 @@ var activeTableTab = "activeTableTab";
 
 @InterfaceAudience.LimitedPrivate(value={"Coprocesssor","Phoenix"})
  @InterfaceStability.Evolving
-public abstract class RpcServer
+public abstract class RpcServer
 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 RpcServerInterface, ConfigurationObserver
 An RPC server that hosts protobuf described Services.
@@ -620,15 +620,6 @@ implements void
 setSecretManager(org.apache.hadoop.security.token.SecretManager secretManager) 
 
-
-protected void
-setupResponse(http://docs.oracle.com/javase/8/docs/api/java/io/ByteArrayOutputStream.html?is-external=true";
 title="class or interface in java.io">ByteArrayOutputStream response,
- ServerCall call,
- http://docs.oracle.com/javase/8/docs/api/java/lang/Throwable.html?is-external=true";
 title="class or interface in java.lang">Throwable t,
- http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String error)
-Setup response for the RPC Call.
-
-
 
 
 
@@ -664,7 +655,7 @@ implements 
 
 LOG
-public static final org.apache.commons.logging.Log LOG
+public static final org.apache.commons.logging.Log LOG
 
 
 
@@ -673,7 +664,7 @@ implements 
 
 CALL_QUEUE_TOO_BIG_EXCEPTION
-protected static final CallQueueTooBigException CALL_QUEUE_TOO_BIG_EXCEPTION
+protected static final CallQueueTooBigException CALL_QUEUE_TOO_BIG_EXCEPTION
 
 
 
@@ -682,7 +673,7 @@ implements 
 
 authorize
-private final boolean authorize
+private final boolean authorize
 
 
 
@@ -691,7 +682,7 @@ implements 
 
 isSecurityEnabled
-protected boolean isSecurityEnabled
+protected boolean isSecurityEnabled
 
 
 
@@ -700,7 +691,7 @@ implements 
 
 CURRENT_VERSION
-public static final byte CURRENT_VERSION
+public static final byte CURRENT_VERSION
 
 See Also:
 Constant
 Field Values
@@ -713,7 +704,7 @@ implements 
 
 FALLBACK_TO_INSECURE_CLIENT_AUTH
-public static final http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String FALLBACK_TO_INSECURE_CLIENT_AUTH
+public static final http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String FALLBACK_TO_INSECURE_CLIENT_AUTH
 Whether we allow a fallback to SIMPLE auth for insecure 
clients when security is enabled.
 
 See Also:
@@ -727,7 +718,7 @@ implements 
 
 DEFAULT_MAX_CALLQUEUE_LENGTH_PER_HANDLER
-protected static final int DEFAULT_MAX_CALLQUEUE_LENGTH_PER_HANDLER
+protected static final int DEFAULT_MAX_CALLQUEUE_LENGTH_PER_HANDLER
 How many calls/handler are allowed in the queue.
 
 See Also:
@@ -741,7 +732,7 @@ implements 
 
 cellBlockBuilder
-protected final CellBlockBuilder cellBlockBuilder
+protected final CellBlockBuilder cellBlockBuilder
 
 
 
@@ -750,7 +741,7 @@ implements 
 
 AUTH_FAILED_FOR
-protected static final http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String AUTH_FAILED_FOR
+protected static final http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String AUTH_FAILED_FOR
 
 See Also:
 Constant
 Field Values
@@ -763,7 +754,7 @@ implements 
 
 AUTH_SUCCESSFUL_FOR
-protected static final http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String AUTH_SUCCESSFUL_FOR
+protected static final

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

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/apache_hbase_reference_guide.pdf
--
diff --git a/apache_hbase_reference_guide.pdf b/apache_hbase_reference_guide.pdf
index 7261a43..2a6b287 100644
--- a/apache_hbase_reference_guide.pdf
+++ b/apache_hbase_reference_guide.pdf
@@ -5,24 +5,24 @@
 /Author (Apache HBase Team)
 /Creator (Asciidoctor PDF 1.5.0.alpha.6, based on Prawn 1.2.1)
 /Producer (Apache HBase Team)
-/CreationDate (D:20170522173528+00'00')
-/ModDate (D:20170522173528+00'00')
+/CreationDate (D:20170523144905+00'00')
+/ModDate (D:20170523144905+00'00')
 >>
 endobj
 2 0 obj
 << /Type /Catalog
 /Pages 3 0 R
 /Names 25 0 R
-/Outlines 4012 0 R
-/PageLabels 4220 0 R
+/Outlines 4019 0 R
+/PageLabels 4227 0 R
 /PageMode /UseOutlines
 /ViewerPreferences [/FitWindow]
 >>
 endobj
 3 0 obj
 << /Type /Pages
-/Count 673
-/Kids [7 0 R 13 0 R 15 0 R 17 0 R 19 0 R 21 0 R 23 0 R 39 0 R 43 0 R 47 0 R 55 
0 R 58 0 R 60 0 R 62 0 R 66 0 R 71 0 R 74 0 R 79 0 R 81 0 R 84 0 R 86 0 R 92 0 
R 101 0 R 106 0 R 108 0 R 120 0 R 123 0 R 130 0 R 136 0 R 145 0 R 154 0 R 166 0 
R 170 0 R 172 0 R 176 0 R 182 0 R 184 0 R 186 0 R 188 0 R 190 0 R 193 0 R 199 0 
R 201 0 R 204 0 R 206 0 R 208 0 R 210 0 R 212 0 R 214 0 R 217 0 R 220 0 R 224 0 
R 226 0 R 228 0 R 230 0 R 232 0 R 234 0 R 236 0 R 238 0 R 245 0 R 247 0 R 249 0 
R 251 0 R 254 0 R 258 0 R 263 0 R 268 0 R 271 0 R 274 0 R 290 0 R 299 0 R 305 0 
R 317 0 R 326 0 R 331 0 R 333 0 R 335 0 R 346 0 R 351 0 R 355 0 R 360 0 R 367 0 
R 378 0 R 390 0 R 404 0 R 413 0 R 415 0 R 417 0 R 423 0 R 434 0 R 445 0 R 456 0 
R 459 0 R 462 0 R 466 0 R 470 0 R 474 0 R 477 0 R 479 0 R 482 0 R 486 0 R 488 0 
R 492 0 R 497 0 R 501 0 R 507 0 R 509 0 R 515 0 R 517 0 R 521 0 R 530 0 R 532 0 
R 536 0 R 539 0 R 542 0 R 545 0 R 559 0 R 566 0 R 573 0 R 585 0 R 591 0 R 599 0 
R 608 0 R 611 0 R 615 0 R 618 0 R 629 0
  R 637 0 R 643 0 R 648 0 R 652 0 R 654 0 R 668 0 R 680 0 R 686 0 R 692 0 R 695 
0 R 704 0 R 712 0 R 716 0 R 721 0 R 726 0 R 728 0 R 730 0 R 732 0 R 740 0 R 749 
0 R 753 0 R 761 0 R 769 0 R 775 0 R 779 0 R 785 0 R 790 0 R 795 0 R 803 0 R 805 
0 R 809 0 R 814 0 R 820 0 R 823 0 R 830 0 R 840 0 R 844 0 R 846 0 R 849 0 R 853 
0 R 858 0 R 861 0 R 873 0 R 877 0 R 882 0 R 890 0 R 895 0 R 899 0 R 903 0 R 905 
0 R 908 0 R 910 0 R 914 0 R 916 0 R 920 0 R 924 0 R 928 0 R 933 0 R 938 0 R 941 
0 R 943 0 R 950 0 R 956 0 R 964 0 R 973 0 R 977 0 R 982 0 R 986 0 R 988 0 R 997 
0 R 1000 0 R 1005 0 R 1008 0 R 1017 0 R 1020 0 R 1026 0 R 1033 0 R 1036 0 R 
1038 0 R 1047 0 R 1049 0 R 1051 0 R 1054 0 R 1056 0 R 1058 0 R 1060 0 R 1062 0 
R 1064 0 R 1068 0 R 1072 0 R 1077 0 R 1079 0 R 1081 0 R 1083 0 R 1085 0 R 1090 
0 R 1099 0 R 1102 0 R 1104 0 R 1106 0 R  0 R 1113 0 R 1116 0 R 1118 0 R 
1120 0 R 1122 0 R 1125 0 R 1130 0 R 1135 0 R 1145 0 R 1150 0 R 1164 0 R 1177 0 
R 1190 0 R 1199 0 R 1213 0 R 1217 0 R 1227 0 R 12
 40 0 R 1243 0 R 1255 0 R 1264 0 R 1271 0 R 1275 0 R 1285 0 R 1290 0 R 1294 0 R 
1300 0 R 1306 0 R 1313 0 R 1321 0 R 1323 0 R 1335 0 R 1337 0 R 1342 0 R 1346 0 
R 1351 0 R 1361 0 R 1367 0 R 1373 0 R 1375 0 R 1377 0 R 1390 0 R 1396 0 R 1404 
0 R 1409 0 R 1421 0 R 1428 0 R 1433 0 R 1443 0 R 1451 0 R 1454 0 R 1460 0 R 
1464 0 R 1467 0 R 1472 0 R 1475 0 R 1479 0 R 1485 0 R 1489 0 R 1494 0 R 1500 0 
R 1504 0 R 1507 0 R 1509 0 R 1517 0 R 1525 0 R 1531 0 R 1536 0 R 1540 0 R 1543 
0 R 1549 0 R 1555 0 R 1560 0 R 1562 0 R 1564 0 R 1567 0 R 1569 0 R 1577 0 R 
1580 0 R 1586 0 R 1594 0 R 1598 0 R 1603 0 R 1609 0 R 1612 0 R 1614 0 R 1616 0 
R 1618 0 R 1625 0 R 1635 0 R 1637 0 R 1639 0 R 1641 0 R 1643 0 R 1646 0 R 1648 
0 R 1650 0 R 1652 0 R 1655 0 R 1657 0 R 1659 0 R 1661 0 R 1665 0 R 1669 0 R 
1678 0 R 1680 0 R 1682 0 R 1684 0 R 1686 0 R 1693 0 R 1695 0 R 1700 0 R 1702 0 
R 1704 0 R 1711 0 R 1716 0 R 1722 0 R 1726 0 R 1729 0 R 1732 0 R 1736 0 R 1738 
0 R 1741 0 R 1743 0 R 1745 0 R 1747 0 R 1751 0 R 1753 0 R 
 1756 0 R 1758 0 R 1760 0 R 1762 0 R 1764 0 R 1772 0 R 1775 0 R 1780 0 R 1782 0 
R 1784 0 R 1786 0 R 1788 0 R 1796 0 R 1807 0 R 1810 0 R 1824 0 R 1836 0 R 1840 
0 R 1846 0 R 1851 0 R 1854 0 R 1859 0 R 1861 0 R 1866 0 R 1868 0 R 1871 0 R 
1873 0 R 1875 0 R 1877 0 R 1879 0 R 1883 0 R 1885 0 R 1889 0 R 1893 0 R 1900 0 
R 1907 0 R 1918 0 R 1932 0 R 1944 0 R 1961 0 R 1965 0 R 1967 0 R 1971 0 R 1988 
0 R 1996 0 R 2003 0 R 2012 0 R 2018 0 R 2028 0 R 2039 0 R 2045 0 R 2054 0 R 
2066 0 R 2083 0 R 2094 0 R 2097 0 R 2106 0 R 2121 0 R 2128 0 R 2131 0 R 2136 0 
R 2141 0 R 2151 0 R 2159 0 R 2162 0 R 2164 0 R 2168 0 R 2183 0 R 2192 0 R 2197 
0 R 2201 0 R 2204 0 R 2206 0 R 2208 0 R 2210 0 R 2212 0 R 2217 0 R 2219 0 R 
2229 0 R 2239 0 R 2246 0 R 2258 0 R 2263 0 R 2267 0 R 2280 0 R 2287 0 R 2293 0 
R 2295 0 R 2305 0 R 2312 0 R 2323 0 R 2327 0 R 2338 0 R 2344 0 R 2354 0 R 2363 
0 R 2371 0 R 2377 0 R 2382 0 R 2386 0 R 2390 0 R 2392 0 R 2398 0 R 2402 0 R 
2406 0 R 2412 0 R 2419 0 R 2424 0 R 2428 0 R 2437 0 R 2442 0 
 R 2447 0 R 2460 0 R 2467 0 R 2470 0 R 247

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

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/Coprocessor.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/Coprocessor.html 
b/devapidocs/org/apache/hadoop/hbase/Coprocessor.html
index 46320f5..c1aeeb1 100644
--- a/devapidocs/org/apache/hadoop/hbase/Coprocessor.html
+++ b/devapidocs/org/apache/hadoop/hbase/Coprocessor.html
@@ -105,7 +105,7 @@ var activeTableTab = "activeTableTab";
 
 
 All Known Implementing Classes:
-AccessController, 
AggregateImplementation, BackupObserver, BaseRowProcessorEndpoint, BulkDeleteEndpoint, ConstraintProcessor, CoprocessorWhitelistMasterObserver,
 ExampleMasterObserverWithMetrics,
 ExampleRegionObserverWithMetrics,
 JMXListener, MultiRowMutationEndpoint, ReplicationObserver, RowCountEndpoint, RSGroupAdminEndpoint, SecureBulkLoadEndpoint, TokenProvider, VisibilityController, VisibilityController.VisibilityReplication,
 WriteSinkCoprocessor, ZooKeeperScanPolicyObserver
+AccessController, 
AggregateImplementation, BackupObserver, BaseRowProcessorEndpoint, BulkDeleteEndpoint, ConstraintProcessor, CoprocessorWhitelistMasterObserver,
 ExampleMasterObserverWithMetrics,
 ExampleRegionObserverWithMetrics,
 JMXListener, MasterSpaceQuotaObserver, 
MultiRowMutationEndpoint, ReplicationObserver, RowCountEndpoint, RSGroupAdminEndpoint, SecureBulkLoadEndpoint, TokenProvider, VisibilityController, VisibilityController.VisibilityReplication,
 WriteSinkCoprocessor, ZooKeeperScanPolicyObserver
 
 
 

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/ScheduledChore.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/ScheduledChore.html 
b/devapidocs/org/apache/hadoop/hbase/ScheduledChore.html
index 5c204cd..b1b817c 100644
--- a/devapidocs/org/apache/hadoop/hbase/ScheduledChore.html
+++ b/devapidocs/org/apache/hadoop/hbase/ScheduledChore.html
@@ -113,7 +113,7 @@ var activeTableTab = "activeTableTab";
 
 
 Direct Known Subclasses:
-BalancerChore, CatalogJanitor, CleanerChore, ClusterStatusChore, ClusterStatusPublisher, CompactedHFilesDischarger, ExpiredMobFileCleanerChore, HealthCheckChore, HeapMemoryManager.HeapMemoryTunerChore,
 HMaster.PeriodicDoMetrics, 
HRegionServer.CompactionChecker, HRegionServer.MovedRegionsCleaner, HRegionServer.PeriodicMemstoreFlusher,
 MobCompactionChore, QuotaCache.QuotaRefresherChore, RegionNormalizerChore, ReplicationMetaCleaner, ReplicationZKNodeCleanerChore, SplitLogManager.TimeoutMonitor, StorefileRefresherChore
+BalancerChore, CatalogJanitor, CleanerChore, ClusterStatusChore, ClusterStatusPublisher, CompactedHFilesDischarger, ExpiredMobFileCleanerChore, FileSystemUtilizationChore, HealthCheckChore, HeapMemoryManager.HeapMemoryTunerChore,
 HMaster.PeriodicDoMetrics, 
HRegionServer.CompactionChecker, HRegionServer.MovedRegionsCleaner, HRegionServer.PeriodicMemstoreFlusher,
 MobCompactionChore, QuotaCache.QuotaRefresherChore, QuotaObserverChore, RegionNormaliz
 erChore, ReplicationMetaCleaner, ReplicationZKNodeCleanerChore, SpaceQuotaRefresherChore, 
SplitLogManager.TimeoutMonitor, StorefileRefresherChore
 
 
 

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/TableName.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/TableName.html 
b/devapidocs/org/apache/hadoop/hbase/TableName.html
index 398962d..0a5b8fc 100644
--- a/devapidocs/org/apache/hadoop/hbase/TableName.html
+++ b/devapidocs/org/apache/hadoop/hbase/TableName.html
@@ -374,7 +374,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/lang/Comparabl
 isLegalNamespaceName(byte[] namespaceName,
 int start,
 int end)
-Valid namespace characters are [a-zA-Z_0-9]
+Valid namespace characters are alphabetic characters, 
numbers, and underscores.
 
 
 
@@ -832,7 +832,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/lang/Comparabl
 public static void isLegalNamespaceName(byte[] namespaceName,
 int start,
 int end)
-Valid namespace characters are [a-zA-Z_0-9]
+Valid namespace characters are alphabetic characters, 
numbers, and underscores.
 
 
 

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/backup/BackupDriver.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/backup/BackupDriver.html 
b/devapidocs/org/apache/hadoop/hbase/backup/BackupDriver.html
index 3a51cbe..e533dc6 100644
--- a/devapidocs/org/apache/hadoop/hbase/backup/BackupDriver.html
+++ b/devapidocs/org/apache/hadoop/hbase/backup/BackupDriver.html
@@ -337,7 +337,7 @@ extends 
 
 addOptions
-protected void addOptions()
+protected 

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

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/apidocs/src-html/org/apache/hadoop/hbase/quotas/SpaceLimitingException.html
--
diff --git 
a/apidocs/src-html/org/apache/hadoop/hbase/quotas/SpaceLimitingException.html 
b/apidocs/src-html/org/apache/hadoop/hbase/quotas/SpaceLimitingException.html
new file mode 100644
index 000..7a52c0e
--- /dev/null
+++ 
b/apidocs/src-html/org/apache/hadoop/hbase/quotas/SpaceLimitingException.html
@@ -0,0 +1,167 @@
+http://www.w3.org/TR/html4/loose.dtd";>
+
+
+Source code
+
+
+
+
+001/*
+002 * Licensed to the Apache Software 
Foundation (ASF) under one or more
+003 * contributor license agreements.  See 
the NOTICE file distributed with
+004 * this work for additional information 
regarding copyright ownership.
+005 * The ASF licenses this file to you 
under the Apache License, Version 2.0
+006 * (the "License"); you may not use this 
file except in compliance with
+007 * the License.  You may obtain a copy of 
the License at
+008 *
+009 * 
http://www.apache.org/licenses/LICENSE-2.0
+010 *
+011 * Unless required by applicable law or 
agreed to in writing, software
+012 * distributed under the License is 
distributed on an "AS IS" BASIS,
+013 * WITHOUT WARRANTIES OR CONDITIONS OF 
ANY KIND, either express or implied.
+014 * See the License for the specific 
language governing permissions and
+015 * limitations under the License.
+016 */
+017package org.apache.hadoop.hbase.quotas;
+018
+019
+020import org.apache.commons.logging.Log;
+021import 
org.apache.commons.logging.LogFactory;
+022import 
org.apache.hadoop.hbase.classification.InterfaceAudience;
+023import 
org.apache.hadoop.hbase.classification.InterfaceStability;
+024
+025/**
+026 * An Exception that is thrown when a 
space quota is in violation.
+027 */
+028@InterfaceAudience.Public
+029@InterfaceStability.Evolving
+030public class SpaceLimitingException 
extends QuotaExceededException {
+031  private static final long 
serialVersionUID = 2319438922387583600L;
+032  private static final Log LOG = 
LogFactory.getLog(SpaceLimitingException.class);
+033  private static final String 
MESSAGE_PREFIX = SpaceLimitingException.class.getName() + ": ";
+034
+035  private final String policyName;
+036
+037  public SpaceLimitingException(String 
msg) {
+038super(parseMessage(msg));
+039
+040// Hack around ResponseConverter 
expecting to invoke a single-arg String constructor
+041// on this class
+042if (msg != null) {
+043  for (SpaceViolationPolicy 
definedPolicy : SpaceViolationPolicy.values()) {
+044if 
(msg.indexOf(definedPolicy.name()) != -1) {
+045  policyName = 
definedPolicy.name();
+046  return;
+047}
+048  }
+049}
+050policyName = null;
+051  }
+052
+053  public SpaceLimitingException(String 
policyName, String msg) {
+054super(msg);
+055this.policyName = policyName;
+056  }
+057
+058  public SpaceLimitingException(String 
policyName, String msg, Throwable e) {
+059super(msg, e);
+060this.policyName = policyName;
+061  }
+062
+063  /**
+064   * Returns the violation policy in 
effect.
+065   *
+066   * @return The violation policy in 
effect.
+067   */
+068  public String getViolationPolicy() {
+069return this.policyName;
+070  }
+071
+072  private static String 
parseMessage(String originalMessage) {
+073// Serialization of the exception 
places a duplicate class name. Try to strip that off if it
+074// exists. Best effort... Looks 
something like:
+075// 
"org.apache.hadoop.hbase.quotas.SpaceLimitingException: NO_INSERTS A Put is 
disallowed due
+076// to a space quota."
+077if (originalMessage != null 
&& originalMessage.startsWith(MESSAGE_PREFIX)) {
+078  // If it starts with the class 
name, rip off the policy too.
+079  try {
+080int index = 
originalMessage.indexOf(' ', MESSAGE_PREFIX.length());
+081return 
originalMessage.substring(index + 1);
+082  } catch (Exception e) {
+083if (LOG.isTraceEnabled()) {
+084  LOG.trace("Failed to trim 
exception message", e);
+085}
+086  }
+087}
+088return originalMessage;
+089  }
+090
+091  @Override
+092  public String getMessage() {
+093return (policyName == null ? 
"(unknown policy)" : policyName) + " " + super.getMessage();
+094  }
+095}
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/apidocs/src-html/org/apache/hadoop/hbase/quotas/SpaceViolationPolicy.html
--
diff --git 
a/apidocs/src-html/org/apache/hadoop/hbase/quotas/SpaceViolationPolicy.html 
b/apidocs/src-html/org/apache/hadoop/hbase/quotas/SpaceViolationPolicy.html
new file mode 100644
index 000..1b919df
--- /dev/null
+++ b/apidocs/src-html/org/apa

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

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.html
 
b/devapidocs/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.html
index 3dc0a70..b9ec622 100644
--- 
a/devapidocs/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.html
+++ 
b/devapidocs/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.html
@@ -74,7 +74,7 @@ var activeTableTab = "activeTableTab";
 
 
 Summary: 
-Nested | 
+Nested | 
 Field | 
 Constr | 
 Method
@@ -115,7 +115,7 @@ var activeTableTab = "activeTableTab";
 
 
 @InterfaceAudience.Private
-public class IncrementalTableBackupClient
+public class IncrementalTableBackupClient
 extends TableBackupClient
 Incremental backup implementation.
  See the execute
 method.
@@ -125,6 +125,21 @@ extends 
 
 
+
+
+
+
+
+Nested Class Summary
+
+
+
+
+Nested classes/interfaces inherited from 
class org.apache.hadoop.hbase.backup.impl.TableBackupClient
+TableBackupClient.Stage
+
+
+
 
 
 
@@ -147,7 +162,7 @@ extends TableBackupClient
-BACKUP_CLIENT_IMPL_CLASS,
 backupId,
 backupInfo,
 backupManager,
 conf,
 conn,
 newTimestamps,
 tableList
+BACKUP_CLIENT_IMPL_CLASS,
 BACKUP_TEST_MODE_STAGE,
 backupId,
 backupInfo,
 backupManager,
 conf,
 conn,
 newTimestamps,
 tableList
 
 
 
@@ -160,10 +175,16 @@ extends 
 Constructors 
 
-Constructor and Description
+Modifier
+Constructor and Description
 
 
-IncrementalTableBackupClient(Connection conn,
+protected 
+IncrementalTableBackupClient() 
+
+
+ 
+IncrementalTableBackupClient(Connection conn,
 http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String backupId,
 BackupRequest request) 
 
@@ -243,7 +264,7 @@ extends TableBackupClient
-addManifest,
 beginBackup,
 cleanupAndRestoreBackupSystem,
 cleanupDistCpLog,
 cleanupExportSnapshotLog,
 cleanupTargetDir,
 completeBackup,
 deleteBackupTableSnapshot,
 deleteSnapshots,
 failBackup,
 getMessage,
 init,
 obtainBackupMetaDataStr, restoreBackupTable,
 snapshotBackupTable,
 snapshotExists
+addManifest,
 beginBackup,
 cleanupAndRestoreBackupSystem,
 cleanupDistCpLog,
 cleanupExportSnapshotLog,
 cleanupTargetDir,
 completeBackup,
 deleteBackupTableSnapshot,
 deleteSnapshots,
 failBackup,
 failStageIf,
 getMessage,
 getTestStage,
 init,
 obtainBackupMetaDataStr,
 restoreBackupTable,
 snapshotBackupTable,
 snapshotExists
 
 
 
@@ -272,7 +293,7 @@ extends 
 
 LOG
-private static final org.apache.commons.logging.Log LOG
+private static final org.apache.commons.logging.Log LOG
 
 
 
@@ -283,13 +304,22 @@ extends 
+
+
+
+
+IncrementalTableBackupClient
+protected IncrementalTableBackupClient()
+
+
 
 
 
 
 
 IncrementalTableBackupClient
-public IncrementalTableBackupClient(Connection conn,
+public IncrementalTableBackupClient(Connection conn,
 http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String backupId,
 BackupRequest request)
  throws http://docs.oracle.com/javase/8/docs/api/java/io/IOException.html?is-external=true";
 title="class or interface in java.io">IOException
@@ -313,7 +343,7 @@ extends 
 
 filterMissingFiles
-protected http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">ListString> filterMissingFiles(http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">ListString> incrBackupFileList)
+protected http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">ListString> filterMissingFiles(http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">ListString> incrBackupFileList)
throws http://docs.oracle.com/javase/8/docs/api/java/io/IOException.html?is-external=true";
 title="class or interface in java.io">IOException
 
 Throws:
@@ -327,7 +357,7 @@ extends 
 
 isActiveWalPath
-protected boolean isActiveWalPath(org.apache.hadoop.fs.Path p)
+protect

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

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/quotas/ActivePolicyEnforcement.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/quotas/ActivePolicyEnforcement.html 
b/devapidocs/org/apache/hadoop/hbase/quotas/ActivePolicyEnforcement.html
new file mode 100644
index 000..b31ef0d
--- /dev/null
+++ b/devapidocs/org/apache/hadoop/hbase/quotas/ActivePolicyEnforcement.html
@@ -0,0 +1,475 @@
+http://www.w3.org/TR/html4/loose.dtd";>
+
+
+
+
+
+ActivePolicyEnforcement (Apache HBase 2.0.0-SNAPSHOT API)
+
+
+
+
+
+var methods = {"i0":10,"i1":10,"i2":10,"i3":10,"i4":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
+
+
+
+
+Prev Class
+Next Class
+
+
+Frames
+No Frames
+
+
+All Classes
+
+
+
+
+
+
+
+Summary: 
+Nested | 
+Field | 
+Constr | 
+Method
+
+
+Detail: 
+Field | 
+Constr | 
+Method
+
+
+
+
+
+
+
+
+org.apache.hadoop.hbase.quotas
+Class 
ActivePolicyEnforcement
+
+
+
+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.quotas.ActivePolicyEnforcement
+
+
+
+
+
+
+
+
+@InterfaceAudience.Private
+ @InterfaceStability.Evolving
+public class ActivePolicyEnforcement
+extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true";
 title="class or interface in java.lang">Object
+A class to ease dealing with tables that have and do not 
have violation policies
+ being enforced. This class is immutable, expect for 
locallyCachedPolicies.
+
+ The locallyCachedPolicies are mutable given the current 
activePolicies
+ and snapshots. It is expected that when a new instance of this 
class is
+ instantiated, we also want to invalidate those previously cached policies (as 
they
+ may now be invalidate if we received new quota usage information).
+
+
+
+
+
+
+
+
+
+
+
+Field Summary
+
+Fields 
+
+Modifier and Type
+Field and Description
+
+
+private http://docs.oracle.com/javase/8/docs/api/java/util/Map.html?is-external=true";
 title="class or interface in java.util">Map
+activePolicies 
+
+
+private SpaceViolationPolicyEnforcementFactory
+factory 
+
+
+private http://docs.oracle.com/javase/8/docs/api/java/util/Map.html?is-external=true";
 title="class or interface in java.util">Map
+locallyCachedPolicies 
+
+
+private RegionServerServices
+rss 
+
+
+private http://docs.oracle.com/javase/8/docs/api/java/util/Map.html?is-external=true";
 title="class or interface in java.util">Map
+snapshots 
+
+
+
+
+
+
+
+
+
+Constructor Summary
+
+Constructors 
+
+Constructor and Description
+
+
+ActivePolicyEnforcement(http://docs.oracle.com/javase/8/docs/api/java/util/Map.html?is-external=true";
 title="class or interface in java.util">Map activePolicies,
+   http://docs.oracle.com/javase/8/docs/api/java/util/Map.html?is-external=true";
 title="class or interface in java.util">Map snapshots,
+   RegionServerServices rss) 
+
+
+ActivePolicyEnforcement(http://docs.oracle.com/javase/8/docs/api/java/util/Map.html?is-external=true";
 title="class or interface in java.util">Map activePolicies,
+   http://docs.oracle.com/javase/8/docs/api/java/util/Map.html?is-external=true";
 title="class or interface in java.util">Map snapshots,
+   RegionServerServices rss,
+   SpaceViolationPolicyEnforcementFactory factory) 
+
+
+
+
+
+
+
+
+
+Method Summary
+
+All Methods Instance Methods Concrete Methods 
+
+Modifier and Type
+Method and Description
+
+
+(package private) http://docs.oracle.com/javase/8/docs/api/java/util/Map.html?is-external=true";
 title="class or interface in java.util">Map
+getLocallyCachedPolicies()
+Returns an unmodifiable version of the policy enforcements 
that were cached because they are
+ not in violation of their 

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

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/client/ConnectionImplementation.MasterServiceState.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/client/ConnectionImplementation.MasterServiceState.html
 
b/devapidocs/org/apache/hadoop/hbase/client/ConnectionImplementation.MasterServiceState.html
index 85f90d1..85d7b38 100644
--- 
a/devapidocs/org/apache/hadoop/hbase/client/ConnectionImplementation.MasterServiceState.html
+++ 
b/devapidocs/org/apache/hadoop/hbase/client/ConnectionImplementation.MasterServiceState.html
@@ -113,7 +113,7 @@ var activeTableTab = "activeTableTab";
 
 
 
-static class ConnectionImplementation.MasterServiceState
+static class ConnectionImplementation.MasterServiceState
 extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true";
 title="class or interface in java.lang">Object
 State of the MasterService connection/setup.
 
@@ -222,7 +222,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 connection
-Connection connection
+Connection connection
 
 
 
@@ -231,7 +231,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 stub
-org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService.BlockingInterface
 stub
+org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService.BlockingInterface
 stub
 
 
 
@@ -240,7 +240,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 userCount
-int userCount
+int userCount
 
 
 
@@ -257,7 +257,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 MasterServiceState
-MasterServiceState(Connection connection)
+MasterServiceState(Connection connection)
 
 
 
@@ -274,7 +274,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 toString
-public http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String toString()
+public http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String toString()
 
 Overrides:
 http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true#toString--";
 title="class or interface in java.lang">toString in 
class http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true";
 title="class or interface in java.lang">Object
@@ -287,7 +287,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 getStub
-http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true";
 title="class or interface in java.lang">Object getStub()
+http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true";
 title="class or interface in java.lang">Object getStub()
 
 
 
@@ -296,7 +296,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 clearStub
-void clearStub()
+void clearStub()
 
 
 
@@ -305,7 +305,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 isMasterRunning
-boolean isMasterRunning()
+boolean isMasterRunning()
  throws http://docs.oracle.com/javase/8/docs/api/java/io/IOException.html?is-external=true";
 title="class or interface in java.io">IOException
 
 Throws:

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/client/ConnectionImplementation.MasterServiceStubMaker.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/client/ConnectionImplementation.MasterServiceStubMaker.html
 
b/devapidocs/org/apache/hadoop/hbase/client/ConnectionImplementation.MasterServiceStubMaker.html
index 0a8e5ea..846a73b 100644
--- 
a/devapidocs/org/apache/hadoop/hbase/client/ConnectionImplementation.MasterServiceStubMaker.html
+++ 
b/devapidocs/org/apache/hadoop/hbase/client/ConnectionImplementation.MasterServiceStubMaker.html
@@ -113,7 +113,7 @@ var activeTableTab = "activeTableTab";
 
 
 
-private final class ConnectionImplementation.MasterServiceStubMaker
+private final class ConnectionImplementation.MasterServiceStubMaker
 extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true";
 title="class or interface in java.lang">Object
 Class to make a MasterServiceStubMaker stub.
 
@@ -197,7 +197,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 MasterServiceStubMaker
-private MasterServiceStubMaker()
+private MasterServiceStubMaker()
 
 
 
@@ -214,7 +214,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 isMasterRunning
-private void isMasterRunning(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService.BlockingInterface stub)
+private void isMasterRunning(org.apache.hadoop.hbase.shaded.protobuf.ge

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

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/master/MetricsMaster.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/master/MetricsMaster.html 
b/devapidocs/org/apache/hadoop/hbase/master/MetricsMaster.html
index c2fd82e..099dbd7 100644
--- a/devapidocs/org/apache/hadoop/hbase/master/MetricsMaster.html
+++ b/devapidocs/org/apache/hadoop/hbase/master/MetricsMaster.html
@@ -18,7 +18,7 @@
 catch(err) {
 }
 //-->
-var methods = {"i0":10,"i1":10,"i2":10};
+var methods = 
{"i0":10,"i1":10,"i2":10,"i3":10,"i4":10,"i5":10,"i6":10,"i7":10,"i8":10};
 var tabs = {65535:["t0","All Methods"],2:["t2","Instance 
Methods"],8:["t4","Concrete Methods"]};
 var altColor = "altColor";
 var rowColor = "rowColor";
@@ -145,6 +145,10 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 masterProcSource 
 
 
+private MetricsMasterQuotaSource
+masterQuotaSource 
+
+
 private MetricsMasterSource
 masterSource 
 
@@ -185,13 +189,47 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 getMetricsProcSource() 
 
 
+MetricsMasterQuotaSource
+getMetricsQuotaSource() 
+
+
 MetricsMasterSource
 getMetricsSource() 
 
-
+
+void
+incrementQuotaObserverTime(long executionTime)
+Sets the execution time of a period of the 
QuotaObserverChore.
+
+
+
 void
 incrementRequests(long inc) 
 
+
+void
+setNumNamespacesInSpaceQuotaViolation(long numNamespacesInViolation)
+Sets the number of namespaces in violation of a space 
quota.
+
+
+
+void
+setNumRegionSizeReports(long numRegionReports)
+Sets the number of region size reports the master currently 
has in memory.
+
+
+
+void
+setNumSpaceQuotas(long numSpaceQuotas)
+Sets the number of space quotas defined.
+
+
+
+void
+setNumTableInSpaceQuotaViolation(long numTablesInViolation)
+Sets the number of table in violation of a space 
quota.
+
+
 
 
 
@@ -235,12 +273,21 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 
-
+
 
 masterProcSource
 private MetricsMasterProcSource masterProcSource
 
 
+
+
+
+
+
+masterQuotaSource
+private MetricsMasterQuotaSource masterQuotaSource
+
+
 
 
 
@@ -255,7 +302,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 MetricsMaster
-public MetricsMaster(MetricsMasterWrapper masterWrapper)
+public MetricsMaster(MetricsMasterWrapper masterWrapper)
 
 
 
@@ -272,7 +319,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 getMetricsSource
-public MetricsMasterSource getMetricsSource()
+public MetricsMasterSource getMetricsSource()
 
 
 
@@ -281,22 +328,103 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 getMetricsProcSource
-public MetricsMasterProcSource getMetricsProcSource()
+public MetricsMasterProcSource getMetricsProcSource()
+
+
+
+
+
+
+
+getMetricsQuotaSource
+public MetricsMasterQuotaSource getMetricsQuotaSource()
 
 
 
 
 
-
+
 
 incrementRequests
-public void incrementRequests(long inc)
+public void incrementRequests(long inc)
 
 Parameters:
 inc - How much to add to requests.
 
 
 
+
+
+
+
+
+setNumSpaceQuotas
+public void setNumSpaceQuotas(long numSpaceQuotas)
+Sets the number of space quotas defined.
+
+See Also:
+MetricsMasterQuotaSource.updateNumSpaceQuotas(long)
+
+
+
+
+
+
+
+
+setNumTableInSpaceQuotaViolation
+public void setNumTableInSpaceQuotaViolation(long numTablesInViolation)
+Sets the number of table in violation of a space 
quota.
+
+See Also:
+MetricsMasterQuotaSource.updateNumTablesInSpaceQuotaViolation(long)
+
+
+
+
+
+
+
+
+setNumNamespacesInSpaceQuotaViolation
+public void setNumNamespacesInSpaceQuotaViolation(long numNamespacesInViolation)
+Sets the number of namespaces in violation of a space 
quota.
+
+See Also:
+MetricsMasterQuotaSource.updateNumNamespacesInSpaceQuotaViolation(long)
+
+
+
+
+
+
+
+
+setNumRegionSizeReports
+public void setNumRegionSizeReports(long numRegionReports)
+Sets the number of region size reports the master currently 
has in memory.
+
+See Also:
+MetricsMasterQuotaSource.updateNumCurrentSpaceQuotaRegionSizeReports(long)
+
+
+
+
+
+
+
+
+incrementQuotaObserverTime
+public void incrementQuotaObserverTime(long executionTime)
+Sets the execution time of a period of the 
QuotaObserverChore.
+
+Parameters:
+executionTime - The execution time in milliseconds.
+See Also:
+MetricsMasterQuotaSource.incrementSpaceQuotaObserverChoreTime(long)
+
+
+
 
 
 

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/master/MetricsMasterProcSourceImpl.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/master/MetricsMasterProcSourceImpl.html 
b/devapidocs/org/apache/hadoop/hbase/master/MetricsMasterProcSourceImpl.html
index 79f6ed7..6059339 100644
--- a/devapidocs/org/apache/hadoop/hbase/master/MetricsMasterProcSourceImpl.html
++

hbase-site git commit: INFRA-10751 Empty commit

Repository: hbase-site
Updated Branches:
  refs/heads/asf-site dab57116f -> 0d37a65d5


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

Branch: refs/heads/asf-site
Commit: 0d37a65d51dbef3f6fce5f05ebd4320d718b3d11
Parents: dab5711
Author: jenkins 
Authored: Tue May 23 15:01:28 2017 +
Committer: jenkins 
Committed: Tue May 23 15:01:28 2017 +

--

--




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

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/backup/impl/BackupCommands.CreateCommand.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/backup/impl/BackupCommands.CreateCommand.html
 
b/devapidocs/org/apache/hadoop/hbase/backup/impl/BackupCommands.CreateCommand.html
index 3e22b3c..afeafa2 100644
--- 
a/devapidocs/org/apache/hadoop/hbase/backup/impl/BackupCommands.CreateCommand.html
+++ 
b/devapidocs/org/apache/hadoop/hbase/backup/impl/BackupCommands.CreateCommand.html
@@ -127,7 +127,7 @@ var activeTableTab = "activeTableTab";
 
 
 
-public static class BackupCommands.CreateCommand
+public static class BackupCommands.CreateCommand
 extends BackupCommands.Command
 
 
@@ -245,7 +245,7 @@ extends 
 
 CreateCommand
-CreateCommand(org.apache.hadoop.conf.Configuration conf,
+CreateCommand(org.apache.hadoop.conf.Configuration conf,
   org.apache.commons.cli.CommandLine cmdline)
 
 
@@ -263,7 +263,7 @@ extends 
 
 requiresNoActiveSession
-protected boolean requiresNoActiveSession()
+protected boolean requiresNoActiveSession()
 Description copied from 
class: BackupCommands.Command
 The command can't be run if active backup session is in 
progress
 
@@ -280,7 +280,7 @@ extends 
 
 execute
-public void execute()
+public void execute()
  throws http://docs.oracle.com/javase/8/docs/api/java/io/IOException.html?is-external=true";
 title="class or interface in java.io">IOException
 
 Overrides:
@@ -296,7 +296,7 @@ extends 
 
 verifyPath
-private boolean verifyPath(http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String path)
+private boolean verifyPath(http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String path)
 
 
 
@@ -305,7 +305,7 @@ extends 
 
 getTablesForSet
-private http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String getTablesForSet(http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String name,
+private http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String getTablesForSet(http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String name,
org.apache.hadoop.conf.Configuration conf)
 throws http://docs.oracle.com/javase/8/docs/api/java/io/IOException.html?is-external=true";
 title="class or interface in java.io">IOException
 
@@ -320,7 +320,7 @@ extends 
 
 printUsage
-protected void printUsage()
+protected void printUsage()
 
 Specified by:
 printUsage in
 class BackupCommands.Command

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/backup/impl/BackupCommands.DeleteCommand.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/backup/impl/BackupCommands.DeleteCommand.html
 
b/devapidocs/org/apache/hadoop/hbase/backup/impl/BackupCommands.DeleteCommand.html
index 0d98fbd..cb2c25b 100644
--- 
a/devapidocs/org/apache/hadoop/hbase/backup/impl/BackupCommands.DeleteCommand.html
+++ 
b/devapidocs/org/apache/hadoop/hbase/backup/impl/BackupCommands.DeleteCommand.html
@@ -127,7 +127,7 @@ var activeTableTab = "activeTableTab";
 
 
 
-private static class BackupCommands.DeleteCommand
+private static class BackupCommands.DeleteCommand
 extends BackupCommands.Command
 
 
@@ -236,7 +236,7 @@ extends 
 
 DeleteCommand
-DeleteCommand(org.apache.hadoop.conf.Configuration conf,
+DeleteCommand(org.apache.hadoop.conf.Configuration conf,
   org.apache.commons.cli.CommandLine cmdline)
 
 
@@ -254,7 +254,7 @@ extends 
 
 requiresNoActiveSession
-protected boolean requiresNoActiveSession()
+protected boolean requiresNoActiveSession()
 Description copied from 
class: BackupCommands.Command
 The command can't be run if active backup session is in 
progress
 
@@ -271,7 +271,7 @@ extends 
 
 execute
-public void execute()
+public void execute()
  throws http://docs.oracle.com/javase/8/docs/api/java/io/IOException.html?is-external=true";
 title="class or interface in java.io">IOException
 
 Overrides:
@@ -287,7 +287,7 @@ extends 
 
 printUsage
-protected void printUsage()
+protected void printUsage()
 
 Specified by:
 printUsage in
 class BackupCommands.Command

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/backup/impl/BackupCommands.DescribeCommand.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/backup/impl/BackupCommands.DescribeCom

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

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/apidocs/src-html/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.html
--
diff --git 
a/apidocs/src-html/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.html
 
b/apidocs/src-html/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.html
index 7231169..61b88c6 100644
--- 
a/apidocs/src-html/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.html
+++ 
b/apidocs/src-html/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.html
@@ -31,468 +31,483 @@
 023import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest;
 024import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
 025import 
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.*;
-026import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequest;
-027import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerResponse;
-028import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerRequest;
-029import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerResponse;
-030import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerRequest;
-031import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerResponse;
-032import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigRequest;
-033import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigResponse;
-034import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersRequest;
-035import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse;
-036import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerRequest;
-037import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse;
-038import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest;
-039import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse;
-040
-041/**
-042 * A short-circuit connection that can 
bypass the RPC layer (serialization, deserialization,
-043 * networking, etc..) when talking to a 
local master
-044 */
-045@InterfaceAudience.Public
-046public class ShortCircuitMasterConnection 
implements MasterKeepAliveConnection {
-047
-048  private final 
MasterService.BlockingInterface stub;
-049
-050  public 
ShortCircuitMasterConnection(MasterService.BlockingInterface stub) {
-051this.stub = stub;
-052  }
+026import 
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest;
+027import 
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse;
+028import 
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest;
+029import 
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse;
+030import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequest;
+031import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerResponse;
+032import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerRequest;
+033import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerResponse;
+034import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerRequest;
+035import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerResponse;
+036import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigRequest;
+037import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigResponse;
+038import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersRequest;
+039import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse;
+040import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerRequest;
+041import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse;
+042import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest;
+043import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse;
+044
+045/**
+046 * A short-circuit connection that can 
bypass the RPC layer (s

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

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/client/ConnectionImplementation.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/client/ConnectionImplementation.html 
b/devapidocs/org/apache/hadoop/hbase/client/ConnectionImplementation.html
index a6ce1bc..c98e496 100644
--- a/devapidocs/org/apache/hadoop/hbase/client/ConnectionImplementation.html
+++ b/devapidocs/org/apache/hadoop/hbase/client/ConnectionImplementation.html
@@ -118,7 +118,7 @@ var activeTableTab = "activeTableTab";
 
 
 @InterfaceAudience.Private
-class ConnectionImplementation
+class ConnectionImplementation
 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 ClusterConnection, http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.html?is-external=true";
 title="class or interface in java.io">Closeable
 Main implementation of Connection 
and ClusterConnection interfaces.
@@ -857,7 +857,7 @@ implements 
 
 RETRIES_BY_SERVER_KEY
-public static final http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String RETRIES_BY_SERVER_KEY
+public static final http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String RETRIES_BY_SERVER_KEY
 
 See Also:
 Constant
 Field Values
@@ -870,7 +870,7 @@ implements 
 
 LOG
-private static final org.apache.commons.logging.Log LOG
+private static final org.apache.commons.logging.Log LOG
 
 
 
@@ -879,7 +879,7 @@ implements 
 
 RESOLVE_HOSTNAME_ON_FAIL_KEY
-private static final http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String RESOLVE_HOSTNAME_ON_FAIL_KEY
+private static final http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String RESOLVE_HOSTNAME_ON_FAIL_KEY
 
 See Also:
 Constant
 Field Values
@@ -892,7 +892,7 @@ implements 
 
 hostnamesCanChange
-private final boolean hostnamesCanChange
+private final boolean hostnamesCanChange
 
 
 
@@ -901,7 +901,7 @@ implements 
 
 pause
-private final long pause
+private final long pause
 
 
 
@@ -910,7 +910,7 @@ implements 
 
 pauseForCQTBE
-private final long pauseForCQTBE
+private final long pauseForCQTBE
 
 
 
@@ -919,7 +919,7 @@ implements 
 
 useMetaReplicas
-private boolean useMetaReplicas
+private boolean useMetaReplicas
 
 
 
@@ -928,7 +928,7 @@ implements 
 
 metaReplicaCallTimeoutScanInMicroSecond
-private final int metaReplicaCallTimeoutScanInMicroSecond
+private final int metaReplicaCallTimeoutScanInMicroSecond
 
 
 
@@ -937,7 +937,7 @@ implements 
 
 numTries
-private final int numTries
+private final int numTries
 
 
 
@@ -946,7 +946,7 @@ implements 
 
 rpcTimeout
-final int rpcTimeout
+final int rpcTimeout
 
 
 
@@ -955,7 +955,7 @@ implements 
 
 nonceGenerator
-private static volatile NonceGenerator nonceGenerator
+private static volatile NonceGenerator nonceGenerator
 Global nonceGenerator shared per client.Currently there's 
no reason to limit its scope.
  Once it's set under nonceGeneratorCreateLock, it is never unset or 
changed.
 
@@ -966,7 +966,7 @@ implements 
 
 nonceGeneratorCreateLock
-private static final http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true";
 title="class or interface in java.lang">Object nonceGeneratorCreateLock
+private static final http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true";
 title="class or interface in java.lang">Object nonceGeneratorCreateLock
 The nonce generator lock. Only taken when creating 
Connection, which gets a private copy.
 
 
@@ -976,7 +976,7 @@ implements 
 
 asyncProcess
-private final AsyncProcess asyncProcess
+private final AsyncProcess asyncProcess
 
 
 
@@ -985,7 +985,7 @@ implements 
 
 stats
-private final ServerStatisticTracker stats
+private final ServerStatisticTracker stats
 
 
 
@@ -994,7 +994,7 @@ implements 
 
 closed
-private volatile boolean closed
+private volatile boolean closed
 
 
 
@@ -1003,7 +1003,7 @@ implements 
 
 aborted
-private volatile boolean aborted
+private volatile boolean aborted
 
 
 
@@ -1012,7 +1012,7 @@ implements 
 
 clusterStatusListener
-ClusterStatusListener clusterStatusListener
+ClusterStatusListener clusterStatusListener
 
 
 
@@ -1021,7 +1021,7 @@ implements 
 
 metaRegionLock
-private final http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true";
 title="class or interface in java.lang">Object metaRegionLock
+private final http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true";
 title="class or interface in java.lang">Object metaRegionLock
 
 
 
@@ -1030,7 +1030,7 @@ implements 
 
 masterAndZKLock
-private final http://docs.oracle.c

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

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/ipc/NettyRpcServer.MessageEncoder.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/ipc/NettyRpcServer.MessageEncoder.html 
b/devapidocs/org/apache/hadoop/hbase/ipc/NettyRpcServer.MessageEncoder.html
deleted file mode 100644
index 0f15520..000
--- a/devapidocs/org/apache/hadoop/hbase/ipc/NettyRpcServer.MessageEncoder.html
+++ /dev/null
@@ -1,338 +0,0 @@
-http://www.w3.org/TR/html4/loose.dtd";>
-
-
-
-
-
-NettyRpcServer.MessageEncoder (Apache HBase 2.0.0-SNAPSHOT API)
-
-
-
-
-
-var methods = {"i0":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
-
-
-
-
-Prev Class
-Next Class
-
-
-Frames
-No Frames
-
-
-All Classes
-
-
-
-
-
-
-
-Summary: 
-Nested | 
-Field | 
-Constr | 
-Method
-
-
-Detail: 
-Field | 
-Constr | 
-Method
-
-
-
-
-
-
-
-
-org.apache.hadoop.hbase.ipc
-Class 
NettyRpcServer.MessageEncoder
-
-
-
-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
-
-
-io.netty.channel.ChannelHandlerAdapter
-
-
-io.netty.channel.ChannelOutboundHandlerAdapter
-
-
-org.apache.hadoop.hbase.ipc.NettyRpcServer.MessageEncoder
-
-
-
-
-
-
-
-
-
-
-
-All Implemented Interfaces:
-io.netty.channel.ChannelHandler, 
io.netty.channel.ChannelOutboundHandler
-
-
-Enclosing class:
-NettyRpcServer
-
-
-
-private class NettyRpcServer.MessageEncoder
-extends io.netty.channel.ChannelOutboundHandlerAdapter
-
-
-
-
-
-
-
-
-
-
-
-Nested Class Summary
-
-
-
-
-Nested classes/interfaces inherited from 
interface io.netty.channel.ChannelHandler
-io.netty.channel.ChannelHandler.Sharable
-
-
-
-
-
-
-
-
-Constructor Summary
-
-Constructors 
-
-Modifier
-Constructor and Description
-
-
-private 
-MessageEncoder() 
-
-
-
-
-
-
-
-
-
-Method Summary
-
-All Methods Instance Methods Concrete Methods 
-
-Modifier and Type
-Method and Description
-
-
-void
-write(io.netty.channel.ChannelHandlerContext ctx,
- http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true";
 title="class or interface in java.lang">Object msg,
- io.netty.channel.ChannelPromise promise) 
-
-
-
-
-
-
-Methods inherited from 
class io.netty.channel.ChannelOutboundHandlerAdapter
-bind, close, connect, deregister, disconnect, flush, read
-
-
-
-
-
-Methods inherited from 
class io.netty.channel.ChannelHandlerAdapter
-exceptionCaught, handlerAdded, handlerRemoved, isSharable
-
-
-
-
-
-Methods inherited from class 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
-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, http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-ext

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

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/master/MetricsMasterWrapperImpl.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/master/MetricsMasterWrapperImpl.html 
b/devapidocs/org/apache/hadoop/hbase/master/MetricsMasterWrapperImpl.html
index d2e7265..b34b28b 100644
--- a/devapidocs/org/apache/hadoop/hbase/master/MetricsMasterWrapperImpl.html
+++ b/devapidocs/org/apache/hadoop/hbase/master/MetricsMasterWrapperImpl.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};
+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,"i16":10,"i17":10};
 var tabs = {65535:["t0","All Methods"],2:["t2","Instance 
Methods"],8:["t4","Concrete Methods"]};
 var altColor = "altColor";
 var rowColor = "rowColor";
@@ -114,7 +114,7 @@ var activeTableTab = "activeTableTab";
 
 
 @InterfaceAudience.Private
-public class MetricsMasterWrapperImpl
+public class MetricsMasterWrapperImpl
 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 MetricsMasterWrapper
 Impl for exposing HMaster Information through JMX
@@ -173,90 +173,106 @@ implements Method and Description
 
 
+(package private) http://docs.oracle.com/javase/8/docs/api/java/util/Map.Entry.html?is-external=true";
 title="class or interface in java.util">Map.EntryLong,http://docs.oracle.com/javase/8/docs/api/java/lang/Long.html?is-external=true";
 title="class or interface in java.lang">Long>
+convertSnapshot(SpaceQuotaSnapshot snapshot) 
+
+
 long
 getActiveTime()
 Get the hbase master active time
 
 
-
+
 double
 getAverageLoad()
 Get Average Load
 
 
-
+
 http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String
 getClusterId()
 Get the Cluster ID
 
 
-
+
 http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String[]
 getCoprocessors()
 Get the co-processors
 
 
-
+
 http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String
 getDeadRegionServers()
 Get the dead region servers
 
 
-
+
 boolean
 getIsActiveMaster()
 Whether this master is the active master
 
 
-
+
 long
 getMergePlanCount()
 Get the number of region merge plans executed.
 
 
-
+
+http://docs.oracle.com/javase/8/docs/api/java/util/Map.html?is-external=true";
 title="class or interface in java.util">MapString,http://docs.oracle.com/javase/8/docs/api/java/util/Map.Entry.html?is-external=true";
 title="class or interface in java.util">Map.EntryLong,http://docs.oracle.com/javase/8/docs/api/java/lang/Long.html?is-external=true";
 title="class or interface in java.lang">Long>>
+getNamespaceSpaceUtilization()
+Gets the space usage and limit for each namespace.
+
+
+
 int
 getNumDeadRegionServers()
 Get the number of dead region servers
 
 
-
+
 int
 getNumRegionServers()
 Get the number of live region servers
 
 
-
+
 long
 getNumWALFiles()
 Get the number of master WAL files.
 
 
-
+
 http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String
 getRegionServers()
 Get the live region servers
 
 
-
+
 http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String
 getServerName()
 Get ServerName
 
 
-
+
 long
 getSplitPlanCount()
 Get the number of region split plans executed.
 
 
-
+
 long
 getStartTime()
 Get hbase master start time
 
 
-
+
+http://docs.oracle.com/javase/8/docs/api/java/util/Map.html?is-external=true";
 title="class or interface in java.util">MapString,http://docs.oracle.com/javase/8/docs/api/java/util/Map.Entry.html?is-external=true";
 title="class or interface in java.util">Map.EntryLong,http://docs.oracle.com/javase/8/docs/api/java/lang/Long.html?is-external=true";
 title="class or interface in java.lang">Long>>
+getTableSpaceUtilization()
+Gets the space usage and limit for each table.
+

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

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/org/apache/hadoop/hbase/ipc/NettyRpcServer.CallWriteListener.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/ipc/NettyRpcServer.CallWriteListener.html 
b/devapidocs/org/apache/hadoop/hbase/ipc/NettyRpcServer.CallWriteListener.html
deleted file mode 100644
index 9cc7e38..000
--- 
a/devapidocs/org/apache/hadoop/hbase/ipc/NettyRpcServer.CallWriteListener.html
+++ /dev/null
@@ -1,331 +0,0 @@
-http://www.w3.org/TR/html4/loose.dtd";>
-
-
-
-
-
-NettyRpcServer.CallWriteListener (Apache HBase 2.0.0-SNAPSHOT 
API)
-
-
-
-
-
-var methods = {"i0":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
-
-
-
-
-Prev Class
-Next Class
-
-
-Frames
-No Frames
-
-
-All Classes
-
-
-
-
-
-
-
-Summary: 
-Nested | 
-Field | 
-Constr | 
-Method
-
-
-Detail: 
-Field | 
-Constr | 
-Method
-
-
-
-
-
-
-
-
-org.apache.hadoop.hbase.ipc
-Class 
NettyRpcServer.CallWriteListener
-
-
-
-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.ipc.NettyRpcServer.CallWriteListener
-
-
-
-
-
-
-
-All Implemented Interfaces:
-io.netty.channel.ChannelFutureListener, 
io.netty.util.concurrent.GenericFutureListener,
 http://docs.oracle.com/javase/8/docs/api/java/util/EventListener.html?is-external=true";
 title="class or interface in java.util">EventListener
-
-
-Enclosing class:
-NettyRpcServer
-
-
-
-private class NettyRpcServer.CallWriteListener
-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 io.netty.channel.ChannelFutureListener
-
-
-
-
-
-
-
-
-
-
-
-Field Summary
-
-Fields 
-
-Modifier and Type
-Field and Description
-
-
-private NettyServerCall
-call 
-
-
-
-
-
-
-Fields inherited from 
interface io.netty.channel.ChannelFutureListener
-CLOSE, CLOSE_ON_FAILURE, FIRE_EXCEPTION_ON_FAILURE
-
-
-
-
-
-
-
-
-Constructor Summary
-
-Constructors 
-
-Constructor and Description
-
-
-CallWriteListener(NettyServerCall call) 
-
-
-
-
-
-
-
-
-
-Method Summary
-
-All Methods Instance Methods Concrete Methods 
-
-Modifier and Type
-Method and Description
-
-
-void
-operationComplete(io.netty.channel.ChannelFuture future) 
-
-
-
-
-
-
-Methods inherited from class 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
-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, http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true#wait-long-int-";
 title="class or interface in java.lang">wait
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-Field Detail
-
-
-
-
-
-

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

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/devapidocs/index-all.html
--
diff --git a/devapidocs/index-all.html b/devapidocs/index-all.html
index aeb7adf..91644d7 100644
--- a/devapidocs/index-all.html
+++ b/devapidocs/index-all.html
@@ -452,6 +452,12 @@
  
 AbstractStateMachineTableProcedure(MasterProcedureEnv,
 ProcedurePrepareLatch) - Constructor for class 
org.apache.hadoop.hbase.master.procedure.AbstractStateMachineTableProcedure
  
+AbstractViolationPolicyEnforcement - Class in 
org.apache.hadoop.hbase.quotas.policies
+
+Abstract implementation for SpaceViolationPolicyEnforcement.
+
+AbstractViolationPolicyEnforcement()
 - Constructor for class org.apache.hadoop.hbase.quotas.policies.AbstractViolationPolicyEnforcement
+ 
 accept(Path)
 - Method in class org.apache.hadoop.hbase.backup.impl.IncrementalBackupManager.NewestLogFilter
  
 accept(Void,
 Throwable) - Method in class org.apache.hadoop.hbase.client.AsyncHBaseAdmin.ProcedureBiConsumer
@@ -829,6 +835,17 @@
  
 activeOperations
 - Variable in class org.apache.hadoop.hbase.regionserver.throttle.PressureAwareThroughputController
  
+activePolicies
 - Variable in class org.apache.hadoop.hbase.quotas.ActivePolicyEnforcement
+ 
+ActivePolicyEnforcement - Class in org.apache.hadoop.hbase.quotas
+
+A class to ease dealing with tables that have and do not 
have violation policies
+ being enforced.
+
+ActivePolicyEnforcement(Map, Map, 
RegionServerServices) - Constructor for class 
org.apache.hadoop.hbase.quotas.ActivePolicyEnforcement
+ 
+ActivePolicyEnforcement(Map, Map, 
RegionServerServices, SpaceViolationPolicyEnforcementFactory) - 
Constructor for class org.apache.hadoop.hbase.quotas.ActivePolicyEnforcement
+ 
 activeReadHandlerCount
 - Variable in class org.apache.hadoop.hbase.ipc.RWQueueRpcExecutor
  
 activeScanHandlerCount
 - Variable in class org.apache.hadoop.hbase.ipc.RWQueueRpcExecutor
@@ -1936,6 +1953,10 @@
  
 addNamespaceQuota(Connection,
 String, QuotaProtos.Quotas) - Static method in class 
org.apache.hadoop.hbase.quotas.QuotaUtil
  
+addNamespaceQuotaTable(TableName)
 - Method in class org.apache.hadoop.hbase.quotas.QuotaObserverChore.TablesWithQuotas
+
+Adds a table with a namespace quota.
+
 addNewMetricIfAbsent(String,
 T, Class) - Method in class 
org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry
  
 addNewTableEntryInMap(Map) - Static method in class 
org.apache.hadoop.hbase.master.procedure.CreateTableProcedure
@@ -2746,6 +2775,8 @@
  
 allChannels
 - Variable in class org.apache.hadoop.hbase.ipc.NettyRpcServer
  
+allChannels
 - Variable in class org.apache.hadoop.hbase.ipc.NettyRpcServerRequestDecoder
+ 
 allCompactedFilesCached
 - Variable in class org.apache.hadoop.hbase.regionserver.StripeStoreFileManager.State
  
 allFiles
 - Variable in class org.apache.hadoop.hbase.master.MasterMobCompactionThread.CompactionRunner
@@ -2983,7 +3014,7 @@
 
 Perform one or more append operations on a row.
 
-append(Region,
 OperationQuota, ClientProtos.MutationProto, CellScanner, long) - 
Method in class org.apache.hadoop.hbase.regionserver.RSRpcServices
+append(Region,
 OperationQuota, ClientProtos.MutationProto, CellScanner, long, 
ActivePolicyEnforceme

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

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/dab57116/apidocs/org/apache/hadoop/hbase/quotas/SpaceViolationPolicy.html
--
diff --git a/apidocs/org/apache/hadoop/hbase/quotas/SpaceViolationPolicy.html 
b/apidocs/org/apache/hadoop/hbase/quotas/SpaceViolationPolicy.html
new file mode 100644
index 000..9adcfc7
--- /dev/null
+++ b/apidocs/org/apache/hadoop/hbase/quotas/SpaceViolationPolicy.html
@@ -0,0 +1,383 @@
+http://www.w3.org/TR/html4/loose.dtd";>
+
+
+
+
+
+SpaceViolationPolicy (Apache HBase 2.0.0-SNAPSHOT API)
+
+
+
+
+
+var methods = {"i0":9,"i1":9};
+var tabs = {65535:["t0","All Methods"],1:["t1","Static 
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
+
+
+
+
+Prev Class
+Next Class
+
+
+Frames
+No Frames
+
+
+All Classes
+
+
+
+
+
+
+
+Summary: 
+Nested | 
+Enum Constants | 
+Field | 
+Method
+
+
+Detail: 
+Enum Constants | 
+Field | 
+Method
+
+
+
+
+
+
+
+
+org.apache.hadoop.hbase.quotas
+Enum 
SpaceViolationPolicy
+
+
+
+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
+
+
+http://docs.oracle.com/javase/8/docs/api/java/lang/Enum.html?is-external=true";
 title="class or interface in java.lang">java.lang.Enum
+
+
+org.apache.hadoop.hbase.quotas.SpaceViolationPolicy
+
+
+
+
+
+
+
+
+
+All Implemented Interfaces:
+http://docs.oracle.com/javase/8/docs/api/java/io/Serializable.html?is-external=true";
 title="class or interface in java.io">Serializable, http://docs.oracle.com/javase/8/docs/api/java/lang/Comparable.html?is-external=true";
 title="class or interface in java.lang">Comparable
+
+
+
+@InterfaceAudience.Public
+public enum SpaceViolationPolicy
+extends http://docs.oracle.com/javase/8/docs/api/java/lang/Enum.html?is-external=true";
 title="class or interface in java.lang">Enum
+Enumeration that represents the action HBase will take when 
a space quota is violated.
+
+ The target for a violation policy is either an HBase table or namespace. In 
the case of a
+ namespace, it is treated as a collection of tables (all tables are subject to 
the same policy).
+
+
+
+
+
+
+
+
+
+
+
+Enum Constant Summary
+
+Enum Constants 
+
+Enum Constant and Description
+
+
+DISABLE
+Disables the table(s).
+
+
+
+NO_INSERTS
+Disallows any updates (but allows deletes and compactions) 
on the table(s).
+
+
+
+NO_WRITES
+Disallows any mutations (but allows compactions) on the 
table(s).
+
+
+
+NO_WRITES_COMPACTIONS
+Disallows any mutations or compactions on the 
table(s).
+
+
+
+
+
+
+
+
+
+
+Method Summary
+
+All Methods Static Methods Concrete Methods 
+
+Modifier and Type
+Method and Description
+
+
+static SpaceViolationPolicy
+valueOf(http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String name)
+Returns the enum constant of this type with the specified 
name.
+
+
+
+static SpaceViolationPolicy[]
+values()
+Returns an array containing the constants of this enum 
type, in
+the order they are declared.
+
+
+
+
+
+
+
+Methods inherited from class java.lang.http://docs.oracle.com/javase/8/docs/api/java/lang/Enum.html?is-external=true";
 title="class or interface in java.lang">Enum
+http://docs.oracle.com/javase/8/docs/api/java/lang/Enum.html?is-external=true#clone--";
 title="class or interface in java.lang">clone, http://docs.oracle.com/javase/8/docs/api/java/lang/Enum.html?is-external=true#compareTo-E-";
 title="class or interface in java.lang">compareTo, http://docs.oracle.com/javase/8/docs/api/java/lang/Enum.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/Enum.html?is-external=true#finalize--";
 title="class or interface in java.lang">finalize, http://docs.oracle.com/javase/8/docs/api/java/lang/Enum.html?is-external=true#getDeclaringClass--";
 title="class or interface in java.lang">getDeclaringClass, http://docs.oracle.com/javase/8/docs/api/java/lang/Enum.html?is-external=true#hashCode--";
 title="class or interface in java.lang">hashCode, http://docs.oracle.com/javase/8/docs/
 api/java/lang/Enum.html?is-external=true#n

hbase git commit: HBASE-18077 Update JUnit licensing to use EPL

Repository: hbase
Updated Branches:
  refs/heads/master 1d0295f4e -> 9e7b0c1a4


HBASE-18077 Update JUnit licensing to use EPL

Signed-off-by: Sean Busbey 


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

Branch: refs/heads/master
Commit: 9e7b0c1a4f24feeecb55498d7926596af9fc284a
Parents: 1d0295f
Author: Mike Drob 
Authored: Thu May 18 19:16:56 2017 -0700
Committer: Sean Busbey 
Committed: Tue May 23 10:12:15 2017 -0500

--
 .../src/main/resources/META-INF/LICENSE.vm  | 262 ++-
 .../src/main/resources/supplemental-models.xml  |   5 +-
 2 files changed, 264 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/9e7b0c1a/hbase-resource-bundle/src/main/resources/META-INF/LICENSE.vm
--
diff --git a/hbase-resource-bundle/src/main/resources/META-INF/LICENSE.vm 
b/hbase-resource-bundle/src/main/resources/META-INF/LICENSE.vm
index a9f0c81..29dd9d5 100644
--- a/hbase-resource-bundle/src/main/resources/META-INF/LICENSE.vm
+++ b/hbase-resource-bundle/src/main/resources/META-INF/LICENSE.vm
@@ -1551,6 +1551,8 @@ You can redistribute it and/or modify it under either the 
terms of the GPL
 #set($bsd3 = [])
 ## gather up CPL 1.0 works
 #set($cpl = [])
+## gather up EPL 1.0 works
+#set($epl = [])
 ## track commons-math
 #set($commons-math-two = false)
 #set($commons-math-three = false)
@@ -1563,7 +1565,7 @@ You can redistribute it and/or modify it under either the 
terms of the GPL
 ## license mentions.
 ##
 ## See this FAQ link for justifications: 
https://www.apache.org/legal/resolved.html
-#set($non_aggregate_fine = [ 'Public Domain', 'New BSD license', 'BSD 
license', 'Mozilla Public License Version 1.1', 'Mozilla Public License Version 
2.0', 'Creative Commons Attribution License, Version 2.5', 'Eclipse Public 
License 1.0'])
+#set($non_aggregate_fine = [ 'Public Domain', 'New BSD license', 'BSD 
license', 'Mozilla Public License Version 1.1', 'Mozilla Public License Version 
2.0', 'Creative Commons Attribution License, Version 2.5' ])
 ## include LICENSE sections for anything not under ASL2.0
 #foreach( ${dep} in ${projects} )
 ## if there are no licenses we'll fail the build later, so
@@ -1647,6 +1649,9 @@ ${dep.scm.url}
 #if(${dep.licenses[0].name.equals("Common Public License Version 1.0")})
 #set($aggregated = $cpl.add($dep))
 #end
+#if(${dep.licenses[0].name.equals("Eclipse Public License 1.0")})
+#set($aggregated = $epl.add($dep))
+#end
 #if(!${aggregated})
 --
 This product includes ${dep.name} licensed under the ${dep.licenses[0].name}.
@@ -2573,4 +2578,259 @@ Common Public License - v 1.0
 #if($jruby)
 #jruby_license()
 #end
+#if(!(${epl.isEmpty()}))
+
+## print all the EPL 1.0 licensed works
+This product includes the following works licensed under the Eclipse Public 
License 1.0:
+
+#foreach($dep in $epl)
+#if( $dep.licenses[0].comments && !$dep.licenses[0].comments.empty )
+  * ${dep.name}, ${dep.licenses[0].comments}
+#else
+  * ${dep.name}
+#end
+#end
+
+  Eclipse Public License - v 1.0
+
+  THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS ECLIPSE
+  PUBLIC LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION
+  OF THE PROGRAM CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT.
+
+  1. DEFINITIONS
+
+  "Contribution" means:
+
+  a) in the case of the initial Contributor, the initial code and
+ documentation distributed under this Agreement, and
+
+  b) in the case of each subsequent Contributor:
+
+  i) changes to the Program, and
+
+  ii) additions to the Program;
+  where such changes and/or additions to the Program
+  originate from and are distributed by that particular
+  Contributor. A Contribution 'originates' from a
+  Contributor if it was added to the Program by such
+  Contributor itself or anyone acting on such
+  Contributor's behalf. Contributions do not include
+  additions to the Program which: (i) are separate modules
+  of software distributed in conjunction with the Program
+  under their own license agreement, and (ii) are not
+  derivative works of the Program.
+
+  "Contributor" means any person or entity that distributes the Program.
+
+  "Licensed Patents" mean patent claims licensable by a Contributor
+  which are necessarily infringed by the use or sale of its
+  Contribution alone or when combined with the Program.
+
+  "Program" means the Contributions distributed in accordance with
+  this Agreement.
+
+  "Recipient" means anyone who receives t

hbase git commit: HBASE-18077 Update JUnit licensing to use EPL

Repository: hbase
Updated Branches:
  refs/heads/branch-1 4249a1f3d -> abf03da41


HBASE-18077 Update JUnit licensing to use EPL

Signed-off-by: Sean Busbey 


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

Branch: refs/heads/branch-1
Commit: abf03da41df57faf8e6426a58822ddf414ba9561
Parents: 4249a1f
Author: Mike Drob 
Authored: Thu May 18 19:16:56 2017 -0700
Committer: Sean Busbey 
Committed: Tue May 23 10:20:44 2017 -0500

--
 .../src/main/resources/META-INF/LICENSE.vm  | 264 ++-
 .../src/main/resources/supplemental-models.xml  |   5 +-
 2 files changed, 266 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/abf03da4/hbase-resource-bundle/src/main/resources/META-INF/LICENSE.vm
--
diff --git a/hbase-resource-bundle/src/main/resources/META-INF/LICENSE.vm 
b/hbase-resource-bundle/src/main/resources/META-INF/LICENSE.vm
index f403c89..9f9afb5 100644
--- a/hbase-resource-bundle/src/main/resources/META-INF/LICENSE.vm
+++ b/hbase-resource-bundle/src/main/resources/META-INF/LICENSE.vm
@@ -1551,6 +1551,8 @@ You can redistribute it and/or modify it under either the 
terms of the GPL
 #set($bsd3 = [])
 ## gather up CPL 1.0 works
 #set($cpl = [])
+## gather up EPL 1.0 works
+#set($epl = [])
 ## track commons-math
 #set($commons-math-two = false)
 #set($commons-math-three = false)
@@ -1561,7 +1563,9 @@ You can redistribute it and/or modify it under either the 
terms of the GPL
 ## Whitelist of licenses that it's safe to not aggregate as above.
 ## Note that this doesn't include ALv2 or the aforementioned aggregate
 ## license mentions.
-#set($non_aggregate_fine = [ 'Public Domain', 'New BSD license', 'BSD 
license', 'Mozilla Public License Version 2.0' ])
+##
+## See this FAQ link for justifications: 
https://www.apache.org/legal/resolved.html
+#set($non_aggregate_fine = [ 'Public Domain', 'New BSD license', 'BSD 
license', 'Mozilla Public License Version 1.1', 'Mozilla Public License Version 
2.0', 'Creative Commons Attribution License, Version 2.5' ])
 ## include LICENSE sections for anything not under ASL2.0
 #foreach( ${dep} in ${projects} )
 #if(${debug-print-included-work-info.equalsIgnoreCase("true")})
@@ -1637,6 +1641,9 @@ ${dep.scm.url}
 #if(${dep.licenses[0].name.equals("Common Public License Version 1.0")})
 #set($aggregated = $cpl.add($dep))
 #end
+#if(${dep.licenses[0].name.equals("Eclipse Public License 1.0")})
+#set($aggregated = $epl.add($dep))
+#end
 #if(!${aggregated})
 --
 This product includes ${dep.name} licensed under the ${dep.licenses[0].name}.
@@ -2563,4 +2570,259 @@ Common Public License - v 1.0
 #if($jruby)
 #jruby_license()
 #end
+#if(!(${epl.isEmpty()}))
+
+## print all the EPL 1.0 licensed works
+This product includes the following works licensed under the Eclipse Public 
License 1.0:
+
+#foreach($dep in $epl)
+#if( $dep.licenses[0].comments && !$dep.licenses[0].comments.empty )
+  * ${dep.name}, ${dep.licenses[0].comments}
+#else
+  * ${dep.name}
+#end
+#end
+
+  Eclipse Public License - v 1.0
+
+  THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS ECLIPSE
+  PUBLIC LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION
+  OF THE PROGRAM CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT.
+
+  1. DEFINITIONS
+
+  "Contribution" means:
+
+  a) in the case of the initial Contributor, the initial code and
+ documentation distributed under this Agreement, and
+
+  b) in the case of each subsequent Contributor:
+
+  i) changes to the Program, and
+
+  ii) additions to the Program;
+  where such changes and/or additions to the Program
+  originate from and are distributed by that particular
+  Contributor. A Contribution 'originates' from a
+  Contributor if it was added to the Program by such
+  Contributor itself or anyone acting on such
+  Contributor's behalf. Contributions do not include
+  additions to the Program which: (i) are separate modules
+  of software distributed in conjunction with the Program
+  under their own license agreement, and (ii) are not
+  derivative works of the Program.
+
+  "Contributor" means any person or entity that distributes the Program.
+
+  "Licensed Patents" mean patent claims licensable by a Contributor
+  which are necessarily infringed by the use or sale of its
+  Contribution alone or when combined with the Program.
+
+  "Program" means the Contributions distributed in accordance with
+  this Agreement.
+
+  "Recipient" 

hbase git commit: HBASE-18077 Update JUnit licensing to use EPL

Repository: hbase
Updated Branches:
  refs/heads/branch-1.3 9d21e89b0 -> d1b1eab10


HBASE-18077 Update JUnit licensing to use EPL

Signed-off-by: Sean Busbey 


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

Branch: refs/heads/branch-1.3
Commit: d1b1eab1036c3f69d06d573f52028aea00cdac5b
Parents: 9d21e89
Author: Mike Drob 
Authored: Thu May 18 19:16:56 2017 -0700
Committer: Sean Busbey 
Committed: Tue May 23 10:22:27 2017 -0500

--
 .../src/main/resources/META-INF/LICENSE.vm  | 264 ++-
 .../src/main/resources/supplemental-models.xml  |   5 +-
 2 files changed, 266 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/d1b1eab1/hbase-resource-bundle/src/main/resources/META-INF/LICENSE.vm
--
diff --git a/hbase-resource-bundle/src/main/resources/META-INF/LICENSE.vm 
b/hbase-resource-bundle/src/main/resources/META-INF/LICENSE.vm
index f403c89..9f9afb5 100644
--- a/hbase-resource-bundle/src/main/resources/META-INF/LICENSE.vm
+++ b/hbase-resource-bundle/src/main/resources/META-INF/LICENSE.vm
@@ -1551,6 +1551,8 @@ You can redistribute it and/or modify it under either the 
terms of the GPL
 #set($bsd3 = [])
 ## gather up CPL 1.0 works
 #set($cpl = [])
+## gather up EPL 1.0 works
+#set($epl = [])
 ## track commons-math
 #set($commons-math-two = false)
 #set($commons-math-three = false)
@@ -1561,7 +1563,9 @@ You can redistribute it and/or modify it under either the 
terms of the GPL
 ## Whitelist of licenses that it's safe to not aggregate as above.
 ## Note that this doesn't include ALv2 or the aforementioned aggregate
 ## license mentions.
-#set($non_aggregate_fine = [ 'Public Domain', 'New BSD license', 'BSD 
license', 'Mozilla Public License Version 2.0' ])
+##
+## See this FAQ link for justifications: 
https://www.apache.org/legal/resolved.html
+#set($non_aggregate_fine = [ 'Public Domain', 'New BSD license', 'BSD 
license', 'Mozilla Public License Version 1.1', 'Mozilla Public License Version 
2.0', 'Creative Commons Attribution License, Version 2.5' ])
 ## include LICENSE sections for anything not under ASL2.0
 #foreach( ${dep} in ${projects} )
 #if(${debug-print-included-work-info.equalsIgnoreCase("true")})
@@ -1637,6 +1641,9 @@ ${dep.scm.url}
 #if(${dep.licenses[0].name.equals("Common Public License Version 1.0")})
 #set($aggregated = $cpl.add($dep))
 #end
+#if(${dep.licenses[0].name.equals("Eclipse Public License 1.0")})
+#set($aggregated = $epl.add($dep))
+#end
 #if(!${aggregated})
 --
 This product includes ${dep.name} licensed under the ${dep.licenses[0].name}.
@@ -2563,4 +2570,259 @@ Common Public License - v 1.0
 #if($jruby)
 #jruby_license()
 #end
+#if(!(${epl.isEmpty()}))
+
+## print all the EPL 1.0 licensed works
+This product includes the following works licensed under the Eclipse Public 
License 1.0:
+
+#foreach($dep in $epl)
+#if( $dep.licenses[0].comments && !$dep.licenses[0].comments.empty )
+  * ${dep.name}, ${dep.licenses[0].comments}
+#else
+  * ${dep.name}
+#end
+#end
+
+  Eclipse Public License - v 1.0
+
+  THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS ECLIPSE
+  PUBLIC LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION
+  OF THE PROGRAM CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT.
+
+  1. DEFINITIONS
+
+  "Contribution" means:
+
+  a) in the case of the initial Contributor, the initial code and
+ documentation distributed under this Agreement, and
+
+  b) in the case of each subsequent Contributor:
+
+  i) changes to the Program, and
+
+  ii) additions to the Program;
+  where such changes and/or additions to the Program
+  originate from and are distributed by that particular
+  Contributor. A Contribution 'originates' from a
+  Contributor if it was added to the Program by such
+  Contributor itself or anyone acting on such
+  Contributor's behalf. Contributions do not include
+  additions to the Program which: (i) are separate modules
+  of software distributed in conjunction with the Program
+  under their own license agreement, and (ii) are not
+  derivative works of the Program.
+
+  "Contributor" means any person or entity that distributes the Program.
+
+  "Licensed Patents" mean patent claims licensable by a Contributor
+  which are necessarily infringed by the use or sale of its
+  Contribution alone or when combined with the Program.
+
+  "Program" means the Contributions distributed in accordance with
+  this Agreement.
+
+  "Recipie

hbase git commit: HBASE-18077 Update JUnit licensing to use EPL

Repository: hbase
Updated Branches:
  refs/heads/branch-1.2 71bf5afa3 -> 6ee7a4932


HBASE-18077 Update JUnit licensing to use EPL

Signed-off-by: Sean Busbey 


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

Branch: refs/heads/branch-1.2
Commit: 6ee7a4932ab0a24956168d6482c30712a247a17a
Parents: 71bf5af
Author: Mike Drob 
Authored: Thu May 18 19:16:56 2017 -0700
Committer: Sean Busbey 
Committed: Tue May 23 10:33:29 2017 -0500

--
 .../src/main/resources/META-INF/LICENSE.vm  | 264 ++-
 .../src/main/resources/supplemental-models.xml  |   5 +-
 2 files changed, 266 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/6ee7a493/hbase-resource-bundle/src/main/resources/META-INF/LICENSE.vm
--
diff --git a/hbase-resource-bundle/src/main/resources/META-INF/LICENSE.vm 
b/hbase-resource-bundle/src/main/resources/META-INF/LICENSE.vm
index f403c89..9f9afb5 100644
--- a/hbase-resource-bundle/src/main/resources/META-INF/LICENSE.vm
+++ b/hbase-resource-bundle/src/main/resources/META-INF/LICENSE.vm
@@ -1551,6 +1551,8 @@ You can redistribute it and/or modify it under either the 
terms of the GPL
 #set($bsd3 = [])
 ## gather up CPL 1.0 works
 #set($cpl = [])
+## gather up EPL 1.0 works
+#set($epl = [])
 ## track commons-math
 #set($commons-math-two = false)
 #set($commons-math-three = false)
@@ -1561,7 +1563,9 @@ You can redistribute it and/or modify it under either the 
terms of the GPL
 ## Whitelist of licenses that it's safe to not aggregate as above.
 ## Note that this doesn't include ALv2 or the aforementioned aggregate
 ## license mentions.
-#set($non_aggregate_fine = [ 'Public Domain', 'New BSD license', 'BSD 
license', 'Mozilla Public License Version 2.0' ])
+##
+## See this FAQ link for justifications: 
https://www.apache.org/legal/resolved.html
+#set($non_aggregate_fine = [ 'Public Domain', 'New BSD license', 'BSD 
license', 'Mozilla Public License Version 1.1', 'Mozilla Public License Version 
2.0', 'Creative Commons Attribution License, Version 2.5' ])
 ## include LICENSE sections for anything not under ASL2.0
 #foreach( ${dep} in ${projects} )
 #if(${debug-print-included-work-info.equalsIgnoreCase("true")})
@@ -1637,6 +1641,9 @@ ${dep.scm.url}
 #if(${dep.licenses[0].name.equals("Common Public License Version 1.0")})
 #set($aggregated = $cpl.add($dep))
 #end
+#if(${dep.licenses[0].name.equals("Eclipse Public License 1.0")})
+#set($aggregated = $epl.add($dep))
+#end
 #if(!${aggregated})
 --
 This product includes ${dep.name} licensed under the ${dep.licenses[0].name}.
@@ -2563,4 +2570,259 @@ Common Public License - v 1.0
 #if($jruby)
 #jruby_license()
 #end
+#if(!(${epl.isEmpty()}))
+
+## print all the EPL 1.0 licensed works
+This product includes the following works licensed under the Eclipse Public 
License 1.0:
+
+#foreach($dep in $epl)
+#if( $dep.licenses[0].comments && !$dep.licenses[0].comments.empty )
+  * ${dep.name}, ${dep.licenses[0].comments}
+#else
+  * ${dep.name}
+#end
+#end
+
+  Eclipse Public License - v 1.0
+
+  THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS ECLIPSE
+  PUBLIC LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION
+  OF THE PROGRAM CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT.
+
+  1. DEFINITIONS
+
+  "Contribution" means:
+
+  a) in the case of the initial Contributor, the initial code and
+ documentation distributed under this Agreement, and
+
+  b) in the case of each subsequent Contributor:
+
+  i) changes to the Program, and
+
+  ii) additions to the Program;
+  where such changes and/or additions to the Program
+  originate from and are distributed by that particular
+  Contributor. A Contribution 'originates' from a
+  Contributor if it was added to the Program by such
+  Contributor itself or anyone acting on such
+  Contributor's behalf. Contributions do not include
+  additions to the Program which: (i) are separate modules
+  of software distributed in conjunction with the Program
+  under their own license agreement, and (ii) are not
+  derivative works of the Program.
+
+  "Contributor" means any person or entity that distributes the Program.
+
+  "Licensed Patents" mean patent claims licensable by a Contributor
+  which are necessarily infringed by the use or sale of its
+  Contribution alone or when combined with the Program.
+
+  "Program" means the Contributions distributed in accordance with
+  this Agreement.
+
+  "Recipie

[04/50] [abbrv] hbase git commit: HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi) Move to a new AssignmentManager, one that describes Assignment using a State Machine built on top

http://git-wip-us.apache.org/repos/asf/hbase/blob/db1dcf3e/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java
--
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java
index c5c6484..8872c63 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
+import org.apache.hadoop.hbase.CategoryBasedTimeout;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
@@ -30,18 +31,19 @@ import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
-import 
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyTableState;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
+import org.junit.rules.TestRule;
 
 @Category({MasterTests.class, MediumTests.class})
 public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
-  @Rule
-  public TestName name = new TestName();
+  @Rule public final TestRule timeout = 
CategoryBasedTimeout.builder().withTimeout(this.getClass()).
+  withLookingForStuckThread(true).build();
+  @Rule public TestName name = new TestName();
 
   @Test(timeout=6)
   public void testModifyTable() throws Exception {
@@ -208,8 +210,7 @@ public class TestModifyTableProcedure extends 
TestTableDDLProcedureBase {
   new ModifyTableProcedure(procExec.getEnvironment(), htd));
 
 // Restart the executor and execute the step twice
-int numberOfSteps = ModifyTableState.values().length;
-MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, 
procId, numberOfSteps);
+MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, 
procId);
 
 // Validate descriptor
 HTableDescriptor currentHtd = 
UTIL.getAdmin().getTableDescriptor(tableName);
@@ -246,8 +247,7 @@ public class TestModifyTableProcedure extends 
TestTableDDLProcedureBase {
   new ModifyTableProcedure(procExec.getEnvironment(), htd));
 
 // Restart the executor and execute the step twice
-int numberOfSteps = ModifyTableState.values().length;
-MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, 
procId, numberOfSteps);
+MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, 
procId);
 
 // Validate descriptor
 HTableDescriptor currentHtd = 
UTIL.getAdmin().getTableDescriptor(tableName);
@@ -282,7 +282,7 @@ public class TestModifyTableProcedure extends 
TestTableDDLProcedureBase {
 long procId = procExec.submitProcedure(
   new ModifyTableProcedure(procExec.getEnvironment(), htd));
 
-int numberOfSteps = 1; // failing at pre operation
+int numberOfSteps = 0; // failing at pre operation
 MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, 
procId, numberOfSteps);
 
 // cf2 should not be present
@@ -315,7 +315,7 @@ public class TestModifyTableProcedure extends 
TestTableDDLProcedureBase {
   new ModifyTableProcedure(procExec.getEnvironment(), htd));
 
 // Restart the executor and rollback the step twice
-int numberOfSteps = 1; // failing at pre operation
+int numberOfSteps = 0; // failing at pre operation
 MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, 
procId, numberOfSteps);
 
 // cf2 should not be present

http://git-wip-us.apache.org/repos/asf/hbase/blob/db1dcf3e/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java
--
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java
index e6e90ef..47b1248 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java
@@ -18,12 +18,16 @@
 
 package org.apache.hadoop.hbase.master.procedure;
 
-import java.util.Random;
+import static org.junit.Assert.

[03/50] [abbrv] hbase git commit: HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi) Move to a new AssignmentManager, one that describes Assignment using a State Machine built on top

http://git-wip-us.apache.org/repos/asf/hbase/blob/db1dcf3e/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
--
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
index 81846df..a64d102 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
@@ -39,7 +39,7 @@ 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.Abortable;
+import org.apache.hadoop.hbase.CategoryBasedTimeout;
 import org.apache.hadoop.hbase.CoordinatedStateManager;
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
@@ -66,23 +66,26 @@ 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.TestReplicasClient.SlowMeCopro;
-import org.apache.hadoop.hbase.coprocessor.MasterObserver;
 import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.MasterObserver;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
-import org.apache.hadoop.hbase.master.AssignmentManager;
+import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.MasterRpcServices;
 import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.master.RegionState.State;
-import org.apache.hadoop.hbase.master.RegionStates;
-import 
org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import 
org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
-import 
org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
+import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
+import org.apache.hadoop.hbase.master.assignment.RegionStates;
+import org.apache.hadoop.hbase.master.NoSuchProcedureException;
+import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
 import 
org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -98,11 +101,11 @@ import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
+import org.junit.rules.TestRule;
 
 /**
  * The below tests are testing split region against a running cluster
@@ -110,8 +113,9 @@ import org.junit.rules.TestName;
 @Category({RegionServerTests.class, LargeTests.class})
 @SuppressWarnings("deprecation")
 public class TestSplitTransactionOnCluster {
-  private static final Log LOG =
-LogFactory.getLog(TestSplitTransactionOnCluster.class);
+  private static final Log LOG = 
LogFactory.getLog(TestSplitTransactionOnCluster.class);
+  @Rule public final TestRule timeout = 
CategoryBasedTimeout.builder().withTimeout(this.getClass()).
+  withLookingForStuckThread(true).build();
   private Admin admin = null;
   private MiniHBaseCluster cluster = null;
   private static final int NB_SERVERS = 3;
@@ -150,8 +154,11 @@ public class TestSplitTransactionOnCluster {
   throws IOException, InterruptedException {
 assertEquals(1, regions.size());
 HRegionInfo hri = regions.get(0).getRegionInfo();
-cluster.getMaster().getAssignmentManager()
-  .waitOnRegionToClearRegionsInTransition(hri, 60);
+try {
+ 

[39/50] [abbrv] hbase git commit: Need to check server when doing ServerCrashProcedure; we had it but I removed it a few patches back... makes for SCPs stamping on each otehr failing ongoing assigns

Need to check server when doing ServerCrashProcedure; we had it but I removed 
it a few patches back... makes for SCPs stamping on each otehr failing ongoing 
assigns


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

Branch: refs/heads/HBASE-14614
Commit: edad94e0f3771f7daa5f74e19842ce955f2d9f6a
Parents: 2e78073
Author: Michael Stack 
Authored: Sat May 13 21:37:52 2017 -0700
Committer: Michael Stack 
Committed: Tue May 23 08:36:53 2017 -0700

--
 .../hbase/master/assignment/AssignProcedure.java  |  8 
 .../master/assignment/RegionTransitionProcedure.java  |  6 ++
 .../hbase/master/assignment/UnassignProcedure.java|  5 +
 .../hbase/master/procedure/ServerCrashException.java  |  7 +--
 .../hbase/master/procedure/ServerCrashProcedure.java  | 14 --
 5 files changed, 36 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/edad94e0/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignProcedure.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignProcedure.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignProcedure.java
index 36f6f08..42ece16 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignProcedure.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignProcedure.java
@@ -327,4 +327,12 @@ public class AssignProcedure extends 
RegionTransitionProcedure {
 super.toStringClassDetails(sb);
 if (this.targetServer != null) sb.append(", 
target=").append(this.targetServer);
   }
+
+  @Override
+  public ServerName getServer(final MasterProcedureEnv env) {
+RegionStateNode node =
+
env.getAssignmentManager().getRegionStates().getRegionNode(this.getRegionInfo());
+if (node == null) return null;
+return node.getRegionLocation();
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/edad94e0/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
index 6dc809b..49124ea 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
@@ -372,4 +372,10 @@ public abstract class RegionTransitionProcedure
 // the client does not know about this procedure.
 return false;
   }
+
+  /**
+   * Used by ServerCrashProcedure to see if this Assign/Unassign needs 
processing.
+   * @return ServerName the Assign or Unassign is going against.
+   */
+  public abstract ServerName getServer(final MasterProcedureEnv env);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/edad94e0/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/UnassignProcedure.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/UnassignProcedure.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/UnassignProcedure.java
index a82a2f5..126718a 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/UnassignProcedure.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/UnassignProcedure.java
@@ -239,4 +239,9 @@ public class UnassignProcedure extends 
RegionTransitionProcedure {
 super.toStringClassDetails(sb);
 sb.append(", server=").append(this.destinationServer);
   }
+
+  @Override
+  public ServerName getServer(final MasterProcedureEnv env) {
+return this.destinationServer;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/edad94e0/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashException.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashException.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashException.java
index dd1874b..26aba9e 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/proced

[13/50] [abbrv] hbase git commit: HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi) Move to a new AssignmentManager, one that describes Assignment using a State Machine built on top

http://git-wip-us.apache.org/repos/asf/hbase/blob/db1dcf3e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
new file mode 100644
index 000..cd8b858
--- /dev/null
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
@@ -0,0 +1,348 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.master.assignment;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
+import org.apache.hadoop.hbase.master.assignment.RegionStates.RegionStateNode;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.procedure.TableProcedureInterface;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
+import 
org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteOperation;
+import 
org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteProcedure;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RegionTransitionState;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
+
+/**
+ * Base class for the Assign and Unassign Procedure.
+ * There can only be one RegionTransitionProcedure per region running at a time
+ * since each procedure takes a lock on the region (see 
MasterProcedureScheduler).
+ *
+ * This procedure is asynchronous and responds to external events.
+ * The AssignmentManager will notify this procedure when the RS completes
+ * the operation and reports the transitioned state
+ * (see the Assign and Unassign class for more details).
+ * Procedures move from the REGION_TRANSITION_QUEUE state when they are
+ * first submitted, to the REGION_TRANSITION_DISPATCH state when the request
+ * to remote server is done. They end in the REGION_TRANSITION_FINISH state.
+ * the 
+ */
+@InterfaceAudience.Private
+public abstract class RegionTransitionProcedure
+extends Procedure
+implements TableProcedureInterface,
+  RemoteProcedure {
+  private static final Log LOG = 
LogFactory.getLog(RegionTransitionProcedure.class);
+
+  protected final AtomicBoolean aborted = new AtomicBoolean(false);
+
+  private RegionTransitionState transitionState =
+  RegionTransitionState.REGION_TRANSITION_QUEUE;
+  private HRegionInfo regionInfo;
+  private volatile boolean lock = false;
+  // Server we assign or unassign from -- the target.
+  protected volatile ServerName server;
+
+  public RegionTransitionProcedure() {
+// Required by the Procedure framework to create the procedure on replay
+super();
+  }
+
+  public RegionTransitionProcedure(final HRegionInfo regionInfo) {
+this.regionInfo = regionInfo;
+  }
+
+  public HRegionInfo getRegionInfo() {
+return regionInfo;
+  }
+
+  protected void setRegionInfo(final HRegionInfo regionInfo) {
+// Setter is for deserialization.
+this.regionInfo = regionInfo;
+  }
+
+  @Override
+  public TableName getTableName() {
+HRegionInfo hri = getRegionInfo();
+return hri != null? hri.getTable(): null;
+  }
+
+  public boolean isMeta() {
+return TableName.isMetaTableName(getTableName());
+  }
+
+  @Override
+  public void toStringClassDetails(final StringBuilder sb) {
+sb.append(getClass().getSimpleName());
+sb.append(" table=");
+sb.append(getTableName());
+sb.append(", region=");
+sb.append(getRegionInfo() == null? null: getRegionInf

[07/50] [abbrv] hbase git commit: HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi) Move to a new AssignmentManager, one that describes Assignment using a State Machine built on top

http://git-wip-us.apache.org/repos/asf/hbase/blob/db1dcf3e/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableFavoredNodes.java
--
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableFavoredNodes.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableFavoredNodes.java
index 3eb65a5..7c0aa74 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableFavoredNodes.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableFavoredNodes.java
@@ -35,18 +35,19 @@ import java.util.concurrent.TimeUnit;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.NamespaceDescriptor;
-import org.apache.hadoop.hbase.favored.FavoredNodeAssignmentHelper;
-import org.apache.hadoop.hbase.favored.FavoredNodesManager;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.favored.FavoredNodeAssignmentHelper;
+import org.apache.hadoop.hbase.favored.FavoredNodeLoadBalancer;
+import org.apache.hadoop.hbase.favored.FavoredNodesManager;
 import org.apache.hadoop.hbase.master.LoadBalancer;
-import org.apache.hadoop.hbase.master.RegionStates;
 import org.apache.hadoop.hbase.master.ServerManager;
 import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer;
 import 
org.apache.hadoop.hbase.master.balancer.LoadOnlyFavoredStochasticBalancer;
@@ -55,9 +56,7 @@ import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.JVMClusterUtil;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
+import org.apache.hadoop.hbase.util.Threads;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -66,6 +65,9 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
 
 @Category({ClientTests.class, MediumTests.class})
 public class TestTableFavoredNodes {
@@ -76,7 +78,6 @@ public class TestTableFavoredNodes {
   private final static int WAIT_TIMEOUT = 6;
   private final static int SLAVES = 8;
   private FavoredNodesManager fnm;
-  private RegionStates regionStates;
   private Admin admin;
 
   private final byte[][] splitKeys = new byte[][] {Bytes.toBytes(1), 
Bytes.toBytes(9)};
@@ -101,8 +102,8 @@ public class TestTableFavoredNodes {
 
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
-TEST_UTIL.cleanupTestDir();
 TEST_UTIL.shutdownMiniCluster();
+TEST_UTIL.cleanupTestDir();
   }
 
   @Before
@@ -111,8 +112,6 @@ public class TestTableFavoredNodes {
 admin = TEST_UTIL.getAdmin();
 admin.setBalancerRunning(false, true);
 admin.enableCatalogJanitor(false);
-regionStates =
-  
TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().getRegionStates();
   }
 
   /*
@@ -165,8 +164,9 @@ public class TestTableFavoredNodes {
   @Test
   public void testSplitTable() throws Exception {
 final TableName tableName = TableName.valueOf(name.getMethodName());
-TEST_UTIL.createTable(tableName, Bytes.toBytes("f"), splitKeys);
+Table t = TEST_UTIL.createTable(tableName, Bytes.toBytes("f"), splitKeys);
 TEST_UTIL.waitUntilAllRegionsAssigned(tableName);
+final int numberOfRegions = admin.getTableRegions(t.getName()).size();
 
 checkIfFavoredNodeInformationIsCorrect(tableName);
 
@@ -176,13 +176,14 @@ public class TestTableFavoredNodes {
 List parentFN = fnm.getFavoredNodes(parent);
 assertNotNull("FN should not be null for region: " + parent, parentFN);
 
+LOG.info("SPLITTING TABLE");
 admin.split(tableName, splitPoint);
 
 TEST_UTIL.waitUntilNoRegionsInTransition(WAIT_TIMEOUT);
-waitUntilTableRegionCountReached(tableName, NUM_REGIONS + 1);
+LOG.info("FINISHED WAITING ON RIT");
+waitUntilTableRegionCountReached(tableName, numberOfRegions + 1);
 
-// All regions should have favored nodes
-checkIfFavoredNodeInformationIsCorrect(tableName);
+// All regions should have favored nodes
checkIfFavoredNodeInformationIsCorrect(tableName);
 
 // Get the daughters of parent.
 HRegionInfo daughter1 = locator.getRegionLocation(parent.getStartKey(), 
true).getRegionInfo();
@@ -210,7 +211,10 @@ public class TestTableFa

[30/50] [abbrv] hbase git commit: Exceptions are not logging identifying details like region; makes it hard to debug who is problem

Exceptions are not logging identifying details like region; makes it hard to 
debug who is problem


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

Branch: refs/heads/HBASE-14614
Commit: c94c44d9c4e82b785f1e4847dcc65706aa19e2f3
Parents: 2c99a03
Author: Michael Stack 
Authored: Sat May 6 11:22:57 2017 -0700
Committer: Michael Stack 
Committed: Tue May 23 08:36:53 2017 -0700

--
 .../master/assignment/AssignmentManager.java  | 18 --
 .../hbase/master/assignment/RegionStates.java |  2 +-
 2 files changed, 9 insertions(+), 11 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/c94c44d9/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
index 2986868..ed55235 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
@@ -983,7 +983,7 @@ public class AssignmentManager implements ServerListener {
   }
 } catch (UnexpectedStateException e) {
   final ServerName serverName = serverNode.getServerName();
-  LOG.warn("Killing " + serverName + ": " + e.getMessage());
+  LOG.warn("KILLING " + serverName + ": " + e.getMessage());
   killRegionServer(serverNode);
 }
   }
@@ -1003,25 +1003,23 @@ public class AssignmentManager implements 
ServerListener {
 synchronized (regionNode) {
   if (regionNode.isInState(State.OPENING, State.OPEN)) {
 if (!regionNode.getRegionLocation().equals(serverName)) {
-  throw new UnexpectedStateException(
-"Reported OPEN on server=" + serverName +
-" but state found says server=" + 
regionNode.getRegionLocation());
+  throw new UnexpectedStateException(regionNode.toString() +
+"reported OPEN on server=" + serverName +
+" but state has otherwise.");
 } else if (regionNode.isInState(State.OPENING)) {
   try {
 if (!reportTransition(regionNode, serverNode, 
TransitionCode.OPENED, 0)) {
-  LOG.warn("Reported OPEN on server=" + serverName +
-" but state found says " + regionNode + " and NO procedure 
is running");
+  LOG.warn(regionNode.toString() + " reported OPEN on server=" 
+ serverName +
+" but state has otherwise AND NO procedure is running");
 }
   } catch (UnexpectedStateException e) {
-LOG.warn("Unexpected exception while trying to report " + 
regionNode +
-  " as open: " + e.getMessage(), e);
+LOG.warn(regionNode.toString() + " reported unexpteced OPEN: " 
+ e.getMessage(), e);
   }
 }
   } else if (!regionNode.isInState(State.CLOSING, State.SPLITTING)) {
 // TODO: We end up killing the RS if we get a report while we 
already
 // transitioned to close or split. we should have a 
timeout/timestamp to compare
-throw new UnexpectedStateException(
-"Reported OPEN but state found says " + regionNode.getState());
+throw new UnexpectedStateException(regionNode.toString() + " 
reported unexpected OPEN");
   }
 }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c94c44d9/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
index aa2627c..1c852c9 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
@@ -173,7 +173,7 @@ public class RegionStates {
 public ServerName setRegionLocation(final ServerName serverName) {
   ServerName lastRegionLocation = this.regionLocation;
   if (serverName == null) {
-LOG.debug("REMOVE tracking when we are set to null", new 
Throwable("DEBUG"));
+LOG.debug("REMOVE tracking when we are set to null " + this, new 
Throwable("DEBUG"));
  

[29/50] [abbrv] hbase git commit: Fix failing hbase-procedure tests

Fix failing hbase-procedure tests


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

Branch: refs/heads/HBASE-14614
Commit: 2c99a03b4164f02aa67dfb278aa12c177c9eb33e
Parents: bd67203
Author: Michael Stack 
Authored: Sat May 6 10:45:58 2017 -0700
Committer: Michael Stack 
Committed: Tue May 23 08:36:53 2017 -0700

--
 .../java/org/apache/hadoop/hbase/procedure2/Procedure.java | 2 +-
 .../org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java  | 6 --
 2 files changed, 5 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/2c99a03b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
--
diff --git 
a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
 
b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
index 5527076..9c47957 100644
--- 
a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
+++ 
b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
@@ -351,7 +351,7 @@ public abstract class Procedure implements 
Comparable {
* @param builder the string builder to use to append the proc specific 
information
*/
   protected void toStringClassDetails(StringBuilder builder) {
-builder.append(getClass().getSimpleName());
+builder.append(getClass().getName());
   }
 
   // ==

http://git-wip-us.apache.org/repos/asf/hbase/blob/2c99a03b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
--
diff --git 
a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
 
b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
index d36be64..e819ae8 100644
--- 
a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
+++ 
b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
@@ -1403,11 +1403,13 @@ public class ProcedureExecutor {
 reExecute = true;
 LOG.info("Short-circuit to rexecute for pid=" + 
procedure.getProcId());
   } else {
-// yield the current procedure, and make the subprocedure runnable
+// Yield the current procedure, and make the subprocedure runnable
+// subprocs may come back 'null'.
 subprocs = initializeChildren(procStack, procedure, subprocs);
 LOG.info("Initialized subprocedures=" +
+  (subprocs == null? null:
 Stream.of(subprocs).map(e -> "{" + e.toString() + "}").
-collect(Collectors.toList()).toString());
+collect(Collectors.toList()).toString()));
   }
 } else if (procedure.getState() == ProcedureState.WAITING_TIMEOUT) {
   if (LOG.isTraceEnabled()) {



[26/50] [abbrv] hbase git commit: HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi) Move to a new AssignmentManager, one that describes Assignment using a State Machine built on top

HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi)
Move to a new AssignmentManager, one that describes Assignment using
a State Machine built on top of ProcedureV2 facility.

Includes four patches from Matteos' repository and then fix up to get it 
all to
pass, filling in some missing functionality, fix of findbugs, fixing bugs, 
etc..

This doc. keeps state on where we are at w/ the new AM:

https://docs.google.com/document/d/1eVKa7FHdeoJ1-9o8yZcOTAQbv0u0bblBlCCzVSIn69g/edit#heading=h.vfdoxqut9lqn
Includes list of tests disabled by this patch with reasons why.

I applied the two patches in one go because applying each independently puts
hbase in a non-working state.

1. HBASE-14616 Procedure v2 - Replace the old AM with the new AM
The basis comes from Matteo's repo here:
  
https://github.com/matteobertozzi/hbase/commit/689227fcbfe8e6588433dbcdabf4526e3d478b2e

Patch replaces old AM with the new under subpackage master.assignment.
Mostly just updating classes to use new AM -- import changes -- rather
than the old. It also removes old AM and supporting classes.
See below for more detail.

2. HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi)
Adds running of remote procedure. Adds batching of remote calls.
Adds support for assign/unassign in procedures. Adds version info
reporting in rpc. Adds start of an AMv2.

3. and 4. are fixes around merge and split.

This work mostly comes from:

https://github.com/matteobertozzi/hbase/commit/3622cba4e331d2fc7bfc1932abb4c9cbf5802efa

Reporting of remote RS version is from here:

https://github.com/matteobertozzi/hbase/commit/ddb4df3964e8298c88c0210e83493aa91ac0942d.patch

And remote dispatch of procedures is from:

https://github.com/matteobertozzi/hbase/commit/186b9e7c4dae61a79509a6c3aad7f80ec61345e5

The split merge patches from here are also melded in:

https://github.com/matteobertozzi/hbase/commit/9a3a95a2c2974842a4849d1ad867e70764e7f707
and 
https://github.com/matteobertozzi/hbase/commit/d6289307a02a777299f65238238a2a8af3253067

Adds testing util for new AM and new sets of tests.

Does a bunch of fixup on logging so its possible to follow a procedures'
narrative by grepping procedure id. We spewed loads of log too on big
transitions such as master fail; fixed.

Details:

M hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java
 Takes List of regionstates on construction rather than a Set.
 NOTE! This is a change in a public class.

M hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
 Add utility getShortNameToLog

M 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
M 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java
 Add support for dispatching assign, split and merge processes.

M hbase-client/src/main/java/org/apache/hadoop/hbase/master/RegionState.java
 Purge old overlapping states: PENDING_OPEN, PENDING_CLOSE, etc.

A 
hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java
 Dispatch remote procedures every 150ms or 32 items -- which ever
 happens first (configurable). Runs a timeout thread. This facility is
 not on yet; will come in as part of a later fix. Currently works a
 region at a time. This class carries notion of a remote procedure and of a 
buffer full of these.
 "hbase.procedure.remote.dispatcher.threadpool.size" with default = 128
 "hbase.procedure.remote.dispatcher.delay.msec" with default = 150ms
 "hbase.procedure.remote.dispatcher.max.queue.size" with default = 32

M 
hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
 Add in support for merge. Remove no-longer used methods.

M hbase-protocol-shaded/src/main/protobuf/Admin.proto 
b/hbase-protocol-shaded/src/main/protobuf/Admin.proto
 Add execute procedures call ExecuteProcedures.

M hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
 Add assign and unassign state support for procedures.

M 
hbase-server/src/main/java/org/apache/hadoop/hbase/client/VersionInfoUtil.java
 Adds getting RS version out of RPC
 Examples: (1.3.4 is 0x0103004, 2.1.0 is 0x0201000)

M hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
 Remove periodic metrics chore. This is done over in new AM now.
 Replace AM with the new.

M 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java
 Have AMv2 handle assigning meta.

M 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
 Extract version number of the server making rpc.

A 
hbase-server/src/main/j

[14/50] [abbrv] hbase git commit: HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi) Move to a new AssignmentManager, one that describes Assignment using a State Machine built on top

http://git-wip-us.apache.org/repos/asf/hbase/blob/db1dcf3e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
new file mode 100644
index 000..177f397
--- /dev/null
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
@@ -0,0 +1,737 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.master.assignment;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Arrays;
+import java.util.ArrayList;
+import java.util.Collection;
+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.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.MetaMutationAnnotation;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.UnknownRegionException;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.MasterSwitchType;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.exceptions.MergeRegionException;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.master.assignment.RegionStates;
+import org.apache.hadoop.hbase.master.CatalogJanitor;
+import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.master.MasterFileSystem;
+import org.apache.hadoop.hbase.master.RegionState;
+import 
org.apache.hadoop.hbase.master.procedure.AbstractStateMachineTableProcedure;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil;
+import org.apache.hadoop.hbase.procedure2.Procedure.LockState;
+import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
+import org.apache.hadoop.hbase.regionserver.StoreFile;
+import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.FSUtils;
+
+/**
+ * The procedure to Merge a region in a table.
+ */
+@InterfaceAudience.Private
+public class MergeTableRegionsProcedure
+extends AbstractStateMachineTableProcedure {
+  private static final Log LOG = 
LogFactory.getLog(MergeTableRegionsProcedure.class);
+
+  private Boolean traceEnabled;
+
+  private ServerName regionLocation;
+  private String regionsToMergeListFullName;
+
+  private HRegionInfo[] regionsToMerge;
+  private HRegionInfo mergedRegion;
+  private boolean forcible;
+
+  public MergeTableRegionsProcedure() {
+// Required by the Procedure framework to create the procedure on replay
+  }
+
+  public MergeTableRegionsProcedure(final MasterProcedureEnv env,
+  final HRegionInfo regionToMergeA, final HRegionInfo regionToMergeB) 
throws IOException {
+this(env, regionToMergeA, regionToMergeB, false);
+  }
+
+  public MergeTableRegionsProcedure(final MasterProcedureEnv env,
+  final HRegionInfo regionToMergeA, final HRegionInfo regionToMergeB,
+  final boolean forcible) throws MergeRegionException {
+this(env, new HRegionInfo[] {regionToMergeA, regionToMergeB}, forcible);
+  }
+
+  public MergeTableRegionsProcedure(final MasterProcedureEnv env,
+  final HRegionInfo[] regionsTo

[37/50] [abbrv] hbase git commit: Fix CatalogTracker. Make it use Procedures doing clean up of Region data on split/merge. Without these changes, ITBLL was failing at larger scale (3-4hours 5B rows) b

http://git-wip-us.apache.org/repos/asf/hbase/blob/5c422d62/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProcedureProtos.java
--
diff --git 
a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProcedureProtos.java
 
b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProcedureProtos.java
index 9780e4f..5c72331 100644
--- 
a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProcedureProtos.java
+++ 
b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProcedureProtos.java
@@ -2533,6 +2533,204 @@ public final class MasterProcedureProtos {
 // @@protoc_insertion_point(enum_scope:hbase.pb.MoveRegionState)
   }
 
+  /**
+   * Protobuf enum {@code hbase.pb.GCRegionState}
+   */
+  public enum GCRegionState
+  implements 
org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
+/**
+ * GC_REGION_PREPARE = 1;
+ */
+GC_REGION_PREPARE(1),
+/**
+ * GC_REGION_ARCHIVE = 2;
+ */
+GC_REGION_ARCHIVE(2),
+/**
+ * GC_REGION_PURGE_METADATA = 3;
+ */
+GC_REGION_PURGE_METADATA(3),
+;
+
+/**
+ * GC_REGION_PREPARE = 1;
+ */
+public static final int GC_REGION_PREPARE_VALUE = 1;
+/**
+ * GC_REGION_ARCHIVE = 2;
+ */
+public static final int GC_REGION_ARCHIVE_VALUE = 2;
+/**
+ * GC_REGION_PURGE_METADATA = 3;
+ */
+public static final int GC_REGION_PURGE_METADATA_VALUE = 3;
+
+
+public final int getNumber() {
+  return value;
+}
+
+/**
+ * @deprecated Use {@link #forNumber(int)} instead.
+ */
+@java.lang.Deprecated
+public static GCRegionState valueOf(int value) {
+  return forNumber(value);
+}
+
+public static GCRegionState forNumber(int value) {
+  switch (value) {
+case 1: return GC_REGION_PREPARE;
+case 2: return GC_REGION_ARCHIVE;
+case 3: return GC_REGION_PURGE_METADATA;
+default: return null;
+  }
+}
+
+public static 
org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
+internalGetValueMap() {
+  return internalValueMap;
+}
+private static final 
org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
+GCRegionState> internalValueMap =
+  new 
org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap()
 {
+public GCRegionState findValueByNumber(int number) {
+  return GCRegionState.forNumber(number);
+}
+  };
+
+public final 
org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
+getValueDescriptor() {
+  return getDescriptor().getValues().get(ordinal());
+}
+public final 
org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
+getDescriptorForType() {
+  return getDescriptor();
+}
+public static final 
org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
+getDescriptor() {
+  return 
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(20);
+}
+
+private static final GCRegionState[] VALUES = values();
+
+public static GCRegionState valueOf(
+
org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
 desc) {
+  if (desc.getType() != getDescriptor()) {
+throw new java.lang.IllegalArgumentException(
+  "EnumValueDescriptor is not for this type.");
+  }
+  return VALUES[desc.getIndex()];
+}
+
+private final int value;
+
+private GCRegionState(int value) {
+  this.value = value;
+}
+
+// @@protoc_insertion_point(enum_scope:hbase.pb.GCRegionState)
+  }
+
+  /**
+   * Protobuf enum {@code hbase.pb.GCMergedRegionsState}
+   */
+  public enum GCMergedRegionsState
+  implements 
org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
+/**
+ * GC_MERGED_REGIONS_PREPARE = 1;
+ */
+GC_MERGED_REGIONS_PREPARE(1),
+/**
+ * GC_MERGED_REGIONS_PURGE = 2;
+ */
+GC_MERGED_REGIONS_PURGE(2),
+/**
+ * GC_REGION_EDIT_METADATA = 3;
+ */
+GC_REGION_EDIT_METADATA(3),
+;
+
+/**
+ * GC_MERGED_REGIONS_PREPARE = 1;
+ */
+public static final int GC_MERGED_REGIONS_PREPARE_VALUE = 1;
+/**
+ * GC_MERGED_REGIONS_PURGE = 2;
+ */
+public static final int GC_MERGED_REGIONS_PURGE_VALUE = 2;
+/**
+ * GC_REGION_EDIT_METADATA = 3;
+ */
+public static final int GC_REGION_EDIT_METADATA_VALUE = 3;
+
+
+public final int getNumber() {
+  return value;
+}
+
+/**
+ * @deprecated Use {@link #forNumber(int)} instead.
+ */
+@java.lang.D

[12/50] [abbrv] hbase git commit: HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi) Move to a new AssignmentManager, one that describes Assignment using a State Machine built on top

http://git-wip-us.apache.org/repos/asf/hbase/blob/db1dcf3e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java
index b53ce45..4d45af3 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java
@@ -21,12 +21,9 @@ package org.apache.hadoop.hbase.master.procedure;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
-import java.util.List;
-import java.util.concurrent.ExecutorService;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotEnabledException;
@@ -34,17 +31,11 @@ import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.constraint.ConstraintException;
-import org.apache.hadoop.hbase.master.AssignmentManager;
-import org.apache.hadoop.hbase.master.BulkAssigner;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
-import org.apache.hadoop.hbase.master.RegionState;
-import org.apache.hadoop.hbase.master.RegionStates;
 import org.apache.hadoop.hbase.master.TableStateManager;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
 import 
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DisableTableState;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.htrace.Trace;
 
 @InterfaceAudience.Private
 public class DisableTableProcedure
@@ -116,12 +107,8 @@ public class DisableTableProcedure
 setNextState(DisableTableState.DISABLE_TABLE_MARK_REGIONS_OFFLINE);
 break;
   case DISABLE_TABLE_MARK_REGIONS_OFFLINE:
-if (markRegionsOffline(env, tableName, true) ==
-MarkRegionOfflineOpResult.MARK_ALL_REGIONS_OFFLINE_SUCCESSFUL) {
-  
setNextState(DisableTableState.DISABLE_TABLE_SET_DISABLED_TABLE_STATE);
-} else {
-  LOG.trace("Retrying later to disable the missing regions");
-}
+
addChildProcedure(env.getAssignmentManager().createUnassignProcedures(tableName));
+setNextState(DisableTableState.DISABLE_TABLE_SET_DISABLED_TABLE_STATE);
 break;
   case DISABLE_TABLE_SET_DISABLED_TABLE_STATE:
 setTableStateToDisabled(env, tableName);
@@ -290,83 +277,6 @@ public class DisableTableProcedure
   }
 
   /**
-   * Mark regions of the table offline with retries
-   * @param env MasterProcedureEnv
-   * @param tableName the target table
-   * @param retryRequired whether to retry if the first run failed
-   * @return whether the operation is fully completed or being interrupted.
-   * @throws IOException
-   */
-  protected static MarkRegionOfflineOpResult markRegionsOffline(
-  final MasterProcedureEnv env,
-  final TableName tableName,
-  final Boolean retryRequired) throws IOException {
-// Dev consideration: add a config to control max number of retry. For 
now, it is hard coded.
-int maxTry = (retryRequired ? 10 : 1);
-MarkRegionOfflineOpResult operationResult =
-MarkRegionOfflineOpResult.BULK_ASSIGN_REGIONS_FAILED;
-do {
-  try {
-operationResult = markRegionsOffline(env, tableName);
-if (operationResult == 
MarkRegionOfflineOpResult.MARK_ALL_REGIONS_OFFLINE_SUCCESSFUL) {
-  break;
-}
-maxTry--;
-  } catch (Exception e) {
-LOG.warn("Received exception while marking regions online. tries left: 
" + maxTry, e);
-maxTry--;
-if (maxTry > 0) {
-  continue; // we still have some retry left, try again.
-}
-throw e;
-  }
-} while (maxTry > 0);
-
-if (operationResult != 
MarkRegionOfflineOpResult.MARK_ALL_REGIONS_OFFLINE_SUCCESSFUL) {
-  LOG.warn("Some or all regions of the Table '" + tableName + "' were 
still online");
-}
-
-return operationResult;
-  }
-
-  /**
-   * Mark regions of the table offline
-   * @param env MasterProcedureEnv
-   * @param tableName the target table
-   * @return whether the operation is fully completed or being interrupted.
-   * @throws IOException
-   */
-  private static MarkRegionOfflineOpResult markRegionsOffline(
-  final MasterProcedureEnv env,
-  final TableName tableName) throws IOException {
-// Get list of online reg

[19/50] [abbrv] hbase git commit: HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi) Move to a new AssignmentManager, one that describes Assignment using a State Machine built on top

http://git-wip-us.apache.org/repos/asf/hbase/blob/db1dcf3e/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/SnapshotProtos.java
--
diff --git 
a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/SnapshotProtos.java
 
b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/SnapshotProtos.java
index 454e3bc..4d5953c 100644
--- 
a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/SnapshotProtos.java
+++ 
b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/SnapshotProtos.java
@@ -357,7 +357,7 @@ public final class SnapshotProtos {
   if (ref instanceof java.lang.String) {
 return (java.lang.String) ref;
   } else {
-org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
 (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) 
ref;
 java.lang.String s = bs.toStringUtf8();
 if (bs.isValidUtf8()) {
@@ -373,7 +373,7 @@ public final class SnapshotProtos {
 getNameBytes() {
   java.lang.Object ref = name_;
   if (ref instanceof java.lang.String) {
-org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b =
+org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
 
org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
 (java.lang.String) ref);
 name_ = b;
@@ -407,7 +407,7 @@ public final class SnapshotProtos {
   if (ref instanceof java.lang.String) {
 return (java.lang.String) ref;
   } else {
-org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
 (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) 
ref;
 java.lang.String s = bs.toStringUtf8();
 if (bs.isValidUtf8()) {
@@ -427,7 +427,7 @@ public final class SnapshotProtos {
 getTableBytes() {
   java.lang.Object ref = table_;
   if (ref instanceof java.lang.String) {
-org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b =
+org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
 
org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
 (java.lang.String) ref);
 table_ = b;
@@ -499,7 +499,7 @@ public final class SnapshotProtos {
   if (ref instanceof java.lang.String) {
 return (java.lang.String) ref;
   } else {
-org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
 (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) 
ref;
 java.lang.String s = bs.toStringUtf8();
 if (bs.isValidUtf8()) {
@@ -515,7 +515,7 @@ public final class SnapshotProtos {
 getOwnerBytes() {
   java.lang.Object ref = owner_;
   if (ref instanceof java.lang.String) {
-org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b =
+org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
 
org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
 (java.lang.String) ref);
 owner_ = b;
@@ -1047,7 +1047,7 @@ public final class SnapshotProtos {
   getNameBytes() {
 java.lang.Object ref = name_;
 if (ref instanceof String) {
-  org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b =
+  org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
   
org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
   (java.lang.String) ref);
   name_ = b;
@@ -1135,7 +1135,7 @@ public final class SnapshotProtos {
   getTableBytes() {
 java.lang.Object ref = table_;
 if (ref instanceof String) {
-  org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b =
+  org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
   
org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
   (java.lang.String) ref);
   table_ = b;
@@ -1323,7 +1323,7 @@ public final class SnapshotProtos {
   getOwnerBytes() {
 java.lang.Object ref = owner_;
 if (ref instanceof String) {
-  org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b =
+  org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
   
org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
   (java.lang.String) ref);
   owner_ = b;
@@ -1

[01/50] [abbrv] hbase git commit: Making more logging trace-level [Forced Update!]

Repository: hbase
Updated Branches:
  refs/heads/HBASE-14614 052fab32a -> f24362c15 (forced update)


Making more logging trace-level


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

Branch: refs/heads/HBASE-14614
Commit: 007d0d2bc41ac4e6033944e20ed8c443a904c4d6
Parents: f101abd
Author: Michael Stack 
Authored: Fri May 12 10:28:39 2017 -0700
Committer: Michael Stack 
Committed: Tue May 23 08:36:53 2017 -0700

--
 .../hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java| 6 --
 .../hadoop/hbase/procedure2/store/wal/WALProcedureStore.java   | 4 ++--
 .../org/apache/hadoop/hbase/zookeeper/RegionServerTracker.java | 4 ++--
 3 files changed, 8 insertions(+), 6 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/007d0d2b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java
--
diff --git 
a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java
 
b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java
index 585762b..2221cfc 100644
--- 
a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java
+++ 
b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java
@@ -167,8 +167,10 @@ public class ProcedureWALFile implements 
Comparable {
   LOG.warn("Making " + archiveDir, ioe);
 }
 Path archivedFile = new Path(archiveDir, logFile.getName());
-LOG.info("ARCHIVED WAL (FIX) " + logFile + " to " + archivedFile);
-fs.rename(logFile, archivedFile);
+LOG.info("ARCHIVED WAL (TODO: FILES ARE NOT PURGED FROM ARCHIVE!) " + 
logFile + " to " + archivedFile);
+if (!fs.rename(logFile, archivedFile)) {
+  LOG.warn("Failed archive of " + logFile);
+}
 // fs.delete(logFile, false);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/007d0d2b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
--
diff --git 
a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
 
b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
index 7eeb2df..300e023 100644
--- 
a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
+++ 
b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
@@ -1079,8 +1079,8 @@ public class WALProcedureStore extends ProcedureStoreBase 
{
   private void removeAllLogs(long lastLogId) {
 if (logs.size() <= 1) return;
 
-if (LOG.isDebugEnabled()) {
-  LOG.debug("Remove all state logs with ID less than " + lastLogId);
+if (LOG.isTraceEnabled()) {
+  LOG.trace("Remove all state logs with ID less than " + lastLogId);
 }
 
 boolean removed = false;

http://git-wip-us.apache.org/repos/asf/hbase/blob/007d0d2b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionServerTracker.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionServerTracker.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionServerTracker.java
index 69cd233..a6a5c17 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionServerTracker.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionServerTracker.java
@@ -88,8 +88,8 @@ public class RegionServerTracker extends ZooKeeperListener {
   int magicLen = ProtobufUtil.lengthOfPBMagic();
   ProtobufUtil.mergeFrom(rsInfoBuilder, data, magicLen, 
data.length - magicLen);
 }
-if (LOG.isDebugEnabled()) {
-  LOG.debug("Added tracking of RS " + nodePath);
+if (LOG.isTraceEnabled()) {
+  LOG.trace("Added tracking of RS " + nodePath);
 }
   } catch (KeeperException e) {
 LOG.warn("Get Rs info port from ephemeral node", e);



[21/50] [abbrv] hbase git commit: HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi) Move to a new AssignmentManager, one that describes Assignment using a State Machine built on top

http://git-wip-us.apache.org/repos/asf/hbase/blob/db1dcf3e/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java
--
diff --git 
a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java
 
b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java
index 2435564..1ccf488 100644
--- 
a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java
+++ 
b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java
@@ -2210,7 +2210,7 @@ public final class QuotaProtos {
* optional .hbase.pb.TimedQuota req_num = 1;
*/
   private 
org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-  
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, 
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder,
 
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder>
+  
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, 
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder,
 
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder>
 
   getReqNumFieldBuilder() {
 if (reqNumBuilder_ == null) {
   reqNumBuilder_ = new 
org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -2328,7 +2328,7 @@ public final class QuotaProtos {
* optional .hbase.pb.TimedQuota req_size = 2;
*/
   private 
org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-  
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, 
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder,
 
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder>
+  
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, 
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder,
 
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder>
 
   getReqSizeFieldBuilder() {
 if (reqSizeBuilder_ == null) {
   reqSizeBuilder_ = new 
org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -2446,7 +2446,7 @@ public final class QuotaProtos {
* optional .hbase.pb.TimedQuota write_num = 3;
*/
   private 
org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-  
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, 
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder,
 
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder>
+  
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, 
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder,
 
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder>
 
   getWriteNumFieldBuilder() {
 if (writeNumBuilder_ == null) {
   writeNumBuilder_ = new 
org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -2564,7 +2564,7 @@ public final class QuotaProtos {
* optional .hbase.pb.TimedQuota write_size = 4;
*/
   private 
org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-  
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, 
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder,
 
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder>
+  
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, 
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder,
 
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder>
 
   getWriteSizeFieldBuilder() {
 if (writeSizeBuilder_ == null) {
   writeSizeBuilder_ = new 
org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -2682,7 +2682,7 @@ public final class QuotaProtos {
* optional .hbase.pb.TimedQuota read_num = 5;
*/
   private 
org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-  
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, 
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder,
 
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder>
+  
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, 
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder,
 
org.apache.hadoop.hbase.shaded.pro

[24/50] [abbrv] hbase git commit: HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi) Move to a new AssignmentManager, one that describes Assignment using a State Machine built on top

http://git-wip-us.apache.org/repos/asf/hbase/blob/db1dcf3e/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/AdminProtos.java
--
diff --git 
a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/AdminProtos.java
 
b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/AdminProtos.java
index 711b9c8..1ffdc52 100644
--- 
a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/AdminProtos.java
+++ 
b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/AdminProtos.java
@@ -9414,53 +9414,70 @@ public final class AdminProtos {
 
   }
 
-  public interface CloseRegionForSplitOrMergeRequestOrBuilder extends
-  // 
@@protoc_insertion_point(interface_extends:hbase.pb.CloseRegionForSplitOrMergeRequest)
+  public interface FlushRegionRequestOrBuilder extends
+  // 
@@protoc_insertion_point(interface_extends:hbase.pb.FlushRegionRequest)
   org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
 /**
- * repeated .hbase.pb.RegionSpecifier region = 1;
+ * required .hbase.pb.RegionSpecifier region = 1;
  */
-
java.util.List
 
-getRegionList();
+boolean hasRegion();
 /**
- * repeated .hbase.pb.RegionSpecifier region = 1;
+ * required .hbase.pb.RegionSpecifier region = 1;
  */
-
org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier 
getRegion(int index);
+
org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier 
getRegion();
 /**
- * repeated .hbase.pb.RegionSpecifier region = 1;
+ * required .hbase.pb.RegionSpecifier region = 1;
  */
-int getRegionCount();
+
org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder
 getRegionOrBuilder();
+
 /**
- * repeated .hbase.pb.RegionSpecifier region = 1;
+ * optional uint64 if_older_than_ts = 2;
  */
-java.util.List
 
-getRegionOrBuilderList();
+boolean hasIfOlderThanTs();
 /**
- * repeated .hbase.pb.RegionSpecifier region = 1;
+ * optional uint64 if_older_than_ts = 2;
  */
-
org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder
 getRegionOrBuilder(
-int index);
+long getIfOlderThanTs();
+
+/**
+ * 
+ * whether to write a marker to WAL even if not flushed
+ * 
+ *
+ * optional bool write_flush_wal_marker = 3;
+ */
+boolean hasWriteFlushWalMarker();
+/**
+ * 
+ * whether to write a marker to WAL even if not flushed
+ * 
+ *
+ * optional bool write_flush_wal_marker = 3;
+ */
+boolean getWriteFlushWalMarker();
   }
   /**
* 
**
-   * Closes the specified region(s) for
-   * split or merge
+   * Flushes the MemStore of the specified region.
+   * 

+ * This method is synchronous. * * - * Protobuf type {@code hbase.pb.CloseRegionForSplitOrMergeRequest} + * Protobuf type {@code hbase.pb.FlushRegionRequest} */ - public static final class CloseRegionForSplitOrMergeRequest extends + public static final class FlushRegionRequest extends org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements - // @@protoc_insertion_point(message_implements:hbase.pb.CloseRegionForSplitOrMergeRequest) - CloseRegionForSplitOrMergeRequestOrBuilder { -// Use CloseRegionForSplitOrMergeRequest.newBuilder() to construct. -private CloseRegionForSplitOrMergeRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) { + // @@protoc_insertion_point(message_implements:hbase.pb.FlushRegionRequest) + FlushRegionRequestOrBuilder { +// Use FlushRegionRequest.newBuilder() to construct. +private FlushRegionRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) { super(builder); } -private CloseRegionForSplitOrMergeRequest() { - region_ = java.util.Collections.emptyList(); +private FlushRegionRequest() { + ifOlderThanTs_ = 0L; + writeFlushWalMarker_ = false; } @java.lang.Override @@ -9468,7 +9485,7 @@ public final class AdminProtos { getUnknownFields() { return this.unknownFields; } -private CloseRegionForSplitOrMergeRequest( +private FlushRegionRequest( org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { @@ -9492,12 +9509,26 @@ public final class AdminProtos { break; } case 10: { - if (!((mutable_bitField0_ & 0x0001)


[22/50] [abbrv] hbase git commit: HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi) Move to a new AssignmentManager, one that describes Assignment using a State Machine built on top

http://git-wip-us.apache.org/repos/asf/hbase/blob/db1dcf3e/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
--
diff --git 
a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
 
b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
index 798932e..d5846ce 100644
--- 
a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
+++ 
b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
@@ -842,7 +842,7 @@ public final class MasterProtos {
* required .hbase.pb.TableName table_name = 1;
*/
   private 
org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-  
org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, 
org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder,
 
org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>
+  
org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, 
org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder,
 
org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>
 
   getTableNameFieldBuilder() {
 if (tableNameBuilder_ == null) {
   tableNameBuilder_ = new 
org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -960,7 +960,7 @@ public final class MasterProtos {
* required .hbase.pb.ColumnFamilySchema column_families = 
2;
*/
   private 
org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-  
org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema,
 
org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema.Builder,
 
org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchemaOrBuilder>
+  
org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema,
 
org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema.Builder,
 
org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchemaOrBuilder>
 
   getColumnFamiliesFieldBuilder() {
 if (columnFamiliesBuilder_ == null) {
   columnFamiliesBuilder_ = new 
org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -2257,7 +2257,7 @@ public final class MasterProtos {
* required .hbase.pb.TableName table_name = 1;
*/
   private 
org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-  
org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, 
org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder,
 
org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>
+  
org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, 
org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder,
 
org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>
 
   getTableNameFieldBuilder() {
 if (tableNameBuilder_ == null) {
   tableNameBuilder_ = new 
org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -3622,7 +3622,7 @@ public final class MasterProtos {
* required .hbase.pb.TableName table_name = 1;
*/
   private 
org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-  
org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, 
org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder,
 
org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>
+  
org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, 
org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder,
 
org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>
 
   getTableNameFieldBuilder() {
 if (tableNameBuilder_ == null) {
   tableNameBuilder_ = new 
org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -3740,7 +3740,7 @@ public final class MasterProtos {
* required .hbase.pb.ColumnFamilySchema column_families = 
2;
*/
   private 
org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-  
org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema,
 
org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema.Builder,
 
org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchemaOrBuilder>
+  
org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.Co

[05/50] [abbrv] hbase git commit: HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi) Move to a new AssignmentManager, one that describes Assignment using a State Machine built on top

http://git-wip-us.apache.org/repos/asf/hbase/blob/db1dcf3e/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStates.java
--
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStates.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStates.java
new file mode 100644
index 000..003dfdd
--- /dev/null
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStates.java
@@ -0,0 +1,224 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.master.assignment;
+
+import static org.junit.Assert.assertEquals;
+
+import java.lang.Thread.UncaughtExceptionHandler;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.Future;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.procedure2.util.StringUtils;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Threads;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({MasterTests.class, MediumTests.class})
+public class TestRegionStates {
+  private static final Log LOG = LogFactory.getLog(TestRegionStates.class);
+
+  protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  private static ThreadPoolExecutor threadPool;
+  private static ExecutorCompletionService executorService;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+threadPool = Threads.getBoundedCachedThreadPool(32, 60L, TimeUnit.SECONDS,
+  Threads.newDaemonThreadFactory("ProcedureDispatcher",
+new UncaughtExceptionHandler() {
+  @Override
+  public void uncaughtException(Thread t, Throwable e) {
+LOG.warn("Failed thread " + t.getName(), e);
+  }
+}));
+executorService = new ExecutorCompletionService(threadPool);
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+threadPool.shutdown();
+  }
+
+  @Before
+  public void testSetup() {
+  }
+
+  @After
+  public void testTearDown() throws Exception {
+while (true) {
+  Future f = executorService.poll();
+  if (f == null) break;
+  f.get();
+}
+  }
+
+  private static void waitExecutorService(final int count) throws Exception {
+for (int i = 0; i < count; ++i) {
+  executorService.take().get();
+}
+  }
+
+  // ==
+  //  Regions related
+  // ==
+
+  @Test
+  public void testRegionDoubleCreation() throws Exception {
+// NOTE: HRegionInfo sort by table first, so we are relying on that
+final TableName TABLE_NAME_A = TableName.valueOf("testOrderedByTableA");
+final TableName TABLE_NAME_B = TableName.valueOf("testOrderedByTableB");
+final TableName TABLE_NAME_C = TableName.valueOf("testOrderedByTableC");
+final RegionStates stateMap = new RegionStates();
+final int NRUNS = 1000;
+final int NSMALL_RUNS = 3;
+
+// add some regions for table B
+for (int i = 0; i < NRUNS; ++i) {
+  addRegionNode(stateMap, TABLE_NAME_B, i);
+}
+// re-add the regions for table B
+for (int i = 0; i < NRUNS; ++i) {
+  addRegionNode(stateMap, TABLE_NAME_B, i);
+}
+waitExecutorService(NRUNS * 2);
+
+// add two other tables A and C that will be placed before and after table 
B (sort order)
+for (int i = 0; i < NSMALL_RUNS; ++i) {
+ 

[28/50] [abbrv] hbase git commit: Rebase with master fixup

Rebase with master fixup


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

Branch: refs/heads/HBASE-14614
Commit: bd672032fc8efe1555eeeb35fbf94e798fb96296
Parents: 924bb80
Author: Michael Stack 
Authored: Fri May 5 21:58:22 2017 -0700
Committer: Michael Stack 
Committed: Tue May 23 08:36:53 2017 -0700

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/bd672032/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java
--
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java
index 79eb7d2..a1b33f7 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java
@@ -49,6 +49,7 @@ import java.util.concurrent.LinkedBlockingQueue;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CategoryBasedTimeout;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.client.Put;
@@ -66,8 +67,10 @@ import org.apache.hadoop.hbase.util.EnvironmentEdge;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.Threads;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.rules.TestRule;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 



[08/50] [abbrv] hbase git commit: HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi) Move to a new AssignmentManager, one that describes Assignment using a State Machine built on top

http://git-wip-us.apache.org/repos/asf/hbase/blob/db1dcf3e/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeRequest.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeRequest.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeRequest.java
new file mode 100644
index 000..e95932b
--- /dev/null
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeRequest.java
@@ -0,0 +1,108 @@
+/**
+ * Copyright The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hadoop.hbase.regionserver;
+
+import java.security.PrivilegedAction;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.security.User;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Handles processing region merges. Put in a queue, owned by HRegionServer.
+ */
+// UNUSED: REMOVE!!!
+@InterfaceAudience.Private
+class RegionMergeRequest implements Runnable {
+  private static final Log LOG = LogFactory.getLog(RegionMergeRequest.class);
+  private final HRegionInfo region_a;
+  private final HRegionInfo region_b;
+  private final HRegionServer server;
+  private final boolean forcible;
+  private final User user;
+
+  RegionMergeRequest(Region a, Region b, HRegionServer hrs, boolean forcible,
+  long masterSystemTime, User user) {
+Preconditions.checkNotNull(hrs);
+this.region_a = a.getRegionInfo();
+this.region_b = b.getRegionInfo();
+this.server = hrs;
+this.forcible = forcible;
+this.user = user;
+  }
+
+  @Override
+  public String toString() {
+return "MergeRequest,regions:" + region_a + ", " + region_b + ", forcible="
++ forcible;
+  }
+
+  private void doMerge() {
+boolean success = false;
+//server.metricsRegionServer.incrMergeRequest();
+
+if (user != null && user.getUGI() != null) {
+  user.getUGI().doAs (new PrivilegedAction() {
+@Override
+public Void run() {
+  requestRegionMerge();
+  return null;
+}
+  });
+} else {
+  requestRegionMerge();
+}
+  }
+
+  private void requestRegionMerge() {
+final TableName table = region_a.getTable();
+if (!table.equals(region_b.getTable())) {
+  LOG.error("Can't merge regions from two different tables: " + region_a + 
", " + region_b);
+  return;
+}
+
+// TODO: fake merged region for compat with the report protocol
+final HRegionInfo merged = new HRegionInfo(table);
+
+// Send the split request to the master. the master will do the validation 
on the split-key.
+// The parent region will be unassigned and the two new regions will be 
assigned.
+// hri_a and hri_b objects may not reflect the regions that will be 
created, those objectes
+// are created just to pass the information to the 
reportRegionStateTransition().
+if (!server.reportRegionStateTransition(TransitionCode.READY_TO_MERGE, 
merged, region_a, region_b)) {
+  LOG.error("Unable to ask master to merge: " + region_a + ", " + 
region_b);
+}
+  }
+
+  @Override
+  public void run() {
+if (this.server.isStopping() || this.server.isStopped()) {
+  LOG.debug("Skipping merge because server is stopping="
+  + this.server.isStopping() + " or stopped=" + 
this.server.isStopped());
+  return;
+}
+
+doMerge();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/db1dcf3e/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
in

[16/50] [abbrv] hbase git commit: HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi) Move to a new AssignmentManager, one that describes Assignment using a State Machine built on top

http://git-wip-us.apache.org/repos/asf/hbase/blob/db1dcf3e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java
deleted file mode 100644
index dcbf5a4..000
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java
+++ /dev/null
@@ -1,1170 +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.master;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Comparator;
-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.SortedSet;
-import java.util.TreeMap;
-import java.util.TreeSet;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-
-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.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.MetaTableAccessor;
-import org.apache.hadoop.hbase.ServerLoad;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.RegionReplicaUtil;
-import org.apache.hadoop.hbase.master.RegionState.State;
-import org.apache.hadoop.hbase.client.TableState;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.util.Pair;
-
-/**
- * Region state accountant. It holds the states of all regions in the memory.
- * In normal scenario, it should match the meta table and the true region 
states.
- *
- * This map is used by AssignmentManager to track region states.
- */
-@InterfaceAudience.Private
-public class RegionStates {
-  private static final Log LOG = LogFactory.getLog(RegionStates.class);
-
-  public final static RegionStateStampComparator REGION_STATE_COMPARATOR =
-new RegionStateStampComparator();
-
-  // This comparator sorts the RegionStates by time stamp then Region name.
-  // Comparing by timestamp alone can lead us to discard different 
RegionStates that happen
-  // to share a timestamp.
-  private static class RegionStateStampComparator implements 
Comparator {
-@Override
-public int compare(RegionState l, RegionState r) {
-  return Long.compare(l.getStamp(), r.getStamp()) == 0 ?
-  Bytes.compareTo(l.getRegion().getRegionName(), 
r.getRegion().getRegionName()) :
-  Long.compare(l.getStamp(), r.getStamp());
-}
-  }
-
-  /**
-   * Regions currently in transition.
-   */
-  final HashMap regionsInTransition = new HashMap<>();
-
-  /**
-   * Region encoded name to state map.
-   * All the regions should be in this map.
-   */
-  private final Map regionStates = new HashMap<>();
-
-  /**
-   * Holds mapping of table -> region state
-   */
-  private final Map> 
regionStatesTableIndex = new HashMap<>();
-
-  /**
-   * Server to regions assignment map.
-   * Contains the set of regions currently assigned to a given server.
-   */
-  private final Map> serverHoldings = new 
HashMap<>();
-
-  /**
-   * Maintains the mapping from the default region to the replica regions.
-   */
-  private final Map> 
defaultReplicaToOtherReplicas = new HashMap<>();
-
-  /**
-   * Region to server assignment map.
-   * Contains the server a given region is currently assigned to.
-   */
-  private final TreeMap regionAssignments = new 
TreeMap<>();
-
-  /**
-   * Encoded region name to server assignment map for re-assignment
-   * purpose. Contains the server a given region is last known assigned
-   * to, which has not completed log splitting, so not assignable.
-   * If a region is currently assigned, this server info in this
-   * map should be the same as that in regi

[18/50] [abbrv] hbase git commit: HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi) Move to a new AssignmentManager, one that describes Assignment using a State Machine built on top

http://git-wip-us.apache.org/repos/asf/hbase/blob/db1dcf3e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
deleted file mode 100644
index 69ebd97..000
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
+++ /dev/null
@@ -1,3053 +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.master;
-
-import com.google.common.annotations.VisibleForTesting;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.NavigableMap;
-import java.util.Random;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.CoordinatedStateException;
-import org.apache.hadoop.hbase.HBaseIOException;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HRegionLocation;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.MetaTableAccessor;
-import org.apache.hadoop.hbase.NotServingRegionException;
-import org.apache.hadoop.hbase.RegionLocations;
-import org.apache.hadoop.hbase.RegionStateListener;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.client.MasterSwitchType;
-import org.apache.hadoop.hbase.client.RegionReplicaUtil;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.TableState;
-import org.apache.hadoop.hbase.executor.EventHandler;
-import org.apache.hadoop.hbase.executor.EventType;
-import org.apache.hadoop.hbase.executor.ExecutorService;
-import org.apache.hadoop.hbase.favored.FavoredNodesManager;
-import org.apache.hadoop.hbase.favored.FavoredNodesPromoter;
-import org.apache.hadoop.hbase.ipc.FailedServerException;
-import org.apache.hadoop.hbase.ipc.RpcClient;
-import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
-import org.apache.hadoop.hbase.master.RegionState.State;
-import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
-import 
org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
-import 
org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
-import org.apache.hadoop.hbase.quotas.QuotaExceededException;
-import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
-import org.apache.hadoop.hbase.regionserver.RegionServerAbortedException;
-import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.util.KeyLocker;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.util.PairOfSameType;
-import org.apache.hadoop.hbase.util.RetryCounter;
-import org.apache.hadoop.hbase.util.Threads;
-import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.hadoop.ip

[02/50] [abbrv] hbase git commit: Fix archiving of pv2 WAL files

Fix archiving of pv2 WAL files


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

Branch: refs/heads/HBASE-14614
Commit: 53b865aa9922143e2e95af09a73814a9e7d60047
Parents: 007d0d2
Author: Michael Stack 
Authored: Fri May 12 13:02:32 2017 -0700
Committer: Michael Stack 
Committed: Tue May 23 08:36:53 2017 -0700

--
 .../procedure2/store/wal/ProcedureWALFile.java  | 28 ++--
 .../procedure2/store/wal/WALProcedureStore.java | 28 +---
 .../org/apache/hadoop/hbase/master/HMaster.java |  4 ++-
 .../assignment/TestAssignmentManager.java   |  2 +-
 4 files changed, 36 insertions(+), 26 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/53b865aa/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java
--
diff --git 
a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java
 
b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java
index 2221cfc..42abe8f 100644
--- 
a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java
+++ 
b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java
@@ -26,7 +26,6 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.procedure2.store.ProcedureStoreTracker;
@@ -156,22 +155,23 @@ public class ProcedureWALFile implements 
Comparable {
 this.logSize += size;
   }
 
-  public void removeFile() throws IOException {
+  public void removeFile(final Path walArchiveDir) throws IOException {
 close();
-// TODO: FIX THIS. MAKE THIS ARCHIVE FORMAL.
-Path archiveDir =
-new Path(logFile.getParent().getParent(), 
HConstants.HFILE_ARCHIVE_DIRECTORY);
-try {
-  fs.mkdirs(archiveDir);
-} catch (IOException ioe) {
-  LOG.warn("Making " + archiveDir, ioe);
+boolean archived = false;
+if (walArchiveDir != null) {
+  Path archivedFile = new Path(walArchiveDir, logFile.getName());
+  LOG.info("ARCHIVED (TODO: FILES ARE NOT PURGED FROM ARCHIVE!) " + 
logFile + " to " + walArchiveDir);
+  if (!fs.rename(logFile, archivedFile)) {
+LOG.warn("Failed archive of " + logFile + ", deleting");
+  } else {
+archived = true;
+  }
 }
-Path archivedFile = new Path(archiveDir, logFile.getName());
-LOG.info("ARCHIVED WAL (TODO: FILES ARE NOT PURGED FROM ARCHIVE!) " + 
logFile + " to " + archivedFile);
-if (!fs.rename(logFile, archivedFile)) {
-  LOG.warn("Failed archive of " + logFile);
+if (!archived) {
+  if (!fs.delete(logFile, false)) {
+LOG.warn("Failed delete of " + logFile);
+  }
 }
-// fs.delete(logFile, false);
   }
 
   public void setProcIds(long minId, long maxId) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/53b865aa/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
--
diff --git 
a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
 
b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
index 300e023..df818fe 100644
--- 
a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
+++ 
b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
@@ -124,6 +124,7 @@ public class WALProcedureStore extends ProcedureStoreBase {
   private final Configuration conf;
   private final FileSystem fs;
   private final Path walDir;
+  private final Path walArchiveDir;
 
   private final AtomicReference syncException = new 
AtomicReference<>();
   private final AtomicBoolean loading = new AtomicBoolean(true);
@@ -185,9 +186,15 @@ public class WALProcedureStore extends ProcedureStoreBase {
 
   public WALProcedureStore(final Configuration conf, final FileSystem fs, 
final Path walDir,
   final LeaseRecovery leaseRecovery) {
+this(conf, fs, walDir, null, leaseRecovery);
+  }
+
+  public WALProcedureStore(final Configuration conf, final FileSystem fs, 
final Path walDir,
+  final Path walArchiveDir, final LeaseRecovery

[31/50] [abbrv] hbase git commit: Debugging around the ask if a region is splittable -- doesn't seem to be working

Debugging around the ask if a region is splittable -- doesn't seem to be working


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

Branch: refs/heads/HBASE-14614
Commit: fa191d9a5d62a17884ab4189ca898e1519193b32
Parents: edad94e
Author: Michael Stack 
Authored: Sun May 14 22:45:05 2017 -0700
Committer: Michael Stack 
Committed: Tue May 23 08:36:53 2017 -0700

--
 .../hbase/master/assignment/SplitTableRegionProcedure.java   | 1 +
 .../main/java/org/apache/hadoop/hbase/regionserver/HRegion.java  | 4 +++-
 2 files changed, 4 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/fa191d9a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
index 4ed1931..6815e9f 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
@@ -365,6 +365,7 @@ public class SplitTableRegionProcedure
 GetRegionInfoResponse response =
 Util.getRegionInfoResponse(env, node.getRegionLocation(), 
node.getRegionInfo());
 splittable = response.hasSplittable() && response.getSplittable();
+LOG.info("REMOVE splittable " + splittable + " " + this + " " + node);
   } catch (IOException e) {
 splittableCheckIOE = e;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa191d9a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index f49e7bf..ac0867c 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -1392,7 +1392,9 @@ public class HRegion implements HeapSize, 
PropagatingConfigurationObserver, Regi
 
   @Override
   public boolean isSplittable() {
-return isAvailable() && !hasReferences();
+boolean result = isAvailable() && !hasReferences();
+LOG.info("ASKED IF SPLITTABLE " + result, new Throwable("LOGGING"));
+return result;
   }
 
   @Override



[11/50] [abbrv] hbase git commit: HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi) Move to a new AssignmentManager, one that describes Assignment using a State Machine built on top

http://git-wip-us.apache.org/repos/asf/hbase/blob/db1dcf3e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MergeTableRegionsProcedure.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MergeTableRegionsProcedure.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MergeTableRegionsProcedure.java
deleted file mode 100644
index 3600fe0..000
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MergeTableRegionsProcedure.java
+++ /dev/null
@@ -1,906 +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.master.procedure;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InterruptedIOException;
-import java.io.OutputStream;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-
-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.DoNotRetryIOException;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.MetaMutationAnnotation;
-import org.apache.hadoop.hbase.RegionLoad;
-import org.apache.hadoop.hbase.ServerLoad;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.UnknownRegionException;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.exceptions.MergeRegionException;
-import org.apache.hadoop.hbase.io.hfile.CacheConfig;
-import org.apache.hadoop.hbase.master.AssignmentManager;
-import org.apache.hadoop.hbase.master.CatalogJanitor;
-import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
-import org.apache.hadoop.hbase.master.MasterFileSystem;
-import org.apache.hadoop.hbase.master.RegionPlan;
-import org.apache.hadoop.hbase.master.RegionState;
-import org.apache.hadoop.hbase.master.RegionStates;
-import org.apache.hadoop.hbase.master.ServerManager;
-import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
-import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
-import 
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsState;
-import 
org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
-import 
org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.FSUtils;
-
-/**
- * The procedure to Merge a region in a table.
- */
-@InterfaceAudience.Private
-public class MergeTableRegionsProcedure
-extends AbstractStateMachineTableProcedure {
-  private static final Log LOG = 
LogFactory.getLog(MergeTableRegionsProcedure.class);
-
-  private Boolean traceEnabled;
-  private AssignmentManager assignmentManager;
-  private int timeout;
-  private ServerName regionLocation;
-  private String regionsToMergeListFullName;
-  private String regionsToMergeListEncodedName;
-
-  private HRegionInfo [] regionsToMerge;
-  private HRegionInfo mergedRegionInfo;
-  private boolean forcible;
-
-  public MergeTableRegionsProcedure() {
-this.traceEnabled = isTraceEnabled();
-this.assignmentManager = null;
-this.timeout = -1;
-this.regionLocation = null;
-this.regionsToMergeListFullName = null;
-this.regionsToMergeListEncodedName = null;
-  }
-
-  public MergeTableRegionsProcedure(
-  final MasterProcedureEnv env,
-  final HRegionInfo[] regionsToMerge,
-  final boolean forcible) throws IOException {
-super(env);

[27/50] [abbrv] hbase git commit: Don't drop old master proc wal files... need to debug

Don't drop old master proc wal files... need to debug


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

Branch: refs/heads/HBASE-14614
Commit: f2a5623d63fa34e8751f1c459f1d7193d3136238
Parents: db1dcf3
Author: Michael Stack 
Authored: Sun Apr 30 21:16:05 2017 -1000
Committer: Michael Stack 
Committed: Tue May 23 08:36:53 2017 -0700

--
 .../hbase/procedure2/store/wal/ProcedureWALFile.java  | 14 +-
 1 file changed, 13 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/f2a5623d/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java
--
diff --git 
a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java
 
b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java
index 012ddeb..585762b 100644
--- 
a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java
+++ 
b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.procedure2.store.ProcedureStoreTracker;
@@ -157,7 +158,18 @@ public class ProcedureWALFile implements 
Comparable {
 
   public void removeFile() throws IOException {
 close();
-fs.delete(logFile, false);
+// TODO: FIX THIS. MAKE THIS ARCHIVE FORMAL.
+Path archiveDir =
+new Path(logFile.getParent().getParent(), 
HConstants.HFILE_ARCHIVE_DIRECTORY);
+try {
+  fs.mkdirs(archiveDir);
+} catch (IOException ioe) {
+  LOG.warn("Making " + archiveDir, ioe);
+}
+Path archivedFile = new Path(archiveDir, logFile.getName());
+LOG.info("ARCHIVED WAL (FIX) " + logFile + " to " + archivedFile);
+fs.rename(logFile, archivedFile);
+// fs.delete(logFile, false);
   }
 
   public void setProcIds(long minId, long maxId) {



[25/50] [abbrv] hbase git commit: HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi) Move to a new AssignmentManager, one that describes Assignment using a State Machine built on top

http://git-wip-us.apache.org/repos/asf/hbase/blob/db1dcf3e/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java
--
diff --git 
a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java
 
b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java
new file mode 100644
index 000..8d5ff3c
--- /dev/null
+++ 
b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java
@@ -0,0 +1,375 @@
+/**
+ * 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.procedure2;
+
+import java.io.IOException;
+import java.lang.Thread.UncaughtExceptionHandler;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.DelayQueue;
+import java.util.concurrent.Future;
+import java.util.concurrent.FutureTask;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.procedure2.util.DelayedUtil;
+import 
org.apache.hadoop.hbase.procedure2.util.DelayedUtil.DelayedContainerWithTimestamp;
+import org.apache.hadoop.hbase.procedure2.util.DelayedUtil.DelayedWithTimeout;
+import org.apache.hadoop.hbase.procedure2.util.StringUtils;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.Threads;
+
+import com.google.common.collect.ArrayListMultimap;
+
+/**
+ * A procedure dispatcher that aggregates and sends after elapsed time or 
after we hit
+ * count threshold. Creates its own threadpool to run RPCs with timeout.
+ * 
+ * Each server queue has a dispatch buffer
+ * Once the dispatch buffer reaches a threshold-size/time we send
+ * 
+ * Call {@link #start()} and then {@link #submitTask(Callable)}. When done,
+ * call {@link #stop()}.
+ */
+@InterfaceAudience.Private
+public abstract class RemoteProcedureDispatcher> {
+  private static final Log LOG = 
LogFactory.getLog(RemoteProcedureDispatcher.class);
+
+  public static final String THREAD_POOL_SIZE_CONF_KEY =
+  "hbase.procedure.remote.dispatcher.threadpool.size";
+  private static final int DEFAULT_THREAD_POOL_SIZE = 128;
+
+  public static final String DISPATCH_DELAY_CONF_KEY =
+  "hbase.procedure.remote.dispatcher.delay.msec";
+  private static final int DEFAULT_DISPATCH_DELAY = 150;
+
+  public static final String DISPATCH_MAX_QUEUE_SIZE_CONF_KEY =
+  "hbase.procedure.remote.dispatcher.max.queue.size";
+  private static final int DEFAULT_MAX_QUEUE_SIZE = 32;
+
+  private final AtomicBoolean running = new AtomicBoolean(false);
+  private final ConcurrentHashMap nodeMap =
+  new ConcurrentHashMap();
+
+  private final int operationDelay;
+  private final int queueMaxSize;
+  private final int corePoolSize;
+
+  private TimeoutExecutorThread timeoutExecutor;
+  private ThreadPoolExecutor threadPool;
+
+  protected RemoteProcedureDispatcher(Configuration conf) {
+this.corePoolSize = conf.getInt(THREAD_POOL_SIZE_CONF_KEY, 
DEFAULT_THREAD_POOL_SIZE);
+this.operationDelay = conf.getInt(DISPATCH_DELAY_CONF_KEY, 
DEFAULT_DISPATCH_DELAY);
+this.queueMaxSize = conf.getInt(DISPATCH_MAX_QUEUE_SIZE_CONF_KEY, 
DEFAULT_MAX_QUEUE_SIZE);
+  }
+
+  public boolean start() {
+if (running.getAndSet(true)) {
+  LOG.warn("Already running");
+  return false;
+}
+
+LOG.info("Starting procedure remote dispatcher; threads=" + 
this.corePoolSize +
+  ", queueMaxSize=" + this.queueMaxSize + ", operationDelay=" + 
this.operationDelay);
+
+// Create the timeout executor
+timeoutExecutor = new TimeoutExecutorThread();
+timeoutExecutor.start();
+
+// Create the thread pool that will execute RPCs
+threadPool = Threads.getBoundedCachedThreadPool(corePoolSize, 60L, 

[06/50] [abbrv] hbase git commit: HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi) Move to a new AssignmentManager, one that describes Assignment using a State Machine built on top

http://git-wip-us.apache.org/repos/asf/hbase/blob/db1dcf3e/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetrics.java
--
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetrics.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetrics.java
index 0084d44..8a216c5 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetrics.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetrics.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.zookeeper.KeeperException;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -55,12 +56,13 @@ public class TestMasterMetrics {
 KeeperException, InterruptedException {
   super(conf, cp);
 }
-
+/*
 @Override
 protected void tryRegionServerReport(
 long reportStartTime, long reportEndTime) {
   // do nothing
 }
+*/
   }
 
   @BeforeClass
@@ -81,7 +83,7 @@ public class TestMasterMetrics {
 }
   }
 
-  @Test(timeout = 30)
+  @Ignore @Test(timeout = 30)
   public void testClusterRequests() throws Exception {
 
 // sending fake request to master to see how metric value has changed
@@ -114,7 +116,7 @@ public class TestMasterMetrics {
 master.stopMaster();
   }
 
-  @Test
+  @Ignore @Test
   public void testDefaultMasterMetrics() throws Exception {
 MetricsMasterSource masterSource = 
master.getMasterMetrics().getMetricsSource();
 metricsHelper.assertGauge( "numRegionServers", 2, masterSource);

http://git-wip-us.apache.org/repos/asf/hbase/blob/db1dcf3e/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java
--
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java
index 6c737e9..737d145 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hbase.master;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.util.Collection;
@@ -191,18 +192,20 @@ public class TestMasterOperationsForRegionReplicas {
   for (int i = 1; i < numSlaves; i++) { //restore the cluster
 TEST_UTIL.getMiniHBaseCluster().startRegionServer();
   }
-
-  //check on alter table
+/* DISABLED! FOR NOW
+  // Check on alter table
   ADMIN.disableTable(tableName);
   assert(ADMIN.isTableDisabled(tableName));
   //increase the replica
   desc.setRegionReplication(numReplica + 1);
   ADMIN.modifyTable(tableName, desc);
   ADMIN.enableTable(tableName);
+  LOG.info(ADMIN.getTableDescriptor(tableName).toString());
   assert(ADMIN.isTableEnabled(tableName));
   List regions = TEST_UTIL.getMiniHBaseCluster().getMaster()
   
.getAssignmentManager().getRegionStates().getRegionsOfTable(tableName);
-  assert(regions.size() == numRegions * (numReplica + 1));
+  assertTrue("regions.size=" + regions.size() + ", numRegions=" + 
numRegions + ", numReplica=" + numReplica,
+  regions.size() == numRegions * (numReplica + 1));
 
   //decrease the replica(earlier, table was modified to have a replica 
count of numReplica + 1)
   ADMIN.disableTable(tableName);
@@ -229,6 +232,7 @@ public class TestMasterOperationsForRegionReplicas {
   assert(defaultReplicas.size() == numRegions);
   Collection counts = new HashSet<>(defaultReplicas.values());
   assert(counts.size() == 1 && counts.contains(new Integer(numReplica)));
+  */
 } finally {
   ADMIN.disableTable(tableName);
   ADMIN.deleteTable(tableName);

http://git-wip-us.apache.org/repos/asf/hbase/blob/db1dcf3e/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java
--
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java
index b59e6ff..23efdb2 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java
@@ -18,15 +18,12 @@
  */
 package org.apache.hadoop.hbase.master;
 
-import static org.junit.Assert.*;
-

[23/50] [abbrv] hbase git commit: HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi) Move to a new AssignmentManager, one that describes Assignment using a State Machine built on top

http://git-wip-us.apache.org/repos/asf/hbase/blob/db1dcf3e/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProcedureProtos.java
--
diff --git 
a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProcedureProtos.java
 
b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProcedureProtos.java
index d7bbd05..9780e4f 100644
--- 
a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProcedureProtos.java
+++ 
b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProcedureProtos.java
@@ -1734,100 +1734,201 @@ public final class MasterProcedureProtos {
   }
 
   /**
-   * Protobuf enum {@code hbase.pb.MergeTableRegionsState}
+   * Protobuf enum {@code hbase.pb.DispatchMergingRegionsState}
*/
-  public enum MergeTableRegionsState
+  public enum DispatchMergingRegionsState
   implements 
org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
 /**
- * MERGE_TABLE_REGIONS_PREPARE = 1;
+ * DISPATCH_MERGING_REGIONS_PREPARE = 1;
  */
-MERGE_TABLE_REGIONS_PREPARE(1),
+DISPATCH_MERGING_REGIONS_PREPARE(1),
 /**
- * MERGE_TABLE_REGIONS_MOVE_REGION_TO_SAME_RS = 2;
+ * DISPATCH_MERGING_REGIONS_PRE_OPERATION = 2;
  */
-MERGE_TABLE_REGIONS_MOVE_REGION_TO_SAME_RS(2),
+DISPATCH_MERGING_REGIONS_PRE_OPERATION(2),
 /**
- * MERGE_TABLE_REGIONS_PRE_MERGE_OPERATION = 3;
+ * DISPATCH_MERGING_REGIONS_MOVE_REGION_TO_SAME_RS = 3;
  */
-MERGE_TABLE_REGIONS_PRE_MERGE_OPERATION(3),
+DISPATCH_MERGING_REGIONS_MOVE_REGION_TO_SAME_RS(3),
 /**
- * MERGE_TABLE_REGIONS_SET_MERGING_TABLE_STATE = 4;
+ * DISPATCH_MERGING_REGIONS_DO_MERGE_IN_RS = 4;
  */
-MERGE_TABLE_REGIONS_SET_MERGING_TABLE_STATE(4),
+DISPATCH_MERGING_REGIONS_DO_MERGE_IN_RS(4),
 /**
- * MERGE_TABLE_REGIONS_CLOSE_REGIONS = 5;
+ * DISPATCH_MERGING_REGIONS_POST_OPERATION = 5;
  */
-MERGE_TABLE_REGIONS_CLOSE_REGIONS(5),
+DISPATCH_MERGING_REGIONS_POST_OPERATION(5),
+;
+
 /**
- * MERGE_TABLE_REGIONS_CREATE_MERGED_REGION = 6;
+ * DISPATCH_MERGING_REGIONS_PREPARE = 1;
  */
-MERGE_TABLE_REGIONS_CREATE_MERGED_REGION(6),
+public static final int DISPATCH_MERGING_REGIONS_PREPARE_VALUE = 1;
 /**
- * MERGE_TABLE_REGIONS_PRE_MERGE_COMMIT_OPERATION = 7;
+ * DISPATCH_MERGING_REGIONS_PRE_OPERATION = 2;
  */
-MERGE_TABLE_REGIONS_PRE_MERGE_COMMIT_OPERATION(7),
+public static final int DISPATCH_MERGING_REGIONS_PRE_OPERATION_VALUE = 2;
 /**
- * MERGE_TABLE_REGIONS_UPDATE_META = 8;
+ * DISPATCH_MERGING_REGIONS_MOVE_REGION_TO_SAME_RS = 3;
  */
-MERGE_TABLE_REGIONS_UPDATE_META(8),
+public static final int 
DISPATCH_MERGING_REGIONS_MOVE_REGION_TO_SAME_RS_VALUE = 3;
 /**
- * MERGE_TABLE_REGIONS_POST_MERGE_COMMIT_OPERATION = 9;
+ * DISPATCH_MERGING_REGIONS_DO_MERGE_IN_RS = 4;
  */
-MERGE_TABLE_REGIONS_POST_MERGE_COMMIT_OPERATION(9),
+public static final int DISPATCH_MERGING_REGIONS_DO_MERGE_IN_RS_VALUE = 4;
 /**
- * MERGE_TABLE_REGIONS_OPEN_MERGED_REGION = 10;
+ * DISPATCH_MERGING_REGIONS_POST_OPERATION = 5;
  */
-MERGE_TABLE_REGIONS_OPEN_MERGED_REGION(10),
+public static final int DISPATCH_MERGING_REGIONS_POST_OPERATION_VALUE = 5;
+
+
+public final int getNumber() {
+  return value;
+}
+
 /**
- * MERGE_TABLE_REGIONS_POST_OPERATION = 11;
+ * @deprecated Use {@link #forNumber(int)} instead.
  */
-MERGE_TABLE_REGIONS_POST_OPERATION(11),
-;
+@java.lang.Deprecated
+public static DispatchMergingRegionsState valueOf(int value) {
+  return forNumber(value);
+}
+
+public static DispatchMergingRegionsState forNumber(int value) {
+  switch (value) {
+case 1: return DISPATCH_MERGING_REGIONS_PREPARE;
+case 2: return DISPATCH_MERGING_REGIONS_PRE_OPERATION;
+case 3: return DISPATCH_MERGING_REGIONS_MOVE_REGION_TO_SAME_RS;
+case 4: return DISPATCH_MERGING_REGIONS_DO_MERGE_IN_RS;
+case 5: return DISPATCH_MERGING_REGIONS_POST_OPERATION;
+default: return null;
+  }
+}
+
+public static 
org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
+internalGetValueMap() {
+  return internalValueMap;
+}
+private static final 
org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
+DispatchMergingRegionsState> internalValueMap =
+  new 
org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap()
 {
+public DispatchMergingRegionsState findValueByNumber(int number) {
+  return DispatchMergingRegionsState.forNumber(number);
+}
+  };
+
+public final 
org.apache.h

[10/50] [abbrv] hbase git commit: HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi) Move to a new AssignmentManager, one that describes Assignment using a State Machine built on top

http://git-wip-us.apache.org/repos/asf/hbase/blob/db1dcf3e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
index 2703947..9c2efa6 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
@@ -19,55 +19,42 @@ package org.apache.hadoop.hbase.master.procedure;
 
 import java.io.IOException;
 import java.io.InputStream;
-import java.io.InterruptedIOException;
 import java.io.OutputStream;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.HashSet;
+import java.util.Iterator;
 import java.util.List;
-import java.util.Set;
-import java.util.concurrent.locks.Lock;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
-import org.apache.hadoop.hbase.client.TableState;
-import org.apache.hadoop.hbase.master.AssignmentManager;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.master.MasterWalManager;
 import org.apache.hadoop.hbase.master.RegionState;
-import org.apache.hadoop.hbase.master.RegionStates;
+import org.apache.hadoop.hbase.master.assignment.AssignProcedure;
+import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
+import org.apache.hadoop.hbase.master.assignment.RegionTransitionProcedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
 import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
 import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
-import 
org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
 import 
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashState;
-import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.hadoop.util.StringUtils;
-import org.apache.zookeeper.KeeperException;
 
 /**
  * Handle crashed server. This is a port to ProcedureV2 of what used to be 
euphemistically called
  * ServerShutdownHandler.
  *
- * The procedure flow varies dependent on whether meta is assigned, if we 
are
- * doing distributed log replay versus distributed log splitting, and if we 
are to split logs at
- * all.
- *
- * This procedure asks that all crashed servers get processed equally; we 
yield after the
- * completion of each successful flow step. We do this so that we do not 
'deadlock' waiting on
- * a region assignment so we can replay edits which could happen if a region 
moved there are edits
- * on two servers for replay.
+ * The procedure flow varies dependent on whether meta is assigned and if 
we are to split logs.
  *
- * TODO: ASSIGN and WAIT_ON_ASSIGN (at least) are not idempotent. Revisit 
when assign is pv2.
- * TODO: We do not have special handling for system tables.
+ * We come in here after ServerManager has noticed a server has expired. 
Procedures
+ * queued on the rpc should have been notified about fail and should be 
concurrently
+ * getting themselves ready to assign elsewhere.
  */
 public class ServerCrashProcedure
 extends StateMachineProcedure
@@ -75,36 +62,6 @@ implements ServerProcedureInterface {
   private static final Log LOG = LogFactory.getLog(ServerCrashProcedure.class);
 
   /**
-   * Configuration key to set how long to wait in ms doing a quick check on 
meta state.
-   */
-  public static final String KEY_SHORT_WAIT_ON_META =
-  "hbase.master.servercrash.short.wait.on.meta.ms";
-
-  public static final int DEFAULT_SHORT_WAIT_ON_META = 1000;
-
-  /**
-   * Configuration key to set how many retries to cycle before we give up on 
meta.
-   * Each attempt will wait at least {@link #KEY_SHORT_WAIT_ON_META} 
milliseconds.
-   */
-  public static final String KEY_RETRIES_ON_META =
-  "hbase.master.servercrash.meta.retries";
-
-  public static final int DEFAULT_RETRIES_ON_META = 10;
-
-  /**
-   * Configuration key to set how long to wait in ms on regions in transition.
-   */
-  public static final S

[15/50] [abbrv] hbase git commit: HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi) Move to a new AssignmentManager, one that describes Assignment using a State Machine built on top

http://git-wip-us.apache.org/repos/asf/hbase/blob/db1dcf3e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
new file mode 100644
index 000..f1c1a40
--- /dev/null
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
@@ -0,0 +1,1792 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.master.assignment;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseIOException;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.PleaseHoldException;
+import org.apache.hadoop.hbase.RegionException;
+import org.apache.hadoop.hbase.RegionStateListener;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
+import org.apache.hadoop.hbase.favored.FavoredNodeLoadBalancer;
+import org.apache.hadoop.hbase.favored.FavoredNodesManager;
+import org.apache.hadoop.hbase.favored.FavoredNodesPromoter;
+import org.apache.hadoop.hbase.master.AssignmentListener;
+import org.apache.hadoop.hbase.master.LoadBalancer;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.master.MetricsAssignmentManager;
+import org.apache.hadoop.hbase.master.NoSuchProcedureException;
+import org.apache.hadoop.hbase.master.RegionPlan;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.master.RegionState.State;
+import org.apache.hadoop.hbase.master.ServerListener;
+import org.apache.hadoop.hbase.master.TableStateManager;
+import org.apache.hadoop.hbase.master.assignment.RegionStates.RegionStateNode;
+import org.apache.hadoop.hbase.master.assignment.RegionStates.ServerState;
+import org.apache.hadoop.hbase.master.assignment.RegionStates.ServerStateNode;
+// TODO: why are they here?
+import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureScheduler;
+import org.apache.hadoop.hbase.master.procedure.ProcedureSyncWait;
+import org.apache.hadoop.hbase.master.procedure.ServerCrashProcedure;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.procedure2.ProcedureInMemoryChore;
+import org.apache.hadoop.hbase.procedure2.util.StringUtils;
+import org.apache.hadoop.hbase.quotas.QuotaExceededException;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionState

  1   2   >