You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by jm...@apache.org on 2015/02/22 21:55:34 UTC

[01/50] [abbrv] hbase git commit: HBASE-12964 Add the ability for hbase-daemon.sh to start in the foreground

Repository: hbase
Updated Branches:
  refs/heads/hbase-11339 8f5dae471 -> fe335b683


HBASE-12964 Add the ability for hbase-daemon.sh to start in the foreground


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

Branch: refs/heads/hbase-11339
Commit: 5c1b08c5ca18a499c0a336e5ebd7c6bcc45a9fad
Parents: 7861e51
Author: Elliott Clark <ec...@apache.org>
Authored: Tue Feb 3 10:58:15 2015 -0800
Committer: Elliott Clark <ec...@apache.org>
Committed: Tue Feb 3 15:09:48 2015 -0800

----------------------------------------------------------------------
 bin/hbase-daemon.sh | 81 +++++++++++++++++++++++++++---------------------
 1 file changed, 46 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/5c1b08c5/bin/hbase-daemon.sh
----------------------------------------------------------------------
diff --git a/bin/hbase-daemon.sh b/bin/hbase-daemon.sh
index 4dcc520..94ab2d1 100755
--- a/bin/hbase-daemon.sh
+++ b/bin/hbase-daemon.sh
@@ -35,7 +35,7 @@
 # Modelled after $HADOOP_HOME/bin/hadoop-daemon.sh
 
 usage="Usage: hbase-daemon.sh [--config <conf-dir>]\
- (start|stop|restart|autorestart) <hbase-command> \
+ (start|stop|restart|autorestart|foreground_start) <hbase-command> \
  <args...>"
 
 # if no args specified, show usage
@@ -74,25 +74,31 @@ hbase_rotate_log ()
     fi
 }
 
-cleanZNode() {
-  if [ -f $HBASE_ZNODE_FILE ]; then
+cleanAfterRun() {
+  if [ -f ${HBASE_PID} ]; then
+    # If the process is still running time to tear it down.
+    kill -9 `cat ${HBASE_PID}` > /dev/null 2>&1
+    rm -f ${HBASE_PID} > /dev/null 2>&1
+  fi
+
+  if [ -f ${HBASE_ZNODE_FILE} ]; then
     if [ "$command" = "master" ]; then
       $bin/hbase master clear > /dev/null 2>&1
     else
       #call ZK to delete the node
-      ZNODE=`cat $HBASE_ZNODE_FILE`
-      $bin/hbase zkcli delete $ZNODE > /dev/null 2>&1
+      ZNODE=`cat ${HBASE_ZNODE_FILE}`
+      $bin/hbase zkcli delete ${ZNODE} > /dev/null 2>&1
     fi
-    rm $HBASE_ZNODE_FILE
+    rm ${HBASE_ZNODE_FILE}
   fi
 }
 
 check_before_start(){
     #ckeck if the process is not running
     mkdir -p "$HBASE_PID_DIR"
-    if [ -f $pid ]; then
-      if kill -0 `cat $pid` > /dev/null 2>&1; then
-        echo $command running as process `cat $pid`.  Stop it first.
+    if [ -f $HBASE_PID ]; then
+      if kill -0 `cat $HBASE_PID` > /dev/null 2>&1; then
+        echo $command running as process `cat $HBASE_PID`.  Stop it first.
         exit 1
       fi
     fi
@@ -153,18 +159,18 @@ if [ -z "${HBASE_SECURITY_LOGGER}" ]; then
 export HBASE_SECURITY_LOGGER=${HBASE_SECURITY_LOGGER:-"INFO,RFAS"}
 fi
 
-logout=$HBASE_LOG_DIR/$HBASE_LOG_PREFIX.out
-loggc=$HBASE_LOG_DIR/$HBASE_LOG_PREFIX.gc
-loglog="${HBASE_LOG_DIR}/${HBASE_LOGFILE}"
-pid=$HBASE_PID_DIR/hbase-$HBASE_IDENT_STRING-$command.pid
+HBASE_LOGOUT=${HBASE_LOGOUT:-"$HBASE_LOG_DIR/$HBASE_LOG_PREFIX.out"}
+HBASE_LOGGC=${HBASE_LOGGC:-"$HBASE_LOG_DIR/$HBASE_LOG_PREFIX.gc"}
+HBASE_LOGLOG=${HBASE_LOGLOG:-"${HBASE_LOG_DIR}/${HBASE_LOGFILE}"}
+HBASE_PID=$HBASE_PID_DIR/hbase-$HBASE_IDENT_STRING-$command.pid
 export HBASE_ZNODE_FILE=$HBASE_PID_DIR/hbase-$HBASE_IDENT_STRING-$command.znode
 export HBASE_START_FILE=$HBASE_PID_DIR/hbase-$HBASE_IDENT_STRING-$command.autorestart
 
 if [ -n "$SERVER_GC_OPTS" ]; then
-  export SERVER_GC_OPTS=${SERVER_GC_OPTS/"-Xloggc:<FILE-PATH>"/"-Xloggc:${loggc}"}
+  export SERVER_GC_OPTS=${SERVER_GC_OPTS/"-Xloggc:<FILE-PATH>"/"-Xloggc:${HBASE_LOGGC}"}
 fi
 if [ -n "$CLIENT_GC_OPTS" ]; then
-  export CLIENT_GC_OPTS=${CLIENT_GC_OPTS/"-Xloggc:<FILE-PATH>"/"-Xloggc:${loggc}"}
+  export CLIENT_GC_OPTS=${CLIENT_GC_OPTS/"-Xloggc:<FILE-PATH>"/"-Xloggc:${HBASE_LOGGC}"}
 fi
 
 # Set default scheduling priority
@@ -179,30 +185,35 @@ case $startStop in
 
 (start)
     check_before_start
-    hbase_rotate_log $logout
-    hbase_rotate_log $loggc
-    echo starting $command, logging to $logout
-    nohup $thiscmd --config "${HBASE_CONF_DIR}" internal_start $command $args < /dev/null > ${logout} 2>&1  &
-    sleep 1; head "${logout}"
+    hbase_rotate_log $HBASE_LOGOUT
+    hbase_rotate_log $HBASE_LOGGC
+    echo starting $command, logging to $HBASE_LOGOUT
+    nohup $thiscmd --config "${HBASE_CONF_DIR}" \
+        foreground_start $command $args < /dev/null > ${HBASE_LOGOUT} 2>&1  &
+    sleep 1; head "${HBASE_LOGOUT}"
   ;;
 
 (autorestart)
     check_before_start
-    hbase_rotate_log $logout
-    hbase_rotate_log $loggc
-    nohup $thiscmd --config "${HBASE_CONF_DIR}" internal_autorestart $command $args < /dev/null > ${logout} 2>&1  &
+    hbase_rotate_log $HBASE_LOGOUT
+    hbase_rotate_log $HBASE_LOGGC
+    nohup $thiscmd --config "${HBASE_CONF_DIR}" \
+        internal_autorestart $command $args < /dev/null > ${HBASE_LOGOUT} 2>&1  &
   ;;
 
-(internal_start)
+(foreground_start)
     # Add to the command log file vital stats on our environment.
-    echo "`date` Starting $command on `hostname`" >> $loglog
-    echo "`ulimit -a`" >> $loglog 2>&1
+    echo "`date` Starting $command on `hostname`" >> ${HBASE_LOGLOG}
+    `ulimit -a` >> "$HBASE_LOGLOG" 2>&1
     nice -n $HBASE_NICENESS "$HBASE_HOME"/bin/hbase \
         --config "${HBASE_CONF_DIR}" \
-        $command "$@" start >> "$logout" 2>&1 &
-    echo $! > $pid
+        $command "$@" start >> ${HBASE_LOGOUT} 2>&1 &
+    echo $! > ${HBASE_PID}
+    # in case the parent shell gets the kill make sure to trap signals.
+    # Only one will get called. Either the trap or the flow will go through.
+    trap cleanAfterRun SIGHUP SIGINT SIGTERM EXIT
     wait
-    cleanZNode
+    cleanAfterRun
   ;;
 
 (internal_autorestart)
@@ -211,7 +222,7 @@ case $startStop in
     while true
       do
         lastLaunchDate=`date +%s`
-        $thiscmd --config "${HBASE_CONF_DIR}" internal_start $command $args
+        $thiscmd --config "${HBASE_CONF_DIR}" foreground_start $command $args
 
         #if the file does not exist it means that it was not stopped properly by the stop command
         if [ ! -f "$HBASE_START_FILE" ]; then
@@ -247,12 +258,12 @@ case $startStop in
 
 (stop)
     rm -f "$HBASE_START_FILE"
-    if [ -f $pid ]; then
-      pidToKill=`cat $pid`
+    if [ -f $HBASE_PID ]; then
+      pidToKill=`cat $HBASE_PID`
       # kill -0 == see if the PID exists
       if kill -0 $pidToKill > /dev/null 2>&1; then
         echo -n stopping $command
-        echo "`date` Terminating $command" >> $loglog
+        echo "`date` Terminating $command" >> $HBASE_LOGLOG
         kill $pidToKill > /dev/null 2>&1
         waitForProcessEnd $pidToKill $command
       else
@@ -260,9 +271,9 @@ case $startStop in
         echo no $command to stop because kill -0 of pid $pidToKill failed with status $retval
       fi
     else
-      echo no $command to stop because no pid file $pid
+      echo no $command to stop because no pid file $HBASE_PID
     fi
-    rm -f $pid
+    rm -f $HBASE_PID
   ;;
 
 (restart)


[30/50] [abbrv] hbase git commit: HBASE-12035 Keep table state in META (Andrey Stepachev)

Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/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
index 221c7a4..e5214ca 100644
--- 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
@@ -40,7 +40,6 @@ 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.Server;
 import org.apache.hadoop.hbase.ServerLoad;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
@@ -147,13 +146,13 @@ public class RegionStates {
   private final TableStateManager tableStateManager;
   private final RegionStateStore regionStateStore;
   private final ServerManager serverManager;
-  private final Server server;
+  private final MasterServices server;
 
   // The maximum time to keep a log split info in region states map
   static final String LOG_SPLIT_TIME = "hbase.master.maximum.logsplit.keeptime";
   static final long DEFAULT_LOG_SPLIT_TIME = 7200000L; // 2 hours
 
-  RegionStates(final Server master, final TableStateManager tableStateManager,
+  RegionStates(final MasterServices master, final TableStateManager tableStateManager,
       final ServerManager serverManager, final RegionStateStore regionStateStore) {
     this.tableStateManager = tableStateManager;
     this.regionStateStore = regionStateStore;
@@ -872,7 +871,7 @@ public class RegionStates {
 
   private int getRegionReplication(HRegionInfo r) throws IOException {
     if (tableStateManager != null) {
-      HTableDescriptor htd = tableStateManager.getTableDescriptors().get(r.getTable());
+      HTableDescriptor htd = server.getTableDescriptors().get(r.getTable());
       if (htd != null) {
         return htd.getRegionReplication();
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SnapshotOfRegionAssignmentFromMeta.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SnapshotOfRegionAssignmentFromMeta.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SnapshotOfRegionAssignmentFromMeta.java
index b03611c..39beba8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SnapshotOfRegionAssignmentFromMeta.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SnapshotOfRegionAssignmentFromMeta.java
@@ -141,7 +141,7 @@ public class SnapshotOfRegionAssignmentFromMeta {
       }
     };
     // Scan hbase:meta to pick up user regions
-    MetaTableAccessor.fullScan(connection, v);
+    MetaTableAccessor.fullScanRegions(connection, v);
     //regionToRegionServerMap = regions;
     LOG.info("Finished to scan the hbase:meta for the current region assignment" +
       "snapshot");

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
index d8199ea..5d1e638 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
@@ -17,19 +17,27 @@
  */
 package org.apache.hadoop.hbase.master;
 
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
 import java.io.IOException;
+import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
 
-import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.TableDescriptor;
 import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.TableState;
 
 /**
@@ -39,24 +47,12 @@ import org.apache.hadoop.hbase.client.TableState;
 @InterfaceAudience.Private
 public class TableStateManager {
   private static final Log LOG = LogFactory.getLog(TableStateManager.class);
-  private final TableDescriptors descriptors;
 
-  private final Map<TableName, TableState.State> tableStates = Maps.newConcurrentMap();
+  private final ReadWriteLock lock = new ReentrantReadWriteLock();
+  private final MasterServices master;
 
   public TableStateManager(MasterServices master) {
-    this.descriptors = master.getTableDescriptors();
-  }
-
-  public void start() throws IOException {
-    Map<String, TableDescriptor> all = descriptors.getAllDescriptors();
-    for (TableDescriptor table : all.values()) {
-      TableName tableName = table.getHTableDescriptor().getTableName();
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Adding table state: " + tableName
-            + ": " + table.getTableState());
-      }
-      tableStates.put(tableName, table.getTableState());
-    }
+    this.master = master;
   }
 
   /**
@@ -67,16 +63,13 @@ public class TableStateManager {
    * @throws IOException
    */
   public void setTableState(TableName tableName, TableState.State newState) throws IOException {
-    synchronized (tableStates) {
-      TableDescriptor descriptor = readDescriptor(tableName);
-      if (descriptor == null) {
-        throw new TableNotFoundException(tableName);
-      }
-      if (descriptor.getTableState() != newState) {
-        writeDescriptor(
-            new TableDescriptor(descriptor.getHTableDescriptor(), newState));
-      }
+    lock.writeLock().lock();
+    try {
+      udpateMetaState(tableName, newState);
+    } finally {
+      lock.writeLock().unlock();
     }
+
   }
 
   /**
@@ -91,21 +84,23 @@ public class TableStateManager {
                                          TableState.State newState,
                                          TableState.State... states)
           throws IOException {
-    synchronized (tableStates) {
-      TableDescriptor descriptor = readDescriptor(tableName);
-      if (descriptor == null) {
+    lock.writeLock().lock();
+    try {
+      TableState currentState = readMetaState(tableName);
+      if (currentState == null) {
         throw new TableNotFoundException(tableName);
       }
-      if (TableState.isInStates(descriptor.getTableState(), states)) {
-        writeDescriptor(
-            new TableDescriptor(descriptor.getHTableDescriptor(), newState));
+      if (currentState.inStates(states)) {
+        udpateMetaState(tableName, newState);
         return true;
       } else {
         return false;
       }
+    } finally {
+      lock.writeLock().unlock();
     }
-  }
 
+  }
 
   /**
    * Set table state to provided but only if table not in specified states
@@ -119,42 +114,36 @@ public class TableStateManager {
                                             TableState.State newState,
                                             TableState.State... states)
           throws IOException {
-    synchronized (tableStates) {
-      TableDescriptor descriptor = readDescriptor(tableName);
-      if (descriptor == null) {
-        throw new TableNotFoundException(tableName);
-      }
-      if (!TableState.isInStates(descriptor.getTableState(), states)) {
-        writeDescriptor(
-            new TableDescriptor(descriptor.getHTableDescriptor(), newState));
-        return true;
-      } else {
-        return false;
-      }
+    TableState currentState = readMetaState(tableName);
+    if (currentState == null) {
+      throw new TableNotFoundException(tableName);
+    }
+    if (!currentState.inStates(states)) {
+      udpateMetaState(tableName, newState);
+      return true;
+    } else {
+      return false;
     }
   }
 
   public boolean isTableState(TableName tableName, TableState.State... states) {
-    TableState.State tableState = null;
     try {
-      tableState = getTableState(tableName);
+      TableState.State tableState = getTableState(tableName);
+      return TableState.isInStates(tableState, states);
     } catch (IOException e) {
-      LOG.error("Unable to get table state, probably table not exists");
+      LOG.error("Unable to get table " + tableName + " state, probably table not exists");
       return false;
     }
-    return tableState != null && TableState.isInStates(tableState, states);
   }
 
   public void setDeletedTable(TableName tableName) throws IOException {
-    TableState.State remove = tableStates.remove(tableName);
-    if (remove == null) {
-      LOG.warn("Moving table " + tableName + " state to deleted but was " +
-              "already deleted");
-    }
+    if (tableName.equals(TableName.META_TABLE_NAME))
+      return;
+    MetaTableAccessor.deleteTableState(master.getConnection(), tableName);
   }
 
   public boolean isTablePresent(TableName tableName) throws IOException {
-    return getTableState(tableName) != null;
+    return readMetaState(tableName) != null;
   }
 
   /**
@@ -164,57 +153,82 @@ public class TableStateManager {
    * @return tables in given states
    * @throws IOException
    */
-  public Set<TableName> getTablesInStates(TableState.State... states) throws IOException {
-    Set<TableName> rv = Sets.newHashSet();
-    for (Map.Entry<TableName, TableState.State> entry : tableStates.entrySet()) {
-      if (TableState.isInStates(entry.getValue(), states))
-        rv.add(entry.getKey());
-    }
+  public Set<TableName> getTablesInStates(final TableState.State... states) throws IOException {
+    final Set<TableName> rv = Sets.newHashSet();
+    MetaTableAccessor.fullScanTables(master.getConnection(), new MetaTableAccessor.Visitor() {
+      @Override
+      public boolean visit(Result r) throws IOException {
+        TableState tableState = MetaTableAccessor.getTableState(r);
+        if (tableState != null && tableState.inStates(states))
+          rv.add(tableState.getTableName());
+        return true;
+      }
+    });
     return rv;
   }
 
+  @Nonnull
   public TableState.State getTableState(TableName tableName) throws IOException {
-    TableState.State tableState = tableStates.get(tableName);
-    if (tableState == null) {
-      TableDescriptor descriptor = readDescriptor(tableName);
-      if (descriptor != null)
-        tableState = descriptor.getTableState();
+    TableState currentState = readMetaState(tableName);
+    if (currentState == null) {
+      throw new TableNotFoundException(tableName);
     }
-    return tableState;
+    return currentState.getState();
   }
 
-  TableDescriptors getTableDescriptors() {
-    return descriptors;
+  protected void udpateMetaState(TableName tableName, TableState.State newState)
+      throws IOException {
+    MetaTableAccessor.updateTableState(master.getConnection(), tableName, newState);
   }
 
-  /**
-   * Write descriptor in place, update cache of states.
-   * Write lock should be hold by caller.
-   *
-   * @param descriptor what to write
-   */
-  private void writeDescriptor(TableDescriptor descriptor) throws IOException {
-    TableName tableName = descriptor.getHTableDescriptor().getTableName();
-    TableState.State state = descriptor.getTableState();
-    descriptors.add(descriptor);
-    LOG.debug("Table " + tableName + " written descriptor for state " + state);
-    tableStates.put(tableName, state);
-    LOG.debug("Table " + tableName + " updated state to " + state);
+  @Nullable
+  protected TableState readMetaState(TableName tableName) throws IOException {
+    if (tableName.equals(TableName.META_TABLE_NAME))
+      return new TableState(tableName, TableState.State.ENABLED);
+    return MetaTableAccessor.getTableState(master.getConnection(), tableName);
   }
 
-  /**
-   * Read current descriptor for table, update cache of states.
-   *
-   * @param table descriptor to read
-   * @return descriptor
-   * @throws IOException
-   */
-  private TableDescriptor readDescriptor(TableName tableName) throws IOException {
-    TableDescriptor descriptor = descriptors.getDescriptor(tableName);
-    if (descriptor == null)
-      tableStates.remove(tableName);
-    else
-      tableStates.put(tableName, descriptor.getTableState());
-    return descriptor;
+  @SuppressWarnings("deprecation")
+  public void start() throws IOException {
+    TableDescriptors tableDescriptors = master.getTableDescriptors();
+    Connection connection = master.getConnection();
+    fixTableStates(tableDescriptors, connection);
+  }
+
+  public static void fixTableStates(TableDescriptors tableDescriptors, Connection connection)
+      throws IOException {
+    final Map<String, TableDescriptor> allDescriptors =
+        tableDescriptors.getAllDescriptors();
+    final Map<String, TableState> states = new HashMap<>();
+    MetaTableAccessor.fullScanTables(connection, new MetaTableAccessor.Visitor() {
+      @Override
+      public boolean visit(Result r) throws IOException {
+        TableState state = MetaTableAccessor.getTableState(r);
+        if (state != null)
+          states.put(state.getTableName().getNameAsString(), state);
+        return true;
+      }
+    });
+    for (Map.Entry<String, TableDescriptor> entry : allDescriptors.entrySet()) {
+      String table = entry.getKey();
+      if (table.equals(TableName.META_TABLE_NAME.getNameAsString()))
+        continue;
+      if (!states.containsKey(table)) {
+        LOG.warn("Found table without state " + table);
+        TableDescriptor td = entry.getValue();
+        TableState.State tds = td.getTableState();
+        if (tds != null) {
+          LOG.warn("Found table with state in descriptor, using that state");
+          MetaTableAccessor.updateTableState(connection, TableName.valueOf(table), tds);
+          LOG.warn("Updating table descriptor");
+          td.setTableState(null);
+          tableDescriptors.add(td);
+        } else {
+          LOG.warn("Found table with no state in descriptor, assuming ENABLED");
+          MetaTableAccessor.updateTableState(connection, TableName.valueOf(table),
+              TableState.State.ENABLED);
+        }
+      }
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/CreateTableHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/CreateTableHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/CreateTableHandler.java
index 2007ed4..b60733e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/CreateTableHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/CreateTableHandler.java
@@ -70,6 +70,7 @@ public class CreateTableHandler extends EventHandler {
   private final AssignmentManager assignmentManager;
   private final TableLockManager tableLockManager;
   private final HRegionInfo [] newRegions;
+  private final MasterServices masterServices;
   private final TableLock tableLock;
   private User activeUser;
 
@@ -82,6 +83,7 @@ public class CreateTableHandler extends EventHandler {
     this.hTableDescriptor = hTableDescriptor;
     this.conf = conf;
     this.newRegions = newRegions;
+    this.masterServices = masterServices;
     this.assignmentManager = masterServices.getAssignmentManager();
     this.tableLockManager = masterServices.getTableLockManager();
 
@@ -209,10 +211,11 @@ public class CreateTableHandler extends EventHandler {
     // 1. Create Table Descriptor
     // using a copy of descriptor, table will be created enabling first
     TableDescriptor underConstruction = new TableDescriptor(
-        this.hTableDescriptor, TableState.State.ENABLING);
+        this.hTableDescriptor);
     Path tempTableDir = FSUtils.getTableDir(tempdir, tableName);
-    new FSTableDescriptors(this.conf).createTableDescriptorForTableDirectory(
-      tempTableDir, underConstruction, false);
+    ((FSTableDescriptors)(masterServices.getTableDescriptors()))
+        .createTableDescriptorForTableDirectory(
+        tempTableDir, underConstruction, false);
     Path tableDir = FSUtils.getTableDir(fileSystemManager.getRootDir(), tableName);
 
     // 2. Create Regions
@@ -223,6 +226,12 @@ public class CreateTableHandler extends EventHandler {
         " to hbase root=" + tableDir);
     }
 
+    // populate descriptors cache to be visible in getAll
+    masterServices.getTableDescriptors().get(tableName);
+
+    MetaTableAccessor.updateTableState(this.server.getConnection(), hTableDescriptor.getTableName(),
+        TableState.State.ENABLING);
+
     if (regionInfos != null && regionInfos.size() > 0) {
       // 4. Add regions to META
       addRegionsToMeta(regionInfos, hTableDescriptor.getRegionReplication());

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TruncateTableHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TruncateTableHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TruncateTableHandler.java
index ee40153..15a5b8f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TruncateTableHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TruncateTableHandler.java
@@ -95,10 +95,10 @@ public class TruncateTableHandler extends DeleteTableHandler {
     AssignmentManager assignmentManager = this.masterServices.getAssignmentManager();
 
     // 1. Create Table Descriptor
-    TableDescriptor underConstruction = new TableDescriptor(
-        this.hTableDescriptor, TableState.State.ENABLING);
+    TableDescriptor underConstruction = new TableDescriptor(this.hTableDescriptor);
     Path tempTableDir = FSUtils.getTableDir(tempdir, this.tableName);
-    new FSTableDescriptors(server.getConfiguration())
+
+    ((FSTableDescriptors)(masterServices.getTableDescriptors()))
       .createTableDescriptorForTableDirectory(tempTableDir, underConstruction, false);
     Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), this.tableName);
 
@@ -123,6 +123,11 @@ public class TruncateTableHandler extends DeleteTableHandler {
         " to hbase root=" + tableDir);
     }
 
+    // populate descriptors cache to be visible in getAll
+    masterServices.getTableDescriptors().get(tableName);
+
+    assignmentManager.getTableStateManager().setTableState(tableName,
+        TableState.State.ENABLING);
     // 4. Add regions to META
     MetaTableAccessor.addRegionsToMeta(masterServices.getConnection(),
       regionInfos, hTableDescriptor.getRegionReplication());

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 90b29ef..c170a65 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -528,8 +528,7 @@ public class HRegionServer extends HasThread implements
     boolean useHBaseChecksum = conf.getBoolean(HConstants.HBASE_CHECKSUM_VERIFICATION, true);
     this.fs = new HFileSystem(this.conf, useHBaseChecksum);
     this.rootDir = FSUtils.getRootDir(this.conf);
-    this.tableDescriptors = new FSTableDescriptors(this.conf,
-      this.fs, this.rootDir, !canUpdateTableDescriptor(), false);
+    this.tableDescriptors = getFsTableDescriptors();
 
     service = new ExecutorService(getServerName().toShortString());
     spanReceiverHost = SpanReceiverHost.getInstance(getConfiguration());
@@ -561,6 +560,11 @@ public class HRegionServer extends HasThread implements
     this.choreService = new ChoreService(getServerName().toString());
   }
 
+  protected TableDescriptors getFsTableDescriptors() throws IOException {
+    return new FSTableDescriptors(this.conf,
+      this.fs, this.rootDir, !canUpdateTableDescriptor(), false);
+  }
+
   protected void login(UserProvider user, String host) throws IOException {
     user.login("hbase.regionserver.keytab.file",
       "hbase.regionserver.kerberos.principal", host);

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java
index a3cfa04..330ead4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java
@@ -39,7 +39,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableDescriptor;
-import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDataManifest;
@@ -357,7 +356,7 @@ public class SnapshotManifest {
       // write a copy of descriptor to the snapshot directory
       new FSTableDescriptors(conf, fs, rootDir)
         .createTableDescriptorForTableDirectory(workingDir, new TableDescriptor(
-            htd, TableState.State.ENABLED), false);
+            htd), false);
     } else {
       LOG.debug("Convert to Single Snapshot Manifest");
       convertToV2SingleManifest();

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
index 7a6811c..cce37d7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
@@ -33,7 +33,6 @@ import com.google.common.primitives.Ints;
 import org.apache.commons.lang.NotImplementedException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -47,7 +46,7 @@ import org.apache.hadoop.hbase.TableDescriptor;
 import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableInfoMissingException;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 
@@ -154,7 +153,7 @@ public class FSTableDescriptors implements TableDescriptors {
     invocations++;
     if (TableName.META_TABLE_NAME.equals(tablename)) {
       cachehits++;
-      return new TableDescriptor(metaTableDescritor, TableState.State.ENABLED);
+      return new TableDescriptor(metaTableDescritor);
     }
     // hbase:meta is already handled. If some one tries to get the descriptor for
     // .logs, .oldlogs or .corrupt throw an exception.
@@ -218,7 +217,7 @@ public class FSTableDescriptors implements TableDescriptors {
       }
       // add hbase:meta to the response
       tds.put(this.metaTableDescritor.getNameAsString(),
-        new TableDescriptor(metaTableDescritor, TableState.State.ENABLED));
+          new TableDescriptor(metaTableDescritor));
     } else {
       LOG.debug("Fetching table descriptors from the filesystem.");
       boolean allvisited = true;
@@ -592,7 +591,7 @@ public class FSTableDescriptors implements TableDescriptors {
         HTableDescriptor htd = HTableDescriptor.parseFrom(content);
         LOG.warn("Found old table descriptor, converting to new format for table " +
             htd.getTableName() + "; NOTE table will be in ENABLED state!");
-        td = new TableDescriptor(htd, TableState.State.ENABLED);
+        td = new TableDescriptor(htd);
         if (rewritePb) rewriteTableDescriptor(fs, status, td);
       } catch (DeserializationException e1) {
         throw new IOException("content=" + Bytes.toShort(content), e);

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
index 8e1d848..a8b60cd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
@@ -53,12 +53,16 @@ import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Multimap;
+import com.google.common.collect.TreeMultimap;
+import com.google.protobuf.ServiceException;
 import org.apache.commons.lang.RandomStringUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -85,15 +89,15 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HConnectable;
 import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HConnectionManager;
 import org.apache.hadoop.hbase.client.MetaScanner;
 import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
 import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase;
@@ -137,13 +141,6 @@ import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.zookeeper.KeeperException;
 
-import com.google.common.base.Joiner;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Multimap;
-import com.google.common.collect.TreeMultimap;
-import com.google.protobuf.ServiceException;
-
 /**
  * HBaseFsck (hbck) is a tool for checking and repairing region consistency and
  * table integrity problems in a corrupted HBase.
@@ -245,7 +242,8 @@ public class HBaseFsck extends Configured implements Closeable {
   // hbase:meta are always checked
   private Set<TableName> tablesIncluded = new HashSet<TableName>();
   private int maxMerge = DEFAULT_MAX_MERGE; // maximum number of overlapping regions to merge
-  private int maxOverlapsToSideline = DEFAULT_OVERLAPS_TO_SIDELINE; // maximum number of overlapping regions to sideline
+  // maximum number of overlapping regions to sideline
+  private int maxOverlapsToSideline = DEFAULT_OVERLAPS_TO_SIDELINE;
   private boolean sidelineBigOverlaps = false; // sideline overlaps with >maxMerge regions
   private Path sidelineDir = null;
 
@@ -267,8 +265,6 @@ public class HBaseFsck extends Configured implements Closeable {
    * to detect and correct consistency (hdfs/meta/deployment) problems.
    */
   private TreeMap<String, HbckInfo> regionInfoMap = new TreeMap<String, HbckInfo>();
-  private TreeSet<TableName> disabledTables =
-    new TreeSet<TableName>();
   // Empty regioninfo qualifiers in hbase:meta
   private Set<Result> emptyRegionInfoQualifiers = new HashSet<Result>();
 
@@ -292,6 +288,8 @@ public class HBaseFsck extends Configured implements Closeable {
 
   private Map<TableName, Set<String>> orphanTableDirs =
       new HashMap<TableName, Set<String>>();
+  private Map<TableName, TableState> tableStates =
+      new HashMap<TableName, TableState>();
 
   /**
    * Constructor
@@ -493,7 +491,7 @@ public class HBaseFsck extends Configured implements Closeable {
     fixes = 0;
     regionInfoMap.clear();
     emptyRegionInfoQualifiers.clear();
-    disabledTables.clear();
+    tableStates.clear();
     errors.clear();
     tablesInfo.clear();
     orphanHdfsDirs.clear();
@@ -577,15 +575,15 @@ public class HBaseFsck extends Configured implements Closeable {
       reportTablesInFlux();
     }
 
+    // Get disabled tables states
+    loadTableStates();
+
     // load regiondirs and regioninfos from HDFS
     if (shouldCheckHdfs()) {
       loadHdfsRegionDirs();
       loadHdfsRegionInfos();
     }
 
-    // Get disabled tables from ZooKeeper
-    loadDisabledTables();
-
     // fix the orphan tables
     fixOrphanTables();
 
@@ -1140,7 +1138,7 @@ public class HBaseFsck extends Configured implements Closeable {
     for (String columnfamimly : columns) {
       htd.addFamily(new HColumnDescriptor(columnfamimly));
     }
-    fstd.createTableDescriptor(new TableDescriptor(htd, TableState.State.ENABLED), true);
+    fstd.createTableDescriptor(new TableDescriptor(htd), true);
     return true;
   }
 
@@ -1188,7 +1186,7 @@ public class HBaseFsck extends Configured implements Closeable {
           if (tableName.equals(htds[j].getTableName())) {
             HTableDescriptor htd = htds[j];
             LOG.info("fixing orphan table: " + tableName + " from cache");
-            fstd.createTableDescriptor(new TableDescriptor(htd, TableState.State.ENABLED), true);
+            fstd.createTableDescriptor(new TableDescriptor(htd), true);
             j++;
             iter.remove();
           }
@@ -1265,6 +1263,8 @@ public class HBaseFsck extends Configured implements Closeable {
       }
 
       TableInfo ti = e.getValue();
+      puts.add(MetaTableAccessor
+          .makePutFromTableState(new TableState(ti.tableName, TableState.State.ENABLED)));
       for (Entry<byte[], Collection<HbckInfo>> spl : ti.sc.getStarts().asMap()
           .entrySet()) {
         Collection<HbckInfo> his = spl.getValue();
@@ -1524,28 +1524,19 @@ public class HBaseFsck extends Configured implements Closeable {
    * @throws ZooKeeperConnectionException
    * @throws IOException
    */
-  private void loadDisabledTables()
+  private void loadTableStates()
   throws IOException {
-    HConnectionManager.execute(new HConnectable<Void>(getConf()) {
-      @Override
-      public Void connect(HConnection connection) throws IOException {
-        TableName[] tables = connection.listTableNames();
-        for (TableName table : tables) {
-          if (connection.getTableState(table)
-              .inStates(TableState.State.DISABLED, TableState.State.DISABLING)) {
-            disabledTables.add(table);
-          }
-        }
-        return null;
-      }
-    });
+    tableStates = MetaTableAccessor.getTableStates(connection);
   }
 
   /**
    * Check if the specified region's table is disabled.
+   * @param tableName table to check status of
    */
-  private boolean isTableDisabled(HRegionInfo regionInfo) {
-    return disabledTables.contains(regionInfo.getTable());
+  private boolean isTableDisabled(TableName tableName) {
+    return tableStates.containsKey(tableName)
+        && tableStates.get(tableName)
+        .inStates(TableState.State.DISABLED, TableState.State.DISABLING);
   }
 
   /**
@@ -1615,15 +1606,24 @@ public class HBaseFsck extends Configured implements Closeable {
         HConstants.EMPTY_START_ROW, false, false);
     if (rl == null) {
       errors.reportError(ERROR_CODE.NULL_META_REGION,
-          "META region or some of its attributes are null.");
+          "META region was not found in Zookeeper");
       return false;
     }
     for (HRegionLocation metaLocation : rl.getRegionLocations()) {
       // Check if Meta region is valid and existing
-      if (metaLocation == null || metaLocation.getRegionInfo() == null ||
-          metaLocation.getHostname() == null) {
+      if (metaLocation == null ) {
+        errors.reportError(ERROR_CODE.NULL_META_REGION,
+            "META region location is null");
+        return false;
+      }
+      if (metaLocation.getRegionInfo() == null) {
+        errors.reportError(ERROR_CODE.NULL_META_REGION,
+            "META location regionInfo is null");
+        return false;
+      }
+      if (metaLocation.getHostname() == null) {
         errors.reportError(ERROR_CODE.NULL_META_REGION,
-            "META region or some of its attributes are null.");
+            "META location hostName is null");
         return false;
       }
       ServerName sn = metaLocation.getServerName();
@@ -1718,6 +1718,55 @@ public class HBaseFsck extends Configured implements Closeable {
       }
     }
     setCheckHdfs(prevHdfsCheck);
+
+    if (shouldCheckHdfs()) {
+      checkAndFixTableStates();
+    }
+  }
+
+  /**
+   * Check and fix table states, assumes full info available:
+   * - tableInfos
+   * - empty tables loaded
+   */
+  private void checkAndFixTableStates() throws IOException {
+    // first check dangling states
+    for (Entry<TableName, TableState> entry : tableStates.entrySet()) {
+      TableName tableName = entry.getKey();
+      TableState tableState = entry.getValue();
+      TableInfo tableInfo = tablesInfo.get(tableName);
+      if (isTableIncluded(tableName)
+          && !tableName.isSystemTable()
+          && tableInfo == null) {
+        if (fixMeta) {
+          MetaTableAccessor.deleteTableState(connection, tableName);
+          TableState state = MetaTableAccessor.getTableState(connection, tableName);
+          if (state != null) {
+            errors.reportError(ERROR_CODE.ORPHAN_TABLE_STATE,
+                tableName + " unable to delete dangling table state " + tableState);
+          }
+        } else {
+          errors.reportError(ERROR_CODE.ORPHAN_TABLE_STATE,
+              tableName + " has dangling table state " + tableState);
+        }
+      }
+    }
+    // check that all tables have states
+    for (TableName tableName : tablesInfo.keySet()) {
+      if (isTableIncluded(tableName) && !tableStates.containsKey(tableName)) {
+        if (fixMeta) {
+          MetaTableAccessor.updateTableState(connection, tableName, TableState.State.ENABLED);
+          TableState newState = MetaTableAccessor.getTableState(connection, tableName);
+          if (newState == null) {
+            errors.reportError(ERROR_CODE.NO_TABLE_STATE,
+                "Unable to change state for table " + tableName + " in meta ");
+          }
+        } else {
+          errors.reportError(ERROR_CODE.NO_TABLE_STATE,
+              tableName + " has no state in meta ");
+        }
+      }
+    }
   }
 
   private void preCheckPermission() throws IOException, AccessDeniedException {
@@ -1961,8 +2010,8 @@ public class HBaseFsck extends Configured implements Closeable {
       hasMetaAssignment && isDeployed && !isMultiplyDeployed &&
       hbi.metaEntry.regionServer.equals(hbi.deployedOn.get(0));
     boolean splitParent =
-      (hbi.metaEntry == null)? false: hbi.metaEntry.isSplit() && hbi.metaEntry.isOffline();
-    boolean shouldBeDeployed = inMeta && !isTableDisabled(hbi.metaEntry);
+        inMeta && hbi.metaEntry.isSplit() && hbi.metaEntry.isOffline();
+    boolean shouldBeDeployed = inMeta && !isTableDisabled(hbi.metaEntry.getTable());
     boolean recentlyModified = inHdfs &&
       hbi.getModTime() + timelag > System.currentTimeMillis();
 
@@ -2744,7 +2793,7 @@ public class HBaseFsck extends Configured implements Closeable {
       // When table is disabled no need to check for the region chain. Some of the regions
       // accidently if deployed, this below code might report some issues like missing start
       // or end regions or region hole in chain and may try to fix which is unwanted.
-      if (disabledTables.contains(this.tableName)) {
+      if (isTableDisabled(this.tableName)) {
         return true;
       }
       int originalErrorsCount = errors.getErrorList().size();
@@ -3534,12 +3583,14 @@ public class HBaseFsck extends Configured implements Closeable {
   public interface ErrorReporter {
     enum ERROR_CODE {
       UNKNOWN, NO_META_REGION, NULL_META_REGION, NO_VERSION_FILE, NOT_IN_META_HDFS, NOT_IN_META,
-      NOT_IN_META_OR_DEPLOYED, NOT_IN_HDFS_OR_DEPLOYED, NOT_IN_HDFS, SERVER_DOES_NOT_MATCH_META, NOT_DEPLOYED,
+      NOT_IN_META_OR_DEPLOYED, NOT_IN_HDFS_OR_DEPLOYED, NOT_IN_HDFS, SERVER_DOES_NOT_MATCH_META,
+      NOT_DEPLOYED,
       MULTI_DEPLOYED, SHOULD_NOT_BE_DEPLOYED, MULTI_META_REGION, RS_CONNECT_FAILURE,
       FIRST_REGION_STARTKEY_NOT_EMPTY, LAST_REGION_ENDKEY_NOT_EMPTY, DUPE_STARTKEYS,
       HOLE_IN_REGION_CHAIN, OVERLAP_IN_REGION_CHAIN, REGION_CYCLE, DEGENERATE_REGION,
       ORPHAN_HDFS_REGION, LINGERING_SPLIT_PARENT, NO_TABLEINFO_FILE, LINGERING_REFERENCE_HFILE,
-      WRONG_USAGE, EMPTY_META_CELL, EXPIRED_TABLE_LOCK, BOUNDARIES_ERROR
+      WRONG_USAGE, EMPTY_META_CELL, EXPIRED_TABLE_LOCK, BOUNDARIES_ERROR, ORPHAN_TABLE_STATE,
+      NO_TABLE_STATE
     }
     void clear();
     void report(String message);

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index 37114b9..1a377fc 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -18,6 +18,11 @@
 package org.apache.hadoop.hbase;
 
 import javax.annotation.Nullable;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 import java.io.File;
 import java.io.IOException;
 import java.io.OutputStream;
@@ -122,10 +127,6 @@ import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.ZooKeeper.States;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
 /**
  * Facility for testing HBase. Replacement for
  * old HBaseTestCase and HBaseClusterTestCase functionality.
@@ -2292,6 +2293,8 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
     Table meta = (HTable) getConnection().getTable(TableName.META_TABLE_NAME);
     Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR);
     List<HRegionInfo> newRegions = new ArrayList<HRegionInfo>(startKeys.length);
+    MetaTableAccessor
+        .updateTableState(getConnection(), htd.getTableName(), TableState.State.ENABLED);
     // add custom ones
     for (int i = 0; i < startKeys.length; i++) {
       int j = (i + 1) % startKeys.length;
@@ -2953,17 +2956,36 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
 
   public String explainTableState(final TableName table, TableState.State state)
       throws IOException {
-    TableState.State tableState =
-        getHBaseCluster().getMaster().getTableStateManager().getTableState(table);
+    TableState tableState = MetaTableAccessor.getTableState(connection, table);
     if (tableState == null) {
-      return "TableState: No state for table " + table;
-    } else if (!tableState.equals(state)) {
-      return "TableState: Not " + state + " state, but " + tableState;
+      return "TableState in META: No table state in META for table " + table
+          + " last state in meta (including deleted is " + findLastTableState(table) + ")";
+    } else if (!tableState.inStates(state)) {
+      return "TableState in META: Not " + state + " state, but " + tableState;
     } else {
-      return "TableState: OK";
+      return "TableState in META: OK";
     }
   }
 
+  @Nullable
+  public TableState findLastTableState(final TableName table) throws IOException {
+    final AtomicReference<TableState> lastTableState = new AtomicReference<>(null);
+    MetaTableAccessor.Visitor visitor = new MetaTableAccessor.Visitor() {
+      @Override
+      public boolean visit(Result r) throws IOException {
+        if (!Arrays.equals(r.getRow(), table.getName()))
+          return false;
+        TableState state = MetaTableAccessor.getTableState(r);
+        if (state != null)
+          lastTableState.set(state);
+        return true;
+      }
+    };
+    MetaTableAccessor
+        .fullScan(connection, visitor, table.getName(), MetaTableAccessor.QueryType.TABLE, true);
+    return lastTableState.get();
+  }
+
   /**
    * Waits for a table to be 'enabled'.  Enabled means that table is set as 'enabled' and the
    * regions have been all assigned.  Will timeout after default period (30 seconds)

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
index e637976..eefb974 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
@@ -228,7 +228,12 @@ public class TestMetaTableAccessor {
     admin.deleteTable(name);
     assertFalse(MetaTableAccessor.tableExists(connection, name));
     assertTrue(MetaTableAccessor.tableExists(connection,
-      TableName.META_TABLE_NAME));
+        TableName.META_TABLE_NAME));
+    UTIL.createTable(name, HConstants.CATALOG_FAMILY);
+    assertTrue(MetaTableAccessor.tableExists(connection, name));
+    admin.disableTable(name);
+    admin.deleteTable(name);
+    assertFalse(MetaTableAccessor.tableExists(connection, name));
   }
 
   @Test public void testGetRegion() throws IOException, InterruptedException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
index 85fbbc6..fd1eff7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.client;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -248,7 +249,8 @@ public class TestAdmin1 {
     this.admin.disableTable(ht.getName());
     assertTrue("Table must be disabled.", TEST_UTIL.getHBaseCluster()
         .getMaster().getAssignmentManager().getTableStateManager().isTableState(
-        ht.getName(), TableState.State.DISABLED));
+            ht.getName(), TableState.State.DISABLED));
+    assertEquals(TableState.State.DISABLED, getStateFromMeta(table));
 
     // Test that table is disabled
     get = new Get(row);
@@ -275,7 +277,8 @@ public class TestAdmin1 {
     this.admin.enableTable(table);
     assertTrue("Table must be enabled.", TEST_UTIL.getHBaseCluster()
         .getMaster().getAssignmentManager().getTableStateManager().isTableState(
-        ht.getName(), TableState.State.ENABLED));
+            ht.getName(), TableState.State.ENABLED));
+    assertEquals(TableState.State.ENABLED, getStateFromMeta(table));
 
     // Test that table is enabled
     try {
@@ -287,6 +290,13 @@ public class TestAdmin1 {
     ht.close();
   }
 
+  private TableState.State getStateFromMeta(TableName table) throws IOException {
+    TableState state =
+        MetaTableAccessor.getTableState(TEST_UTIL.getConnection(), table);
+    assertNotNull(state);
+    return state.getState();
+  }
+
   @Test (timeout=300000)
   public void testDisableAndEnableTables() throws IOException {
     final byte [] row = Bytes.toBytes("row");
@@ -318,6 +328,10 @@ public class TestAdmin1 {
       ok = true;
     }
 
+    assertEquals(TableState.State.DISABLED, getStateFromMeta(table1));
+    assertEquals(TableState.State.DISABLED, getStateFromMeta(table2));
+
+
     assertTrue(ok);
     this.admin.enableTables("testDisableAndEnableTable.*");
 
@@ -336,18 +350,23 @@ public class TestAdmin1 {
 
     ht1.close();
     ht2.close();
+
+    assertEquals(TableState.State.ENABLED, getStateFromMeta(table1));
+    assertEquals(TableState.State.ENABLED, getStateFromMeta(table2));
   }
 
   @Test (timeout=300000)
   public void testCreateTable() throws IOException {
     HTableDescriptor [] tables = admin.listTables();
     int numTables = tables.length;
-    TEST_UTIL.createTable(TableName.valueOf("testCreateTable"), HConstants.CATALOG_FAMILY).close();
+    TableName tableName = TableName.valueOf("testCreateTable");
+    TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY).close();
     tables = this.admin.listTables();
     assertEquals(numTables + 1, tables.length);
     assertTrue("Table must be enabled.", TEST_UTIL.getHBaseCluster()
         .getMaster().getAssignmentManager().getTableStateManager().isTableState(
-        TableName.valueOf("testCreateTable"), TableState.State.ENABLED));
+            tableName, TableState.State.ENABLED));
+    assertEquals(TableState.State.ENABLED, getStateFromMeta(tableName));
   }
 
   @Test (timeout=300000)
@@ -405,6 +424,7 @@ public class TestAdmin1 {
     Table table = TEST_UTIL.getConnection().getTable(htd.getTableName());
     HTableDescriptor confirmedHtd = table.getTableDescriptor();
     assertEquals(htd.compareTo(confirmedHtd), 0);
+    MetaTableAccessor.fullScanMetaAndPrint(TEST_UTIL.getConnection());
     table.close();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
index 83ff822..8e60353 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
@@ -18,10 +18,8 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.assertErrors;
-import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.doFsck;
-import static org.junit.Assert.*;
-
+import javax.annotation.Nullable;
+import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
@@ -39,21 +37,28 @@ import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.regionserver.StorefileRefresherChore;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.HBaseFsck;
-import org.apache.hadoop.hbase.util.HBaseFsckRepair;
 import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE;
+import org.apache.hadoop.hbase.util.HBaseFsckRepair;
 import org.apache.hadoop.hbase.util.hbck.HbckTestingUtil;
 import org.apache.hadoop.hbase.zookeeper.LoadBalancerTracker;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.zookeeper.KeeperException;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.assertErrors;
+import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.doFsck;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 /**
  * Tests the scenarios where replicas are enabled for the meta table
  */
@@ -224,7 +229,8 @@ public class TestMetaWithReplicas {
     stopMasterAndValidateReplicaCount(2, 3);
   }
 
-  private void stopMasterAndValidateReplicaCount(int originalReplicaCount, int newReplicaCount)
+  private void stopMasterAndValidateReplicaCount(final int originalReplicaCount,
+      final int newReplicaCount)
       throws Exception {
     ServerName sn = TEST_UTIL.getHBaseClusterInterface().getClusterStatus().getMaster();
     TEST_UTIL.getHBaseClusterInterface().stopMaster(sn);
@@ -235,16 +241,7 @@ public class TestMetaWithReplicas {
         newReplicaCount);
     TEST_UTIL.getHBaseClusterInterface().startMaster(sn.getHostname(), 0);
     TEST_UTIL.getHBaseClusterInterface().waitForActiveAndReadyMaster();
-    int count = 0;
-    do {
-      metaZnodes = TEST_UTIL.getZooKeeperWatcher().getMetaReplicaNodes();
-      Thread.sleep(10);
-      count++;
-      // wait for the count to be different from the originalReplicaCount. When the 
-      // replica count is reduced, that will happen when the master unassigns excess
-      // replica, and deletes the excess znodes
-    } while (metaZnodes.size() == originalReplicaCount && count < 1000);
-    assert(metaZnodes.size() == newReplicaCount);
+    TEST_UTIL.waitFor(10000, predicateMetaHasReplicas(newReplicaCount));
     // also check if hbck returns without errors
     TEST_UTIL.getConfiguration().setInt(HConstants.META_REPLICAS_NUM,
         newReplicaCount);
@@ -252,6 +249,46 @@ public class TestMetaWithReplicas {
     HbckTestingUtil.assertNoErrors(hbck);
   }
 
+  private Waiter.ExplainingPredicate<Exception> predicateMetaHasReplicas(
+      final int newReplicaCount) {
+    return new Waiter.ExplainingPredicate<Exception>() {
+      @Override
+      public String explainFailure() throws Exception {
+        return checkMetaLocationAndExplain(newReplicaCount);
+      }
+
+      @Override
+      public boolean evaluate() throws Exception {
+        return checkMetaLocationAndExplain(newReplicaCount) == null;
+      }
+    };
+  }
+
+  @Nullable
+  private String checkMetaLocationAndExplain(int originalReplicaCount)
+      throws KeeperException, IOException {
+    List<String> metaZnodes = TEST_UTIL.getZooKeeperWatcher().getMetaReplicaNodes();
+    if (metaZnodes.size() == originalReplicaCount) {
+      RegionLocations rl = ((ClusterConnection) TEST_UTIL.getConnection())
+          .locateRegion(TableName.META_TABLE_NAME,
+              HConstants.EMPTY_START_ROW, false, false);
+      for (HRegionLocation location : rl.getRegionLocations()) {
+        if (location == null) {
+          return "Null location found in " + rl.toString();
+        }
+        if (location.getRegionInfo() == null) {
+          return "Null regionInfo for location " + location;
+        }
+        if (location.getHostname() == null) {
+          return "Null hostName for location " + location;
+        }
+      }
+      return null; // OK
+    }
+    return "Replica count is not as expected " + originalReplicaCount + " <> " + metaZnodes.size()
+        + "(" + metaZnodes.toString() + ")";
+  }
+
   @Test
   public void testHBaseFsckWithMetaReplicas() throws Exception {
     HBaseFsck hbck = HbckTestingUtil.doFsck(TEST_UTIL.getConfiguration(), false);

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
index 6d98c52..b0bd6f6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.regionserver.TestHRegionServerBulkLoad;
 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.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
 import org.junit.AfterClass;
@@ -130,7 +131,8 @@ public class TestReplicaWithCluster {
 
   @AfterClass
   public static void afterClass() throws Exception {
-    HTU2.shutdownMiniCluster();
+    if (HTU2 != null)
+      HTU2.shutdownMiniCluster();
     HTU.shutdownMiniCluster();
   }
 
@@ -213,7 +215,6 @@ public class TestReplicaWithCluster {
       SlowMeCopro.sleepTime.set(0);
     }
 
-    HTU.getHBaseCluster().stopMaster(0);
     Admin admin = HTU.getHBaseAdmin();
     nHdt =admin.getTableDescriptor(hdt.getTableName());
     Assert.assertEquals("fams=" + Arrays.toString(nHdt.getColumnFamilies()),
@@ -221,7 +222,6 @@ public class TestReplicaWithCluster {
 
     admin.disableTable(hdt.getTableName());
     admin.deleteTable(hdt.getTableName());
-    HTU.getHBaseCluster().startMaster();
     admin.close();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java
index f1c080d..eb72220 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java
@@ -643,7 +643,8 @@ public class TestAssignmentManagerOnCluster {
       if (hri != null && serverName != null) {
         am.regionOnline(hri, serverName);
       }
-      am.getTableStateManager().setTableState(table, TableState.State.DISABLED);
+      am.getTableStateManager().setTableState(table, TableState.State.ENABLED);
+      TEST_UTIL.getHBaseAdmin().disableTable(table);
       TEST_UTIL.deleteTable(table);
     }
   }
@@ -1164,7 +1165,7 @@ public class TestAssignmentManagerOnCluster {
         tableNameList.add(TableName.valueOf(name + "_" + i));
       }
     }
-    List<Result> metaRows = MetaTableAccessor.fullScanOfMeta(admin.getConnection());
+    List<Result> metaRows = MetaTableAccessor.fullScanRegions(admin.getConnection());
     int count = 0;
     // Check all 100 rows are in meta
     for (Result result : metaRows) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
index e09583a..8ed49ff 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
@@ -1043,8 +1043,7 @@ public class TestCatalogJanitor {
   }
 
   private TableDescriptor createTableDescriptor() {
-    TableDescriptor htd = new TableDescriptor(createHTableDescriptor(), TableState.State.ENABLED);
-    return htd;
+    return new TableDescriptor(createHTableDescriptor());
   }
 
   private MultiResponse buildMultiResponse(MultiRequest req) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/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 6307c4c..ca9bc9c 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
@@ -47,8 +47,6 @@ import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Table;
@@ -134,7 +132,7 @@ public class TestMasterOperationsForRegionReplicas {
         }
       }
 
-      List<Result> metaRows = MetaTableAccessor.fullScanOfMeta(ADMIN.getConnection());
+      List<Result> metaRows = MetaTableAccessor.fullScanRegions(ADMIN.getConnection());
       int numRows = 0;
       for (Result result : metaRows) {
         RegionLocations locations = MetaTableAccessor.getRegionLocations(result);
@@ -297,7 +295,7 @@ public class TestMasterOperationsForRegionReplicas {
         return true;
       }
     };
-    MetaTableAccessor.fullScan(connection, visitor);
+    MetaTableAccessor.fullScanRegions(connection, visitor);
     assert(count.get() == numRegions);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionStates.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionStates.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionStates.java
index ce61e40..99e1709 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionStates.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionStates.java
@@ -56,7 +56,7 @@ public class TestRegionStates {
   @Test (timeout=10000)
   public void testCanMakeProgressThoughMetaIsDown()
   throws IOException, InterruptedException, BrokenBarrierException {
-    Server server = mock(Server.class);
+    MasterServices server = mock(MasterServices.class);
     when(server.getServerName()).thenReturn(ServerName.valueOf("master,1,1"));
     Connection connection = mock(ClusterConnection.class);
     // Set up a table that gets 'stuck' when we try to fetch a row from the meta table.
@@ -101,7 +101,7 @@ public class TestRegionStates {
 
   @Test
   public void testWeDontReturnDrainingServersForOurBalancePlans() throws Exception {
-    Server server = mock(Server.class);
+    MasterServices server = mock(MasterServices.class);
     when(server.getServerName()).thenReturn(ServerName.valueOf("master,1,1"));
     Configuration configuration = mock(Configuration.class);
     when(server.getConfiguration()).thenReturn(configuration);

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
index 8296e81..51436b4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
@@ -2520,6 +2520,7 @@ public class TestAccessController extends SecureTestUtil {
       assertTrue(existingPerms.size() > 1);
       TEST_UTIL.getHBaseAdmin().disableTable(TEST_TABLE.getTableName());
       TEST_UTIL.truncateTable(TEST_TABLE.getTableName());
+      TEST_UTIL.waitTableAvailable(TEST_TABLE.getTableName());
       List<UserPermission> perms = AccessControlClient.getUserPermissions(conf,
         TEST_TABLE.getTableName().getNameAsString());
       assertTrue(perms != null);

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java
index 9a7db90..7600388 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java
@@ -212,6 +212,7 @@ public class TestCoprocessorScanPolicy {
     // should be gone now
     assertEquals(0, r.size());
     t.close();
+    EnvironmentEdgeManager.reset();
   }
 
   public static class ScanObserver extends BaseRegionObserver {

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java
index a99daf2..c09982e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java
@@ -76,7 +76,7 @@ public class TestFSTableDescriptors {
   public void testCreateAndUpdate() throws IOException {
     Path testdir = UTIL.getDataTestDir("testCreateAndUpdate");
     HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("testCreate"));
-    TableDescriptor td = new TableDescriptor(htd, TableState.State.ENABLED);
+    TableDescriptor td = new TableDescriptor(htd);
     FileSystem fs = FileSystem.get(UTIL.getConfiguration());
     FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, testdir);
     assertTrue(fstd.createTableDescriptor(td));
@@ -113,7 +113,7 @@ public class TestFSTableDescriptors {
     assertTrue(!fs.exists(p1));
     int i2 = FSTableDescriptors.getTableInfoSequenceId(p2);
     assertTrue(i2 == i1 + 1);
-    td = new TableDescriptor(htd, TableState.State.DISABLED);
+    td = new TableDescriptor(htd);
     Path p3 = fstd.updateTableDescriptor(td);
     // Assert we cleaned up the old file.
     assertTrue(!fs.exists(p2));
@@ -172,7 +172,7 @@ public class TestFSTableDescriptors {
     final String name = "testReadingHTDFromFS";
     FileSystem fs = FileSystem.get(UTIL.getConfiguration());
     HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name));
-    TableDescriptor td = new TableDescriptor(htd, TableState.State.ENABLED);
+    TableDescriptor td = new TableDescriptor(htd);
     Path rootdir = UTIL.getDataTestDir(name);
     FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, rootdir);
     fstd.createTableDescriptor(td);
@@ -187,7 +187,7 @@ public class TestFSTableDescriptors {
     Path rootdir = UTIL.getDataTestDir(name);
     FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, rootdir);
     HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name));
-    TableDescriptor td = new TableDescriptor(htd, TableState.State.ENABLED);
+    TableDescriptor td = new TableDescriptor(htd);
     Path descriptorFile = fstd.updateTableDescriptor(td);
     try (FSDataOutputStream out = fs.create(descriptorFile, true)) {
       out.write(htd.toByteArray());
@@ -222,8 +222,8 @@ public class TestFSTableDescriptors {
     final int count = 10;
     // Write out table infos.
     for (int i = 0; i < count; i++) {
-      TableDescriptor htd = new TableDescriptor(new HTableDescriptor(name + i),
-          TableState.State.ENABLED);
+      TableDescriptor htd = new TableDescriptor(
+          new HTableDescriptor(TableName.valueOf(name + i)));
       htds.createTableDescriptor(htd);
     }
 
@@ -420,7 +420,7 @@ public class TestFSTableDescriptors {
     Path testdir = UTIL.getDataTestDir("testCreateTableDescriptorUpdatesIfThereExistsAlready");
     HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(
         "testCreateTableDescriptorUpdatesIfThereExistsAlready"));
-    TableDescriptor td = new TableDescriptor(htd, TableState.State.ENABLED);
+    TableDescriptor td = new TableDescriptor(htd);
     FileSystem fs = FileSystem.get(UTIL.getConfiguration());
     FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, testdir);
     assertTrue(fstd.createTableDescriptor(td));

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java
index 33bd337..0d3a94e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java
@@ -157,6 +157,7 @@ public class TestHBaseFsck {
     conf.setInt("hbase.hconnection.threads.max", 2 * POOL_SIZE);
     conf.setInt("hbase.hconnection.threads.core", POOL_SIZE);
     conf.setInt("hbase.hbck.close.timeout", 2 * REGION_ONLINE_TIMEOUT);
+    conf.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, 2 * REGION_ONLINE_TIMEOUT);
     TEST_UTIL.startMiniCluster(3);
 
     tableExecutorService = new ThreadPoolExecutor(1, POOL_SIZE, 60, TimeUnit.SECONDS,
@@ -1402,7 +1403,7 @@ public class TestHBaseFsck {
     HBaseFsck hbck = doFsck(conf, false);
     assertErrors(hbck, new ERROR_CODE[] {ERROR_CODE.NOT_IN_HDFS,
         ERROR_CODE.NOT_IN_HDFS, ERROR_CODE.NOT_IN_HDFS,
-        ERROR_CODE.NOT_IN_HDFS,});
+        ERROR_CODE.NOT_IN_HDFS, ERROR_CODE.ORPHAN_TABLE_STATE, });
     // holes are separate from overlap groups
     assertEquals(0, hbck.getOverlapGroups(table).size());
 
@@ -1445,6 +1446,34 @@ public class TestHBaseFsck {
   }
 
   /**
+   * when the hbase.version file missing, It is fix the fault.
+   */
+  @Test (timeout=180000)
+  public void testNoTableState() throws Exception {
+    // delete the hbase.version file
+    TableName table =
+        TableName.valueOf("testNoTableState");
+    try {
+      setupTable(table);
+      // make sure data in regions, if in wal only there is no data loss
+      admin.flush(table);
+
+      MetaTableAccessor.deleteTableState(TEST_UTIL.getConnection(), table);
+
+      // test
+      HBaseFsck hbck = doFsck(conf, false);
+      assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.NO_TABLE_STATE });
+      // fix table state missing
+      doFsck(conf, true);
+
+      assertNoErrors(doFsck(conf, false));
+      assertTrue(TEST_UTIL.getHBaseAdmin().isTableEnabled(table));
+    } finally {
+      cleanupTable(table);
+    }
+  }
+
+  /**
    * The region is not deployed when the table is disabled.
    */
   @Test (timeout=180000)

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java
index 349bf56..e940425 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java
@@ -285,16 +285,9 @@ public class OfflineMetaRebuildTestCore {
    * @return # of entries in meta.
    */
   protected int scanMeta() throws IOException {
-    int count = 0;
-    Table meta = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME);
-    ResultScanner scanner = meta.getScanner(new Scan());
-    LOG.info("Table: " + meta.getName());
-    for (Result res : scanner) {
-      LOG.info(Bytes.toString(res.getRow()));
-      count++;
-    }
-    meta.close();
-    return count;
+    LOG.info("Scanning META");
+    MetaTableAccessor.fullScanMetaAndPrint(TEST_UTIL.getConnection());
+    return MetaTableAccessor.fullScanRegions(TEST_UTIL.getConnection()).size();
   }
 
   protected HTableDescriptor[] getTables(final Configuration configuration) throws IOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildBase.java
index a3d323c..fc22292 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildBase.java
@@ -20,11 +20,13 @@ package org.apache.hadoop.hbase.util.hbck;
 import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.assertErrors;
 import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.doFsck;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
 import java.util.Arrays;
 
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
@@ -70,13 +72,20 @@ public class TestOfflineMetaRebuildBase extends OfflineMetaRebuildTestCore {
     TEST_UTIL.restartHBaseCluster(3);
     try (Connection connection = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())) {
       Admin admin = connection.getAdmin();
-      admin.enableTable(table);
+      if (admin.isTableDisabled(table))
+        admin.enableTable(table);
       LOG.info("Waiting for no more RIT");
       TEST_UTIL.waitUntilNoRegionsInTransition(60000);
       LOG.info("No more RIT in ZK, now doing final test verification");
 
       // everything is good again.
-      assertEquals(5, scanMeta());
+      assertEquals(5, scanMeta()); // including table state rows
+      TableName[] tableNames = TEST_UTIL.getHBaseAdmin().listTableNames();
+      for (TableName tableName : tableNames) {
+        HTableDescriptor tableDescriptor = TEST_UTIL.getHBaseAdmin().getTableDescriptor(tableName);
+        assertNotNull(tableDescriptor);
+        assertTrue(TEST_UTIL.getHBaseAdmin().isTableEnabled(tableName));
+      }
       HTableDescriptor[] htbls = admin.listTables();
       LOG.info("Tables present after restart: " + Arrays.toString(htbls));
       assertEquals(1, htbls.length);


[11/50] [abbrv] hbase git commit: HBASE-12956 Binding to 0.0.0.0 is broken after HBASE-10569

Posted by jm...@apache.org.
HBASE-12956 Binding to 0.0.0.0 is broken after HBASE-10569

Signed-off-by: Enis Soztutar <en...@apache.org>


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

Branch: refs/heads/hbase-11339
Commit: 3b56d2a0bc36f9dcb901bb709b8d9ae58df955ff
Parents: 57319c5
Author: Esteban Gutierrez <es...@cloudera.com>
Authored: Fri Feb 6 11:50:07 2015 -0800
Committer: Enis Soztutar <en...@apache.org>
Committed: Fri Feb 6 14:35:20 2015 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/ipc/RpcServer.java    | 18 ++++++++++--------
 .../hadoop/hbase/regionserver/RSRpcServices.java  | 13 +++++++------
 .../hadoop/hbase/TestHBaseTestingUtility.java     | 12 ++++++++++++
 3 files changed, 29 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/3b56d2a0/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
index 5eb9a52..b469fac 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
@@ -196,7 +196,7 @@ public class RpcServer implements RpcServerInterface {
   static final ThreadLocal<MonitoredRPCHandler> MONITORED_RPC
       = new ThreadLocal<MonitoredRPCHandler>();
 
-  protected final InetSocketAddress isa;
+  protected final InetSocketAddress bindAddress;
   protected int port;                             // port we listen on
   private int readThreads;                        // number of read threads
   protected int maxIdleTime;                      // the maximum idle time after
@@ -525,8 +525,8 @@ public class RpcServer implements RpcServerInterface {
       acceptChannel = ServerSocketChannel.open();
       acceptChannel.configureBlocking(false);
 
-      // Bind the server socket to the local host and port
-      bind(acceptChannel.socket(), isa, backlogLength);
+      // Bind the server socket to the binding addrees (can be different from the default interface)
+      bind(acceptChannel.socket(), bindAddress, backlogLength);
       port = acceptChannel.socket().getLocalPort(); //Could be an ephemeral port
       // create a selector;
       selector= Selector.open();
@@ -534,7 +534,8 @@ public class RpcServer implements RpcServerInterface {
       readers = new Reader[readThreads];
       readPool = Executors.newFixedThreadPool(readThreads,
         new ThreadFactoryBuilder().setNameFormat(
-          "RpcServer.reader=%d,port=" + port).setDaemon(true).build());
+          "RpcServer.reader=%d,bindAddress=" + bindAddress.getHostName() +
+          ",port=" + port).setDaemon(true).build());
       for (int i = 0; i < readThreads; ++i) {
         Reader reader = new Reader();
         readers[i] = reader;
@@ -1873,17 +1874,18 @@ public class RpcServer implements RpcServerInterface {
    * instance else pass null for no authentication check.
    * @param name Used keying this rpc servers' metrics and for naming the Listener thread.
    * @param services A list of services.
-   * @param isa Where to listen
+   * @param bindAddres Where to listen
    * @throws IOException
    */
   public RpcServer(final Server server, final String name,
       final List<BlockingServiceAndInterface> services,
-      final InetSocketAddress isa, Configuration conf,
+      final InetSocketAddress bindAddress, Configuration conf,
       RpcScheduler scheduler)
-  throws IOException {
+      throws IOException {
+
     this.server = server;
     this.services = services;
-    this.isa = isa;
+    this.bindAddress = bindAddress;
     this.conf = conf;
     this.socketSendBufferSize = 0;
     this.maxQueueSize =

http://git-wip-us.apache.org/repos/asf/hbase/blob/3b56d2a0/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index 68530d3..3653cfb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -781,6 +781,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       HConstants.DEFAULT_REGIONSERVER_PORT);
     // Creation of a HSA will force a resolve.
     InetSocketAddress initialIsa = new InetSocketAddress(hostname, port);
+    InetSocketAddress bindAddress = new InetSocketAddress(
+      rs.conf.get("hbase.regionserver.ipc.address", hostname), port);
     if (initialIsa.getAddress() == null) {
       throw new IllegalArgumentException("Failed resolve of " + initialIsa);
     }
@@ -789,7 +791,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     // Set how many times to retry talking to another server over HConnection.
     ConnectionUtils.setServerSideHConnectionRetriesConfig(rs.conf, name, LOG);
     rpcServer = new RpcServer(rs, name, getServices(),
-      initialIsa, // BindAddress is IP we got for this server.
+      bindAddress, // use final bindAddress for this server.
       rs.conf,
       rpcSchedulerFactory.create(rs.conf, this, rs));
 
@@ -800,17 +802,16 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       HConstants.HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE_KEY,
       HConstants.DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE);
 
-    // Set our address.
-    isa = rpcServer.getListenerAddress();
+    // Set our address, however we need the final port that was given to rpcServer
+    isa = new InetSocketAddress(initialIsa.getHostName(), rpcServer.getListenerAddress().getPort());
     rpcServer.setErrorHandler(this);
     rs.setName(name);
   }
 
   public static String getHostname(Configuration conf) throws UnknownHostException {
-    return conf.get("hbase.regionserver.ipc.address",
-        Strings.domainNamePointerToHostName(DNS.getDefaultHost(
+    return Strings.domainNamePointerToHostName(DNS.getDefaultHost(
             conf.get("hbase.regionserver.dns.interface", "default"),
-            conf.get("hbase.regionserver.dns.nameserver", "default"))));
+            conf.get("hbase.regionserver.dns.nameserver", "default")));
   }
 
   RegionScanner getScanner(long scannerId) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/3b56d2a0/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java
index abbcb4c..56720a3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java
@@ -125,6 +125,18 @@ public class TestHBaseTestingUtility {
     }
   }
 
+  @Test
+  public void testMiniClusterBindToWildcard() throws Exception {
+    HBaseTestingUtility hbt = new HBaseTestingUtility();
+    hbt.getConfiguration().set("hbase.regionserver.ipc.address", "0.0.0.0");
+    MiniHBaseCluster cluster = hbt.startMiniCluster();
+    try {
+      assertEquals(1, cluster.getLiveRegionServerThreads().size());
+    } finally {
+      hbt.shutdownMiniCluster();
+    }
+  }
+
   /**
    *  Test that we can start and stop multiple time a cluster
    *   with the same HBaseTestingUtility.


[23/50] [abbrv] hbase git commit: HBASE-12973 RegionCoprocessorEnvironment should provide HRegionInfo directly

Posted by jm...@apache.org.
HBASE-12973 RegionCoprocessorEnvironment should provide HRegionInfo directly


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

Branch: refs/heads/hbase-11339
Commit: 1e6afa2f3c7b8515146e7bf2bce032f8f65da7ff
Parents: 1f830be
Author: Andrew Purtell <ap...@apache.org>
Authored: Mon Feb 9 17:57:40 2015 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Mon Feb 9 17:57:40 2015 -0800

----------------------------------------------------------------------
 .../hbase/coprocessor/RegionCoprocessorEnvironment.java       | 7 +++++--
 .../hadoop/hbase/regionserver/RegionCoprocessorHost.java      | 5 +++++
 .../hadoop/hbase/security/token/TestTokenAuthentication.java  | 6 ++++++
 3 files changed, 16 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/1e6afa2f/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionCoprocessorEnvironment.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionCoprocessorEnvironment.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionCoprocessorEnvironment.java
index db6312f..ccb16bf 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionCoprocessorEnvironment.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionCoprocessorEnvironment.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
 
@@ -34,10 +35,12 @@ public interface RegionCoprocessorEnvironment extends CoprocessorEnvironment {
   /** @return the region associated with this coprocessor */
   HRegion getRegion();
 
+  /** @return region information for the region this coprocessor is running on */
+  HRegionInfo getRegionInfo();
+
   /** @return reference to the region server services */
   RegionServerServices getRegionServerServices();
 
   /** @return shared data between all instances of this coprocessor */
-  ConcurrentMap<String, Object> getSharedData();
-
+  ConcurrentMap<String, Object> getSharedData(); 
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1e6afa2f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java
index 87c8b9e..a32a478 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java
@@ -168,6 +168,11 @@ public class RegionCoprocessorHost
       return latencies;
     }
 
+    @Override
+    public HRegionInfo getRegionInfo() {
+      return region.getRegionInfo();
+    }
+
   }
 
   static class TableCoprocessorAttribute {

http://git-wip-us.apache.org/repos/asf/hbase/blob/1e6afa2f/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
index 22f1a24..3bd20b2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.CoordinatedStateManager;
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
@@ -244,6 +245,11 @@ public class TestTokenAuthentication {
         public ClassLoader getClassLoader() {
           return Thread.currentThread().getContextClassLoader();
         }
+
+        @Override
+        public HRegionInfo getRegionInfo() {
+          return null;
+        }
       });
 
       started = true;


[38/50] [abbrv] hbase git commit: HBASE-12585 Fix refguide so it does hbase 1.0 style API everywhere with callout on how we used to do it in pre-1.0

Posted by jm...@apache.org.
HBASE-12585 Fix refguide so it does hbase 1.0 style API everywhere
with callout on how we used to do it in pre-1.0


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

Branch: refs/heads/hbase-11339
Commit: a0f2bc07b27a4119c356b5e2d111cf8e93dfb00e
Parents: a8d325e
Author: Misty Stanley-Jones <ms...@cloudera.com>
Authored: Tue Feb 10 13:14:25 2015 +1000
Committer: Misty Stanley-Jones <ms...@cloudera.com>
Committed: Wed Feb 11 16:07:35 2015 +1000

----------------------------------------------------------------------
 src/main/asciidoc/_chapters/architecture.adoc   | 38 ++++++++--------
 src/main/asciidoc/_chapters/configuration.adoc  |  4 +-
 src/main/asciidoc/_chapters/datamodel.adoc      |  2 +-
 src/main/asciidoc/_chapters/hbase_apis.adoc     | 11 ++---
 src/main/asciidoc/_chapters/ops_mgt.adoc        |  8 ++--
 src/main/asciidoc/_chapters/performance.adoc    | 26 +++++------
 src/main/asciidoc/_chapters/schema_design.adoc  | 12 ++---
 src/main/asciidoc/_chapters/security.adoc       | 47 +++++++++++++-------
 src/main/asciidoc/_chapters/tracing.adoc        |  7 +--
 .../asciidoc/_chapters/troubleshooting.adoc     |  1 +
 src/main/asciidoc/_chapters/unit_testing.adoc   | 14 +++---
 11 files changed, 92 insertions(+), 78 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a0f2bc07/src/main/asciidoc/_chapters/architecture.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/architecture.adoc b/src/main/asciidoc/_chapters/architecture.adoc
index 1833cfc..bae4a23 100644
--- a/src/main/asciidoc/_chapters/architecture.adoc
+++ b/src/main/asciidoc/_chapters/architecture.adoc
@@ -202,24 +202,24 @@ HBaseConfiguration conf2 = HBaseConfiguration.create();
 HTable table2 = new HTable(conf2, "myTable");
 ----
 
-For more information about how connections are handled in the HBase client, see link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HConnectionManager.html[HConnectionManager].
+For more information about how connections are handled in the HBase client, see link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/ConnectionFactory.html[ConnectionFactory].
 
 [[client.connection.pooling]]
 ===== Connection Pooling
 
-For applications which require high-end multithreaded access (e.g., web-servers or application servers that may serve many application threads in a single JVM), you can pre-create an `HConnection`, as shown in the following example:
+For applications which require high-end multithreaded access (e.g., web-servers or application servers that may serve many application threads in a single JVM), you can pre-create a `Connection`, as shown in the following example:
 
-.Pre-Creating a `HConnection`
+.Pre-Creating a `Connection`
 ====
 [source,java]
 ----
 // Create a connection to the cluster.
-HConnection connection = HConnectionManager.createConnection(Configuration);
-HTableInterface table = connection.getTable("myTable");
-// use table as needed, the table returned is lightweight
-table.close();
-// use the connection for other access to the cluster
-connection.close();
+Configuration conf = HBaseConfiguration.create();
+try (Connection connection = ConnectionFactory.createConnection(conf)) {
+  try (Table table = connection.getTable(TableName.valueOf(tablename)) {
+    // use table as needed, the table returned is lightweight
+  }
+}
 ----
 ====
 
@@ -228,22 +228,20 @@ Constructing HTableInterface implementation is very lightweight and resources ar
 .`HTablePool` is Deprecated
 [WARNING]
 ====
-Previous versions of this guide discussed `HTablePool`, which was deprecated in HBase 0.94, 0.95, and 0.96, and removed in 0.98.1, by link:https://issues.apache.org/jira/browse/HBASE-6580[HBASE-6500].
-Please use link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HConnection.html[HConnection] instead.
+Previous versions of this guide discussed `HTablePool`, which was deprecated in HBase 0.94, 0.95, and 0.96, and removed in 0.98.1, by link:https://issues.apache.org/jira/browse/HBASE-6580[HBASE-6500], or `HConnection`, which is deprecated in HBase 1.0 by `Connection`.
+Please use link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Connection.html[Connection] instead.
 ====
 
 [[client.writebuffer]]
 === WriteBuffer and Batch Methods
 
-If <<perf.hbase.client.autoflush>> is turned off on link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HTable.html[HTable], ``Put``s are sent to RegionServers when the writebuffer is filled.
-The writebuffer is 2MB by default.
-Before an (H)Table instance is discarded, either `close()` or `flushCommits()` should be invoked so Puts will not be lost.
+In HBase 1.0 and later, link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HTable.html[HTable] is deprecated in favor of link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html[Table]. `Table` does not use autoflush. To do buffered writes, use the BufferedMutator class.
 
-NOTE: `htable.delete(Delete);` does not go in the writebuffer! This only applies to Puts.
+Before a `Table` or `HTable` instance is discarded, invoke either `close()` or `flushCommits()`, so `Put`s will not be lost.
 
 For additional information on write durability, review the link:../acid-semantics.html[ACID semantics] page.
 
-For fine-grained control of batching of ``Put``s or ``Delete``s, see the link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HTable.html#batch%28java.util.List%29[batch] methods on HTable.
+For fine-grained control of batching of ``Put``s or ``Delete``s, see the link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html#batch%28java.util.List%29[batch] methods on Table.
 
 [[client.external]]
 === External Clients
@@ -523,7 +521,7 @@ The methods exposed by `HMasterInterface` are primarily metadata-oriented method
 
 * Table (createTable, modifyTable, removeTable, enable, disable)
 * ColumnFamily (addColumn, modifyColumn, removeColumn)
-* Region (move, assign, unassign) For example, when the `HBaseAdmin` method `disableTable` is invoked, it is serviced by the Master server.
+* Region (move, assign, unassign) For example, when the `Admin` method `disableTable` is invoked, it is serviced by the Master server.
 
 [[master.processes]]
 === Processes
@@ -557,7 +555,7 @@ In a distributed cluster, a RegionServer runs on a <<arch.hdfs.dn>>.
 The methods exposed by `HRegionRegionInterface` contain both data-oriented and region-maintenance methods:
 
 * Data (get, put, delete, next, etc.)
-* Region (splitRegion, compactRegion, etc.) For example, when the `HBaseAdmin` method `majorCompact` is invoked on a table, the client is actually iterating through all regions for the specified table and requesting a major compaction directly to each region.
+* Region (splitRegion, compactRegion, etc.) For example, when the `Admin` method `majorCompact` is invoked on a table, the client is actually iterating through all regions for the specified table and requesting a major compaction directly to each region.
 
 [[regionserver.arch.processes]]
 === Processes
@@ -2310,7 +2308,7 @@ Ensure to set the following for all clients (and servers) that will use region r
   <name>hbase.client.primaryCallTimeout.multiget</name>
   <value>10000</value>
   <description>
-      The timeout (in microseconds), before secondary fallback RPC’s are submitted for multi-get requests (HTable.get(List<Get>)) with Consistency.TIMELINE to the secondary replicas of the regions. Defaults to 10ms. Setting this lower will increase the number of RPC’s, but will lower the p99 latencies.
+      The timeout (in microseconds), before secondary fallback RPC’s are submitted for multi-get requests (Table.get(List<Get>)) with Consistency.TIMELINE to the secondary replicas of the regions. Defaults to 10ms. Setting this lower will increase the number of RPC’s, but will lower the p99 latencies.
   </description>
 </property>
 <property>
@@ -2346,7 +2344,7 @@ flush 't1'
 
 [source,java]
 ----
-HTableDescriptor htd = new HTableDesctiptor(TableName.valueOf(“test_table”));
+HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(“test_table”));
 htd.setRegionReplication(2);
 ...
 admin.createTable(htd);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a0f2bc07/src/main/asciidoc/_chapters/configuration.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/configuration.adoc b/src/main/asciidoc/_chapters/configuration.adoc
index 4a9835a..6f8858d 100644
--- a/src/main/asciidoc/_chapters/configuration.adoc
+++ b/src/main/asciidoc/_chapters/configuration.adoc
@@ -626,7 +626,7 @@ Configuration config = HBaseConfiguration.create();
 config.set("hbase.zookeeper.quorum", "localhost");  // Here we are running zookeeper locally
 ----
 
-If multiple ZooKeeper instances make up your ZooKeeper ensemble, they may be specified in a comma-separated list (just as in the _hbase-site.xml_ file). This populated `Configuration` instance can then be passed to an link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HTable.html[HTable], and so on.
+If multiple ZooKeeper instances make up your ZooKeeper ensemble, they may be specified in a comma-separated list (just as in the _hbase-site.xml_ file). This populated `Configuration` instance can then be passed to an link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html[Table], and so on.
 
 [[example_config]]
 == Example Configurations
@@ -867,7 +867,7 @@ See the entry for `hbase.hregion.majorcompaction` in the <<compaction.parameters
 ====
 Major compactions are absolutely necessary for StoreFile clean-up.
 Do not disable them altogether.
-You can run major compactions manually via the HBase shell or via the link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HBaseAdmin.html#majorCompact%28java.lang.String%29[HBaseAdmin API].
+You can run major compactions manually via the HBase shell or via the http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Admin.html#majorCompact(org.apache.hadoop.hbase.TableName)[Admin API].
 ====
 
 For more information about compactions and the compaction file selection process, see <<compaction,compaction>>

http://git-wip-us.apache.org/repos/asf/hbase/blob/a0f2bc07/src/main/asciidoc/_chapters/datamodel.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/datamodel.adoc b/src/main/asciidoc/_chapters/datamodel.adoc
index 91e6be8..74238ca 100644
--- a/src/main/asciidoc/_chapters/datamodel.adoc
+++ b/src/main/asciidoc/_chapters/datamodel.adoc
@@ -316,7 +316,7 @@ Note that generally the easiest way to specify a specific stop point for a scan
 === Delete
 
 link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Delete.html[Delete] removes a row from a table.
-Deletes are executed via link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html#delete(org.apache.hadoop.hbase.client.Delete)[HTable.delete].
+Deletes are executed via link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html#delete(org.apache.hadoop.hbase.client.Delete)[Table.delete].
 
 HBase does not modify data in place, and so deletes are handled by creating new markers called _tombstones_.
 These tombstones, along with the dead values, are cleaned up on major compactions.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a0f2bc07/src/main/asciidoc/_chapters/hbase_apis.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/hbase_apis.adoc b/src/main/asciidoc/_chapters/hbase_apis.adoc
index 7fe0d3e..d73de61 100644
--- a/src/main/asciidoc/_chapters/hbase_apis.adoc
+++ b/src/main/asciidoc/_chapters/hbase_apis.adoc
@@ -38,11 +38,9 @@ See <<external_apis>> for more information.
 
 .Create a Table Using Java
 ====
-This example has been tested on HBase 0.96.1.1.
 
 [source,java]
 ----
-
 package com.example.hbase.admin;
 
 import java.io.IOException;
@@ -51,7 +49,7 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
 import org.apache.hadoop.conf.Configuration;
 
@@ -59,7 +57,7 @@ import static com.example.hbase.Constants.*;
 
 public class CreateSchema {
 
-  public static void createOrOverwrite(HBaseAdmin admin, HTableDescriptor table) throws IOException {
+  public static void createOrOverwrite(Admin admin, HTableDescriptor table) throws IOException {
     if (admin.tableExists(table.getName())) {
       admin.disableTable(table.getName());
       admin.deleteTable(table.getName());
@@ -69,7 +67,7 @@ public class CreateSchema {
 
   public static void createSchemaTables (Configuration config) {
     try {
-      final HBaseAdmin admin = new HBaseAdmin(config);
+      final Admin admin = new Admin(config);
       HTableDescriptor table = new HTableDescriptor(TableName.valueOf(TABLE_NAME));
       table.addFamily(new HColumnDescriptor(CF_DEFAULT).setCompressionType(Algorithm.SNAPPY));
 
@@ -90,14 +88,13 @@ public class CreateSchema {
 
 .Add, Modify, and Delete a Table
 ====
-This example has been tested on HBase 0.96.1.1.
 
 [source,java]
 ----
 public static void upgradeFrom0 (Configuration config) {
 
   try {
-    final HBaseAdmin admin = new HBaseAdmin(config);
+    final Admin admin = new Admin(config);
     TableName tableName = TableName.valueOf(TABLE_ASSETMETA);
     HTableDescriptor table_assetmeta = new HTableDescriptor(tableName);
     table_assetmeta.addFamily(new HColumnDescriptor(CF_DEFAULT).setCompressionType(Algorithm.SNAPPY));

http://git-wip-us.apache.org/repos/asf/hbase/blob/a0f2bc07/src/main/asciidoc/_chapters/ops_mgt.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/ops_mgt.adoc b/src/main/asciidoc/_chapters/ops_mgt.adoc
index 852e76b..1402f52 100644
--- a/src/main/asciidoc/_chapters/ops_mgt.adoc
+++ b/src/main/asciidoc/_chapters/ops_mgt.adoc
@@ -106,7 +106,7 @@ private static final int ERROR_EXIT_CODE = 4;
 ----
 
 Here are some examples based on the following given case.
-There are two HTable called test-01 and test-02, they have two column family cf1 and cf2 respectively, and deployed on the 3 RegionServers.
+There are two Table objects called test-01 and test-02, they have two column family cf1 and cf2 respectively, and deployed on the 3 RegionServers.
 see following table.
 
 [cols="1,1,1", options="header"]
@@ -665,7 +665,7 @@ The LoadTestTool has received many updates in recent HBase releases, including s
 [[ops.regionmgt.majorcompact]]
 === Major Compaction
 
-Major compactions can be requested via the HBase shell or link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HBaseAdmin.html#majorCompact%28java.lang.String%29[HBaseAdmin.majorCompact].
+Major compactions can be requested via the HBase shell or link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Admin.html#majorCompact%28java.lang.String%29[Admin.majorCompact].
 
 Note: major compactions do NOT do region merges.
 See <<compaction,compaction>> for more information about compactions.
@@ -1352,7 +1352,7 @@ A single WAL edit goes through several steps in order to be replicated to a slav
 . The edit is tagged with the master's UUID and added to a buffer.
   When the buffer is filled, or the reader reaches the end of the file, the buffer is sent to a random region server on the slave cluster.
 . The region server reads the edits sequentially and separates them into buffers, one buffer per table.
-  After all edits are read, each buffer is flushed using link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HTable.html[HTable], HBase's normal client.
+  After all edits are read, each buffer is flushed using link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html[Table], HBase's normal client.
   The master's UUID and the UUIDs of slaves which have already consumed the data are preserved in the edits they are applied, in order to prevent replication loops.
 . In the master, the offset for the WAL that is currently being replicated is registered in ZooKeeper.
 
@@ -1994,7 +1994,7 @@ or in code it would be as follows:
 
 [source,java]
 ----
-void rename(HBaseAdmin admin, String oldTableName, String newTableName) {
+void rename(Admin admin, String oldTableName, String newTableName) {
   String snapshotName = randomName();
   admin.disableTable(oldTableName);
   admin.snapshot(snapshotName, oldTableName);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a0f2bc07/src/main/asciidoc/_chapters/performance.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/performance.adoc b/src/main/asciidoc/_chapters/performance.adoc
index 36b3c70..48b647c 100644
--- a/src/main/asciidoc/_chapters/performance.adoc
+++ b/src/main/asciidoc/_chapters/performance.adoc
@@ -439,7 +439,7 @@ When people get started with HBase they have a tendency to write code that looks
 [source,java]
 ----
 Get get = new Get(rowkey);
-Result r = htable.get(get);
+Result r = table.get(get);
 byte[] b = r.getValue(Bytes.toBytes("cf"), Bytes.toBytes("attr"));  // returns current version of value
 ----
 
@@ -452,7 +452,7 @@ public static final byte[] CF = "cf".getBytes();
 public static final byte[] ATTR = "attr".getBytes();
 ...
 Get get = new Get(rowkey);
-Result r = htable.get(get);
+Result r = table.get(get);
 byte[] b = r.getValue(CF, ATTR);  // returns current version of value
 ----
 
@@ -475,7 +475,7 @@ A useful pattern to speed up the bulk import process is to pre-create empty regi
 Be somewhat conservative in this, because too-many regions can actually degrade performance.
 
 There are two different approaches to pre-creating splits.
-The first approach is to rely on the default `HBaseAdmin` strategy (which is implemented in `Bytes.split`)...
+The first approach is to rely on the default `Admin` strategy (which is implemented in `Bytes.split`)...
 
 [source,java]
 ----
@@ -511,12 +511,12 @@ The default value of `hbase.regionserver.optionallogflushinterval` is 1000ms.
 [[perf.hbase.client.autoflush]]
 === HBase Client: AutoFlush
 
-When performing a lot of Puts, make sure that setAutoFlush is set to false on your link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HTable.html[HTable] instance.
+When performing a lot of Puts, make sure that setAutoFlush is set to false on your link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html[Table] instance.
 Otherwise, the Puts will be sent one at a time to the RegionServer.
-Puts added via `htable.add(Put)` and `htable.add( <List> Put)` wind up in the same write buffer.
+Puts added via `table.add(Put)` and `table.add( <List> Put)` wind up in the same write buffer.
 If `autoFlush = false`, these messages are not sent until the write-buffer is filled.
 To explicitly flush the messages, call `flushCommits`.
-Calling `close` on the `HTable` instance will invoke `flushCommits`.
+Calling `close` on the `Table` instance will invoke `flushCommits`.
 
 [[perf.hbase.client.putwal]]
 === HBase Client: Turn off WAL on Puts
@@ -553,7 +553,7 @@ If all your data is being written to one region at a time, then re-read the sect
 
 Also, if you are pre-splitting regions and all your data is _still_ winding up in a single region even though your keys aren't monotonically increasing, confirm that your keyspace actually works with the split strategy.
 There are a variety of reasons that regions may appear "well split" but won't work with your data.
-As the HBase client communicates directly with the RegionServers, this can be obtained via link:hhttp://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HTable.html#getRegionLocation(byte[])[HTable.getRegionLocation].
+As the HBase client communicates directly with the RegionServers, this can be obtained via link:hhttp://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html#getRegionLocation(byte[])[Table.getRegionLocation].
 
 See <<precreate.regions>>, as well as <<perf.configurations>>
 
@@ -622,14 +622,14 @@ Always have ResultScanner processing enclosed in try/catch blocks.
 ----
 Scan scan = new Scan();
 // set attrs...
-ResultScanner rs = htable.getScanner(scan);
+ResultScanner rs = table.getScanner(scan);
 try {
   for (Result r = rs.next(); r != null; r = rs.next()) {
   // process result...
 } finally {
   rs.close();  // always close the ResultScanner!
 }
-htable.close();
+table.close();
 ----
 
 [[perf.hbase.client.blockcache]]
@@ -761,16 +761,16 @@ In this case, special care must be taken to regularly perform major compactions
 As is documented in <<datamodel>>, marking rows as deleted creates additional StoreFiles which then need to be processed on reads.
 Tombstones only get cleaned up with major compactions.
 
-See also <<compaction>> and link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HBaseAdmin.html#majorCompact%28java.lang.String%29[HBaseAdmin.majorCompact].
+See also <<compaction>> and link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Admin.html#majorCompact%28java.lang.String%29[Admin.majorCompact].
 
 [[perf.deleting.rpc]]
 === Delete RPC Behavior
 
-Be aware that `htable.delete(Delete)` doesn't use the writeBuffer.
+Be aware that `Table.delete(Delete)` doesn't use the writeBuffer.
 It will execute an RegionServer RPC with each invocation.
-For a large number of deletes, consider `htable.delete(List)`.
+For a large number of deletes, consider `Table.delete(List)`.
 
-See http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HTable.html#delete%28org.apache.hadoop.hbase.client.Delete%29
+See http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html#delete%28org.apache.hadoop.hbase.client.Delete%29
 
 [[perf.hdfs]]
 == HDFS

http://git-wip-us.apache.org/repos/asf/hbase/blob/a0f2bc07/src/main/asciidoc/_chapters/schema_design.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/schema_design.adoc b/src/main/asciidoc/_chapters/schema_design.adoc
index c930616..28f28a5 100644
--- a/src/main/asciidoc/_chapters/schema_design.adoc
+++ b/src/main/asciidoc/_chapters/schema_design.adoc
@@ -32,7 +32,7 @@ A good general introduction on the strength and weaknesses modelling on the vari
 [[schema.creation]]
 ==  Schema Creation
 
-HBase schemas can be created or updated using the <<shell>> or by using link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HBaseAdmin.html[HBaseAdmin] in the Java API.
+HBase schemas can be created or updated using the <<shell>> or by using link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Admin.html[Admin] in the Java API.
 
 Tables must be disabled when making ColumnFamily modifications, for example:
 
@@ -40,7 +40,7 @@ Tables must be disabled when making ColumnFamily modifications, for example:
 ----
 
 Configuration config = HBaseConfiguration.create();
-HBaseAdmin admin = new HBaseAdmin(conf);
+Admin admin = new Admin(conf);
 String table = "myTable";
 
 admin.disableTable(table);
@@ -308,7 +308,7 @@ This is a fairly common question on the HBase dist-list so it pays to get the ro
 === Relationship Between RowKeys and Region Splits
 
 If you pre-split your table, it is _critical_ to understand how your rowkey will be distributed across the region boundaries.
-As an example of why this is important, consider the example of using displayable hex characters as the lead position of the key (e.g., "0000000000000000" to "ffffffffffffffff"). Running those key ranges through `Bytes.split` (which is the split strategy used when creating regions in `HBaseAdmin.createTable(byte[] startKey, byte[] endKey, numRegions)` for 10 regions will generate the following splits...
+As an example of why this is important, consider the example of using displayable hex characters as the lead position of the key (e.g., "0000000000000000" to "ffffffffffffffff"). Running those key ranges through `Bytes.split` (which is the split strategy used when creating regions in `Admin.createTable(byte[] startKey, byte[] endKey, numRegions)` for 10 regions will generate the following splits...
 
 ----
 
@@ -340,7 +340,7 @@ To conclude this example, the following is an example of how appropriate splits
 
 [source,java]
 ----
-public static boolean createTable(HBaseAdmin admin, HTableDescriptor table, byte[][] splits)
+public static boolean createTable(Admin admin, HTableDescriptor table, byte[][] splits)
 throws IOException {
   try {
     admin.createTable( table, splits );
@@ -400,7 +400,7 @@ Take that into consideration when making your design, as well as block size for
 
 === Counters
 
-One supported datatype that deserves special mention are "counters" (i.e., the ability to do atomic increments of numbers). See link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HTable.html#increment%28org.apache.hadoop.hbase.client.Increment%29[Increment] in HTable.
+One supported datatype that deserves special mention are "counters" (i.e., the ability to do atomic increments of numbers). See link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html#increment%28org.apache.hadoop.hbase.client.Increment%29[Increment] in `Table`.
 
 Synchronization on counters are done on the RegionServer, not in the client.
 
@@ -630,7 +630,7 @@ The rowkey of LOG_TYPES would be:
 * [type] (e.g., byte indicating hostname vs. event-type)
 * [bytes] variable length bytes for raw hostname or event-type.
 
-A column for this rowkey could be a long with an assigned number, which could be obtained by using an link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HTable.html#incrementColumnValue%28byte[],%20byte[],%20byte[],%20long%29[HBase counter].
+A column for this rowkey could be a long with an assigned number, which could be obtained by using an link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html#incrementColumnValue%28byte[],%20byte[],%20byte[],%20long%29[HBase counter].
 
 So the resulting composite rowkey would be:
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a0f2bc07/src/main/asciidoc/_chapters/security.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/security.adoc b/src/main/asciidoc/_chapters/security.adoc
index 21698fa..9cffbdb 100644
--- a/src/main/asciidoc/_chapters/security.adoc
+++ b/src/main/asciidoc/_chapters/security.adoc
@@ -131,14 +131,19 @@ To do so, add the following to the `hbase-site.xml` file on every client:
 </property>
 ----
 
-This configuration property can also be set on a per connection basis.
-Set it in the `Configuration` supplied to `HTable`:
+This configuration property can also be set on a per-connection basis.
+Set it in the `Configuration` supplied to `Table`:
 
 [source,java]
 ----
 Configuration conf = HBaseConfiguration.create();
+Connection connection = ConnectionFactory.createConnection(conf);
 conf.set("hbase.rpc.protection", "privacy");
-HTable table = new HTable(conf, tablename);
+try (Connection connection = ConnectionFactory.createConnection(conf)) {
+  try (Table table = connection.getTable(TableName.valueOf(tablename)) {
+  .... do your stuff
+  }
+}
 ----
 
 Expect a ~10% performance penalty for encrypted communication.
@@ -881,18 +886,24 @@ public static void grantOnTable(final HBaseTestingUtility util, final String use
   SecureTestUtil.updateACLs(util, new Callable<Void>() {
     @Override
     public Void call() throws Exception {
-      HTable acl = new HTable(util.getConfiguration(), AccessControlLists.ACL_TABLE_NAME);
-      try {
-        BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW);
-        AccessControlService.BlockingInterface protocol =
-            AccessControlService.newBlockingStub(service);
-        ProtobufUtil.grant(protocol, user, table, family, qualifier, actions);
-      } finally {
-        acl.close();
+      Configuration conf = HBaseConfiguration.create();
+      Connection connection = ConnectionFactory.createConnection(conf);
+      try (Connection connection = ConnectionFactory.createConnection(conf)) {
+        try (Table table = connection.getTable(TableName.valueOf(tablename)) {
+          AccessControlLists.ACL_TABLE_NAME);
+          try {
+            BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW);
+            AccessControlService.BlockingInterface protocol =
+                AccessControlService.newBlockingStub(service);
+            ProtobufUtil.grant(protocol, user, table, family, qualifier, actions);
+          } finally {
+            acl.close();
+          }
+          return null;
+        }
       }
-      return null;
     }
-  });
+  }
 }
 ----
 
@@ -931,7 +942,9 @@ public static void revokeFromTable(final HBaseTestingUtility util, final String
   SecureTestUtil.updateACLs(util, new Callable<Void>() {
     @Override
     public Void call() throws Exception {
-      HTable acl = new HTable(util.getConfiguration(), AccessControlLists.ACL_TABLE_NAME);
+      Configuration conf = HBaseConfiguration.create();
+      Connection connection = ConnectionFactory.createConnection(conf);
+      Table acl = connection.getTable(util.getConfiguration(), AccessControlLists.ACL_TABLE_NAME);
       try {
         BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW);
         AccessControlService.BlockingInterface protocol =
@@ -1215,9 +1228,11 @@ The correct way to apply cell level labels is to do so in the application code w
 ====
 [source,java]
 ----
-static HTable createTableAndWriteDataWithLabels(TableName tableName, String... labelExps)
+static Table createTableAndWriteDataWithLabels(TableName tableName, String... labelExps)
     throws Exception {
-  HTable table = null;
+  Configuration conf = HBaseConfiguration.create();
+  Connection connection = ConnectionFactory.createConnection(conf);
+  Table table = NULL;
   try {
     table = TEST_UTIL.createTable(tableName, fam);
     int i = 1;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a0f2bc07/src/main/asciidoc/_chapters/tracing.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/tracing.adoc b/src/main/asciidoc/_chapters/tracing.adoc
index 9a4a811..6bb8065 100644
--- a/src/main/asciidoc/_chapters/tracing.adoc
+++ b/src/main/asciidoc/_chapters/tracing.adoc
@@ -124,8 +124,9 @@ For example, if you wanted to trace all of your get operations, you change this:
 
 [source,java]
 ----
-
-HTable table = new HTable(conf, "t1");
+Configuration config = HBaseConfiguration.create();
+Connection connection = ConnectionFactory.createConnection(config);
+Table table = connection.getTable(TableName.valueOf("t1"));
 Get get = new Get(Bytes.toBytes("r1"));
 Result res = table.get(get);
 ----
@@ -137,7 +138,7 @@ into:
 
 TraceScope ts = Trace.startSpan("Gets", Sampler.ALWAYS);
 try {
-  HTable table = new HTable(conf, "t1");
+  Table table = connection.getTable(TableName.valueOf("t1"));
   Get get = new Get(Bytes.toBytes("r1"));
   Result res = table.get(get);
 } finally {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a0f2bc07/src/main/asciidoc/_chapters/troubleshooting.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/troubleshooting.adoc b/src/main/asciidoc/_chapters/troubleshooting.adoc
index 6d35f1d..1776c9e 100644
--- a/src/main/asciidoc/_chapters/troubleshooting.adoc
+++ b/src/main/asciidoc/_chapters/troubleshooting.adoc
@@ -627,6 +627,7 @@ This issue is caused by bugs in the MIT Kerberos replay_cache component, link:ht
 These bugs caused the old version of krb5-server to erroneously block subsequent requests sent from a Principal.
 This caused krb5-server to block the connections sent from one Client (one HTable instance with multi-threading connection instances for each RegionServer); Messages, such as `Request is a replay (34)`, are logged in the client log You can ignore the messages, because HTable will retry 5 * 10 (50) times for each failed connection by default.
 HTable will throw IOException if any connection to the RegionServer fails after the retries, so that the user client code for HTable instance can handle it further.
+NOTE: `HTable` is deprecated in HBase 1.0, in favor of `Table`.
 
 Alternatively, update krb5-server to a version which solves these issues, such as krb5-server-1.10.3.
 See JIRA link:https://issues.apache.org/jira/browse/HBASE-10379[HBASE-10379] for more details.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a0f2bc07/src/main/asciidoc/_chapters/unit_testing.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/unit_testing.adoc b/src/main/asciidoc/_chapters/unit_testing.adoc
index 1ffedf1..3f70001 100644
--- a/src/main/asciidoc/_chapters/unit_testing.adoc
+++ b/src/main/asciidoc/_chapters/unit_testing.adoc
@@ -42,7 +42,7 @@ This example will add unit tests to the following example class:
 
 public class MyHBaseDAO {
 
-    public static void insertRecord(HTableInterface table, HBaseTestObj obj)
+    public static void insertRecord(Table.getTable(table), HBaseTestObj obj)
     throws Exception {
         Put put = createPut(obj);
         table.put(put);
@@ -129,17 +129,19 @@ Next, add a `@RunWith` annotation to your test class, to direct it to use Mockit
 
 @RunWith(MockitoJUnitRunner.class)
 public class TestMyHBaseDAO{
-  @Mock 
-  private HTableInterface table;
   @Mock
-  private HTablePool hTablePool;
+  Configuration config = HBaseConfiguration.create();
+  @Mock
+  Connection connection = ConnectionFactory.createConnection(config);
+  @Mock 
+  private Table table;
   @Captor
   private ArgumentCaptor putCaptor;
 
   @Test
   public void testInsertRecord() throws Exception {
     //return mock table when getTable is called
-    when(hTablePool.getTable("tablename")).thenReturn(table);
+    when(connection.getTable(TableName.valueOf("tablename")).thenReturn(table);
     //create test object and make a call to the DAO that needs testing
     HBaseTestObj obj = new HBaseTestObj();
     obj.setRowKey("ROWKEY-1");
@@ -162,7 +164,7 @@ This code populates `HBaseTestObj` with ``ROWKEY-1'', ``DATA-1'', ``DATA-2'' as
 It then inserts the record into the mocked table.
 The Put that the DAO would have inserted is captured, and values are tested to verify that they are what you expected them to be.
 
-The key here is to manage htable pool and htable instance creation outside the DAO.
+The key here is to manage Connection and Table instance creation outside the DAO.
 This allows you to mock them cleanly and test Puts as shown above.
 Similarly, you can now expand into other operations such as Get, Scan, or Delete.
 


[25/50] [abbrv] hbase git commit: HBASE-12747 IntegrationTestMTTR will OOME if launched with mvn verify (Abhishek Singh Chouhan)

Posted by jm...@apache.org.
HBASE-12747 IntegrationTestMTTR will OOME if launched with mvn verify (Abhishek Singh Chouhan)


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

Branch: refs/heads/hbase-11339
Commit: 200ec5b191262ac356639b0390d7d72ab93feef3
Parents: f97c00f
Author: Andrew Purtell <ap...@apache.org>
Authored: Mon Feb 9 18:54:43 2015 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Mon Feb 9 18:54:43 2015 -0800

----------------------------------------------------------------------
 hbase-it/pom.xml | 4 +++-
 pom.xml          | 7 +++++--
 2 files changed, 8 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/200ec5b1/hbase-it/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-it/pom.xml b/hbase-it/pom.xml
index a527e77..4522f9c 100644
--- a/hbase-it/pom.xml
+++ b/hbase-it/pom.xml
@@ -35,6 +35,8 @@
     <!-- Test inclusion patterns used by failsafe configuration -->
     <unittest.include>**/Test*.java</unittest.include>
     <integrationtest.include>**/IntegrationTest*.java</integrationtest.include>
+    <!-- To Run Tests with a particular Xmx Value use -Dfailsafe.Xmx=XXXg -->
+    <failsafe.Xmx>3g</failsafe.Xmx>
     <!-- To run a single integration test, use -Dit.test=IntegrationTestXXX -->
   </properties>
 
@@ -146,7 +148,7 @@
             <!-- TODO: failsafe does timeout, but verify does not fail the build because of the timeout.
                  I believe it is a failsafe bug, we may consider using surefire -->
             <forkedProcessTimeoutInSeconds>1800</forkedProcessTimeoutInSeconds>
-            <argLine>-enableassertions -Xmx1900m
+            <argLine>-enableassertions -Xmx${failsafe.Xmx}
               -Djava.security.egd=file:/dev/./urandom</argLine>
             <testFailureIgnore>false</testFailureIgnore>
           </configuration>

http://git-wip-us.apache.org/repos/asf/hbase/blob/200ec5b1/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 975f1f8..132215d 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1086,11 +1086,14 @@
     <test.output.tofile>true</test.output.tofile>
     <surefire.timeout>900</surefire.timeout>
     <test.exclude.pattern></test.exclude.pattern>
-    <hbase-surefire.argLine>-enableassertions -XX:MaxDirectMemorySize=1G -Xmx1900m
+    <!-- default Xmx value is 1900m. Use -Dsurefire.Xmx=xxg to run tests with different JVM Xmx value -->
+    <surefire.Xmx>1900m</surefire.Xmx>
+    <surefire.cygwinXmx>1900m</surefire.cygwinXmx>
+    <hbase-surefire.argLine>-enableassertions -XX:MaxDirectMemorySize=1G -Xmx${surefire.Xmx}
       -XX:MaxPermSize=256m -Djava.security.egd=file:/dev/./urandom -Djava.net.preferIPv4Stack=true
       -Djava.awt.headless=true
     </hbase-surefire.argLine>
-    <hbase-surefire.cygwin-argline>-enableassertions -Xmx1900m -XX:MaxPermSize=256m
+    <hbase-surefire.cygwin-argline>-enableassertions -Xmx${surefire.cygwinXmx} -XX:MaxPermSize=256m
       -Djava.security.egd=file:/dev/./urandom -Djava.net.preferIPv4Stack=true
       "-Djava.library.path=${hadoop.library.path};${java.library.path}"
     </hbase-surefire.cygwin-argline>


[50/50] [abbrv] hbase git commit: Merge branch 'master' (2/11/15) into hbase-11339

Posted by jm...@apache.org.
Merge branch 'master' (2/11/15) into hbase-11339

Conflicts:
	hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
	hbase-client/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptor.java
	hbase-common/src/main/java/org/apache/hadoop/hbase/TagType.java
	hbase-common/src/main/resources/hbase-default.xml
	hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java
	hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
	hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestAcidGuarantees.java
	hbase-server/src/main/java/org/apache/hadoop/hbase/io/HFileLink.java
	hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java
	hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/DisabledTableSnapshotHandler.java
	hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreEngine.java
	hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
	hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
	hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
	hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
	hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java
	hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/StripeCompactor.java
	hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
	hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java
	hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotReferenceUtil.java
	hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java
	pom.xml
	src/main/docbkx/book.xml
	src/main/docbkx/ops_mgt.xml
	src/main/docbkx/schema_design.xml


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

Branch: refs/heads/hbase-11339
Commit: fe335b683c79e87278962f7ce911a792d7775581
Parents: 8f5dae4 3dd220f
Author: Jonathan M Hsieh <jm...@apache.org>
Authored: Sun Feb 22 12:54:45 2015 -0800
Committer: Jonathan M Hsieh <jm...@apache.org>
Committed: Sun Feb 22 12:54:45 2015 -0800

----------------------------------------------------------------------
 .arcconfig                                      |    9 +-
 README.txt                                      |   12 +-
 bin/considerAsDead.sh                           |   63 +
 bin/graceful_stop.sh                            |   46 +-
 bin/hbase                                       |   50 +-
 bin/hbase-daemon.sh                             |   92 +-
 bin/hbase.cmd                                   |   31 +-
 bin/region_mover.rb                             |  175 +-
 bin/region_status.rb                            |    4 +-
 bin/regionservers.sh                            |   29 +-
 bin/shutdown_regionserver.rb                    |   52 +
 conf/hbase-env.cmd                              |    6 +
 conf/hbase-env.sh                               |   10 +-
 conf/log4j.properties                           |    1 +
 conf/regionservers                              |    1 +
 dev-support/check_compatibility.sh              |  269 +
 dev-support/checkstyle_report.py                |   63 +
 dev-support/findHangingTest.sh                  |   40 -
 dev-support/findHangingTests.py                 |   54 +
 dev-support/hbase_docker.sh                     |  162 +
 dev-support/hbase_docker/Dockerfile             |   50 +
 dev-support/hbase_docker/README.md              |   39 +
 dev-support/jdiffHBasePublicAPI.sh              |   31 +-
 dev-support/jdiffHBasePublicAPI_common.sh       |    0
 dev-support/jenkinsEnv.sh                       |    0
 dev-support/make_patch.sh                       |  142 +
 dev-support/publish_hbase_website.sh            |  239 +
 dev-support/rebase_all_git_branches.sh          |  202 +
 dev-support/test-patch.properties               |    6 +-
 dev-support/test-patch.sh                       |  191 +-
 hbase-annotations/pom.xml                       |   43 +
 hbase-annotations/src/main/asciidoc/.gitignore  |    0
 .../hbase/classification/InterfaceAudience.java |   73 +
 .../classification/InterfaceStability.java      |   66 +
 ...ExcludePrivateAnnotationsStandardDoclet.java |   62 +
 .../IncludePublicAnnotationsStandardDoclet.java |   68 +
 .../classification/tools/RootDocProcessor.java  |  230 +
 .../classification/tools/StabilityOptions.java  |   70 +
 .../hbase/testclassification/ClientTests.java   |   41 +
 .../testclassification/CoprocessorTests.java    |   41 +
 .../hbase/testclassification/FilterTests.java   |   41 +
 .../hbase/testclassification/FlakeyTests.java   |   40 +
 .../hbase/testclassification/IOTests.java       |   41 +
 .../testclassification/IntegrationTests.java    |   39 +
 .../hbase/testclassification/LargeTests.java    |   38 +
 .../testclassification/MapReduceTests.java      |   40 +
 .../hbase/testclassification/MasterTests.java   |   40 +
 .../hbase/testclassification/MediumTests.java   |   37 +
 .../hbase/testclassification/MiscTests.java     |   40 +
 .../hbase/testclassification/RPCTests.java      |   40 +
 .../testclassification/RegionServerTests.java   |   41 +
 .../testclassification/ReplicationTests.java    |   40 +
 .../hbase/testclassification/RestTests.java     |   41 +
 .../hbase/testclassification/SecurityTests.java |   42 +
 .../hbase/testclassification/SmallTests.java    |   34 +
 .../VerySlowMapReduceTests.java                 |   42 +
 .../VerySlowRegionServerTests.java              |   42 +
 hbase-assembly/pom.xml                          |   10 +
 hbase-assembly/src/main/asciidoc/.gitignore     |    0
 hbase-assembly/src/main/assembly/components.xml |   14 +
 hbase-checkstyle/pom.xml                        |   73 +
 hbase-checkstyle/src/main/asciidoc/.gitignore   |    0
 .../resources/hbase/checkstyle-suppressions.xml |   36 +
 .../src/main/resources/hbase/checkstyle.xml     |   65 +
 hbase-client/pom.xml                            |  126 +-
 hbase-client/src/main/asciidoc/.gitignore       |    0
 .../java/org/apache/hadoop/hbase/Abortable.java |   10 +-
 .../java/org/apache/hadoop/hbase/Chore.java     |  142 -
 .../hadoop/hbase/ClockOutOfSyncException.java   |    4 +-
 .../java/org/apache/hadoop/hbase/ClusterId.java |    7 +-
 .../org/apache/hadoop/hbase/ClusterStatus.java  |   80 +-
 .../hadoop/hbase/CoordinatedStateException.java |    2 +-
 .../org/apache/hadoop/hbase/Coprocessor.java    |    2 +-
 .../hadoop/hbase/CoprocessorEnvironment.java    |    2 +-
 .../hadoop/hbase/DoNotRetryIOException.java     |    7 +-
 .../hadoop/hbase/DroppedSnapshotException.java  |    8 +-
 .../apache/hadoop/hbase/HColumnDescriptor.java  |  272 +-
 .../org/apache/hadoop/hbase/HRegionInfo.java    |  162 +-
 .../apache/hadoop/hbase/HRegionLocation.java    |    7 +-
 .../apache/hadoop/hbase/HTableDescriptor.java   |  355 +-
 .../hbase/InvalidFamilyOperationException.java  |    6 +-
 .../apache/hadoop/hbase/KeepDeletedCells.java   |   50 +
 .../hadoop/hbase/MasterNotRunningException.java |    6 +-
 .../apache/hadoop/hbase/MetaTableAccessor.java  |  773 +-
 .../hadoop/hbase/NamespaceExistException.java   |    4 +-
 .../hbase/NamespaceNotFoundException.java       |    4 +-
 .../hbase/NotAllMetaRegionsOnlineException.java |    4 +-
 .../hadoop/hbase/NotServingRegionException.java |    8 +-
 .../hadoop/hbase/PleaseHoldException.java       |    5 +-
 .../apache/hadoop/hbase/RegionException.java    |    5 +-
 .../org/apache/hadoop/hbase/RegionLoad.java     |   27 +-
 .../apache/hadoop/hbase/RegionLocations.java    |    2 +-
 .../hadoop/hbase/RegionTooBusyException.java    |    4 +-
 .../org/apache/hadoop/hbase/ServerLoad.java     |   25 +-
 .../org/apache/hadoop/hbase/ServerName.java     |  402 -
 .../java/org/apache/hadoop/hbase/Stoppable.java |   38 -
 .../hadoop/hbase/TableExistsException.java      |    5 +-
 .../hadoop/hbase/TableInfoMissingException.java |    4 +-
 .../hadoop/hbase/TableNotDisabledException.java |    5 +-
 .../hadoop/hbase/TableNotEnabledException.java  |    5 +-
 .../hadoop/hbase/TableNotFoundException.java    |    5 +-
 .../hadoop/hbase/UnknownRegionException.java    |    7 +-
 .../hadoop/hbase/UnknownScannerException.java   |    4 +-
 .../hadoop/hbase/YouAreDeadException.java       |    8 +-
 .../hbase/ZooKeeperConnectionException.java     |    6 +-
 .../hbase/client/AbstractClientScanner.java     |    2 +-
 .../org/apache/hadoop/hbase/client/Action.java  |    5 +-
 .../org/apache/hadoop/hbase/client/Admin.java   |  341 +-
 .../org/apache/hadoop/hbase/client/Append.java  |   63 +-
 .../hadoop/hbase/client/AsyncProcess.java       |  263 +-
 .../apache/hadoop/hbase/client/Attributes.java  |    8 +-
 .../hadoop/hbase/client/BufferedMutator.java    |  129 +
 .../hbase/client/BufferedMutatorImpl.java       |  258 +
 .../hbase/client/BufferedMutatorParams.java     |  110 +
 .../apache/hadoop/hbase/client/Cancellable.java |   31 +
 .../hadoop/hbase/client/ClientIdGenerator.java  |   11 +-
 .../hadoop/hbase/client/ClientScanner.java      |   91 +-
 .../client/ClientSmallReversedScanner.java      |   19 +-
 .../hadoop/hbase/client/ClientSmallScanner.java |   16 +-
 .../hadoop/hbase/client/ClusterConnection.java  |   44 +-
 .../hbase/client/ClusterStatusListener.java     |   25 +-
 .../apache/hadoop/hbase/client/Connection.java  |  171 +
 .../hadoop/hbase/client/ConnectionAdapter.java  |   60 +-
 .../hadoop/hbase/client/ConnectionCallable.java |   56 +
 .../hadoop/hbase/client/ConnectionFactory.java  |  243 +
 .../hadoop/hbase/client/ConnectionManager.java  |  549 +-
 .../hadoop/hbase/client/ConnectionUtils.java    |   46 +-
 .../apache/hadoop/hbase/client/Consistency.java |    4 +-
 .../hadoop/hbase/client/DelayingRunner.java     |  116 +
 .../client/DelegatingRetryingCallable.java      |    5 +-
 .../org/apache/hadoop/hbase/client/Delete.java  |  165 +-
 .../hbase/client/DoNotRetryRegionException.java |   42 +
 .../apache/hadoop/hbase/client/Durability.java  |    4 +-
 .../apache/hadoop/hbase/client/FailureInfo.java |   60 +
 .../client/FastFailInterceptorContext.java      |  123 +
 .../org/apache/hadoop/hbase/client/Get.java     |   56 +-
 .../apache/hadoop/hbase/client/HBaseAdmin.java  |  975 +-
 .../hadoop/hbase/client/HConnectable.java       |    6 +-
 .../apache/hadoop/hbase/client/HConnection.java |  119 +-
 .../hadoop/hbase/client/HConnectionKey.java     |    7 +-
 .../hadoop/hbase/client/HConnectionManager.java |   70 +-
 .../hadoop/hbase/client/HRegionLocator.java     |  148 +
 .../org/apache/hadoop/hbase/client/HTable.java  |  564 +-
 .../hadoop/hbase/client/HTableFactory.java      |   11 +-
 .../hadoop/hbase/client/HTableInterface.java    |   69 +-
 .../hbase/client/HTableInterfaceFactory.java    |   10 +-
 .../hadoop/hbase/client/HTableMultiplexer.java  |  513 +-
 .../apache/hadoop/hbase/client/HTableUtil.java  |  137 -
 .../apache/hadoop/hbase/client/Increment.java   |   81 +-
 .../hadoop/hbase/client/IsolationLevel.java     |    4 +-
 .../hadoop/hbase/client/MasterCallable.java     |   60 +
 .../hbase/client/MasterKeepAliveConnection.java |    2 +-
 .../apache/hadoop/hbase/client/MetaCache.java   |   10 +-
 .../apache/hadoop/hbase/client/MetaScanner.java |  160 +-
 .../apache/hadoop/hbase/client/MultiAction.java |   24 +-
 .../hadoop/hbase/client/MultiResponse.java      |    8 +-
 .../hbase/client/MultiServerCallable.java       |   20 +-
 .../apache/hadoop/hbase/client/Mutation.java    |  138 +-
 .../client/NoOpRetryableCallerInterceptor.java  |   68 +
 .../client/NoOpRetryingInterceptorContext.java  |   44 +
 .../client/NoServerForRegionException.java      |    7 +-
 .../hadoop/hbase/client/NonceGenerator.java     |    2 +-
 .../apache/hadoop/hbase/client/Operation.java   |    4 +-
 .../hbase/client/OperationWithAttributes.java   |   21 +-
 .../client/PerClientRandomNonceGenerator.java   |    2 +-
 .../client/PreemptiveFastFailInterceptor.java   |  405 +
 .../org/apache/hadoop/hbase/client/Put.java     |   53 +-
 .../org/apache/hadoop/hbase/client/Query.java   |   49 +-
 .../client/RegionAdminServiceCallable.java      |    4 +
 .../client/RegionCoprocessorServiceExec.java    |    7 +-
 .../hadoop/hbase/client/RegionLocator.java      |   24 +-
 .../hbase/client/RegionOfflineException.java    |    4 +-
 .../hadoop/hbase/client/RegionReplicaUtil.java  |   12 +-
 .../hbase/client/RegionServerCallable.java      |   14 +-
 .../apache/hadoop/hbase/client/Registry.java    |   14 +-
 .../hadoop/hbase/client/RegistryFactory.java    |   49 +
 .../org/apache/hadoop/hbase/client/Result.java  |  134 +-
 .../client/ResultBoundedCompletionService.java  |  165 +
 .../hadoop/hbase/client/ResultScanner.java      |    9 +-
 .../hadoop/hbase/client/ResultStatsUtil.java    |   76 +
 .../hbase/client/RetriesExhaustedException.java |   11 +-
 .../RetriesExhaustedWithDetailsException.java   |   12 +-
 .../hadoop/hbase/client/RetryingCallable.java   |    5 +-
 .../hbase/client/RetryingCallerInterceptor.java |   98 +
 .../RetryingCallerInterceptorContext.java       |   69 +
 .../RetryingCallerInterceptorFactory.java       |   80 +
 .../hbase/client/ReversedClientScanner.java     |    7 +-
 .../hbase/client/ReversedScannerCallable.java   |   19 +-
 .../org/apache/hadoop/hbase/client/Row.java     |    4 +-
 .../hadoop/hbase/client/RowMutations.java       |    6 +-
 .../hadoop/hbase/client/RowTooBigException.java |   37 +
 .../hadoop/hbase/client/RpcRetryingCaller.java  |  198 +-
 .../hbase/client/RpcRetryingCallerFactory.java  |   57 +-
 .../hbase/client/RpcRetryingCallerImpl.java     |  238 +
 .../RpcRetryingCallerWithReadReplicas.java      |  181 +-
 .../org/apache/hadoop/hbase/client/Scan.java    |  210 +-
 .../hadoop/hbase/client/ScannerCallable.java    |   24 +-
 .../client/ScannerCallableWithReplicas.java     |  137 +-
 .../hbase/client/ScannerTimeoutException.java   |    4 +-
 .../hbase/client/ServerStatisticTracker.java    |   74 +
 .../client/StatsTrackingRpcRetryingCaller.java  |   77 +
 .../org/apache/hadoop/hbase/client/Table.java   |   80 +-
 .../hadoop/hbase/client/TableConfiguration.java |  122 +
 .../apache/hadoop/hbase/client/TableState.java  |  220 +
 .../hbase/client/UnmodifyableHRegionInfo.java   |    4 +-
 .../client/UnmodifyableHTableDescriptor.java    |   31 +-
 .../hbase/client/WrongRowIOException.java       |    6 +-
 .../client/ZooKeeperKeepAliveConnection.java    |    4 +-
 .../hadoop/hbase/client/ZooKeeperRegistry.java  |   56 +-
 .../client/backoff/ClientBackoffPolicy.java     |   42 +
 .../backoff/ClientBackoffPolicyFactory.java     |   59 +
 .../backoff/ExponentialClientBackoffPolicy.java |  104 +
 .../hbase/client/backoff/ServerStatistics.java  |   74 +
 .../client/coprocessor/AggregationClient.java   |  137 +-
 .../hadoop/hbase/client/coprocessor/Batch.java  |   10 +-
 .../BigDecimalColumnInterpreter.java            |    4 +-
 .../coprocessor/DoubleColumnInterpreter.java    |    7 +-
 .../coprocessor/LongColumnInterpreter.java      |    2 +-
 .../coprocessor/SecureBulkLoadClient.java       |  117 +-
 .../hbase/client/coprocessor/package-info.java  |   22 +-
 .../hbase/client/metrics/ScanMetrics.java       |    2 +-
 .../hadoop/hbase/client/package-info.java       |  207 +-
 .../client/replication/ReplicationAdmin.java    |  200 +-
 .../hbase/coprocessor/ColumnInterpreter.java    |   13 +-
 .../hbase/coprocessor/CoprocessorException.java |    4 +-
 .../exceptions/ConnectionClosingException.java  |   59 +
 .../exceptions/DeserializationException.java    |   43 -
 .../exceptions/FailedSanityCheckException.java  |    5 +
 .../hadoop/hbase/exceptions/HBaseException.java |   44 -
 .../hbase/exceptions/LockTimeoutException.java  |    9 +-
 .../hbase/exceptions/MergeRegionException.java  |   10 +-
 .../exceptions/OperationConflictException.java  |    4 +-
 .../OutOfOrderScannerNextException.java         |    2 +-
 .../exceptions/PreemptiveFastFailException.java |   70 +
 .../exceptions/RegionInRecoveryException.java   |    4 +-
 .../hbase/exceptions/RegionMovedException.java  |    4 +-
 .../exceptions/RegionOpeningException.java      |    7 +-
 .../exceptions/UnknownProtocolException.java    |    7 +-
 .../apache/hadoop/hbase/executor/EventType.java |    2 +-
 .../hadoop/hbase/executor/ExecutorType.java     |    4 +-
 .../hadoop/hbase/filter/BinaryComparator.java   |    7 +-
 .../hbase/filter/BinaryPrefixComparator.java    |    7 +-
 .../hadoop/hbase/filter/BitComparator.java      |    7 +-
 .../hbase/filter/ByteArrayComparable.java       |    6 +-
 .../hbase/filter/ColumnCountGetFilter.java      |    4 +-
 .../hbase/filter/ColumnPaginationFilter.java    |    4 +-
 .../hadoop/hbase/filter/ColumnPrefixFilter.java |    4 +-
 .../hadoop/hbase/filter/ColumnRangeFilter.java  |    4 +-
 .../hadoop/hbase/filter/CompareFilter.java      |    9 +-
 .../hbase/filter/DependentColumnFilter.java     |   18 +-
 .../hadoop/hbase/filter/FamilyFilter.java       |    4 +-
 .../org/apache/hadoop/hbase/filter/Filter.java  |   26 +-
 .../apache/hadoop/hbase/filter/FilterBase.java  |   28 +-
 .../apache/hadoop/hbase/filter/FilterList.java  |   91 +-
 .../hadoop/hbase/filter/FilterWrapper.java      |   35 +-
 .../hadoop/hbase/filter/FirstKeyOnlyFilter.java |    4 +-
 .../FirstKeyValueMatchingQualifiersFilter.java  |   10 +-
 .../hadoop/hbase/filter/FuzzyRowFilter.java     |  156 +-
 .../hbase/filter/InclusiveStopFilter.java       |    6 +-
 .../filter/IncompatibleFilterException.java     |    4 +-
 .../hbase/filter/InvalidRowFilterException.java |    4 +-
 .../hadoop/hbase/filter/KeyOnlyFilter.java      |   30 +-
 .../hadoop/hbase/filter/LongComparator.java     |   87 +
 .../hbase/filter/MultiRowRangeFilter.java       |  508 +
 .../filter/MultipleColumnPrefixFilter.java      |    4 +-
 .../hadoop/hbase/filter/NullComparator.java     |    4 +-
 .../apache/hadoop/hbase/filter/PageFilter.java  |   14 +-
 .../hadoop/hbase/filter/ParseConstants.java     |    6 +-
 .../apache/hadoop/hbase/filter/ParseFilter.java |   16 +-
 .../hadoop/hbase/filter/PrefixFilter.java       |   12 +-
 .../hadoop/hbase/filter/QualifierFilter.java    |    8 +-
 .../hadoop/hbase/filter/RandomRowFilter.java    |    4 +-
 .../hbase/filter/RegexStringComparator.java     |  318 +-
 .../apache/hadoop/hbase/filter/RowFilter.java   |    8 +-
 .../filter/SingleColumnValueExcludeFilter.java  |    4 +-
 .../hbase/filter/SingleColumnValueFilter.java   |   10 +-
 .../apache/hadoop/hbase/filter/SkipFilter.java  |    6 +-
 .../hbase/filter/SubstringComparator.java       |    7 +-
 .../hadoop/hbase/filter/TimestampsFilter.java   |   15 +-
 .../apache/hadoop/hbase/filter/ValueFilter.java |   11 +-
 .../hadoop/hbase/filter/WhileMatchFilter.java   |    4 +-
 .../hadoop/hbase/ipc/AbstractRpcClient.java     |  290 +
 .../org/apache/hadoop/hbase/ipc/AsyncCall.java  |  135 +
 .../hadoop/hbase/ipc/AsyncRpcChannel.java       |  777 ++
 .../apache/hadoop/hbase/ipc/AsyncRpcClient.java |  402 +
 .../hbase/ipc/AsyncServerResponseHandler.java   |  130 +
 .../hadoop/hbase/ipc/BadAuthException.java      |    6 +-
 .../hadoop/hbase/ipc/BlockingRpcCallback.java   |    4 +-
 .../java/org/apache/hadoop/hbase/ipc/Call.java  |  127 +
 .../hadoop/hbase/ipc/CallTimeoutException.java  |   35 +
 .../hbase/ipc/CallerDisconnectedException.java  |    4 +-
 .../apache/hadoop/hbase/ipc/ConnectionId.java   |   81 +
 .../hadoop/hbase/ipc/CoprocessorRpcChannel.java |   17 +-
 .../DelegatingPayloadCarryingRpcController.java |    4 +-
 .../hadoop/hbase/ipc/FailedServerException.java |   37 +
 .../apache/hadoop/hbase/ipc/FailedServers.java  |   79 +
 .../hbase/ipc/FatalConnectionException.java     |    6 +-
 .../org/apache/hadoop/hbase/ipc/IPCUtil.java    |   26 +-
 .../hbase/ipc/MasterCoprocessorRpcChannel.java  |    4 +-
 .../hbase/ipc/PayloadCarryingRpcController.java |   15 +-
 .../hbase/ipc/RegionCoprocessorRpcChannel.java  |    6 +-
 .../ipc/RegionServerCoprocessorRpcChannel.java  |   74 +
 .../hbase/ipc/RemoteWithExtrasException.java    |   10 +-
 .../org/apache/hadoop/hbase/ipc/RpcClient.java  | 1738 +---
 .../hadoop/hbase/ipc/RpcClientFactory.java      |   69 +
 .../apache/hadoop/hbase/ipc/RpcClientImpl.java  | 1275 +++
 .../hadoop/hbase/ipc/RpcControllerFactory.java  |    8 +-
 .../hbase/ipc/ServerNotRunningYetException.java |    6 +-
 .../hadoop/hbase/ipc/ServerRpcController.java   |   23 +-
 .../hbase/ipc/StoppedRpcClientException.java    |    4 +-
 .../hbase/ipc/TimeLimitedRpcController.java     |   75 +-
 .../ipc/UnsupportedCellCodecException.java      |    6 +-
 .../UnsupportedCompressionCodecException.java   |    6 +-
 .../hadoop/hbase/ipc/WrongVersionException.java |    6 +-
 .../apache/hadoop/hbase/master/RegionState.java |   82 +-
 .../hadoop/hbase/protobuf/ProtobufUtil.java     |  354 +-
 .../hadoop/hbase/protobuf/RequestConverter.java |  105 +-
 .../hbase/protobuf/ResponseConverter.java       |   37 +-
 .../quotas/InvalidQuotaSettingsException.java   |   32 +
 .../hbase/quotas/QuotaExceededException.java    |   34 +
 .../apache/hadoop/hbase/quotas/QuotaFilter.java |  110 +
 .../hadoop/hbase/quotas/QuotaRetriever.java     |  185 +
 .../apache/hadoop/hbase/quotas/QuotaScope.java  |   43 +
 .../hadoop/hbase/quotas/QuotaSettings.java      |  124 +
 .../hbase/quotas/QuotaSettingsFactory.java      |  267 +
 .../hadoop/hbase/quotas/QuotaTableUtil.java     |  412 +
 .../apache/hadoop/hbase/quotas/QuotaType.java   |   31 +
 .../hadoop/hbase/quotas/ThrottleSettings.java   |  106 +
 .../hadoop/hbase/quotas/ThrottleType.java       |   34 +
 .../hbase/quotas/ThrottlingException.java       |  170 +
 .../hadoop/hbase/regionserver/BloomType.java    |    6 +-
 .../hbase/regionserver/LeaseException.java      |    4 +-
 .../NoSuchColumnFamilyException.java            |    4 +-
 .../RegionAlreadyInTransitionException.java     |   39 -
 .../hbase/regionserver/RegionOpeningState.java  |    2 +-
 .../RegionServerRunningException.java           |    4 +-
 .../RegionServerStoppedException.java           |    6 +-
 .../regionserver/WrongRegionException.java      |    6 +-
 .../wal/FailedLogCloseException.java            |    8 +-
 .../wal/FailedSyncBeforeLogCloseException.java  |    6 +-
 .../hbase/replication/ReplicationException.java |    4 +-
 .../hbase/replication/ReplicationFactory.java   |    4 +-
 .../hbase/replication/ReplicationListener.java  |    4 +-
 .../hbase/replication/ReplicationPeer.java      |    5 +-
 .../replication/ReplicationPeerConfig.java      |    4 +-
 .../replication/ReplicationPeerZKImpl.java      |   70 +-
 .../hbase/replication/ReplicationPeers.java     |    5 +-
 .../replication/ReplicationPeersZKImpl.java     |   45 +-
 .../hbase/replication/ReplicationQueueInfo.java |   11 +-
 .../hbase/replication/ReplicationQueues.java    |   26 +-
 .../replication/ReplicationQueuesClient.java    |    9 +-
 .../ReplicationQueuesClientZKImpl.java          |    6 +-
 .../replication/ReplicationQueuesZKImpl.java    |   78 +-
 .../replication/ReplicationStateZKBase.java     |    4 +-
 .../hbase/replication/ReplicationTracker.java   |    4 +-
 .../replication/ReplicationTrackerZKImpl.java   |    4 +-
 .../hbase/security/AccessDeniedException.java   |    4 +-
 .../hadoop/hbase/security/AuthMethod.java       |    6 +-
 .../hadoop/hbase/security/EncryptionUtil.java   |    6 +-
 .../hbase/security/HBaseSaslRpcClient.java      |   57 +-
 .../hbase/security/SaslClientHandler.java       |  353 +
 .../hadoop/hbase/security/SaslStatus.java       |    2 +-
 .../apache/hadoop/hbase/security/SaslUtil.java  |    6 +-
 .../hbase/security/SecureBulkLoadUtil.java      |    2 +-
 .../hadoop/hbase/security/SecurityInfo.java     |    2 +-
 .../security/access/AccessControlClient.java    |  274 +-
 .../security/access/AccessControlConstants.java |    4 +-
 .../hbase/security/access/Permission.java       |   22 +-
 .../hbase/security/access/TablePermission.java  |   13 +-
 .../hbase/security/access/UserPermission.java   |   11 +-
 .../token/AuthenticationTokenIdentifier.java    |   13 +-
 .../token/AuthenticationTokenSelector.java      |    6 +-
 .../hadoop/hbase/security/token/TokenUtil.java  |  374 +
 .../security/visibility/Authorizations.java     |   10 +-
 .../security/visibility/CellVisibility.java     |    4 +-
 .../visibility/InvalidLabelException.java       |    4 +-
 .../visibility/LabelAlreadyExistsException.java |    4 +-
 .../security/visibility/VisibilityClient.java   |  228 +-
 .../visibility/VisibilityConstants.java         |    8 +-
 .../VisibilityControllerNotReadyException.java  |    4 +-
 .../visibility/VisibilityLabelsValidator.java   |    2 +-
 .../ClientSnapshotDescriptionUtils.java         |    2 +-
 .../snapshot/CorruptedSnapshotException.java    |    4 +-
 .../hbase/snapshot/ExportSnapshotException.java |    4 +-
 .../hbase/snapshot/HBaseSnapshotException.java  |    6 +-
 .../snapshot/RestoreSnapshotException.java      |    4 +-
 .../snapshot/SnapshotCreationException.java     |    4 +-
 .../snapshot/SnapshotDoesNotExistException.java |    4 +-
 .../hbase/snapshot/SnapshotExistsException.java |    6 +-
 .../snapshot/TablePartiallyOpenException.java   |    8 +-
 .../snapshot/UnknownSnapshotException.java      |    6 +-
 .../hbase/util/FileSystemVersionException.java  |    4 +-
 .../org/apache/hadoop/hbase/util/HasThread.java |  100 -
 .../apache/hadoop/hbase/util/JsonMapper.java    |    4 +-
 .../org/apache/hadoop/hbase/util/PoolMap.java   |    5 +-
 .../org/apache/hadoop/hbase/util/Sleeper.java   |  115 -
 .../org/apache/hadoop/hbase/util/Writables.java |    8 +-
 .../hadoop/hbase/zookeeper/EmptyWatcher.java    |    2 +-
 .../hadoop/hbase/zookeeper/HQuorumPeer.java     |   31 +-
 .../hbase/zookeeper/MasterAddressTracker.java   |  106 +-
 .../hbase/zookeeper/MetaTableLocator.java       |  239 +-
 .../hbase/zookeeper/RecoverableZooKeeper.java   |   26 +-
 .../hadoop/hbase/zookeeper/ZKClusterId.java     |    2 +-
 .../apache/hadoop/hbase/zookeeper/ZKConfig.java |   26 +-
 .../hadoop/hbase/zookeeper/ZKLeaderManager.java |    2 +-
 .../zookeeper/ZKTableStateClientSideReader.java |  168 -
 .../apache/hadoop/hbase/zookeeper/ZKUtil.java   |  159 +-
 .../hbase/zookeeper/ZooKeeperListener.java      |    2 +-
 .../hbase/zookeeper/ZooKeeperNodeTracker.java   |    2 +-
 .../hbase/zookeeper/ZooKeeperWatcher.java       |  102 +-
 .../hadoop/hbase/TestHColumnDescriptor.java     |  138 +
 .../hadoop/hbase/TestHTableDescriptor.java      |  263 +
 .../hbase/TestInterfaceAudienceAnnotations.java |  273 +
 .../hadoop/hbase/TestRegionLocations.java       |    4 +
 .../hadoop/hbase/client/TestAsyncProcess.java   |  146 +-
 .../hadoop/hbase/client/TestAttributes.java     |    5 +-
 .../client/TestClientExponentialBackoff.java    |  143 +
 .../hbase/client/TestClientNoCluster.java       |  100 +-
 .../hadoop/hbase/client/TestDelayingRunner.java |   62 +
 .../hbase/client/TestDeleteTimeStamp.java       |    5 +-
 .../client/TestFastFailWithoutTestUtil.java     |  613 ++
 .../org/apache/hadoop/hbase/client/TestGet.java |    5 +-
 .../hadoop/hbase/client/TestIncrement.java      |    5 +-
 .../hadoop/hbase/client/TestOperation.java      |   57 +-
 .../hadoop/hbase/client/TestPutDotHas.java      |    5 +-
 .../apache/hadoop/hbase/client/TestScan.java    |    5 +-
 .../hbase/client/TestSnapshotFromAdmin.java     |   11 +-
 .../hadoop/hbase/filter/TestLongComparator.java |   37 +
 .../apache/hadoop/hbase/ipc/TestIPCUtil.java    |    7 +-
 .../ipc/TestPayloadCarryingRpcController.java   |    5 +-
 .../hbase/security/TestEncryptionUtil.java      |   11 +-
 .../hadoop/hbase/util/BuilderStyleTest.java     |  104 +
 hbase-common/pom.xml                            |   61 +-
 hbase-common/src/main/asciidoc/.gitignore       |    0
 .../java/org/apache/hadoop/hbase/AuthUtil.java  |  103 +
 .../apache/hadoop/hbase/BaseConfigurable.java   |    2 +
 .../main/java/org/apache/hadoop/hbase/Cell.java |    6 +-
 .../org/apache/hadoop/hbase/CellComparator.java |  210 +-
 .../org/apache/hadoop/hbase/CellScannable.java  |    4 +-
 .../org/apache/hadoop/hbase/CellScanner.java    |    7 +-
 .../java/org/apache/hadoop/hbase/CellUtil.java  |  379 +-
 .../org/apache/hadoop/hbase/ChoreService.java   |  368 +
 .../hadoop/hbase/CompoundConfiguration.java     |   27 +-
 .../apache/hadoop/hbase/HBaseConfiguration.java |   25 +-
 .../apache/hadoop/hbase/HBaseIOException.java   |    6 +-
 .../hadoop/hbase/HBaseInterfaceAudience.java    |   22 +-
 .../org/apache/hadoop/hbase/HConstants.java     |  189 +-
 .../java/org/apache/hadoop/hbase/KeyValue.java  |   43 +-
 .../apache/hadoop/hbase/KeyValueTestUtil.java   |    2 +-
 .../org/apache/hadoop/hbase/KeyValueUtil.java   |   90 +-
 .../hadoop/hbase/MetaMutationAnnotation.java    |    3 +
 .../hadoop/hbase/NamespaceDescriptor.java       |   14 +-
 .../org/apache/hadoop/hbase/ScheduledChore.java |  330 +
 .../org/apache/hadoop/hbase/ServerName.java     |  402 +
 .../apache/hadoop/hbase/SettableSequenceId.java |    6 +-
 .../apache/hadoop/hbase/SettableTimestamp.java  |   43 +
 .../java/org/apache/hadoop/hbase/Stoppable.java |   38 +
 .../java/org/apache/hadoop/hbase/TableName.java |   12 +-
 .../main/java/org/apache/hadoop/hbase/Tag.java  |   27 +-
 .../java/org/apache/hadoop/hbase/TagType.java   |   10 +-
 .../apache/hadoop/hbase/VersionAnnotation.java  |   13 +-
 .../apache/hadoop/hbase/codec/BaseDecoder.java  |    6 +-
 .../apache/hadoop/hbase/codec/BaseEncoder.java  |    4 +-
 .../apache/hadoop/hbase/codec/CellCodec.java    |    5 +-
 .../hadoop/hbase/codec/CellCodecWithTags.java   |    7 +-
 .../org/apache/hadoop/hbase/codec/Codec.java    |    8 +-
 .../hadoop/hbase/codec/CodecException.java      |    4 +-
 .../hadoop/hbase/codec/KeyValueCodec.java       |   13 +-
 .../hbase/codec/KeyValueCodecWithTags.java      |   15 +-
 .../exceptions/DeserializationException.java    |   43 +
 .../hadoop/hbase/exceptions/HBaseException.java |   44 +
 .../hadoop/hbase/io/ByteBufferInputStream.java  |  101 +
 .../hadoop/hbase/io/ByteBufferOutputStream.java |   12 +-
 .../hadoop/hbase/io/CellOutputStream.java       |   11 +-
 .../org/apache/hadoop/hbase/io/HeapSize.java    |    2 +-
 .../hadoop/hbase/io/ImmutableBytesWritable.java |    8 +-
 .../hadoop/hbase/io/LimitInputStream.java       |    7 +-
 .../hadoop/hbase/io/SizedCellScanner.java       |    7 +-
 .../hadoop/hbase/io/TagCompressionContext.java  |    4 +-
 .../org/apache/hadoop/hbase/io/TimeRange.java   |   18 +-
 .../hadoop/hbase/io/compress/Compression.java   |    4 +-
 .../io/compress/ReusableStreamGzipCodec.java    |    2 +-
 .../apache/hadoop/hbase/io/crypto/Cipher.java   |    4 +-
 .../hadoop/hbase/io/crypto/CipherProvider.java  |    4 +-
 .../apache/hadoop/hbase/io/crypto/Context.java  |    4 +-
 .../hadoop/hbase/io/crypto/Decryptor.java       |    4 +-
 .../hbase/io/crypto/DefaultCipherProvider.java  |    4 +-
 .../hadoop/hbase/io/crypto/Encryption.java      |    6 +-
 .../hadoop/hbase/io/crypto/Encryptor.java       |    4 +-
 .../hadoop/hbase/io/crypto/KeyProvider.java     |    4 +-
 .../hbase/io/crypto/KeyStoreKeyProvider.java    |    6 +-
 .../apache/hadoop/hbase/io/crypto/aes/AES.java  |    4 +-
 .../hbase/io/crypto/aes/AESDecryptor.java       |    4 +-
 .../hbase/io/crypto/aes/AESEncryptor.java       |    4 +-
 .../io/encoding/BufferedDataBlockEncoder.java   |   81 +-
 .../hbase/io/encoding/CompressionState.java     |    2 +-
 .../io/encoding/CopyKeyDataBlockEncoder.java    |   32 +-
 .../hbase/io/encoding/DataBlockEncoder.java     |    7 +-
 .../hbase/io/encoding/DataBlockEncoding.java    |    4 +-
 .../hbase/io/encoding/DiffKeyDeltaEncoder.java  |   70 +-
 .../hbase/io/encoding/EncodedDataBlock.java     |    2 +-
 .../EncoderBufferTooSmallException.java         |    2 +-
 .../hadoop/hbase/io/encoding/EncodingState.java |    8 +-
 .../hbase/io/encoding/FastDiffDeltaEncoder.java |   86 +-
 .../io/encoding/HFileBlockDecodingContext.java  |    8 +-
 .../HFileBlockDefaultDecodingContext.java       |   11 +-
 .../HFileBlockDefaultEncodingContext.java       |    2 +-
 .../io/encoding/HFileBlockEncodingContext.java  |    2 +-
 .../io/encoding/PrefixKeyDeltaEncoder.java      |   79 +-
 .../io/hadoopbackport/ThrottledInputStream.java |   32 +-
 .../apache/hadoop/hbase/io/hfile/BlockType.java |   10 +-
 .../hadoop/hbase/io/hfile/HFileContext.java     |   35 +-
 .../hbase/io/hfile/HFileContextBuilder.java     |   11 +-
 .../apache/hadoop/hbase/io/util/Dictionary.java |    2 +-
 .../hbase/io/util/HeapMemorySizeUtil.java       |    6 +-
 .../hadoop/hbase/io/util/LRUDictionary.java     |    2 +-
 .../hadoop/hbase/io/util/StreamUtils.java       |   22 +-
 .../org/apache/hadoop/hbase/security/User.java  |   60 +-
 .../hadoop/hbase/security/UserProvider.java     |    5 +-
 .../hbase/trace/HBaseHTraceConfiguration.java   |   37 +-
 .../hadoop/hbase/trace/SpanReceiverHost.java    |   57 +-
 .../org/apache/hadoop/hbase/types/DataType.java |    8 +-
 .../hadoop/hbase/types/FixedLengthWrapper.java  |    4 +-
 .../apache/hadoop/hbase/types/OrderedBlob.java  |    4 +-
 .../hadoop/hbase/types/OrderedBlobVar.java      |    4 +-
 .../hadoop/hbase/types/OrderedBytesBase.java    |    4 +-
 .../hadoop/hbase/types/OrderedFloat32.java      |    4 +-
 .../hadoop/hbase/types/OrderedFloat64.java      |    4 +-
 .../apache/hadoop/hbase/types/OrderedInt16.java |    4 +-
 .../apache/hadoop/hbase/types/OrderedInt32.java |    4 +-
 .../apache/hadoop/hbase/types/OrderedInt64.java |    4 +-
 .../apache/hadoop/hbase/types/OrderedInt8.java  |    4 +-
 .../hadoop/hbase/types/OrderedNumeric.java      |    4 +-
 .../hadoop/hbase/types/OrderedString.java       |    4 +-
 .../org/apache/hadoop/hbase/types/PBType.java   |    9 +-
 .../org/apache/hadoop/hbase/types/RawByte.java  |    4 +-
 .../org/apache/hadoop/hbase/types/RawBytes.java |    4 +-
 .../hadoop/hbase/types/RawBytesFixedLength.java |   13 +-
 .../hadoop/hbase/types/RawBytesTerminated.java  |   13 +-
 .../apache/hadoop/hbase/types/RawDouble.java    |    4 +-
 .../org/apache/hadoop/hbase/types/RawFloat.java |    4 +-
 .../apache/hadoop/hbase/types/RawInteger.java   |    4 +-
 .../org/apache/hadoop/hbase/types/RawLong.java  |    4 +-
 .../org/apache/hadoop/hbase/types/RawShort.java |    4 +-
 .../apache/hadoop/hbase/types/RawString.java    |    4 +-
 .../hbase/types/RawStringFixedLength.java       |   14 +-
 .../hadoop/hbase/types/RawStringTerminated.java |   14 +-
 .../org/apache/hadoop/hbase/types/Struct.java   |    4 +-
 .../hadoop/hbase/types/StructBuilder.java       |    4 +-
 .../hadoop/hbase/types/StructIterator.java      |    4 +-
 .../hadoop/hbase/types/TerminatedWrapper.java   |    4 +-
 .../org/apache/hadoop/hbase/types/Union2.java   |    4 +-
 .../org/apache/hadoop/hbase/types/Union3.java   |    4 +-
 .../org/apache/hadoop/hbase/types/Union4.java   |    4 +-
 .../hadoop/hbase/util/AbstractByteRange.java    |    5 +
 .../hbase/util/AbstractPositionedByteRange.java |   32 +-
 .../apache/hadoop/hbase/util/Addressing.java    |   65 +-
 .../apache/hadoop/hbase/util/ArrayUtils.java    |    2 +-
 .../apache/hadoop/hbase/util/AtomicUtils.java   |   63 +
 .../org/apache/hadoop/hbase/util/Base64.java    |   12 +-
 .../hbase/util/BoundedCompletionService.java    |    3 +
 .../hadoop/hbase/util/ByteBufferArray.java      |    2 +-
 .../hadoop/hbase/util/ByteBufferUtils.java      |   62 +-
 .../org/apache/hadoop/hbase/util/ByteRange.java |    6 +-
 .../hadoop/hbase/util/ByteRangeUtils.java       |    4 +-
 .../org/apache/hadoop/hbase/util/Bytes.java     |  533 +-
 .../hadoop/hbase/util/ChecksumFactory.java      |   10 +-
 .../apache/hadoop/hbase/util/ChecksumType.java  |   12 +-
 .../hadoop/hbase/util/ClassLoaderBase.java      |    2 +-
 .../org/apache/hadoop/hbase/util/ClassSize.java |    8 +-
 .../org/apache/hadoop/hbase/util/Classes.java   |    2 +-
 .../hadoop/hbase/util/CollectionUtils.java      |    8 +-
 .../hadoop/hbase/util/ConcatenatedLists.java    |    4 +-
 .../hadoop/hbase/util/ConcurrentIndex.java      |   16 +-
 .../hbase/util/CoprocessorClassLoader.java      |    2 +-
 .../org/apache/hadoop/hbase/util/Counter.java   |    4 +-
 .../hbase/util/DefaultEnvironmentEdge.java      |    4 +-
 .../apache/hadoop/hbase/util/DrainBarrier.java  |    2 +-
 .../hadoop/hbase/util/DynamicClassLoader.java   |    2 +-
 .../hadoop/hbase/util/EnvironmentEdge.java      |    4 +-
 .../hbase/util/EnvironmentEdgeManager.java      |    2 +-
 .../apache/hadoop/hbase/util/ExceptionUtil.java |    3 +
 .../hadoop/hbase/util/FastLongHistogram.java    |  233 +
 .../org/apache/hadoop/hbase/util/HasThread.java |  100 +
 .../java/org/apache/hadoop/hbase/util/Hash.java |    4 +-
 .../hbase/util/IncrementingEnvironmentEdge.java |    2 +-
 .../apache/hadoop/hbase/util/IterableUtils.java |    2 +-
 .../java/org/apache/hadoop/hbase/util/JVM.java  |   11 +-
 .../apache/hadoop/hbase/util/JenkinsHash.java   |    6 +-
 .../org/apache/hadoop/hbase/util/KeyLocker.java |    2 +-
 .../org/apache/hadoop/hbase/util/MD5Hash.java   |    4 +-
 .../org/apache/hadoop/hbase/util/Methods.java   |    2 +-
 .../apache/hadoop/hbase/util/MurmurHash.java    |    4 +-
 .../apache/hadoop/hbase/util/MurmurHash3.java   |    5 +
 .../org/apache/hadoop/hbase/util/Order.java     |    4 +-
 .../apache/hadoop/hbase/util/OrderedBytes.java  |    5 +-
 .../java/org/apache/hadoop/hbase/util/Pair.java |    4 +-
 .../hadoop/hbase/util/PairOfSameType.java       |    4 +-
 .../hadoop/hbase/util/PositionedByteRange.java  |    7 +-
 .../apache/hadoop/hbase/util/PrettyPrinter.java |    9 +-
 .../hbase/util/ReadOnlyByteRangeException.java  |    5 +
 .../hadoop/hbase/util/ReflectionUtils.java      |    2 +-
 .../apache/hadoop/hbase/util/RetryCounter.java  |    2 +-
 .../hadoop/hbase/util/RetryCounterFactory.java  |    2 +-
 .../hadoop/hbase/util/SimpleByteRange.java      |   11 +-
 .../hbase/util/SimpleMutableByteRange.java      |    4 +-
 .../hbase/util/SimplePositionedByteRange.java   |    4 +-
 .../util/SimplePositionedMutableByteRange.java  |    4 +-
 .../org/apache/hadoop/hbase/util/Sleeper.java   |  124 +
 .../org/apache/hadoop/hbase/util/Strings.java   |    2 +-
 .../org/apache/hadoop/hbase/util/Threads.java   |   47 +-
 .../org/apache/hadoop/hbase/util/Triple.java    |    2 +-
 .../apache/hadoop/hbase/util/VersionInfo.java   |   21 +-
 .../hbase/util/test/LoadTestKVGenerator.java    |   15 +-
 .../hbase/util/test/RedundantKVGenerator.java   |    2 +-
 .../src/main/resources/hbase-default.xml        |   78 +-
 hbase-common/src/saveVersion.sh                 |   14 +-
 .../org/apache/hadoop/hbase/ClassFinder.java    |   41 +-
 .../apache/hadoop/hbase/ClassTestFinder.java    |    2 -
 .../hadoop/hbase/HBaseCommonTestingUtility.java |   49 +-
 .../apache/hadoop/hbase/IntegrationTests.java   |   39 -
 .../org/apache/hadoop/hbase/LargeTests.java     |   38 -
 .../org/apache/hadoop/hbase/MediumTests.java    |   37 -
 .../apache/hadoop/hbase/ResourceChecker.java    |    5 +-
 .../hbase/ResourceCheckerJUnitListener.java     |    3 +-
 .../org/apache/hadoop/hbase/SmallTests.java     |   34 -
 .../apache/hadoop/hbase/TestCellComparator.java |   78 +-
 .../org/apache/hadoop/hbase/TestCellUtil.java   |  112 +-
 .../apache/hadoop/hbase/TestChoreService.java   |  882 ++
 .../apache/hadoop/hbase/TestClassFinder.java    |  117 +-
 .../hadoop/hbase/TestCompoundConfiguration.java |   25 +-
 .../hadoop/hbase/TestHBaseConfiguration.java    |    8 +-
 .../org/apache/hadoop/hbase/TestKeyValue.java   |    9 +-
 .../java/org/apache/hadoop/hbase/Waiter.java    |   42 +-
 .../hadoop/hbase/codec/TestCellCodec.java       |   11 +-
 .../hbase/codec/TestCellCodecWithTags.java      |    5 +-
 .../hadoop/hbase/codec/TestKeyValueCodec.java   |    7 +-
 .../hbase/codec/TestKeyValueCodecWithTags.java  |    5 +-
 .../hbase/io/TestByteBufferInputStream.java     |   82 +
 .../hbase/io/TestTagCompressionContext.java     |    5 +-
 .../hbase/io/crypto/KeyProviderForTesting.java  |    1 +
 .../hbase/io/crypto/TestCipherProvider.java     |   10 +-
 .../hadoop/hbase/io/crypto/TestEncryption.java  |    8 +-
 .../hadoop/hbase/io/crypto/TestKeyProvider.java |    6 +-
 .../io/crypto/TestKeyStoreKeyProvider.java      |    5 +-
 .../hadoop/hbase/io/crypto/aes/TestAES.java     |   10 +-
 .../hadoop/hbase/io/util/TestLRUDictionary.java |    9 +-
 .../hbase/types/TestFixedLengthWrapper.java     |    5 +-
 .../hadoop/hbase/types/TestOrderedBlob.java     |    5 +-
 .../hadoop/hbase/types/TestOrderedBlobVar.java  |    5 +-
 .../hadoop/hbase/types/TestOrderedString.java   |    5 +-
 .../hadoop/hbase/types/TestRawString.java       |    5 +-
 .../apache/hadoop/hbase/types/TestStruct.java   |    5 +-
 .../hbase/types/TestStructNullExtension.java    |    5 +-
 .../hbase/types/TestTerminatedWrapper.java      |    5 +-
 .../apache/hadoop/hbase/types/TestUnion2.java   |    5 +-
 .../apache/hadoop/hbase/util/TestBase64.java    |    6 +-
 .../util/TestByteRangeWithKVSerialization.java  |    5 +-
 .../org/apache/hadoop/hbase/util/TestBytes.java |   10 +-
 .../hbase/util/TestConcatenatedLists.java       |   14 +-
 .../hbase/util/TestCoprocessorClassLoader.java  |    5 +-
 .../apache/hadoop/hbase/util/TestCounter.java   |    5 +-
 .../hadoop/hbase/util/TestDrainBarrier.java     |   11 +-
 .../hbase/util/TestDynamicClassLoader.java      |    5 +-
 .../hbase/util/TestEnvironmentEdgeManager.java  |   18 +-
 .../hbase/util/TestFastLongHistogram.java       |  100 +
 .../apache/hadoop/hbase/util/TestKeyLocker.java |    9 +-
 .../hbase/util/TestLoadTestKVGenerator.java     |    6 +-
 .../org/apache/hadoop/hbase/util/TestOrder.java |    5 +-
 .../hadoop/hbase/util/TestOrderedBytes.java     |    5 +-
 .../hadoop/hbase/util/TestShowProperties.java   |    9 +-
 .../hbase/util/TestSimpleMutableByteRange.java  |    5 +-
 .../TestSimplePositionedMutableByteRange.java   |    5 +-
 .../apache/hadoop/hbase/util/TestThreads.java   |    9 +-
 hbase-examples/README.txt                       |    4 +-
 hbase-examples/pom.xml                          |   41 +-
 hbase-examples/src/main/asciidoc/.gitignore     |    0
 .../client/example/BufferedMutatorExample.java  |  119 +
 .../example/ZooKeeperScanPolicyObserver.java    |    8 +-
 .../hadoop/hbase/mapreduce/IndexBuilder.java    |    4 +-
 .../apache/hadoop/hbase/thrift/DemoClient.java  |  105 +-
 .../hadoop/hbase/thrift/HttpDoAsClient.java     |  284 +
 .../apache/hadoop/hbase/thrift2/DemoClient.java |   92 +-
 .../example/TestBulkDeleteProtocol.java         |   52 +-
 .../example/TestRowCountEndpoint.java           |   17 +-
 .../TestZooKeeperScanPolicyObserver.java        |    8 +-
 .../hbase/mapreduce/TestMapReduceExamples.java  |    5 +-
 .../apache/hadoop/hbase/types/TestPBCell.java   |    4 +
 hbase-hadoop-compat/pom.xml                     |   23 +-
 .../src/main/asciidoc/.gitignore                |    0
 .../ipc/MetricsHBaseServerSourceFactory.java    |    4 +-
 .../master/MetricsMasterFileSystemSource.java   |    6 +-
 .../regionserver/MetricsRegionServerSource.java |   48 +-
 .../MetricsRegionServerWrapper.java             |   29 +-
 .../wal/MetricsEditsReplaySource.java           |    2 +-
 .../regionserver/wal/MetricsWALSource.java      |   21 +-
 .../MetricsReplicationSinkSource.java           |   29 +
 .../MetricsReplicationSourceFactory.java        |   25 +
 .../MetricsReplicationSourceSource.java         |   46 +
 .../regionserver/wal/TestMetricsHLogSource.java |   32 -
 .../regionserver/wal/TestMetricsWALSource.java  |   32 +
 hbase-hadoop2-compat/pom.xml                    |   56 +-
 .../src/main/asciidoc/.gitignore                |    0
 .../MetricsHBaseServerSourceFactoryImpl.java    |    4 +-
 .../hbase/ipc/MetricsHBaseServerSourceImpl.java |    2 +-
 .../apache/hadoop/hbase/mapreduce/JobUtil.java  |    4 +-
 .../MetricsAssignmentManagerSourceImpl.java     |    2 +-
 .../MetricsMasterFilesystemSourceImpl.java      |    2 +-
 .../master/MetricsMasterSourceFactoryImpl.java  |    2 +-
 .../hbase/master/MetricsMasterSourceImpl.java   |    2 +-
 .../hbase/master/MetricsSnapshotSourceImpl.java |    2 +-
 .../balancer/MetricsBalancerSourceImpl.java     |    2 +-
 .../hadoop/hbase/metrics/BaseSourceImpl.java    |    6 +-
 .../hadoop/hbase/metrics/MBeanSourceImpl.java   |    2 +-
 .../MetricsRegionAggregateSourceImpl.java       |    2 +-
 .../MetricsRegionServerSourceFactoryImpl.java   |    2 +-
 .../MetricsRegionServerSourceImpl.java          |   49 +-
 .../regionserver/MetricsRegionSourceImpl.java   |    2 +-
 .../wal/MetricsEditsReplaySourceImpl.java       |    2 +-
 .../regionserver/wal/MetricsWALSourceImpl.java  |   24 +-
 .../MetricsReplicationGlobalSourceSource.java   |   98 +
 .../MetricsReplicationSinkSourceImpl.java       |   47 +
 .../MetricsReplicationSourceFactoryImpl.java    |   38 +
 .../MetricsReplicationSourceImpl.java           |    2 +-
 .../MetricsReplicationSourceSourceImpl.java     |  128 +
 .../hbase/rest/MetricsRESTSourceImpl.java       |    2 +-
 .../MetricsThriftServerSourceFactoryImpl.java   |    2 +-
 .../thrift/MetricsThriftServerSourceImpl.java   |    2 +-
 .../hadoop/metrics2/impl/JmxCacheBuster.java    |    5 +-
 .../metrics2/lib/DynamicMetricsRegistry.java    |    2 +-
 .../metrics2/lib/MetricMutableQuantiles.java    |    2 +-
 .../metrics2/lib/MetricsExecutorImpl.java       |    2 +-
 .../hadoop/metrics2/lib/MutableHistogram.java   |    2 +-
 .../hadoop/metrics2/util/MetricQuantile.java    |    4 +-
 .../metrics2/util/MetricSampleQuantiles.java    |    2 +-
 ...regionserver.MetricsReplicationSourceFactory |    1 +
 .../master/TestMetricsMasterSourceImpl.java     |    3 -
 ...TestMetricsReplicationSourceFactoryImpl.java |   35 +
 hbase-it/pom.xml                                |   43 +-
 hbase-it/src/main/asciidoc/.gitignore           |    0
 .../org/apache/hadoop/hbase/ClusterManager.java |   51 +-
 .../hadoop/hbase/DistributedHBaseCluster.java   |  199 +-
 .../hadoop/hbase/HBaseClusterManager.java       |   38 +-
 .../hbase/IntegrationTestAcidGuarantees.java    |  305 +-
 .../hadoop/hbase/IntegrationTestBase.java       |    3 +
 .../hadoop/hbase/IntegrationTestIngest.java     |   10 +-
 .../IntegrationTestIngestStripeCompactions.java |    1 +
 .../hbase/IntegrationTestIngestWithACL.java     |    1 +
 .../IntegrationTestIngestWithEncryption.java    |   46 +-
 .../hbase/IntegrationTestIngestWithMOB.java     |    1 +
 .../hbase/IntegrationTestIngestWithTags.java    |    1 +
 ...tegrationTestIngestWithVisibilityLabels.java |    1 +
 .../hbase/IntegrationTestLazyCfLoading.java     |    8 +-
 .../hbase/IntegrationTestManyRegions.java       |    2 +-
 .../hbase/IntegrationTestMetaReplicas.java      |  108 +
 .../hbase/IntegrationTestRegionReplicaPerf.java |  142 +-
 .../hadoop/hbase/IntegrationTestsDriver.java    |    1 +
 .../StripeCompactionsPerformanceEvaluation.java |    2 +-
 .../hadoop/hbase/chaos/actions/Action.java      |   43 +-
 .../chaos/actions/BatchRestartRsAction.java     |    4 +-
 .../chaos/actions/ChangeEncodingAction.java     |    4 +-
 .../actions/RestartRsHoldingTableAction.java    |   29 +-
 .../actions/RollingBatchRestartRsAction.java    |    2 +-
 .../hbase/chaos/factories/MonkeyFactory.java    |    4 +
 .../chaos/factories/NoKillMonkeyFactory.java    |   87 +
 .../SlowDeterministicMonkeyFactory.java         |    1 -
 .../StressAssignmentManagerMonkeyFactory.java   |   81 +
 .../hbase/chaos/monkies/CalmChaosMonkey.java    |    5 +
 .../hadoop/hbase/chaos/monkies/ChaosMonkey.java |    6 +
 .../chaos/monkies/PolicyBasedChaosMonkey.java   |    6 +
 .../policies/TwoConcurrentActionPolicy.java     |   97 +
 .../mapreduce/IntegrationTestBulkLoad.java      |  103 +-
 .../mapreduce/IntegrationTestImportTsv.java     |  194 +-
 .../IntegrationTestTableMapReduceUtil.java      |    4 +-
 ...IntegrationTestTableSnapshotInputFormat.java |    2 +-
 .../hadoop/hbase/mttr/IntegrationTestMTTR.java  |   85 +-
 .../test/IntegrationTestBigLinkedList.java      |  542 +-
 ...egrationTestBigLinkedListWithVisibility.java |   34 +-
 .../test/IntegrationTestLoadAndVerify.java      |   54 +-
 ...undedMultiGetRequestsWithRegionReplicas.java |    5 +-
 ...stTimeBoundedRequestsWithRegionReplicas.java |   22 +-
 ...tionTestWithCellVisibilityLoadAndVerify.java |    9 +-
 .../trace/IntegrationTestSendTraceRequests.java |   22 +-
 hbase-prefix-tree/pom.xml                       |   57 +-
 hbase-prefix-tree/src/main/asciidoc/.gitignore  |    0
 .../codec/prefixtree/PrefixTreeBlockMeta.java   |    8 +-
 .../hbase/codec/prefixtree/PrefixTreeCodec.java |   11 +-
 .../codec/prefixtree/PrefixTreeSeeker.java      |  220 +-
 .../prefixtree/decode/ArraySearcherPool.java    |    4 +-
 .../codec/prefixtree/decode/DecoderFactory.java |    2 +-
 .../PrefixTreeArrayReversibleScanner.java       |    3 +-
 .../decode/PrefixTreeArrayScanner.java          |   29 +-
 .../decode/PrefixTreeArraySearcher.java         |   15 +-
 .../codec/prefixtree/decode/PrefixTreeCell.java |   20 +-
 .../decode/column/ColumnNodeReader.java         |    2 +-
 .../prefixtree/decode/column/ColumnReader.java  |    2 +-
 .../prefixtree/decode/row/RowNodeReader.java    |    4 +-
 .../decode/timestamp/MvccVersionDecoder.java    |    2 +-
 .../decode/timestamp/TimestampDecoder.java      |    2 +-
 .../codec/prefixtree/encode/EncoderFactory.java |    2 +-
 .../codec/prefixtree/encode/EncoderPool.java    |    4 +-
 .../prefixtree/encode/EncoderPoolImpl.java      |    2 +-
 .../prefixtree/encode/PrefixTreeEncoder.java    |    2 +-
 .../encode/column/ColumnNodeWriter.java         |    2 +-
 .../encode/column/ColumnSectionWriter.java      |    6 +-
 .../encode/other/CellTypeEncoder.java           |    2 +-
 .../prefixtree/encode/other/ColumnNodeType.java |    2 +-
 .../prefixtree/encode/other/LongEncoder.java    |    6 +-
 .../prefixtree/encode/row/RowNodeWriter.java    |    2 +-
 .../prefixtree/encode/row/RowSectionWriter.java |    2 +-
 .../encode/tokenize/TokenDepthComparator.java   |    2 +-
 .../prefixtree/encode/tokenize/Tokenizer.java   |    4 +-
 .../encode/tokenize/TokenizerNode.java          |    8 +-
 .../tokenize/TokenizerRowSearchPosition.java    |   10 +-
 .../tokenize/TokenizerRowSearchResult.java      |    2 +-
 .../prefixtree/scanner/CellScannerPosition.java |    2 +-
 .../codec/prefixtree/scanner/CellSearcher.java  |    2 +-
 .../scanner/ReversibleCellScanner.java          |    4 +-
 .../hbase/util/byterange/ByteRangeSet.java      |    2 +-
 .../util/byterange/impl/ByteRangeHashSet.java   |    4 +-
 .../util/byterange/impl/ByteRangeTreeSet.java   |    4 +-
 .../hadoop/hbase/util/vint/UFIntTool.java       |   16 +-
 .../hadoop/hbase/util/vint/UVIntTool.java       |    2 +-
 .../hadoop/hbase/util/vint/UVLongTool.java      |    2 +-
 .../hbase/codec/keyvalue/TestKeyValueTool.java  |    7 +-
 .../prefixtree/blockmeta/TestBlockMeta.java     |    5 +-
 .../codec/prefixtree/builder/TestTokenizer.java |    5 +-
 .../codec/prefixtree/builder/TestTreeDepth.java |    5 +-
 .../prefixtree/column/TestColumnBuilder.java    |    5 +-
 .../prefixtree/row/TestPrefixTreeSearcher.java  |   33 +-
 .../hbase/codec/prefixtree/row/TestRowData.java |    6 +-
 .../codec/prefixtree/row/TestRowEncoder.java    |    9 +-
 .../row/data/TestRowDataComplexQualifiers.java  |    2 +-
 .../prefixtree/row/data/TestRowDataDeeper.java  |   48 +-
 .../data/TestRowDataDifferentTimestamps.java    |   72 +-
 .../row/data/TestRowDataExerciseFInts.java      |   84 +-
 .../prefixtree/row/data/TestRowDataNub.java     |   52 +-
 .../data/TestRowDataQualifierByteOrdering.java  |   18 +-
 .../row/data/TestRowDataSearchWithPrefix.java   |   74 +
 .../row/data/TestRowDataSearcherRowMiss.java    |   24 +-
 .../row/data/TestRowDataSingleQualifier.java    |   12 +-
 .../prefixtree/row/data/TestRowDataTrivial.java |   12 +-
 .../prefixtree/row/data/TestRowDataUrls.java    |    2 +-
 .../row/data/TestRowDataUrlsExample.java        |   60 +-
 .../timestamp/TestTimestampEncoder.java         |    5 +-
 .../hadoop/hbase/util/bytes/TestByteRange.java  |    5 +-
 .../hadoop/hbase/util/vint/TestFIntTool.java    |    6 +-
 .../hadoop/hbase/util/vint/TestVIntTool.java    |    6 +-
 .../hadoop/hbase/util/vint/TestVLongTool.java   |    6 +-
 hbase-protocol/pom.xml                          |    5 +
 hbase-protocol/src/main/asciidoc/.gitignore     |    0
 .../protobuf/HBaseZeroCopyByteString.java       |    2 +-
 .../hadoop/hbase/protobuf/ProtobufMagic.java    |   90 +
 .../hbase/protobuf/generated/AdminProtos.java   | 1103 ++-
 .../hbase/protobuf/generated/ClientProtos.java  | 1507 ++-
 .../protobuf/generated/ClusterStatusProtos.java |  259 +-
 .../protobuf/generated/ComparatorProtos.java    |  764 +-
 .../hbase/protobuf/generated/FilterProtos.java  | 1451 ++-
 .../hbase/protobuf/generated/HBaseProtos.java   | 1783 +++-
 .../hbase/protobuf/generated/MasterProtos.java  | 8813 +++++++++++++++---
 .../hbase/protobuf/generated/QuotaProtos.java   | 4378 +++++++++
 .../hbase/protobuf/generated/RPCProtos.java     |   16 +-
 .../generated/RegionServerStatusProtos.java     |   16 +-
 .../generated/VisibilityLabelsProtos.java       | 1121 ++-
 .../hbase/protobuf/generated/WALProtos.java     | 3455 ++++---
 .../protobuf/generated/ZooKeeperProtos.java     |  325 +-
 .../apache/hadoop/hbase/util/ByteStringer.java  |    2 +
 hbase-protocol/src/main/protobuf/Admin.proto    |   13 +
 hbase-protocol/src/main/protobuf/Client.proto   |   19 +
 .../src/main/protobuf/ClusterStatus.proto       |    5 +
 .../src/main/protobuf/Comparator.proto          |    5 +
 hbase-protocol/src/main/protobuf/Filter.proto   |   11 +
 hbase-protocol/src/main/protobuf/HBase.proto    |   29 +
 hbase-protocol/src/main/protobuf/Master.proto   |   60 +-
 hbase-protocol/src/main/protobuf/Quota.proto    |   73 +
 hbase-protocol/src/main/protobuf/RPC.proto      |    2 +-
 .../src/main/protobuf/RegionServerStatus.proto  |    2 +-
 .../src/main/protobuf/VisibilityLabels.proto    |   10 +
 hbase-protocol/src/main/protobuf/WAL.proto      |   32 +-
 .../src/main/protobuf/ZooKeeper.proto           |    6 +-
 hbase-rest/pom.xml                              |  406 +
 hbase-rest/src/main/asciidoc/.gitignore         |    0
 .../org/apache/hadoop/hbase/rest/Constants.java |   78 +
 .../hadoop/hbase/rest/ExistsResource.java       |   75 +
 .../apache/hadoop/hbase/rest/MetricsREST.java   |  103 +
 .../hadoop/hbase/rest/MultiRowResource.java     |  108 +
 .../hbase/rest/ProtobufMessageHandler.java      |   46 +
 .../hbase/rest/ProtobufStreamingUtil.java       |  102 +
 .../apache/hadoop/hbase/rest/RESTServer.java    |  254 +
 .../apache/hadoop/hbase/rest/RESTServlet.java   |  161 +
 .../hadoop/hbase/rest/RESTServletContainer.java |   76 +
 .../hadoop/hbase/rest/RegionsResource.java      |  108 +
 .../apache/hadoop/hbase/rest/ResourceBase.java  |   96 +
 .../hadoop/hbase/rest/ResourceConfig.java       |   31 +
 .../hadoop/hbase/rest/ResultGenerator.java      |   50 +
 .../apache/hadoop/hbase/rest/RootResource.java  |  106 +
 .../apache/hadoop/hbase/rest/RowResource.java   |  595 ++
 .../hadoop/hbase/rest/RowResultGenerator.java   |  122 +
 .../org/apache/hadoop/hbase/rest/RowSpec.java   |  407 +
 .../hbase/rest/ScannerInstanceResource.java     |  201 +
 .../hadoop/hbase/rest/ScannerResource.java      |  164 +
 .../hbase/rest/ScannerResultGenerator.java      |  191 +
 .../hadoop/hbase/rest/SchemaResource.java       |  248 +
 .../rest/StorageClusterStatusResource.java      |  109 +
 .../rest/StorageClusterVersionResource.java     |   79 +
 .../apache/hadoop/hbase/rest/TableResource.java |  196 +
 .../hadoop/hbase/rest/TableScanResource.java    |  168 +
 .../hadoop/hbase/rest/VersionResource.java      |  104 +
 .../apache/hadoop/hbase/rest/client/Client.java |  525 ++
 .../hadoop/hbase/rest/client/Cluster.java       |  110 +
 .../hadoop/hbase/rest/client/RemoteAdmin.java   |  401 +
 .../hadoop/hbase/rest/client/RemoteHTable.java  |  858 ++
 .../hadoop/hbase/rest/client/Response.java      |  155 +
 .../hadoop/hbase/rest/filter/AuthFilter.java    |   82 +
 .../hbase/rest/filter/GZIPRequestStream.java    |   58 +
 .../hbase/rest/filter/GZIPRequestWrapper.java   |   52 +
 .../hbase/rest/filter/GZIPResponseStream.java   |   78 +
 .../hbase/rest/filter/GZIPResponseWrapper.java  |  147 +
 .../hadoop/hbase/rest/filter/GzipFilter.java    |   85 +
 .../hadoop/hbase/rest/model/CellModel.java      |  209 +
 .../hadoop/hbase/rest/model/CellSetModel.java   |  152 +
 .../hbase/rest/model/ColumnSchemaModel.java     |  241 +
 .../hadoop/hbase/rest/model/RowModel.java       |  151 +
 .../hadoop/hbase/rest/model/ScannerModel.java   |  870 ++
 .../rest/model/StorageClusterStatusModel.java   |  790 ++
 .../rest/model/StorageClusterVersionModel.java  |   78 +
 .../hadoop/hbase/rest/model/TableInfoModel.java |  159 +
 .../hadoop/hbase/rest/model/TableListModel.java |  113 +
 .../hadoop/hbase/rest/model/TableModel.java     |   84 +
 .../hbase/rest/model/TableRegionModel.java      |  196 +
 .../hbase/rest/model/TableSchemaModel.java      |  360 +
 .../hadoop/hbase/rest/model/VersionModel.java   |  209 +
 .../org/apache/hadoop/hbase/rest/package.html   | 1660 ++++
 .../rest/protobuf/generated/CellMessage.java    |  731 ++
 .../rest/protobuf/generated/CellSetMessage.java | 1521 +++
 .../protobuf/generated/ColumnSchemaMessage.java | 1904 ++++
 .../rest/protobuf/generated/ScannerMessage.java | 1578 ++++
 .../generated/StorageClusterStatusMessage.java  | 3955 ++++++++
 .../protobuf/generated/TableInfoMessage.java    | 1802 ++++
 .../protobuf/generated/TableListMessage.java    |  547 ++
 .../protobuf/generated/TableSchemaMessage.java  | 2125 +++++
 .../rest/protobuf/generated/VersionMessage.java | 1147 +++
 .../rest/provider/JAXBContextResolver.java      |   89 +
 .../hbase/rest/provider/JacksonProvider.java    |   31 +
 .../consumer/ProtobufMessageBodyConsumer.java   |   88 +
 .../producer/PlainTextMessageBodyProducer.java  |   74 +
 .../producer/ProtobufMessageBodyProducer.java   |   81 +
 .../resources/hbase-webapps/rest/index.html     |   20 +
 .../main/resources/hbase-webapps/rest/rest.jsp  |  117 +
 .../org/apache/hadoop/hbase/rest/XMLSchema.xsd  |  181 +
 .../hbase/rest/protobuf/CellMessage.proto       |   25 +
 .../hbase/rest/protobuf/CellSetMessage.proto    |   28 +
 .../rest/protobuf/ColumnSchemaMessage.proto     |   31 +
 .../hbase/rest/protobuf/ScannerMessage.proto    |   32 +
 .../protobuf/StorageClusterStatusMessage.proto  |   51 +
 .../hbase/rest/protobuf/TableInfoMessage.proto  |   30 +
 .../hbase/rest/protobuf/TableListMessage.proto  |   22 +
 .../rest/protobuf/TableSchemaMessage.proto      |   33 +
 .../hbase/rest/protobuf/VersionMessage.proto    |   26 +
 .../apache/hadoop/hbase/rest/DummyFilter.java   |   64 +
 .../hbase/rest/HBaseRESTTestingUtility.java     |   98 +
 .../hbase/rest/PerformanceEvaluation.java       | 1549 +++
 .../hadoop/hbase/rest/RowResourceBase.java      |  482 +
 .../apache/hadoop/hbase/rest/TestDeleteRow.java |   97 +
 .../hbase/rest/TestGZIPResponseWrapper.java     |  118 +
 .../hbase/rest/TestGetAndPutResource.java       |  583 ++
 .../hadoop/hbase/rest/TestGzipFilter.java       |  161 +
 .../hadoop/hbase/rest/TestMultiRowResource.java |  181 +
 .../hadoop/hbase/rest/TestResourceFilter.java   |   62 +
 .../hadoop/hbase/rest/TestScannerResource.java  |  366 +
 .../hbase/rest/TestScannersWithFilters.java     | 1002 ++
 .../hbase/rest/TestScannersWithLabels.java      |  239 +
 .../hadoop/hbase/rest/TestSchemaResource.java   |  192 +
 .../hadoop/hbase/rest/TestStatusResource.java   |  135 +
 .../hadoop/hbase/rest/TestTableResource.java    |  264 +
 .../apache/hadoop/hbase/rest/TestTableScan.java |  615 ++
 .../hadoop/hbase/rest/TestVersionResource.java  |  179 +
 .../rest/client/TestRemoteAdminRetries.java     |  166 +
 .../rest/client/TestRemoteHTableRetries.java    |  194 +
 .../hbase/rest/client/TestRemoteTable.java      |  533 ++
 .../hadoop/hbase/rest/model/TestCellModel.java  |   75 +
 .../hbase/rest/model/TestCellSetModel.java      |  139 +
 .../hbase/rest/model/TestColumnSchemaModel.java |   78 +
 .../hadoop/hbase/rest/model/TestModelBase.java  |  135 +
 .../hadoop/hbase/rest/model/TestRowModel.java   |   76 +
 .../hbase/rest/model/TestScannerModel.java      |  110 +
 .../model/TestStorageClusterStatusModel.java    |  145 +
 .../model/TestStorageClusterVersionModel.java   |   54 +
 .../hbase/rest/model/TestTableInfoModel.java    |   89 +
 .../hbase/rest/model/TestTableListModel.java    |   66 +
 .../hbase/rest/model/TestTableRegionModel.java  |   88 +
 .../hbase/rest/model/TestTableSchemaModel.java  |  110 +
 .../hbase/rest/model/TestVersionModel.java      |   71 +
 hbase-rest/src/test/resources/hbase-site.xml    |  150 +
 hbase-rest/src/test/resources/hdfs-site.xml     |   32 +
 hbase-rest/src/test/resources/log4j.properties  |   66 +
 hbase-rest/src/test/resources/mapred-queues.xml |   75 +
 hbase-rest/src/test/resources/mapred-site.xml   |   34 +
 hbase-rest/src/test/resources/zoo.cfg           |   43 +
 hbase-server/pom.xml                            |  124 +-
 hbase-server/src/main/asciidoc/.gitignore       |    0
 .../tmpl/master/BackupMasterStatusTmpl.jamon    |    9 +-
 .../hbase/tmpl/master/MasterStatusTmpl.jamon    |   71 +-
 .../tmpl/regionserver/BlockCacheTmpl.jamon      |    8 +-
 .../tmpl/regionserver/BlockCacheViewTmpl.jamon  |    8 +-
 .../hbase/tmpl/regionserver/RSStatusTmpl.jamon  |   19 +-
 .../tmpl/regionserver/RegionListTmpl.jamon      |    2 +
 .../tmpl/regionserver/ServerMetricsTmpl.jamon   |   19 +-
 .../hadoop/hbase/CoordinatedStateManager.java   |   10 +-
 .../hbase/CoordinatedStateManagerFactory.java   |    9 +-
 .../hadoop/hbase/DaemonThreadFactory.java       |    8 +-
 .../hadoop/hbase/HDFSBlocksDistribution.java    |    2 +-
 .../apache/hadoop/hbase/HealthCheckChore.java   |    6 +-
 .../org/apache/hadoop/hbase/HealthReport.java   |    2 +-
 .../apache/hadoop/hbase/InterProcessLock.java   |    2 +-
 .../hadoop/hbase/InterProcessReadWriteLock.java |    2 +-
 .../org/apache/hadoop/hbase/JMXListener.java    |    2 +-
 .../apache/hadoop/hbase/LocalHBaseCluster.java  |   30 +-
 .../java/org/apache/hadoop/hbase/Server.java    |   21 +-
 .../apache/hadoop/hbase/SplitLogCounters.java   |    8 +-
 .../org/apache/hadoop/hbase/SplitLogTask.java   |    4 +-
 .../apache/hadoop/hbase/TableDescriptor.java    |  165 +
 .../apache/hadoop/hbase/TableDescriptors.java   |   38 +-
 .../apache/hadoop/hbase/TableStateManager.java  |  115 -
 .../org/apache/hadoop/hbase/TagRewriteCell.java |  202 +
 .../apache/hadoop/hbase/ZKNamespaceManager.java |    2 +-
 .../hadoop/hbase/backup/HFileArchiver.java      |    4 +-
 .../backup/example/HFileArchiveManager.java     |    3 +-
 .../example/LongTermArchivingHFileCleaner.java  |   14 +-
 .../example/TableHFileArchiveTracker.java       |    2 +-
 .../backup/example/ZKTableArchiveClient.java    |    8 +-
 .../hbase/client/ClientSideRegionScanner.java   |   14 +-
 .../hbase/client/CoprocessorHConnection.java    |    7 +-
 .../hadoop/hbase/client/HTableWrapper.java      |   89 +-
 .../hbase/client/TableSnapshotScanner.java      |   11 +-
 .../client/coprocessor/RowProcessorClient.java  |    4 +-
 .../apache/hadoop/hbase/codec/MessageCodec.java |    5 +-
 .../hadoop/hbase/conf/ConfigurationManager.java |  139 +
 .../hbase/conf/ConfigurationObserver.java       |   38 +
 .../conf/PropagatingConfigurationObserver.java  |   43 +
 .../hadoop/hbase/constraint/BaseConstraint.java |    2 +-
 .../hadoop/hbase/constraint/Constraint.java     |    3 +-
 .../hbase/constraint/ConstraintException.java   |   14 +-
 .../hbase/constraint/ConstraintProcessor.java   |    2 +-
 .../hadoop/hbase/constraint/Constraints.java    |   11 +-
 .../BaseCoordinatedStateManager.java            |   10 +-
 .../SplitLogManagerCoordination.java            |    7 +-
 .../SplitLogWorkerCoordination.java             |   10 +-
 .../ZKSplitLogManagerCoordination.java          |  124 +-
 .../coordination/ZkCoordinatedStateManager.java |   20 +-
 .../ZkSplitLogWorkerCoordination.java           |   26 +-
 .../coprocessor/AggregateImplementation.java    |   31 +-
 .../BaseMasterAndRegionObserver.java            |  112 +-
 .../hbase/coprocessor/BaseMasterObserver.java   |  113 +-
 .../hbase/coprocessor/BaseRegionObserver.java   |   67 +-
 .../coprocessor/BaseRegionServerObserver.java   |   28 +-
 .../coprocessor/BaseRowProcessorEndpoint.java   |    4 +-
 .../hbase/coprocessor/BaseWALObserver.java      |   27 +-
 .../hbase/coprocessor/BulkLoadObserver.java     |   54 +
 .../hbase/coprocessor/CoprocessorHost.java      |  107 +-
 .../hbase/coprocessor/CoprocessorService.java   |    4 +-
 .../hbase/coprocessor/EndpointObserver.java     |    4 +-
 .../MasterCoprocessorEnvironment.java           |    4 +-
 .../hbase/coprocessor/MasterObserver.java       |  200 +-
 .../coprocessor/MultiRowMutationEndpoint.java   |    4 +-
 .../hbase/coprocessor/ObserverContext.java      |    4 +-
 .../RegionCoprocessorEnvironment.java           |   11 +-
 .../hbase/coprocessor/RegionObserver.java       |  105 +-
 .../hbase/coprocessor/RegionServerObserver.java |   47 +
 .../SingletonCoprocessorService.java            |   34 +
 .../coprocessor/WALCoprocessorEnvironment.java  |   10 +-
 .../hadoop/hbase/coprocessor/WALObserver.java   |   68 +-
 .../hbase/errorhandling/ForeignException.java   |    6 +-
 .../ForeignExceptionDispatcher.java             |    4 +-
 .../errorhandling/ForeignExceptionListener.java |    4 +-
 .../errorhandling/ForeignExceptionSnare.java    |    3 +-
 .../hbase/errorhandling/TimeoutException.java   |    6 +-
 .../errorhandling/TimeoutExceptionInjector.java |   20 +-
 .../hadoop/hbase/executor/EventHandler.java     |   48 +-
 .../hadoop/hbase/executor/ExecutorService.java  |   48 +-
 .../org/apache/hadoop/hbase/fs/HFileSystem.java |   16 +-
 .../hbase/http/AdminAuthorizedServlet.java      |    4 +-
 .../hadoop/hbase/http/FilterInitializer.java    |    2 +-
 .../apache/hadoop/hbase/http/HttpConfig.java    |   29 +-
 .../apache/hadoop/hbase/http/HttpServer.java    |  128 +-
 .../apache/hadoop/hbase/http/InfoServer.java    |   32 +-
 .../apache/hadoop/hbase/http/NoCacheFilter.java |    4 +
 .../hbase/http/ServerConfigurationKeys.java     |    4 +-
 .../hbase/http/SslSocketConnectorSecure.java    |   58 +
 .../hadoop/hbase/http/conf/ConfServlet.java     |    4 +-
 .../hadoop/hbase/http/jmx/JMXJsonServlet.java   |  284 +-
 .../hadoop/hbase/http/jmx/package-info.java     |    2 +-
 .../hbase/http/lib/StaticUserWebFilter.java     |    4 +
 .../hadoop/hbase/http/lib/package-info.java     |    4 +-
 .../apache/hadoop/hbase/http/log/LogLevel.java  |    4 +-
 .../apache/hadoop/hbase/http/package-info.java  |    2 +-
 .../hbase/io/FSDataInputStreamWrapper.java      |    2 +-
 .../org/apache/hadoop/hbase/io/FileLink.java    |   52 +-
 .../org/apache/hadoop/hbase/io/HFileLink.java   |   78 +-
 .../org/apache/hadoop/hbase/io/HLogLink.java    |   69 -
 .../hadoop/hbase/io/HalfStoreFileReader.java    |   14 +-
 .../org/apache/hadoop/hbase/io/Reference.java   |   25 +-
 .../org/apache/hadoop/hbase/io/WALLink.java     |   69 +
 .../hadoop/hbase/io/WritableWithSize.java       |    2 +-
 .../hbase/io/hfile/AbstractHFileReader.java     |    2 +-
 .../hbase/io/hfile/AbstractHFileWriter.java     |   48 +-
 .../hadoop/hbase/io/hfile/BlockCache.java       |    2 +-
 .../hadoop/hbase/io/hfile/BlockCacheKey.java    |    4 +-
 .../hadoop/hbase/io/hfile/BlockCacheUtil.java   |    2 +-
 .../hbase/io/hfile/BlockCachesIterator.java     |    4 +-
 .../hadoop/hbase/io/hfile/BlockPriority.java    |    2 +-
 .../hadoop/hbase/io/hfile/CacheConfig.java      |   83 +-
 .../hadoop/hbase/io/hfile/CacheStats.java       |    4 +-
 .../apache/hadoop/hbase/io/hfile/Cacheable.java |    4 +-
 .../hbase/io/hfile/CacheableDeserializer.java   |    2 +-
 .../hfile/CacheableDeserializerIdManager.java   |    2 +-
 .../hadoop/hbase/io/hfile/CachedBlock.java      |    4 +-
 .../hadoop/hbase/io/hfile/ChecksumUtil.java     |    9 +-
 .../hbase/io/hfile/CombinedBlockCache.java      |    4 +-
 .../hbase/io/hfile/CorruptHFileException.java   |    2 +-
 .../hadoop/hbase/io/hfile/FixedFileTrailer.java |    2 +-
 .../org/apache/hadoop/hbase/io/hfile/HFile.java |   58 +-
 .../hadoop/hbase/io/hfile/HFileBlock.java       |  643 +-
 .../hadoop/hbase/io/hfile/HFileBlockIndex.java  |   53 +-
 .../hbase/io/hfile/HFileDataBlockEncoder.java   |    8 +-
 .../io/hfile/HFileDataBlockEncoderImpl.java     |    8 +-
 .../hbase/io/hfile/HFilePrettyPrinter.java      |  168 +-
 .../hadoop/hbase/io/hfile/HFileReaderV2.java    |   86 +-
 .../hadoop/hbase/io/hfile/HFileReaderV3.java    |    7 +-
 .../hadoop/hbase/io/hfile/HFileScanner.java     |    9 +-
 .../hadoop/hbase/io/hfile/HFileWriterV2.java    |   99 +-
 .../hadoop/hbase/io/hfile/HFileWriterV3.java    |   55 +-
 .../hbase/io/hfile/InlineBlockWriter.java       |    2 +-
 .../hbase/io/hfile/InvalidHFileException.java   |    2 +-
 .../hadoop/hbase/io/hfile/LruBlockCache.java    |  119 +-
 .../hadoop/hbase/io/hfile/LruCachedBlock.java   |    2 +-
 .../hbase/io/hfile/LruCachedBlockQueue.java     |    2 +-
 .../hbase/io/hfile/NoOpDataBlockEncoder.java    |   23 +-
 .../hbase/io/hfile/ResizableBlockCache.java     |    2 +-
 .../hbase/io/hfile/bucket/BucketAllocator.java  |    2 +-
 .../hfile/bucket/BucketAllocatorException.java  |    4 +-
 .../hbase/io/hfile/bucket/BucketCache.java      |   31 +-
 .../hbase/io/hfile/bucket/BucketCacheStats.java |    2 +-
 .../io/hfile/bucket/ByteBufferIOEngine.java     |    4 +-
 .../io/hfile/bucket/CacheFullException.java     |    2 +-
 .../hbase/io/hfile/bucket/CachedEntryQueue.java |    2 +-
 .../hbase/io/hfile/bucket/FileIOEngine.java     |    2 +-
 .../hadoop/hbase/io/hfile/bucket/IOEngine.java  |    4 +-
 .../hbase/io/hfile/bucket/UniqueIndexMap.java   |    2 +-
 .../hbase/io/hfile/bucket/package-info.java     |    2 +-
 .../hbase/ipc/BalancedQueueRpcExecutor.java     |   70 +-
 .../apache/hadoop/hbase/ipc/BufferChain.java    |    4 +-
 .../org/apache/hadoop/hbase/ipc/CallRunner.java |   16 +-
 .../org/apache/hadoop/hbase/ipc/Delayable.java  |    2 +-
 .../hbase/ipc/EmptyServiceNameException.java    |    2 +-
 .../hadoop/hbase/ipc/HBaseRPCErrorHandler.java  |   12 +-
 .../hadoop/hbase/ipc/MetricsHBaseServer.java    |    4 +-
 .../ipc/MetricsHBaseServerWrapperImpl.java      |   16 +-
 .../hadoop/hbase/ipc/PriorityFunction.java      |    7 +-
 .../hadoop/hbase/ipc/RWQueueRpcExecutor.java    |   38 +-
 .../apache/hadoop/hbase/ipc/RequestContext.java |    6 +-
 .../apache/hadoop/hbase/ipc/RpcCallContext.java |    2 +-
 .../apache/hadoop/hbase/ipc/RpcExecutor.java    |   94 +-
 .../apache/hadoop/hbase/ipc/RpcScheduler.java   |    4 +-
 .../hadoop/hbase/ipc/RpcSchedulerContext.java   |    4 +-
 .../org/apache/hadoop/hbase/ipc/RpcServer.java  |  100 +-
 .../hadoop/hbase/ipc/RpcServerInterface.java    |    8 +-
 .../hadoop/hbase/ipc/SimpleRpcScheduler.java    |   37 +-
 .../hbase/ipc/UnknownServiceException.java      |    2 +-
 .../org/apache/hadoop/hbase/mapred/Driver.java  |   13 +-
 .../hadoop/hbase/mapred/GroupingTableMap.java   |    5 +-
 .../hadoop/hbase/mapred/HRegionPartitioner.java |   28 +-
 .../hadoop/hbase/mapred/IdentityTableMap.java   |    5 +-
 .../hbase/mapred/IdentityTableReduce.java       |    5 +-
 .../apache/hadoop/hbase/mapred/RowCounter.java  |    8 +-
 .../hadoop/hbase/mapred/TableInputFormat.java   |   12 +-
 .../hbase/mapred/TableInputFormatBase.java      |   17 +-
 .../apache/hadoop/hbase/mapred/TableMap.java    |    5 +-
 .../hadoop/hbase/mapred/TableMapReduceUtil.java |   53 +-
 .../hadoop/hbase/mapred/TableOutputFormat.java  |   75 +-
 .../hadoop/hbase/mapred/TableRecordReader.java  |   11 +-
 .../hbase/mapred/TableRecordReaderImpl.java     |   13 +-
 .../apache/hadoop/hbase/mapred/TableReduce.java |    5 +-
 .../hbase/mapred/TableSnapshotInputFormat.java  |    4 +-
 .../apache/hadoop/hbase/mapred/TableSplit.java  |   15 +-
 .../hadoop/hbase/mapred/package-info.java       |    2 +-
 .../hadoop/hbase/mapreduce/CellCounter.java     |   52 +-
 .../hadoop/hbase/mapreduce/CellCreator.java     |   39 +-
 .../hadoop/hbase/mapreduce/CopyTable.java       |  164 +-
 .../DefaultVisibilityExpressionResolver.java    |   82 +-
 .../apache/hadoop/hbase/mapreduce/Driver.java   |    7 +-
 .../apache/hadoop/hbase/mapreduce/Export.java   |   10 +-
 .../hbase/mapreduce/GroupingTableMapper.java    |    4 +-
 .../hbase/mapreduce/HFileOutputFormat.java      |   39 +-
 .../hbase/mapreduce/HFileOutputFormat2.java     |  144 +-
 .../hadoop/hbase/mapreduce/HLogInputFormat.java |  237 +-
 .../hbase/mapreduce/HRegionPartitioner.java     |   23 +-
 .../hbase/mapreduce/IdentityTableMapper.java    |    4 +-
 .../hbase/mapreduce/IdentityTableReducer.java   |   11 +-
 .../apache/hadoop/hbase/mapreduce/Import.java   |  100 +-
 .../hadoop/hbase/mapreduce/ImportTsv.java       |  302 +-
 .../hadoop/hbase/mapreduce/JarFinder.java       |  176 +
 .../hbase/mapreduce/KeyValueSerialization.java  |    2 +-
 .../hbase/mapreduce/KeyValueSortReducer.java    |    6 +-
 .../hbase/mapreduce/LoadIncrementalHFiles.java  |  199 +-
 .../hbase/mapreduce/MultiTableInputFormat.java  |    4 +-
 .../mapreduce/MultiTableInputFormatBase.java    |  135 +-
 .../hbase/mapreduce/MultiTableOutputFormat.java |   50 +-
 .../hadoop/hbase/mapreduce/PutCombiner.java     |   13 +-
 .../hadoop/hbase/mapreduce/PutSortReducer.java  |    4 +-
 .../hbase/mapreduce/ResultSerialization.java    |    2 +-
 .../hadoop/hbase/mapreduce/RowCounter.java      |   28 +-
 .../mapreduce/SimpleTotalOrderPartitioner.java  |    4 +-
 .../hbase/mapreduce/TableInputFormat.java       |   81 +-
 .../hbase/mapreduce/TableInputFormatBase.java   |  545 +-
 .../hbase/mapreduce/TableMapReduceUtil.java     |  183 +-
 .../hadoop/hbase/mapreduce/TableMapper.java     |    6 +-
 .../hbase/mapreduce/TableOutputCommitter.java   |    4 +-
 .../hbase/mapreduce/TableOutputFormat.java      |   62 +-
 .../hbase/mapreduce/TableRecordReader.java      |   21 +-
 .../hbase/mapreduce/TableRecordReaderImpl.java  |   12 +-
 .../hadoop/hbase/mapreduce/TableReducer.java    |    4 +-
 .../mapreduce/TableSnapshotInputFormat.java     |   21 +-
 .../mapreduce/TableSnapshotInputFormatImpl.java |    4 +-
 .../hadoop/hbase/mapreduce/TableSplit.java      |   15 +-
 .../hadoop/hbase/mapreduce/TextSortReducer.java |   28 +-
 .../hbase/mapreduce/TsvImporterMapper.java      |   30 +-
 .../hbase/mapreduce/TsvImporterTextMapper.java  |    4 +-
 .../mapreduce/VisibilityExpressionResolver.java |    6 +-
 .../hadoop/hbase/mapreduce/WALInputFormat.java  |  287 +
 .../hadoop/hbase/mapreduce/WALPlayer.java       |  164 +-
 .../mapreduce/hadoopbackport/JarFinder.java     |  175 -
 .../hadoop/hbase/mapreduce/package-info.java    |    2 +-
 .../replication/VerifyReplication.java          |   98 +-
 .../hbase/master/ActiveMasterManager.java       |   10 +-
 .../hadoop/hbase/master/AssignCallable.java     |    8 +-
 .../hadoop/hbase/master/AssignmentListener.java |    3 +-
 .../hadoop/hbase/master/AssignmentManager.java  | 1217 ++-
 .../master/AssignmentVerificationReport.java    |    2 +-
 .../hadoop/hbase/master/BulkAssigner.java       |    2 +-
 .../apache/hadoop/hbase/master/BulkReOpen.java  |    2 +-
 .../hadoop/hbase/master/CatalogJanitor.java     |   28 +-
 .../hbase/master/ClusterStatusPublisher.java    |   88 +-
 .../apache/hadoop/hbase/master/DeadServer.java  |    2 +-
 .../master/ExpiredMobFileCleanerChore.java      |    9 +-
 .../hbase/master/GeneralBulkAssigner.java       |    2 +-
 .../org/apache/hadoop/hbase/master/HMaster.java |  647 +-
 .../hadoop/hbase/master/HMasterCommandLine.java |   29 +-
 .../hadoop/hbase/master/LoadBalancer.java       |   14 +-
 .../hbase/master/MasterCoprocessorHost.java     |  223 +-
 .../hadoop/hbase/master/MasterDumpServlet.java  |   31 +-
 .../hadoop/hbase/master/MasterFileSystem.java   |   76 +-
 .../hadoop/hbase/master/MasterRpcServices.java  |  184 +-
 .../hadoop/hbase/master/MasterServices.java     |   29 +-
 .../hbase/master/MasterStatusServlet.java       |    7 +-
 .../hadoop/hbase/master/MetricsMaster.java      |    4 +-
 .../hbase/master/MetricsMasterFileSystem.java   |    4 +-
 .../hbase/master/MetricsMasterWrapperImpl.java  |    2 +-
 .../hbase/master/MobFileCompactionChore.java    |   10 +-
 .../apache/hadoop/hbase/master/RackManager.java |    2 +-
 .../hbase/master/RegionPlacementMaintainer.java |   50 +-
 .../apache/hadoop/hbase/master/RegionPlan.java  |    4 +-
 .../hadoop/hbase/master/RegionStateStore.java   |   15 +-
 .../hadoop/hbase/master/RegionStates.java       |  310 +-
 .../hadoop/hbase/master/ServerListener.java     |    3 +-
 .../hadoop/hbase/master/ServerManager.java      |  166 +-
 .../SnapshotOfRegionAssignmentFromMeta.java     |   18 +-
 .../hadoop/hbase/master/SnapshotSentinel.java   |    4 +-
 .../hadoop/hbase/master/SplitLogManager.java    |   86 +-
 .../hadoop/hbase/master/TableLockManager.java   |    2 +-
 .../hbase/master/TableNamespaceManager.java     |  101 +-
 .../hadoop/hbase/master/TableStateManager.java  |  223 +
 .../hadoop/hbase/master/UnAssignCallable.java   |    2 +-
 .../hbase/master/balancer/BalancerChore.java    |   16 +-
 .../hbase/master/balancer/BaseLoadBalancer.java |  508 +-
 .../hbase/master/balancer/ClusterLoadState.java |   22 +-
 .../master/balancer/ClusterStatusChore.java     |   15 +-
 .../balancer/FavoredNodeAssignmentHelper.java   |   36 +-
 .../balancer/FavoredNodeLoadBalancer.java       |   31 +-
 .../hbase/master/balancer/FavoredNodesPlan.java |    2 +-
 .../master/balancer/LoadBalancerFactory.java    |    2 +-
 .../hbase/master/balancer/ServerAndLoad.java    |    2 +-
 .../master/balancer/SimpleLoadBalancer.java     |   59 +-
 .../master/balancer/StochasticLoadBalancer.java |  128 +-
 .../cleaner/BaseHFileCleanerDelegate.java       |    2 +-
 .../master/cleaner/BaseLogCleanerDelegate.java  |    2 +-
 .../hbase/master/cleaner/CleanerChore.java      |   18 +-
 .../master/cleaner/FileCleanerDelegate.java     |    2 +-
 .../hbase/master/cleaner/HFileCleaner.java      |    2 +-
 .../hbase/master/cleaner/HFileLinkCleaner.java  |   27 +-
 .../hadoop/hbase/master/cleaner/LogCleaner.java |    8 +-
 .../master/cleaner/TimeToLiveHFileCleaner.java  |    5 +-
 .../master/cleaner/TimeToLiveLogCleaner.java    |    7 +-
 .../master/handler/CreateTableHandler.java      |  129 +-
 .../master/handler/DeleteTableHandler.java      |  186 +-
 .../master/handler/DisableTableHandler.java     |   36 +-
 .../handler/DispatchMergingRegionHandler.java   |    4 +-
 .../master/handler/EnableTableHandler.java      |   93 +-
 .../hbase/master/handler/LogReplayHandler.java  |   10 +-
 .../handler/MetaServerShutdownHandler.java      |   12 +-
 .../master/handler/ModifyTableHandler.java      |   42 +-
 .../master/handler/ServerShutdownHandler.java   |   92 +-
 .../master/handler/TableAddFamilyHandler.java   |    8 +-
 .../handler/TableDeleteFamilyHandler.java       |    4 +-
 .../hbase/master/handler/TableEventHandler.java |   62 +-
 .../handler/TableModifyFamilyHandler.java       |    4 +-
 .../hbase/master/handler/TotesHRegionInfo.java  |    5 +-
 .../master/handler/TruncateTableHandler.java    |   96 +-
 .../master/snapshot/CloneSnapshotHandler.java   |    9 +-
 .../snapshot/DisabledTableSnapshotHandler.java  |    6 +-
 .../snapshot/EnabledTableSnapshotHandler.java   |    4 +-
 .../master/snapshot/MasterSnapshotVerifier.java |   17 +-
 .../master/snapshot/RestoreSnapshotHandler.java |   13 +-
 .../master/snapshot/SnapshotFileCache.java      |   84 +-
 .../master/snapshot/SnapshotHFileCleaner.java   |   21 +-
 .../master/snapshot/SnapshotLogCleaner.java     |   26 +-
 .../hbase/master/snapshot/SnapshotManager.java  |   97 +-
 .../master/snapshot/TakeSnapshotHandler.java    |   14 +-
 .../hbase/metrics/ExactCounterMetric.java       |  155 -
 .../hadoop/hbase/metrics/MetricsMBeanBase.java  |  243 -
 .../hadoop/hbase/metrics/MetricsRate.java       |   89 -
 .../hadoop/hbase/metrics/MetricsString.java     |   59 -
 .../PersistentMetricsTimeVaryingRate.java       |  141 -
 .../metrics/histogram/MetricsHistogram.java     |  240 -
 .../hbase/migration/NamespaceUpgrade.java       |  574 --
 .../hadoop/hbase/mob/DefaultMobCompactor.java   |   56 +-
 .../org/apache/hadoop/hbase/mob/MobUtils.java   |    2 +-
 .../PartitionedMobFileCompactor.java            |    2 +-
 .../hadoop/hbase/mob/mapreduce/SweepMapper.java |   14 +-
 .../hadoop/hbase/monitoring/LogMonitoring.java  |    2 +-
 .../MemoryBoundedLogMessageBuffer.java          |    2 +-
 .../hbase/monitoring/MonitoredRPCHandler.java   |    2 +-
 .../monitoring/MonitoredRPCHandlerImpl.java     |    4 +-
 .../hadoop/hbase/monitoring/MonitoredTask.java  |    2 +-
 .../hbase/monitoring/MonitoredTaskImpl.java     |    2 +-
 .../hbase/monitoring/StateDumpServlet.java      |   10 +-
 .../hadoop/hbase/monitoring/TaskMonitor.java    |    2 +-
 .../hbase/monitoring/ThreadMonitoring.java      |    2 +-
 .../hbase/namespace/NamespaceAuditor.java       |  166 +
 .../hbase/namespace/NamespaceStateManager.java  |  227 +
 .../namespace/NamespaceTableAndRegionInfo.java  |  119 +
 .../hbase/procedure/MasterProcedureManager.java |   29 +-
 .../procedure/MasterProcedureManagerHost.java   |    5 +-
 .../hadoop/hbase/procedure/Procedure.java       |    2 +-
 .../hbase/procedure/ProcedureCoordinator.java   |    2 +-
 .../procedure/ProcedureCoordinatorRpcs.java     |    2 +-
 .../hbase/procedure/ProcedureManager.java       |    4 +-
 .../hbase/procedure/ProcedureManagerHost.java   |    4 +-
 .../hadoop/hbase/procedure/ProcedureMember.java |    2 +-
 .../hbase/procedure/ProcedureMemberRpcs.java    |    3 +-
 .../procedure/RegionServerProcedureManager.java |    4 +-
 .../RegionServerProcedureManagerHost.java       |    4 +-
 .../hbase/procedure/SubprocedureFactory.java    |    2 +-
 .../procedure/ZKProcedureCoordinatorRpcs.java   |    3 +-
 .../hbase/procedure/ZKProcedureMemberRpcs.java  |    2 +-
 .../hadoop/hbase/procedure/ZKProcedureUtil.java |    4 +-
 .../procedure/flush/FlushTableSubprocedure.java |    5 +-
 .../flush/MasterFlushTableProcedureManager.java |   26 +-
 .../RegionServerFlushTableProcedureManager.java |    5 +-
 .../hbase/protobuf/ReplicationProtbufUtil.java  |   52 +-
 .../hbase/quotas/DefaultOperationQuota.java     |  144 +
 .../hadoop/hbase/quotas/MasterQuotaManager.java |  484 +
 .../hadoop/hbase/quotas/NoopOperationQuota.java |   84 +
 .../hadoop/hbase/quotas/NoopQuotaLimiter.java   |   90 +
 .../hadoop/hbase/quotas/OperationQuota.java     |  128 +
 .../apache/hadoop/hbase/quotas/QuotaCache.java  |  326 +
 .../hadoop/hbase/quotas/QuotaLimiter.java       |   80 +
 .../hbase/quotas/QuotaLimiterFactory.java       |   39 +
 .../apache/hadoop/hbase/quotas/QuotaState.java  |  119 +
 .../apache/hadoop/hbase/quotas/QuotaUtil.java   |  311 +
 .../apache/hadoop/hbase/quotas/RateLimiter.java |  181 +
 .../hbase/quotas/RegionServerQuotaManager.java  |  199 +
 .../hbase/quotas/RegionStateListener.java       |   54 +
 .../hbase/quotas/RegionStateListener.java~HEAD  |   54 +
 .../quotas/RegionStateListener.java~HEAD_0      |   54 +
 .../quotas/RegionStateListener.java~jon_master  |   54 +
 .../quotas/RegionStateListener.java~master      |   54 +
 .../hadoop/hbase/quotas/TimeBasedLimiter.java   |  206 +
 .../hadoop/hbase/quotas/UserQuotaState.java     |  202 +
 .../AnnotationReadingPriorityFunction.java      |   10 +-
 .../hbase/regionserver/BaseRowProcessor.java    |    5 +
 .../hbase/regionserver/CellSkipListSet.java     |  185 +
 .../regionserver/ChangedReadersObserver.java    |    2 +-
 .../hadoop/hbase/regionserver/ColumnCount.java  |    2 +-
 .../hbase/regionserver/ColumnTracker.java       |    2 +-
 .../hbase/regionserver/CompactSplitThread.java  |  162 +-
 .../hbase/regionserver/CompactionRequestor.java |    2 +-
 .../hbase/regionserver/CompactionTool.java      |   26 +-
 .../ConstantSizeRegionSplitPolicy.java          |    5 +-
 .../regionserver/DefaultHeapMemoryTuner.java    |    2 +-
 .../hbase/regionserver/DefaultMemStore.java     |  364 +-
 .../hbase/regionserver/DefaultStoreEngine.java  |   23 +-
 .../regionserver/DefaultStoreFileManager.java   |   27 +-
 .../hbase/regionserver/DefaultStoreFlusher.java |   13 +-
 .../hbase/regionserver/DeleteTracker.java       |    2 +-
 .../DelimitedKeyPrefixRegionSplitPolicy.java    |   24 +-
 .../regionserver/DisabledRegionSplitPolicy.java |    4 +
 .../regionserver/ExplicitColumnTracker.java     |    2 +-
 .../regionserver/FavoredNodesForRegion.java     |    2 +-
 .../regionserver/FlushAllStoresPolicy.java      |   35 +
 .../regionserver/FlushLargeStoresPolicy.java    |  108 +
 .../hadoop/hbase/regionserver/FlushPolicy.java  |   49 +
 .../hbase/regionserver/FlushPolicyFactory.java  |   76 +
 .../regionserver/FlushRequestListener.java      |    2 +-
 .../hbase/regionserver/FlushRequester.java      |   17 +-
 .../GetClosestRowBeforeTracker.java             |   27 +-
 .../hadoop/hbase/regionserver/HMobStore.java    |    8 +-
 .../hadoop/hbase/regionserver/HRegion.java      | 1653 ++--
 .../hbase/regionserver/HRegionFileSystem.java   |   70 +-
 .../hbase/regionserver/HRegionServer.java       |  778 +-
 .../regionserver/HRegionServerCommandLine.java  |    2 +-
 .../hadoop/hbase/regionserver/HStore.java       |  214 +-
 .../hbase/regionserver/HeapMemStoreLAB.java     |    2 +-
 .../hbase/regionserver/HeapMemoryManager.java   |   81 +-
 .../hbase/regionserver/HeapMemoryTuner.java     |    8 +-
 ...IncreasingToUpperBoundRegionSplitPolicy.java |    2 +
 .../hadoop/hbase/regionserver/InternalScan.java |   21 +-
 .../hbase/regionserver/InternalScanner.java     |    2 +-
 .../KeyPrefixRegionSplitPolicy.java             |   52 +-
 .../hadoop/hbase/regionserver/KeyValueHeap.java |    2 +-
 .../hbase/regionserver/KeyValueScanner.java     |    2 +-
 .../hbase/regionserver/KeyValueSkipListSet.java |  184 -
 .../hbase/regionserver/LastSequenceId.java      |    8 +-
 .../hbase/regionserver/LeaseListener.java       |    2 +-
 .../hadoop/hbase/regionserver/Leases.java       |    4 +-
 .../hadoop/hbase/regionserver/LogRoller.java    |  117 +-
 .../hadoop/hbase/regionserver/LruHashMap.java   |    2 +-
 .../hadoop/hbase/regionserver/MemStore.java     |    2 +-
 .../hbase/regionserver/MemStoreChunkPool.java   |    7 +-
 .../hbase/regionserver/MemStoreFlusher.java     |  121 +-
 .../hadoop/hbase/regionserver/MemStoreLAB.java  |    2 +-
 .../hbase/regionserver/MemStoreSnapshot.java    |    2 +-
 .../hbase/regionserver/MetaLogRoller.java       |   38 -
 .../hbase/regionserver/MetricsRegion.java       |    2 +-
 .../hbase/regionserver/MetricsRegionServer.java |   20 +-
 .../MetricsRegionServerWrapperImpl.java         |   71 +-
 .../regionserver/MetricsRegionWrapperImpl.java  |    2 +
 .../MiniBatchOperationInProgress.java           |   13 +-
 .../regionserver/MultiRowMutationProcessor.java |   13 +-
 .../MultiVersionConsistencyControl.java         |    2 +-
 .../hbase/regionserver/NoOpHeapMemoryTuner.java |    2 +-
 .../regionserver/NonLazyKeyValueScanner.java    |    2 +-
 .../NonReversedNonLazyKeyValueScanner.java      |    2 +-
 .../hbase/regionserver/OnlineRegions.java       |    2 +-
 .../hbase/regionserver/OperationStatus.java     |    2 +-
 .../hbase/regionserver/RSDumpServlet.java       |   33 +-
 .../hbase/regionserver/RSRpcServices.java       |  420 +-
 .../hbase/regionserver/RSStatusServlet.java     |   10 +-
 .../regionserver/RegionCoprocessorHost.java     |  207 +-
 .../hbase/regionserver/RegionMergeRequest.java  |    2 +-
 .../regionserver/RegionMergeTransaction.java    |    4 +-
 .../hbase/regionserver/RegionScanner.java       |   11 +-
 .../regionserver/RegionServerAccounting.java    |    4 +-
 .../RegionServerCoprocessorHost.java            |   90 +
 .../regionserver/RegionServerServices.java      |   45 +-
 .../hbase/regionserver/RegionSplitPolicy.java   |   17 +-
 .../hbase/regionserver/ReplicationService.java  |    2 +-
 .../regionserver/ReplicationSinkService.java    |    2 +-
 .../regionserver/ReplicationSourceService.java  |    2 +-
 .../regionserver/ReversedKeyValueHeap.java      |    2 +-
 .../regionserver/ReversedRegionScannerImpl.java |    4 +-
 .../regionserver/ReversedStoreScanner.java      |    2 +-
 .../hadoop/hbase/regionserver/RowProcessor.java |   10 +-
 .../hbase/regionserver/RowTooBigException.java  |   13 +-
 .../hbase/regionserver/RpcSchedulerFactory.java |   10 +
 .../hbase/regionserver/ScanDeleteTracker.java   |    2 +-
 .../hadoop/hbase/regionserver/ScanInfo.java     |    9 +-
 .../hbase/regionserver/ScanQueryMatcher.java    |   47 +-
 .../hadoop/hbase/regionserver/ScanType.java     |    7 +-
 .../regionserver/ScanWildcardColumnTracker.java |    2 +-
 .../hadoop/hbase/regionserver/SequenceId.java   |    2 +-
 .../hbase/regionserver/ServerNonceManager.java  |   20 +-
 .../hadoop/hbase/regionserver/ShutdownHook.java |    2 +-
 .../regionserver/SimpleRpcSchedulerFactory.java |   32 +-
 .../hbase/regionserver/SplitLogWorker.java      |   32 +-
 .../hadoop/hbase/regionserver/SplitRequest.java |   30 +-
 .../hbase/regionserver/SplitTransaction.java    |  180 +-
 .../apache/hadoop/hbase/regionserver/Store.java |   34 +-
 .../regionserver/StoreConfigInformation.java    |    4 +-
 .../hadoop/hbase/regionserver/StoreEngine.java  |    2 +-
 .../hadoop/hbase/regionserver/StoreFile.java    |  116 +-
 .../hbase/regionserver/StoreFileInfo.java       |  110 +-
 .../hbase/regionserver/StoreFileManager.java    |   12 +-
 .../hbase/regionserver/StoreFileScanner.java    |    4 +-
 .../hbase/regionserver/StoreFlushContext.java   |    2 +-
 .../hadoop/hbase/regionserver/StoreFlusher.java |    7 +-
 .../hadoop/hbase/regionserver/StoreScanner.java |   28 +-
 .../hadoop/hbase/regionserver/StoreUtils.java   |    4 +-
 .../regionserver/StorefileRefresherChore.java   |   22 +-
 .../regionserver/StripeMultiFileWriter.java     |   90 +-
 .../hbase/regionserver/StripeStoreConfig.java   |    2 +-
 .../hbase/regionserver/StripeStoreEngine.java   |   11 +-
 .../regionserver/StripeStoreFileManager.java    |   65 +-
 .../hbase/regionserver/StripeStoreFlusher.java  |    2 +
 .../hbase/regionserver/TimeRangeTracker.java    |   88 +-
 .../regionserver/UnexpectedStateException.java  |    2 +-
 .../compactions/CompactionConfiguration.java    |   70 +-
 .../compactions/CompactionContext.java          |    5 +-
 .../compactions/CompactionPolicy.java           |   12 +-
 .../compactions/CompactionProgress.java         |    2 +-
 .../compactions/CompactionRequest.java          |    4 +-
 .../CompactionThroughputController.java         |   52 +
 .../CompactionThroughputControllerFactory.java  |   61 +
 .../regionserver/compactions/Compactor.java     |  151 +-
 .../compactions/CurrentHourProvider.java        |    2 +-
 .../compactions/DefaultCompactor.java           |   69 +-
 .../compactions/ExploringCompactionPolicy.java  |    2 +-
 .../NoLimitCompactionThroughputController.java  |   66 +
 .../regionserver/compactions/OffPeakHours.java  |    6 +-
 ...sureAwareCompactionThroughputController.java |  263 +
 .../compactions/RatioBasedCompactionPolicy.java |   45 +-
 .../compactions/StripeCompactionPolicy.java     |   19 +-
 .../compactions/StripeCompactor.java            |   30 +-
 .../regionserver/handler/CloseMetaHandler.java  |    2 +-
 .../handler/CloseRegionHandler.java             |    4 +-
 .../handler/HLogSplitterHandler.java            |  106 -
 .../regionserver/handler/OpenMetaHandler.java   |    2 +-
 .../regionserver/handler/OpenRegionHandler.java |    2 +-
 .../handler/ParallelSeekHandler.java            |    2 +-
 .../handler/WALSplitterHandler.java             |  106 +
 .../snapshot/FlushSnapshotSubprocedure.java     |    4 +-
 .../snapshot/RegionServerSnapshotManager.java   |    9 +-
 .../regionserver/wal/CompressionContext.java    |   13 +-
 .../hbase/regionserver/wal/Compressor.java      |   20 +-
 .../hadoop/hbase/regionserver/wal/FSHLog.java   |  951 +-
 .../hbase/regionserver/wal/FSWALEntry.java      |   42 +-
 .../hadoop/hbase/regionserver/wal/HLog.java     |  444 -
 .../hbase/regionserver/wal/HLogFactory.java     |  205 -
 .../hadoop/hbase/regionserver/wal/HLogKey.java  |  432 +-
 .../regionserver/wal/HLogPrettyPrinter.java     |  327 +-
 .../hbase/regionserver/wal/HLogSplitter.java    | 2073 ----
 .../hadoop/hbase/regionserver/wal/HLogUtil.java |  304 -
 .../regionserver/wal/KeyValueCompression.java   |    2 +
 .../hbase/regionserver/wal/MetricsWAL.java      |   30 +-
 .../regionserver/wal/MetricsWALEditsReplay.java |    3 +-
 .../regionserver/wal/ProtobufLogReader.java     |   44 +-
 .../regionserver/wal/ProtobufLogWriter.java     |   30 +-
 .../hbase/regionserver/wal/ReaderBase.java      |   31 +-
 .../hbase/regionserver/wal/RingBufferTruck.java |    6 +-
 .../wal/SecureProtobufLogReader.java            |    8 +-
 .../wal/SecureProtobufLogWriter.java            |   13 +-
 .../regionserver/wal/SecureWALCellCodec.java    |   53 +-
 .../regionserver/wal/SequenceFileLogReader.java |   25 +-
 .../hbase/regionserver/wal/SyncFuture.java      |    6 +-
 .../regionserver/wal/WALActionsListener.java    |   82 +-
 .../hbase/regionserver/wal/WALCellCodec.java    |   56 +-
 .../regionserver/wal/WALCoprocessorHost.java    |   69 +-
 .../hadoop/hbase/regionserver/wal/WALEdit.java  |   95 +-
 .../regionserver/wal/WALEditsReplaySink.java    |   43 +-
 .../hadoop/hbase/regionserver/wal/WALUtil.java  |  140 +
 .../hbase/regionserver/wal/WriterBase.java      |    6 +-
 .../replication/BaseReplicationEndpoint.java    |    2 +-
 .../hbase/replication/ChainWALEntryFilter.java  |    9 +-
 .../replication/HBaseReplicationEndpoint.java   |   16 +-
 .../hbase/replication/ReplicationEndpoint.java  |   10 +-
 .../hbase/replication/ScopeWALEntryFilter.java  |   22 +-
 .../replication/SystemTableWALEntryFilter.java  |    4 +-
 .../replication/TableCfWALEntryFilter.java      |   24 +-
 .../hbase/replication/WALEntryFilter.java       |   12 +-
 .../master/ReplicationLogCleaner.java           |   31 +-
 .../HBaseInterClusterReplicationEndpoint.java   |   12 +-
 .../replication/regionserver/MetricsSink.java   |   18 +-
 .../replication/regionserver/MetricsSource.java |   82 +-
 .../RegionReplicaReplicationEndpoint.java       |   37 +-
 .../replication/regionserver/Replication.java   |   47 +-
 .../ReplicationHLogReaderManager.java           |  144 -
 .../regionserver/ReplicationSink.java           |   21 +-
 .../regionserver/ReplicationSinkManager.java    |    4 +-
 .../regionserver/ReplicationSource.java         |   59 +-
 .../ReplicationSourceInterface.java             |    2 +-
 .../regionserver/ReplicationSourceManager.java  |  148 +-
 .../regionserver/ReplicationSyncUp.java         |   20 +-
 .../regionserver/ReplicationThrottler.java      |    2 +-
 .../ReplicationWALReaderManager.java            |  145 +
 .../org/apache/hadoop/hbase/rest/Constants.java |   78 -
 .../hadoop/hbase/rest/ExistsResource.java       |   75 -
 .../apache/hadoop/hbase/rest/MetricsREST.java   |  103 -
 .../hadoop/hbase/rest/MultiRowResource.java     |  108 -
 .../hbase/rest/ProtobufMessageHandler.java      |   46 -
 .../hbase/rest/ProtobufStreamingUtil.java       |  102 -
 .../apache/hadoop/hbase/rest/RESTServer.java    |  253 -
 .../apache/hadoop/hbase/rest/RESTServlet.java   |  142 -
 .../hadoop/hbase/rest/RESTServletContainer.java |   76 -
 .../hadoop/hbase/rest/RegionsResource.java      |  104 -
 .../apache/hadoop/hbase/rest/ResourceBase.java  |   93 -
 .../hadoop/hbase/rest/ResourceConfig.java       |   31 -
 .../hadoop/hbase/rest/ResultGenerator.java      |   50 -
 .../apache/hadoop/hbase/rest/RootResource.java  |  106 -
 .../apache/hadoop/hbase/rest/RowResource.java   |  598 --
 .../hadoop/hbase/rest/RowResultGenerator.java   |  122 -
 .../org/apache/hadoop/hbase/rest/RowSpec.java   |  408 -
 .../hbase/rest/ScannerInstanceResource.java     |  201 -
 .../hadoop/hbase/rest/ScannerResource.java      |  164 -
 .../hbase/rest/ScannerResultGenerator.java      |  191 -
 .../hadoop/hbase/rest/SchemaResource.java       |  252 -
 .../rest/StorageClusterStatusResource.java      |  109 -
 .../rest/StorageClusterVersionResource.java     |   79 -
 .../apache/hadoop/hbase/rest/TableResource.java |  196 -
 .../hadoop/hbase/rest/TableScanResource.java    |  168 -
 .../hadoop/hbase/rest/VersionResource.java      |  104 -
 .../apache/hadoop/hbase/rest/client/Client.java |  506 -
 .../hadoop/hbase/rest/client/Cluster.java       |  103 -
 .../hadoop/hbase/rest/client/RemoteAdmin.java   |  401 -
 .../hadoop/hbase/rest/client/RemoteHTable.java  |  856 --
 .../hadoop/hbase/rest/client/Response.java      |  155 -
 .../hadoop/hbase/rest/filter/AuthFilter.java    |   82 -
 .../hbase/rest/filter/GZIPRequestStream.java    |   58 -
 .../hbase/rest/filter/GZIPRequestWrapper.java   |   52 -
 .../hbase/rest/filter/GZIPResponseStream.java   |   78 -
 .../hbase/rest/filter/GZIPResponseWrapper.java  |  147 -
 .../hadoop/hbase/rest/filter/GzipFilter.java    |   84 -
 .../hadoop/hbase/rest/model/CellModel.java      |  207 -
 .../hadoop/hbase/rest/model/CellSetModel.java   |  152 -
 .../hbase/rest/model/ColumnSchemaModel.java     |  241 -
 .../hadoop/hbase/rest/model/RowModel.java       |  151 -
 .../hadoop/hbase/rest/model/ScannerModel.java   |  852 --
 .../rest/model/StorageClusterStatusModel.java   |  790 --
 .../rest/model/StorageClusterVersionModel.java  |   78 -
 .../hadoop/hbase/rest/model/TableInfoModel.java |  159 -
 .../hadoop/hbase/rest/model/TableListModel.java |  113 -
 .../hadoop/hbase/rest/model/TableModel.java     |   84 -
 .../hbase/rest/model/TableRegionModel.java      |  196 -
 .../hbase/rest/model/TableSchemaModel.java      |  361 -
 .../hadoop/hbase/rest/model/VersionModel.java   |  209 -
 .../org/apache/hadoop/hbase/rest/package.html   | 1660 ----
 .../rest/protobuf/generated/CellMessage.java    |  731 --
 .../rest/protobuf/generated/CellSetMessage.java | 1521 ---
 .../protobuf/generated/ColumnSchemaMessage.java | 1904 ----
 .../rest/protobuf/generated/ScannerMessage.java | 1578 ----
 .../generated/StorageClusterStatusMessage.java  | 3955 --------
 .../protobuf/generated/TableInfoMessage.java    | 1802 ----
 .../protobuf/generated/TableListMessage.java    |  547 --
 .../protobuf/generated/TableSchemaMessage.java  | 2125 -----
 .../rest/protobuf/generated/VersionMessage.java | 1147 ---
 .../rest/provider/JAXBContextResolver.java      |   89 -
 .../hbase/rest/provider/JacksonProvider.java    |   31 -
 .../consumer/ProtobufMessageBodyConsumer.java   |   88 -
 .../producer/PlainTextMessageBodyProducer.java  |   74 -
 .../producer/ProtobufMessageBodyProducer.java   |   81 -
 .../hbase/security/HBasePolicyProvider.java     |    2 +-
 .../hbase/security/HBaseSaslRpcServer.java      |    2 +-
 .../hadoop/hbase/security/SecurityUtil.java     |   12 +-
 .../security/access/AccessControlFilter.java    |    2 +-
 .../security/access/AccessControlLists.java     |  203 +-
 .../hbase/security/access/AccessController.java |  544 +-
 .../hbase/security/access/AuthResult.java       |   13 +-
 .../CodeToClassAndBackFor96Migration.java       |    2 +-
 .../HbaseObjectWritableFor96Migration.java      |   43 +-
 .../security/access/SecureBulkLoadEndpoint.java |  149 +-
 .../hbase/security/access/TableAuthManager.java |   84 +-
 .../security/access/ZKPermissionWatcher.java    |   51 +-
 .../hbase/security/token/AuthenticationKey.java |    2 +-
 .../token/AuthenticationTokenSecretManager.java |    3 +-
 .../hbase/security/token/FsDelegationToken.java |    6 +-
 .../hbase/security/token/TokenProvider.java     |    2 +-
 .../hadoop/hbase/security/token/TokenUtil.java  |  198 -
 .../hbase/security/token/ZKSecretWatcher.java   |    4 +-
 .../visibility/DefaultScanLabelGenerator.java   |   93 -
 .../DefaultVisibilityLabelServiceImpl.java      |  262 +-
 .../DefinedSetFilterScanLabelGenerator.java     |   98 +
 .../visibility/EnforcingScanLabelGenerator.java |   10 +-
 .../security/visibility/ExpressionExpander.java |    4 +-
 .../security/visibility/ExpressionParser.java   |    2 +-
 .../FeedUserAuthScanLabelGenerator.java         |   76 +
 .../security/visibility/ParseException.java     |    2 +-
 .../security/visibility/ScanLabelGenerator.java |    4 +-
 .../visibility/SimpleScanLabelGenerator.java    |    2 +-
 .../visibility/VisibilityController.java        |  348 +-
 .../visibility/VisibilityExpEvaluator.java      |    4 +-
 .../visibility/VisibilityLabelFilter.java       |    4 +-
 .../VisibilityLabelOrdinalProvider.java         |    9 +-
 .../visibility/VisibilityLabelService.java      |   49 +-
 .../VisibilityLabelServiceManager.java          |    2 +-
 .../visibility/VisibilityLabelsCache.java       |   72 +-
 .../VisibilityReplicationEndpoint.java          |  158 +
 .../visibility/VisibilityScanDeleteTracker.java |    2 +-
 .../security/visibility/VisibilityUtils.java    |   84 +-
 .../visibility/ZKVisibilityLabelWatcher.java    |   21 +-
 .../visibility/expression/ExpressionNode.java   |    2 +-
 .../expression/LeafExpressionNode.java          |    2 +-
 .../expression/NonLeafExpressionNode.java       |    7 +-
 .../visibility/expression/Operator.java         |    2 +-
 .../hadoop/hbase/snapshot/CreateSnapshot.java   |   86 +
 .../hadoop/hbase/snapshot/ExportSnapshot.java   |   55 +-
 .../hbase/snapshot/RestoreSnapshotHelper.java   |   20 +-
 .../snapshot/SnapshotDescriptionUtils.java      |   14 +-
 .../hadoop/hbase/snapshot/SnapshotInfo.java     |   15 +-
 .../hadoop/hbase/snapshot/SnapshotManifest.java |   11 +-
 .../hbase/snapshot/SnapshotManifestV1.java      |    4 +-
 .../hbase/snapshot/SnapshotManifestV2.java      |    2 +-
 .../hbase/snapshot/SnapshotReferenceUtil.java   |   18 +-
 .../org/apache/hadoop/hbase/tool/Canary.java    |  174 +-
 .../hadoop/hbase/util/AbstractHBaseTool.java    |   18 +-
 .../apache/hadoop/hbase/util/BloomFilter.java   |    2 +-
 .../hadoop/hbase/util/BloomFilterBase.java      |    2 +-
 .../hadoop/hbase/util/BloomFilterFactory.java   |    5 +-
 .../hadoop/hbase/util/BloomFilterWriter.java    |    2 +-
 .../util/BoundedPriorityBlockingQueue.java      |    9 +-
 .../hadoop/hbase/util/ByteBloomFilter.java      |   16 +-
 .../hbase/util/CancelableProgressable.java      |    2 +-
 .../hbase/util/CollectionBackedScanner.java     |   40 +-
 .../hadoop/hbase/util/CompoundBloomFilter.java  |    4 +-
 .../hbase/util/CompoundBloomFilterBase.java     |    2 +-
 .../hbase/util/CompoundBloomFilterWriter.java   |    6 +-
 .../hadoop/hbase/util/CompressionTest.java      |   44 +-
 .../hadoop/hbase/util/ConnectionCache.java      |   19 +-
 .../hadoop/hbase/util/DirectMemoryUtils.java    |    4 +-
 .../hadoop/hbase/util/EncryptionTest.java       |  158 +
 .../apache/hadoop/hbase/util/FSHDFSUtils.java   |    6 +-
 .../apache/hadoop/hbase/util/FSMapRUtils.java   |    2 +-
 .../hadoop/hbase/util/FSRegionScanner.java      |   11 +-
 .../FSTableDescriptorMigrationToSubdir.java     |    1 -
 .../hadoop/hbase/util/FSTableDescriptors.java   |  420 +-
 .../org/apache/hadoop/hbase/util/FSUtils.java   |  303 +-
 .../org/apache/hadoop/hbase/util/FSVisitor.java |   42 +-
 .../apache/hadoop/hbase/util/HBaseConfTool.java |    2 +-
 .../org/apache/hadoop/hbase/util/HBaseFsck.java |  588 +-
 .../hadoop/hbase/util/HBaseFsckRepair.java      |   73 +-
 .../hadoop/hbase/util/HFileArchiveUtil.java     |    7 +-
 .../hadoop/hbase/util/HFileV1Detector.java      |    4 +-
 .../org/apache/hadoop/hbase/util/HMerge.java    |   35 +-
 .../apache/hadoop/hbase/util/HashedBytes.java   |    4 +-
 .../hadoop/hbase/util/HttpServerUtil.java       |    2 +-
 .../org/apache/hadoop/hbase/util/IdLock.java    |    2 +-
 .../apache/hadoop/hbase/util/InfoServer.java    |  132 -
 .../org/apache/hadoop/hbase/util/JSONBean.java  |  387 +
 .../hadoop/hbase/util/JVMClusterUtil.java       |    8 +-
 .../hadoop/hbase/util/JvmPauseMonitor.java      |    4 +-
 .../apache/hadoop/hbase/util/JvmVersion.java    |    4 +-
 .../org/apache/hadoop/hbase/util/KeyRange.java  |    2 +-
 .../hbase/util/ManualEnvironmentEdge.java       |    4 +-
 .../util/MapreduceDependencyClasspathTool.java  |    3 +
 .../org/apache/hadoop/hbase/util/Merge.java     |   19 +-
 .../org/apache/hadoop/hbase/util/MetaUtils.java |   41 +-
 .../hadoop/hbase/util/ModifyRegionUtils.java    |   68 +-
 .../hadoop/hbase/util/MultiHConnection.java     |   38 +-
 .../hadoop/hbase/util/MunkresAssignment.java    |    6 +-
 .../org/apache/hadoop/hbase/util/ProtoUtil.java |    2 +-
 .../hadoop/hbase/util/RegionSizeCalculator.java |  106 +-
 .../hbase/util/RegionSplitCalculator.java       |    2 +-
 .../hadoop/hbase/util/RegionSplitter.java       |  658 +-
 .../hadoop/hbase/util/ServerCommandLine.java    |    2 +-
 .../hbase/util/ServerRegionReplicaUtil.java     |    5 +-
 .../hadoop/hbase/util/ShutdownHookManager.java  |    1 -
 .../hadoop/hbase/util/SortedCopyOnWriteSet.java |    3 +-
 .../hadoop/hbase/util/YammerHistogramUtils.java |   80 +
 .../hadoop/hbase/util/ZKDataMigrator.java       |  299 +-
 .../hbase/util/hbck/HFileCorruptionChecker.java |    2 +-
 .../hbase/util/hbck/OfflineMetaRepair.java      |   17 +-
 .../wal/BoundedRegionGroupingProvider.java      |  106 +
 .../hadoop/hbase/wal/DefaultWALProvider.java    |  366 +
 .../hadoop/hbase/wal/DisabledWALProvider.java   |  221 +
 .../hbase/wal/RegionGroupingProvider.java       |  212 +
 .../java/org/apache/hadoop/hbase/wal/WAL.java   |  275 +
 .../org/apache/hadoop/hbase/wal/WALFactory.java |  437 +
 .../org/apache/hadoop/hbase/wal/WALKey.java     |  553 ++
 .../hadoop/hbase/wal/WALPrettyPrinter.java      |  403 +
 .../apache/hadoop/hbase/wal/WALProvider.java    |   82 +
 .../apache/hadoop/hbase/wal/WALSplitter.java    | 2248 +++++
 .../hbase/zookeeper/ClusterStatusTracker.java   |    9 +-
 .../hbase/zookeeper/DeletionListener.java       |    2 +-
 .../hbase/zookeeper/DrainingServerTracker.java  |    2 +-
 .../hbase/zookeeper/LoadBalancerTracker.java    |    2 +-
 .../hbase/zookeeper/MiniZooKeeperCluster.java   |   13 +-
 .../zookeeper/RecoveringRegionWatcher.java      |    4 +-
 .../hbase/zookeeper/RegionServerTracker.java    |   28 +-
 .../hadoop/hbase/zookeeper/ZKServerTool.java    |    7 +-
 .../hadoop/hbase/zookeeper/ZKSplitLog.java      |    9 +-
 .../hbase/zookeeper/ZKTableStateManager.java    |  330 -
 .../hbase/zookeeper/ZooKeeperMainServer.java    |    5 +-
 .../zookeeper/lock/ZKInterProcessLockBase.java  |    2 +-
 .../zookeeper/lock/ZKInterProcessReadLock.java  |    2 +-
 .../lock/ZKInterProcessReadWriteLock.java       |    2 +-
 .../zookeeper/lock/ZKInterProcessWriteLock.java |    2 +-
 .../hadoop/hbase/replication/package.html       |    2 +-
 .../org/apache/hadoop/hbase/thrift/package.html |   14 +-
 .../resources/hbase-webapps/master/snapshot.jsp |   39 +-
 .../resources/hbase-webapps/master/table.jsp    |   71 +-
 .../resources/hbase-webapps/rest/index.html     |   20 -
 .../main/resources/hbase-webapps/rest/rest.jsp  |  117 -
 .../static/jumping-orca_rotated_12percent.png   |  Bin 2401 -> 2401 bytes
 .../org/apache/hadoop/hbase/rest/XMLSchema.xsd  |  181 -
 .../hbase/rest/protobuf/CellMessage.proto       |   25 -
 .../hbase/rest/protobuf/CellSetMessage.proto    |   28 -
 .../rest/protobuf/ColumnSchemaMessage.proto     |   31 -
 .../hbase/rest/protobuf/ScannerMessage.proto    |   32 -
 .../protobuf/StorageClusterStatusMessage.proto  |   51 -
 .../hbase/rest/protobuf/TableInfoMessage.proto  |   30 -
 .../hbase/rest/protobuf/TableListMessage.proto  |   22 -
 .../rest/protobuf/TableSchemaMessage.proto      |   33 -
 .../hbase/rest/protobuf/VersionMessage.proto    |   26 -
 hbase-server/src/test/data/0000000000000016310  |  Bin 0 -> 11776703 bytes
 .../org/apache/hadoop/hbase/HBaseCluster.java   |   24 +-
 .../org/apache/hadoop/hbase/HBaseTestCase.java  |   35 +-
 .../hadoop/hbase/HBaseTestingUtility.java       |  898 +-
 .../hbase/HFilePerformanceEvaluation.java       |  250 +-
 .../hbase/MapFilePerformanceEvaluation.java     |    6 +
 .../apache/hadoop/hbase/MetaMockingUtil.java    |    4 +-
 .../apache/hadoop/hbase/MiniHBaseCluster.java   |   13 +-
 .../hadoop/hbase/MockRegionServerServices.java  |   74 +-
 .../hadoop/hbase/PerformanceEvaluation.java     |  267 +-
 .../hbase/PerformanceEvaluationCommons.java     |   15 +-
 .../hadoop/hbase/ScanPerformanceEvaluation.java |    9 +-
 .../apache/hadoop/hbase/TestAcidGuarantees.java |  411 +
 .../hadoop/hbase/TestCheckTestClasses.java      |    4 +-
 .../hadoop/hbase/TestClusterBootOrder.java      |    4 +-
 .../org/apache/hadoop/hbase/TestCompare.java    |    4 +-
 .../TestFSTableDescriptorForceCreation.java     |   10 +-
 .../hadoop/hbase/TestFullLogReconstruction.java |   11 +-
 .../hadoop/hbase/TestGlobalMemStoreSize.java    |   21 +-
 .../hadoop/hbase/TestHBaseTestingUtility.java   |   53 +-
 .../hadoop/hbase/TestHColumnDescriptor.java     |  115 -
 .../TestHColumnDescriptorDefaultVersions.java   |   13 +-
 .../hbase/TestHDFSBlocksDistribution.java       |    4 +-
 .../hadoop/hbase/TestHRegionLocation.java       |    4 +-
 .../hadoop/hbase/TestHTableDescriptor.java      |  210 -
 .../org/apache/hadoop/hbase/TestIOFencing.java  |   61 +-
 .../hbase/TestIPv6NIOServerSocketChannel.java   |    4 +-
 .../apache/hadoop/hbase/TestInfoServers.java    |   17 +-
 .../apache/hadoop/hbase/TestJMXListener.java    |    4 +-
 .../hadoop/hbase/TestLocalHBaseCluster.java     |    4 +-
 .../hadoop/hbase/TestMetaTableAccessor.java     |  185 +-
 .../hbase/TestMetaTableAccessorNoCluster.java   |   10 +-
 .../hadoop/hbase/TestMetaTableLocator.java      |   28 +-
 .../apache/hadoop/hbase/TestMultiVersions.java  |   70 +-
 .../org/apache/hadoop/hbase/TestNamespace.java  |    7 +-
 .../hadoop/hbase/TestNodeHealthCheckChore.java  |    4 +-
 .../hadoop/hbase/TestPerformanceEvaluation.java |    4 +-
 .../hadoop/hbase/TestRegionRebalancing.java     |  141 +-
 .../apache/hadoop/hbase/TestSerialization.java  |    9 +-
 .../org/apache/hadoop/hbase/TestServerLoad.java |   18 +-
 .../org/apache/hadoop/hbase/TestServerName.java |    4 +-
 .../hadoop/hbase/TestTableDescriptor.java       |   56 +
 .../org/apache/hadoop/hbase/TestZooKeeper.java  |   34 +-
 .../hadoop/hbase/backup/TestHFileArchiving.java |   38 +-
 .../TestZooKeeperTableArchiveClient.java        |   22 +-
 .../hbase/client/HConnectionTestingUtility.java |   28 +-
 .../apache/hadoop/hbase/client/TestAdmin.java   | 1779 ----
 .../apache/hadoop/hbase/client/TestAdmin1.java  | 1315 +++
 .../apache/hadoop/hbase/client/TestAdmin2.java  |  729 ++
 .../hadoop/hbase/client/TestCheckAndMutate.java |  103 +
 .../client/TestClientOperationInterrupt.java    |   12 +-
 .../hadoop/hbase/client/TestClientPushback.java |  153 +
 .../client/TestClientScannerRPCTimeout.java     |   15 +-
 .../hadoop/hbase/client/TestClientTimeouts.java |   78 +-
 .../client/TestCloneSnapshotFromClient.java     |   48 +-
 ...oneSnapshotFromClientWithRegionReplicas.java |    5 +-
 .../hbase/client/TestConnectionUtils.java       |    5 +-
 .../hadoop/hbase/client/TestFastFail.java       |  310 +
 .../hadoop/hbase/client/TestFromClientSide.java |  577 +-
 .../hbase/client/TestFromClientSide3.java       |   46 +-
 .../hbase/client/TestFromClientSideNoCodec.java |    9 +-
 .../TestFromClientSideWithCoprocessor.java      |    5 +-
 .../hbase/client/TestHBaseAdminNoCluster.java   |  262 +-
 .../org/apache/hadoop/hbase/client/TestHCM.java |  313 +-
 .../hbase/client/TestHTableMultiplexer.java     |   70 +-
 .../client/TestHTableMultiplexerFlushCache.java |  115 +
 .../hadoop/hbase/client/TestHTableUtil.java     |  130 -
 .../hbase/client/TestIntraRowPagination.java    |   11 +-
 .../hadoop/hbase/client/TestMetaScanner.java    |   52 +-
 .../hbase/client/TestMetaWithReplicas.java      |  430 +
 .../client/TestMobCloneSnapshotFromClient.java  |    9 +-
 .../TestMobRestoreSnapshotFromClient.java       |   11 +-
 .../TestMobSnapshotCloneIndependence.java       |    8 +-
 .../hbase/client/TestMobSnapshotFromClient.java |    5 +-
 .../hadoop/hbase/client/TestMultiParallel.java  |  136 +-
 .../hbase/client/TestMultipleTimestamps.java    |   36 +-
 .../client/TestPutDeleteEtcCellIteration.java   |    5 +-
 .../hadoop/hbase/client/TestPutWithDelete.java  |   24 +-
 .../hbase/client/TestReplicaWithCluster.java    |   27 +-
 .../hadoop/hbase/client/TestReplicasClient.java |  140 +-
 .../client/TestRestoreSnapshotFromClient.java   |   23 +-
 ...oreSnapshotFromClientWithRegionReplicas.java |    5 +-
 .../apache/hadoop/hbase/client/TestResult.java  |    5 +-
 .../hbase/client/TestRpcControllerFactory.java  |   16 +-
 .../hadoop/hbase/client/TestScannerTimeout.java |   26 +-
 .../client/TestScannersFromClientSide.java      |   18 +-
 .../client/TestSnapshotCloneIndependence.java   |   31 +-
 .../hbase/client/TestSnapshotFromClient.java    |    9 +-
 ...estSnapshotFromClientWithRegionReplicas.java |    5 +-
 .../hbase/client/TestSnapshotMetadata.java      |   22 +-
 .../hbase/client/TestTableSnapshotScanner.java  |    9 +-
 .../hbase/client/TestTimestampsFilter.java      |   20 +-
 .../hbase/client/TestUpdateConfiguration.java   |   77 +
 .../replication/TestReplicationAdmin.java       |   60 +-
 .../hadoop/hbase/codec/CodecPerformance.java    |    6 +-
 .../hbase/codec/TestCellMessageCodec.java       |    7 +-
 .../hbase/conf/TestConfigurationManager.java    |  137 +
 .../hadoop/hbase/constraint/TestConstraint.java |   16 +-
 .../hbase/constraint/TestConstraints.java       |    5 +-
 .../coprocessor/SampleRegionWALObserver.java    |   98 +-
 .../hbase/coprocessor/SimpleRegionObserver.java |   49 +-
 .../coprocessor/TestAggregateProtocol.java      |   17 +-
 .../TestBatchCoprocessorEndpoint.java           |   22 +-
 .../TestBigDecimalColumnInterpreter.java        |   85 +-
 .../hbase/coprocessor/TestClassLoading.java     |   20 +-
 .../coprocessor/TestCoprocessorEndpoint.java    |   20 +-
 .../coprocessor/TestCoprocessorInterface.java   |   12 +-
 .../hbase/coprocessor/TestCoprocessorStop.java  |    6 +-
 .../TestDoubleColumnInterpreter.java            |   10 +-
 .../hbase/coprocessor/TestHTableWrapper.java    |   21 +-
 ...TestMasterCoprocessorExceptionWithAbort.java |    8 +-
 ...estMasterCoprocessorExceptionWithRemove.java |    6 +-
 .../hbase/coprocessor/TestMasterObserver.java   |  204 +-
 .../coprocessor/TestOpenTableInCoprocessor.java |   41 +-
 .../coprocessor/TestRegionObserverBypass.java   |   14 +-
 .../TestRegionObserverInterface.java            |  101 +-
 .../TestRegionObserverScannerOpenHook.java      |   27 +-
 .../coprocessor/TestRegionObserverStacking.java |    8 +-
 .../TestRegionServerCoprocessorEndpoint.java    |  108 +
 ...gionServerCoprocessorExceptionWithAbort.java |   21 +-
 ...ionServerCoprocessorExceptionWithRemove.java |   23 +-
 .../coprocessor/TestRegionServerObserver.java   |   10 +-
 .../coprocessor/TestRowProcessorEndpoint.java   |   17 +-
 .../hbase/coprocessor/TestWALObserver.java      |  283 +-
 .../DummyRegionServerEndpointProtos.java        | 1151 +++
 .../TestForeignExceptionDispatcher.java         |    5 +-
 .../TestForeignExceptionSerialization.java      |    5 +-
 .../TestTimeoutExceptionInjector.java           |    5 +-
 .../hbase/executor/TestExecutorService.java     |    5 +-
 .../hbase/filter/FilterTestingCluster.java      |  134 +
 .../hadoop/hbase/filter/TestBitComparator.java  |    5 +-
 .../filter/TestColumnPaginationFilter.java      |    5 +-
 .../hbase/filter/TestColumnPrefixFilter.java    |   20 +-
 .../hbase/filter/TestColumnRangeFilter.java     |   11 +-
 .../filter/TestComparatorSerialization.java     |    5 +-
 .../hbase/filter/TestDependentColumnFilter.java |   34 +-
 .../apache/hadoop/hbase/filter/TestFilter.java  |   35 +-
 .../hadoop/hbase/filter/TestFilterList.java     |   27 +-
 .../hbase/filter/TestFilterSerialization.java   |   20 +-
 .../hbase/filter/TestFilterWithScanLimits.java  |  105 +-
 .../hadoop/hbase/filter/TestFilterWrapper.java  |   23 +-
 ...stFirstKeyValueMatchingQualifiersFilter.java |    5 +-
 .../TestFuzzyRowAndColumnRangeFilter.java       |   14 +-
 .../hadoop/hbase/filter/TestFuzzyRowFilter.java |  272 +-
 .../hbase/filter/TestInclusiveStopFilter.java   |   11 +-
 .../filter/TestInvocationRecordFilter.java      |   39 +-
 .../hbase/filter/TestMultiRowRangeFilter.java   |  465 +
 .../filter/TestMultipleColumnPrefixFilter.java  |   22 +-
 .../hadoop/hbase/filter/TestNullComparator.java |    5 +-
 .../hadoop/hbase/filter/TestPageFilter.java     |    5 +-
 .../hadoop/hbase/filter/TestParseFilter.java    |   19 +-
 .../hadoop/hbase/filter/TestPrefixFilter.java   |   10 +-
 .../hbase/filter/TestRandomRowFilter.java       |   10 +-
 .../hbase/filter/TestRegexComparator.java       |  197 +
 .../hadoop/hbase/filter/TestScanRowPrefix.java  |  279 +
 .../TestSingleColumnValueExcludeFilter.java     |    5 +-
 .../filter/TestSingleColumnValueFilter.java     |   25 +-
 .../hadoop/hbase/fs/TestBlockReorder.java       |  135 +-
 .../hbase/http/HttpServerFunctionalTest.java    |    3 -
 .../hadoop/hbase/http/TestGlobalFilter.java     |    8 +-
 .../hadoop/hbase/http/TestHtmlQuoting.java      |    7 +-
 .../hadoop/hbase/http/TestHttpRequestLog.java   |    7 +-
 .../hbase/http/TestHttpRequestLogAppender.java  |    6 +-
 .../hadoop/hbase/http/TestHttpServer.java       |    5 +-
 .../hbase/http/TestHttpServerLifecycle.java     |   28 +-
 .../hbase/http/TestHttpServerWebapps.java       |    6 +-
 .../hadoop/hbase/http/TestPathFilter.java       |    8 +-
 .../hadoop/hbase/http/TestSSLHttpServer.java    |    5 +-
 .../hadoop/hbase/http/TestServletFilter.java    |    5 +-
 .../hadoop/hbase/http/conf/TestConfServlet.java |    6 +-
 .../hbase/http/jmx/TestJMXJsonServlet.java      |    5 +-
 .../hbase/http/lib/TestStaticUserWebFilter.java |    5 +-
 .../hadoop/hbase/http/log/TestLogLevel.java     |    6 +-
 .../apache/hadoop/hbase/io/TestFileLink.java    |    5 +-
 .../apache/hadoop/hbase/io/TestHFileLink.java   |   20 +-
 .../hbase/io/TestHalfStoreFileReader.java       |    5 +-
 .../apache/hadoop/hbase/io/TestHeapSize.java    |   26 +-
 .../hbase/io/TestImmutableBytesWritable.java    |    5 +-
 .../apache/hadoop/hbase/io/TestReference.java   |    5 +-
 .../encoding/TestBufferedDataBlockEncoder.java  |    5 +-
 .../hbase/io/encoding/TestChangingEncoding.java |   87 +-
 .../io/encoding/TestDataBlockEncoders.java      |    5 +-
 .../hbase/io/encoding/TestEncodedSeekers.java   |    7 +-
 .../encoding/TestLoadAndSwitchEncodeOnDisk.java |    9 +-
 .../hbase/io/encoding/TestPrefixTree.java       |  256 +-
 .../io/encoding/TestPrefixTreeEncoding.java     |   19 +-
 .../encoding/TestSeekToBlockWithEncoders.java   |    5 +-
 .../hadoop/hbase/io/hfile/KeySampler.java       |   16 +-
 .../hbase/io/hfile/TestBlockCacheReporting.java |    5 +-
 .../hadoop/hbase/io/hfile/TestCacheConfig.java  |    8 +-
 .../hadoop/hbase/io/hfile/TestCacheOnWrite.java |  117 +-
 .../hbase/io/hfile/TestCachedBlockQueue.java    |    5 +-
 .../hadoop/hbase/io/hfile/TestChecksum.java     |   31 +-
 .../hbase/io/hfile/TestFixedFileTrailer.java    |    4 +-
 .../io/hfile/TestForceCacheImportantBlocks.java |   24 +-
 .../apache/hadoop/hbase/io/hfile/TestHFile.java |   25 +-
 .../hadoop/hbase/io/hfile/TestHFileBlock.java   |  111 +-
 .../io/hfile/TestHFileBlockCompatibility.java   |   39 +-
 .../hbase/io/hfile/TestHFileBlockIndex.java     |   63 +-
 .../io/hfile/TestHFileDataBlockEncoder.java     |    5 +-
 .../hbase/io/hfile/TestHFileEncryption.java     |   23 +-
 .../TestHFileInlineToRootChunkConversion.java   |   13 +-
 .../hbase/io/hfile/TestHFilePerformance.java    |  455 -
 .../hadoop/hbase/io/hfile/TestHFileSeek.java    |   16 +-
 .../hbase/io/hfile/TestHFileWriterV2.java       |   19 +-
 .../hbase/io/hfile/TestHFileWriterV3.java       |   20 +-
 .../hfile/TestLazyDataBlockDecompression.java   |  232 +
 .../hbase/io/hfile/TestLruBlockCache.java       |    5 +-
 .../hbase/io/hfile/TestLruCachedBlock.java      |    5 +-
 .../hadoop/hbase/io/hfile/TestPrefetch.java     |    5 +-
 .../hadoop/hbase/io/hfile/TestReseekTo.java     |    5 +-
 .../TestScannerSelectionUsingKeyRange.java      |   10 +-
 .../io/hfile/TestScannerSelectionUsingTTL.java  |   14 +-
 .../hadoop/hbase/io/hfile/TestSeekTo.java       |   26 +-
 .../hbase/io/hfile/bucket/TestBucketCache.java  |    5 +-
 .../io/hfile/bucket/TestBucketWriterThread.java |   46 +-
 .../io/hfile/bucket/TestByteBufferIOEngine.java |    5 +-
 .../hbase/io/hfile/bucket/TestFileIOEngine.java |    6 +-
 .../hadoop/hbase/ipc/TestBufferChain.java       |    5 +-
 .../apache/hadoop/hbase/ipc/TestCallRunner.java |    5 +-
 .../apache/hadoop/hbase/ipc/TestDelayedRpc.java |   20 +-
 .../hadoop/hbase/ipc/TestHBaseClient.java       |    9 +-
 .../org/apache/hadoop/hbase/ipc/TestIPC.java    |  318 +-
 .../hadoop/hbase/ipc/TestProtoBufRpc.java       |    9 +-
 .../hbase/ipc/TestRpcHandlerException.java      |  192 +
 .../apache/hadoop/hbase/ipc/TestRpcMetrics.java |    9 +-
 .../hbase/ipc/TestSimpleRpcScheduler.java       |    5 +-
 .../apache/hadoop/hbase/mapred/TestDriver.java  |    6 +-
 .../hbase/mapred/TestGroupingTableMap.java      |    5 +-
 .../hbase/mapred/TestIdentityTableMap.java      |    5 +-
 .../hadoop/hbase/mapred/TestRowCounter.java     |    5 +-
 .../hadoop/hbase/mapred/TestSplitTable.java     |   25 +-
 .../hbase/mapred/TestTableInputFormat.java      |   42 +-
 .../hadoop/hbase/mapred/TestTableMapReduce.java |   16 +-
 .../hbase/mapred/TestTableMapReduceUtil.java    |   20 +-
 .../mapred/TestTableSnapshotInputFormat.java    |    5 +-
 .../TableSnapshotInputFormatTestBase.java       |    5 +-
 .../hadoop/hbase/mapreduce/TestCellCounter.java |  184 +-
 .../hadoop/hbase/mapreduce/TestCopyTable.java   |   92 +-
 .../mapreduce/TestGroupingTableMapper.java      |    5 +-
 .../hbase/mapreduce/TestHFileOutputFormat.java  |  114 +-
 .../hbase/mapreduce/TestHFileOutputFormat2.java |  183 +-
 .../hbase/mapreduce/TestHLogRecordReader.java   |  234 +-
 .../hbase/mapreduce/TestHRegionPartitioner.java |   10 +-
 .../hbase/mapreduce/TestImportExport.java       |  186 +-
 .../TestImportTSVWithOperationAttributes.java   |   23 +-
 .../hbase/mapreduce/TestImportTSVWithTTLs.java  |  173 +
 .../TestImportTSVWithVisibilityLabels.java      |   34 +-
 .../hadoop/hbase/mapreduce/TestImportTsv.java   |   68 +-
 .../hbase/mapreduce/TestImportTsvParser.java    |    5 +-
 .../hadoop/hbase/mapreduce/TestJarFinder.java   |  132 +
 .../mapreduce/TestLoadIncrementalHFiles.java    |   86 +-
 .../TestLoadIncrementalHFilesSplitRecovery.java |  364 +-
 ...oadIncrementalHFilesUseSecurityEndPoint.java |   39 +
 .../mapreduce/TestMultiTableInputFormat.java    |   18 +-
 .../mapreduce/TestMultithreadedTableMapper.java |   45 +-
 .../hadoop/hbase/mapreduce/TestRowCounter.java  |   90 +-
 .../TestSecureLoadIncrementalHFiles.java        |    7 +-
 ...ecureLoadIncrementalHFilesSplitRecovery.java |   14 +-
 .../TestSimpleTotalOrderPartitioner.java        |    4 +-
 .../mapreduce/TestTableInputFormatBase.java     |   55 +
 .../mapreduce/TestTableInputFormatScan1.java    |   94 +-
 .../mapreduce/TestTableInputFormatScan2.java    |    5 +-
 .../mapreduce/TestTableInputFormatScanBase.java |   45 +-
 .../hbase/mapreduce/TestTableMapReduce.java     |   29 +-
 .../hbase/mapreduce/TestTableMapReduceBase.java |   20 +-
 .../hbase/mapreduce/TestTableMapReduceUtil.java |   13 +-
 .../mapreduce/TestTableSnapshotInputFormat.java |   26 +-
 .../hadoop/hbase/mapreduce/TestTableSplit.java  |   21 +-
 .../hbase/mapreduce/TestTimeRangeMapRed.java    |   81 +-
 .../hadoop/hbase/mapreduce/TestWALPlayer.java   |   63 +-
 .../hbase/mapreduce/TestWALRecordReader.java    |  269 +
 .../mapreduce/hadoopbackport/TestJarFinder.java |  132 -
 .../hadoop/hbase/master/MockRegionServer.java   |   67 +-
 .../hbase/master/TestActiveMasterManager.java   |   20 +-
 .../hbase/master/TestAssignmentListener.java    |    8 +-
 .../master/TestAssignmentManagerOnCluster.java  |  178 +-
 .../hadoop/hbase/master/TestCatalogJanitor.java |  111 +-
 .../hbase/master/TestClockSkewDetection.java    |   24 +-
 .../master/TestClusterStatusPublisher.java      |    5 +-
 .../hadoop/hbase/master/TestDeadServer.java     |    5 +-
 .../master/TestDistributedLogSplitting.java     |  249 +-
 .../hadoop/hbase/master/TestGetInfoPort.java    |   55 +
 .../master/TestGetLastFlushedSequenceId.java    |   99 +
 .../hbase/master/TestHMasterCommandLine.java    |    5 +-
 .../hbase/master/TestHMasterRPCException.java   |    9 +-
 .../apache/hadoop/hbase/master/TestMaster.java  |   33 +-
 .../hadoop/hbase/master/TestMasterFailover.java |   68 +-
 .../TestMasterFailoverBalancerPersistence.java  |    5 +-
 .../hbase/master/TestMasterFileSystem.java      |    5 +-
 .../hadoop/hbase/master/TestMasterMetrics.java  |    9 +-
 .../hbase/master/TestMasterMetricsWrapper.java  |    7 +-
 .../hbase/master/TestMasterNoCluster.java       |   15 +-
 .../TestMasterOperationsForRegionReplicas.java  |  121 +-
 .../TestMasterRestartAfterDisablingTable.java   |   26 +-
 .../hadoop/hbase/master/TestMasterShutdown.java |   66 +-
 .../hbase/master/TestMasterStatusServlet.java   |   27 +-
 .../hbase/master/TestMasterTransitions.java     |   36 +-
 .../hbase/master/TestRegionPlacement.java       |  161 +-
 .../hbase/master/TestRegionPlacement2.java      |  196 +
 .../hadoop/hbase/master/TestRegionPlan.java     |    5 +-
 .../hadoop/hbase/master/TestRegionState.java    |    7 +-
 .../hadoop/hbase/master/TestRegionStates.java   |  145 +
 .../hadoop/hbase/master/TestRestartCluster.java |   32 +-
 .../hadoop/hbase/master/TestRollingRestart.java |   14 +-
 .../hbase/master/TestSplitLogManager.java       |   55 +-
 .../hbase/master/TestTableLockManager.java      |   24 +-
 .../hbase/master/balancer/BalancerTestBase.java |   46 +-
 .../master/balancer/TestBaseLoadBalancer.java   |   94 +-
 .../balancer/TestDefaultLoadBalancer.java       |   56 +-
 .../TestFavoredNodeAssignmentHelper.java        |    5 +-
 .../master/balancer/TestServerAndLoad.java      |    5 +-
 .../balancer/TestStochasticLoadBalancer.java    |   68 +-
 .../hbase/master/cleaner/TestCleanerChore.java  |    5 +-
 .../hbase/master/cleaner/TestHFileCleaner.java  |   16 +-
 .../master/cleaner/TestHFileLinkCleaner.java    |   25 +-
 .../hbase/master/cleaner/TestLogsCleaner.java   |   24 +-
 .../master/cleaner/TestSnapshotFromMaster.java  |   10 +-
 .../master/handler/TestCreateTableHandler.java  |    5 +-
 .../master/handler/TestEnableTableHandler.java  |  154 +
 .../handler/TestTableDeleteFamilyHandler.java   |   32 +-
 .../TestTableDescriptorModification.java        |   12 +-
 .../master/snapshot/TestSnapshotFileCache.java  |  152 +-
 .../snapshot/TestSnapshotHFileCleaner.java      |    5 +-
 .../master/snapshot/TestSnapshotLogCleaner.java |    5 +-
 .../master/snapshot/TestSnapshotManager.java    |    5 +-
 .../hbase/metrics/TestMetricsHistogram.java     |  112 -
 .../hadoop/hbase/mob/TestCachedMobFile.java     |    2 +-
 .../hbase/mob/TestDefaultMobStoreFlusher.java   |    2 +-
 .../hbase/mob/TestExpiredMobFileCleaner.java    |    2 +-
 .../hbase/mob/TestMobDataBlockEncoding.java     |    2 +-
 .../apache/hadoop/hbase/mob/TestMobFile.java    |    2 +-
 .../hadoop/hbase/mob/TestMobFileCache.java      |    3 +-
 .../hadoop/hbase/mob/TestMobFileName.java       |    2 +-
 .../filecompactions/TestMobFileCompactor.java   |    2 +-
 ...TestPartitionedMobFileCompactionRequest.java |    2 +-
 .../TestPartitionedMobFileCompactor.java        |    2 +-
 .../hbase/mob/mapreduce/TestMobSweepJob.java    |    2 +-
 .../hbase/mob/mapreduce/TestMobSweepMapper.java |    4 +-
 .../mob/mapreduce/TestMobSweepReducer.java      |    2 +-
 .../hbase/mob/mapreduce/TestMobSweeper.java     |    2 +-
 .../TestMemoryBoundedLogMessageBuffer.java      |    5 +-
 .../hbase/monitoring/TestTaskMonitor.java       |    5 +-
 .../hbase/namespace/TestNamespaceAuditor.java   |  479 +
 .../hadoop/hbase/procedure/TestProcedure.java   |    5 +-
 .../procedure/TestProcedureCoordinator.java     |    5 +-
 .../hbase/procedure/TestProcedureManager.java   |    5 +-
 .../hbase/procedure/TestProcedureMember.java    |    6 +-
 .../hadoop/hbase/procedure/TestZKProcedure.java |   11 +-
 .../procedure/TestZKProcedureControllers.java   |    5 +-
 .../hadoop/hbase/protobuf/TestProtobufUtil.java |    6 +-
 .../hbase/protobuf/TestReplicationProtobuf.java |    5 +-
 .../hadoop/hbase/quotas/TestQuotaAdmin.java     |  218 +
 .../hadoop/hbase/quotas/TestQuotaState.java     |  236 +
 .../hadoop/hbase/quotas/TestQuotaTableUtil.java |  185 +
 .../hadoop/hbase/quotas/TestQuotaThrottle.java  |  423 +
 .../hadoop/hbase/quotas/TestRateLimiter.java    |  115 +
 .../MetricsRegionServerWrapperStub.java         |   27 +-
 .../hbase/regionserver/TestAtomicOperation.java |   29 +-
 .../hbase/regionserver/TestBlocksRead.java      |   23 +-
 .../hbase/regionserver/TestBlocksScanned.java   |    8 +-
 .../hadoop/hbase/regionserver/TestBulkLoad.java |  312 +
 .../regionserver/TestCacheOnWriteInSchema.java  |   25 +-
 .../hbase/regionserver/TestCellSkipListSet.java |  151 +
 .../hbase/regionserver/TestClusterId.java       |    5 +-
 .../hbase/regionserver/TestColumnSeeking.java   |   10 +-
 .../hbase/regionserver/TestCompaction.java      |   39 +-
 .../hbase/regionserver/TestCompactionState.java |   31 +-
 .../TestCompactionWithCoprocessor.java          |    5 +-
 .../regionserver/TestCompoundBloomFilter.java   |    5 +-
 .../TestDefaultCompactSelection.java            |   28 +-
 .../hbase/regionserver/TestDefaultMemStore.java |  101 +-
 .../regionserver/TestDefaultStoreEngine.java    |    5 +-
 .../hbase/regionserver/TestDeleteMobTable.java  |    2 +-
 .../regionserver/TestEncryptionKeyRotation.java |    8 +-
 .../TestEncryptionRandomKeying.java             |   12 +-
 .../TestEndToEndSplitTransaction.java           |   91 +-
 .../regionserver/TestExplicitColumnTracker.java |    4 +-
 .../hbase/regionserver/TestFSErrorsExposed.java |   51 +-
 .../regionserver/TestFlushRegionEntry.java      |   10 +-
 .../regionserver/TestGetClosestAtOrBefore.java  |   29 +-
 .../hbase/regionserver/TestHMobStore.java       |   16 +-
 .../hadoop/hbase/regionserver/TestHRegion.java  |  907 +-
 .../hbase/regionserver/TestHRegionBusyWait.java |   96 -
 .../regionserver/TestHRegionFileSystem.java     |    5 +-
 .../hbase/regionserver/TestHRegionInfo.java     |   16 +-
 .../regionserver/TestHRegionOnCluster.java      |   15 +-
 .../regionserver/TestHRegionServerBulkLoad.java |   80 +-
 .../regionserver/TestHeapMemoryManager.java     |   49 +-
 .../hbase/regionserver/TestJoinedScanners.java  |   39 +-
 .../hbase/regionserver/TestKeepDeletes.java     |  175 +-
 .../hbase/regionserver/TestKeyValueHeap.java    |   35 +-
 .../regionserver/TestKeyValueScanFixture.java   |    5 +-
 .../regionserver/TestKeyValueSkipListSet.java   |  151 -
 .../hbase/regionserver/TestMajorCompaction.java |   13 +-
 .../regionserver/TestMasterAddressTracker.java  |   10 +-
 .../regionserver/TestMemStoreChunkPool.java     |   13 +-
 .../hbase/regionserver/TestMemStoreLAB.java     |    5 +-
 .../hbase/regionserver/TestMetricsRegion.java   |    5 +-
 .../regionserver/TestMetricsRegionServer.java   |    5 +-
 .../hbase/regionserver/TestMinVersions.java     |   36 +-
 .../TestMiniBatchOperationInProgress.java       |    5 +-
 .../hbase/regionserver/TestMinorCompaction.java |   11 +-
 .../hbase/regionserver/TestMobCompaction.java   |    4 +-
 .../hbase/regionserver/TestMobStoreScanner.java |    2 +-
 .../regionserver/TestMultiColumnScanner.java    |    7 +-
 .../TestMultiVersionConsistencyControl.java     |    5 +-
 .../hbase/regionserver/TestParallelPut.java     |    5 +-
 .../regionserver/TestPerColumnFamilyFlush.java  |  658 ++
 .../hbase/regionserver/TestPriorityRpc.java     |   17 +-
 .../hbase/regionserver/TestQosFunction.java     |    7 +-
 .../hbase/regionserver/TestQueryMatcher.java    |   43 +-
 .../TestRSKilledWhenInitializing.java           |    7 +-
 .../hbase/regionserver/TestRSStatusServlet.java |   16 +-
 .../hbase/regionserver/TestRecoveredEdits.java  |  177 +
 .../regionserver/TestRegionFavoredNodes.java    |   11 +-
 .../TestRegionMergeTransaction.java             |   42 +-
 .../TestRegionMergeTransactionOnCluster.java    |  135 +-
 .../hbase/regionserver/TestRegionReplicas.java  |   81 +-
 .../regionserver/TestRegionServerMetrics.java   |  140 +-
 .../regionserver/TestRegionServerNoMaster.java  |   73 +-
 .../TestRegionServerOnlineConfigChange.java     |  212 +
 .../regionserver/TestRegionSplitPolicy.java     |    5 +-
 .../regionserver/TestResettingCounters.java     |   10 +-
 .../regionserver/TestReversibleScanners.java    |   23 +-
 .../hbase/regionserver/TestRowTooBig.java       |   19 +-
 .../regionserver/TestSCVFWithMiniCluster.java   |   11 +-
 .../regionserver/TestScanDeleteTracker.java     |   17 +-
 .../TestScanWildcardColumnTracker.java          |    4 +-
 .../regionserver/TestScanWithBloomError.java    |    9 +-
 .../hadoop/hbase/regionserver/TestScanner.java  |   21 +-
 .../regionserver/TestScannerWithBulkload.java   |  148 +-
 .../regionserver/TestSeekOptimizations.java     |    7 +-
 .../regionserver/TestServerCustomProtocol.java  |  248 +-
 .../regionserver/TestServerNonceManager.java    |   16 +-
 .../hbase/regionserver/TestSplitLogWorker.java  |   19 +-
 .../regionserver/TestSplitTransaction.java      |   75 +-
 .../TestSplitTransactionOnCluster.java          |  370 +-
 .../hadoop/hbase/regionserver/TestStore.java    |   65 +-
 .../hbase/regionserver/TestStoreFile.java       |   48 +-
 .../hbase/regionserver/TestStoreFileInfo.java   |   43 +-
 .../TestStoreFileRefresherChore.java            |   19 +-
 .../TestStoreFileScannerWithTagCompression.java |    5 +-
 .../hbase/regionserver/TestStoreScanner.java    |   14 +-
 .../hbase/regionserver/TestStripeCompactor.java |   18 +-
 .../regionserver/TestStripeStoreEngine.java     |   31 +-
 .../TestStripeStoreFileManager.java             |    6 +-
 .../hadoop/hbase/regionserver/TestTags.java     |  136 +-
 .../regionserver/TestTimeRangeTracker.java      |  111 +
 .../hbase/regionserver/TestWideScanner.java     |   10 +-
 .../compactions/PerfTestCompactionPolicies.java |    5 +-
 .../TestCompactionWithThroughputController.java |  294 +
 .../compactions/TestOffPeakHours.java           |   13 +-
 .../compactions/TestStripeCompactionPolicy.java |   62 +-
 .../hbase/regionserver/wal/FaultyHLog.java      |   70 -
 .../wal/FaultySequenceFileLogReader.java        |   12 +-
 .../wal/HLogPerformanceEvaluation.java          |  569 --
 .../regionserver/wal/HLogUtilsForTests.java     |   43 -
 .../regionserver/wal/InstrumentedLogWriter.java |   43 +
 .../wal/InstrumentedSequenceFileLogWriter.java  |   40 -
 .../regionserver/wal/SequenceFileLogWriter.java |   24 +-
 .../hbase/regionserver/wal/TestCompressor.java  |    5 +-
 .../wal/TestCustomWALCellCodec.java             |    6 +-
 .../hbase/regionserver/wal/TestDurability.java  |   60 +-
 .../hbase/regionserver/wal/TestFSHLog.java      |  469 +
 .../hadoop/hbase/regionserver/wal/TestHLog.java | 1339 ---
 .../regionserver/wal/TestHLogFiltering.java     |  153 -
 .../hbase/regionserver/wal/TestHLogMethods.java |  169 -
 .../wal/TestHLogReaderOnSecureHLog.java         |  193 -
 .../hbase/regionserver/wal/TestHLogSplit.java   | 1500 ---
 .../wal/TestHLogSplitCompressed.java            |   35 -
 .../wal/TestKeyValueCompression.java            |    5 +-
 .../regionserver/wal/TestLogRollAbort.java      |   57 +-
 .../regionserver/wal/TestLogRollPeriod.java     |   46 +-
 .../hbase/regionserver/wal/TestLogRolling.java  |  237 +-
 .../wal/TestLogRollingNoCluster.java            |   36 +-
 .../hbase/regionserver/wal/TestMetricsWAL.java  |   56 +
 .../hbase/regionserver/wal/TestProtobufLog.java |  209 +
 .../wal/TestReadOldRootAndMetaEdits.java        |   44 +-
 .../hbase/regionserver/wal/TestSecureHLog.java  |  128 -
 .../regionserver/wal/TestSecureWALReplay.java   |   11 +-
 .../wal/TestWALActionsListener.java             |   67 +-
 .../wal/TestWALCellCodecWithCompression.java    |    5 +-
 .../hbase/regionserver/wal/TestWALReplay.java   |  211 +-
 .../wal/TestWALReplayCompressed.java            |    5 +-
 .../replication/TestMasterReplication.java      |   35 +-
 .../replication/TestMultiSlaveReplication.java  |   98 +-
 .../replication/TestPerTableCFReplication.java  |  365 +-
 .../hbase/replication/TestReplicationBase.java  |   49 +-
 ...estReplicationChangingPeerRegionservers.java |   17 +-
 .../TestReplicationDisableInactivePeer.java     |    5 +-
 .../replication/TestReplicationEndpoint.java    |   94 +-
 .../TestReplicationKillMasterRS.java            |    5 +-
 .../TestReplicationKillMasterRSCompressed.java  |    7 +-
 .../replication/TestReplicationKillRS.java      |    9 +-
 .../replication/TestReplicationKillSlaveRS.java |    6 +-
 .../replication/TestReplicationSmallTests.java  |   60 +-
 .../replication/TestReplicationSource.java      |   35 +-
 .../replication/TestReplicationStateZKImpl.java |   27 +-
 .../replication/TestReplicationSyncUpTool.java  |   57 +-
 .../TestReplicationTrackerZKImpl.java           |   72 +-
 .../TestReplicationWALEntryFilters.java         |   73 +-
 .../replication/TestReplicationWithTags.java    |   44 +-
 .../TestRegionReplicaReplicationEndpoint.java   |   52 +-
 ...egionReplicaReplicationEndpointNoMaster.java |   32 +-
 .../TestReplicationHLogReaderManager.java       |  240 -
 .../regionserver/TestReplicationSink.java       |   26 +-
 .../TestReplicationSinkManager.java             |    5 +-
 .../TestReplicationSourceManager.java           |   82 +-
 .../regionserver/TestReplicationThrottler.java  |    6 +-
 .../TestReplicationWALReaderManager.java        |  221 +
 .../apache/hadoop/hbase/rest/DummyFilter.java   |   64 -
 .../hbase/rest/HBaseRESTTestingUtility.java     |   98 -
 .../hbase/rest/PerformanceEvaluation.java       | 1524 ---
 .../hadoop/hbase/rest/RowResourceBase.java      |  482 -
 .../apache/hadoop/hbase/rest/TestDeleteRow.java |   99 -
 .../hbase/rest/TestGZIPResponseWrapper.java     |  117 -
 .../hbase/rest/TestGetAndPutResource.java       |  585 --
 .../hadoop/hbase/rest/TestGzipFilter.java       |  162 -
 .../hadoop/hbase/rest/TestMultiRowResource.java |  182 -
 .../hadoop/hbase/rest/TestResourceFilter.java   |   61 -
 .../hadoop/hbase/rest/TestScannerResource.java  |  355 -
 .../hbase/rest/TestScannersWithFilters.java     | 1000 --
 .../hbase/rest/TestScannersWithLabels.java      |  239 -
 .../hadoop/hbase/rest/TestSchemaResource.java   |  193 -
 .../hadoop/hbase/rest/TestStatusResource.java   |  117 -
 .../hadoop/hbase/rest/TestTableResource.java    |  262 -
 .../apache/hadoop/hbase/rest/TestTableScan.java |  615 --
 .../hadoop/hbase/rest/TestVersionResource.java  |  179 -
 .../rest/client/TestRemoteAdminRetries.java     |  165 -
 .../rest/client/TestRemoteHTableRetries.java    |  193 -
 .../hbase/rest/client/TestRemoteTable.java      |  539 --
 .../hadoop/hbase/rest/model/TestCellModel.java  |   84 -
 .../hbase/rest/model/TestCellSetModel.java      |  146 -
 .../hbase/rest/model/TestColumnSchemaModel.java |   86 -
 .../hadoop/hbase/rest/model/TestModelBase.java  |  134 -
 .../hadoop/hbase/rest/model/TestRowModel.java   |   79 -
 .../hbase/rest/model/TestScannerModel.java      |  109 -
 .../model/TestStorageClusterStatusModel.java    |  145 -
 .../model/TestStorageClusterVersionModel.java   |   60 -
 .../hbase/rest/model/TestTableInfoModel.java    |   96 -
 .../hbase/rest/model/TestTableListModel.java    |   73 -
 .../hbase/rest/model/TestTableRegionModel.java  |   93 -
 .../hbase/rest/model/TestTableSchemaModel.java  |  117 -
 .../hbase/rest/model/TestVersionModel.java      |   80 -
 .../hbase/security/HBaseKerberosUtils.java      |    4 +-
 .../hbase/security/TestHBaseSaslRpcClient.java  |    5 +-
 .../hadoop/hbase/security/TestSecureRPC.java    |   11 +-
 .../apache/hadoop/hbase/security/TestUser.java  |   11 +-
 .../TestUsersOperationsWithSecureHadoop.java    |    5 +-
 .../hbase/security/access/SecureTestUtil.java   |  240 +-
 .../access/TestAccessControlFilter.java         |   24 +-
 .../security/access/TestAccessController.java   |  739 +-
 .../security/access/TestAccessController2.java  |  216 +
 .../access/TestCellACLWithMultipleVersions.java |  786 +-
 .../hbase/security/access/TestCellACLs.java     |   73 +-
 .../security/access/TestNamespaceCommands.java  |  394 +-
 .../access/TestScanEarlyTermination.java        |   34 +-
 .../security/access/TestTablePermissions.java   |   11 +-
 .../access/TestZKPermissionsWatcher.java        |    6 +-
 .../security/token/TestAuthenticationKey.java   |    5 +-
 .../security/token/TestTokenAuthentication.java |   67 +-
 .../security/token/TestZKSecretWatcher.java     |    5 +-
 .../ExpAsStringVisibilityLabelServiceImpl.java  |  216 +-
 .../LabelFilteringScanLabelGenerator.java       |    2 +-
 ...adTestDataGeneratorWithVisibilityLabels.java |    2 +-
 .../TestDefaultScanLabelGeneratorStack.java     |  255 +
 .../TestEnforcingScanLabelGenerator.java        |   36 +-
 .../visibility/TestExpressionExpander.java      |    5 +-
 .../visibility/TestExpressionParser.java        |    5 +-
 ...sibilityLabelReplicationWithExpAsString.java |  221 +
 .../visibility/TestVisibilityLabels.java        |   77 +-
 ...sibilityLabelsOpWithDifferentUsersNoACL.java |    5 +-
 .../TestVisibilityLabelsReplication.java        |  489 +
 .../visibility/TestVisibilityLabelsWithACL.java |   20 +-
 ...VisibilityLabelsWithCustomVisLabService.java |    5 +-
 ...ibilityLabelsWithDefaultVisLabelService.java |   68 +-
 .../TestVisibilityLabelsWithDeletes.java        |  930 +-
 ...isibilityLabelsWithDistributedLogReplay.java |    5 +-
 .../TestVisibilityLabelsWithSLGStack.java       |    9 +-
 .../TestVisibilityLablesWithGroups.java         |  346 +
 .../TestVisibilityWithCheckAuths.java           |   45 +-
 .../hbase/snapshot/MobSnapshotTestingUtils.java |   12 +-
 .../hbase/snapshot/SnapshotTestingUtils.java    |   51 +-
 .../hbase/snapshot/TestExportSnapshot.java      |   15 +-
 .../snapshot/TestFlushSnapshotFromClient.java   |   34 +-
 .../hbase/snapshot/TestMobExportSnapshot.java   |    2 +-
 .../TestMobFlushSnapshotFromClient.java         |   10 +-
 .../TestMobRestoreFlushSnapshotFromClient.java  |    9 +-
 .../snapshot/TestMobRestoreSnapshotHelper.java  |    2 +-
 .../snapshot/TestMobSecureExportSnapshot.java   |   53 +
 .../TestRestoreFlushSnapshotFromClient.java     |   14 +-
 .../snapshot/TestRestoreSnapshotHelper.java     |    5 +-
 .../snapshot/TestSecureExportSnapshot.java      |    7 +-
 .../snapshot/TestSnapshotDescriptionUtils.java  |    6 +-
 .../hadoop/hbase/trace/TestHTraceHooks.java     |   38 +-
 .../util/LoadTestDataGeneratorWithTags.java     |    2 +-
 .../apache/hadoop/hbase/util/LoadTestTool.java  |   20 +-
 .../apache/hadoop/hbase/util/MockServer.java    |   12 +-
 .../hadoop/hbase/util/MultiThreadedAction.java  |   58 +-
 .../hadoop/hbase/util/MultiThreadedReader.java  |   15 +-
 .../hbase/util/MultiThreadedReaderWithACL.java  |    9 +-
 .../hadoop/hbase/util/MultiThreadedUpdater.java |   13 +-
 .../hbase/util/MultiThreadedUpdaterWithACL.java |   17 +-
 .../hadoop/hbase/util/MultiThreadedWriter.java  |    5 +-
 .../hbase/util/MultiThreadedWriterBase.java     |    4 +-
 .../hbase/util/MultiThreadedWriterWithACL.java  |    9 +-
 .../util/ProcessBasedLocalHBaseCluster.java     |    7 +-
 .../hadoop/hbase/util/RestartMetaTest.java      |   11 +-
 .../hbase/util/StoppableImplementation.java     |    4 +-
 .../util/TestBoundedPriorityBlockingQueue.java  |    5 +-
 .../hadoop/hbase/util/TestByteBloomFilter.java  |    5 +-
 .../hadoop/hbase/util/TestByteBufferUtils.java  |    5 +-
 .../hadoop/hbase/util/TestCompressionTest.java  |    5 +-
 .../hbase/util/TestCoprocessorScanPolicy.java   |   11 +-
 .../hbase/util/TestDefaultEnvironmentEdge.java  |    5 +-
 .../hadoop/hbase/util/TestEncryptionTest.java   |  140 +
 .../hadoop/hbase/util/TestFSHDFSUtils.java      |    5 +-
 .../hbase/util/TestFSTableDescriptors.java      |  222 +-
 .../apache/hadoop/hbase/util/TestFSUtils.java   |  162 +-
 .../apache/hadoop/hbase/util/TestFSVisitor.java |   28 +-
 .../apache/hadoop/hbase/util/TestHBaseFsck.java |  833 +-
 .../hbase/util/TestHBaseFsckComparator.java     |    5 +-
 .../hbase/util/TestHBaseFsckEncryption.java     |   12 +-
 .../hadoop/hbase/util/TestHFileArchiveUtil.java |    5 +-
 .../apache/hadoop/hbase/util/TestIdLock.java    |    5 +-
 .../util/TestIncrementingEnvironmentEdge.java   |    5 +-
 .../hadoop/hbase/util/TestMergeTable.java       |   34 +-
 .../apache/hadoop/hbase/util/TestMergeTool.java |   91 +-
 .../hbase/util/TestMiniClusterLoadEncoded.java  |    5 +-
 .../hbase/util/TestMiniClusterLoadParallel.java |    5 +-
 .../util/TestMiniClusterLoadSequential.java     |   10 +-
 .../apache/hadoop/hbase/util/TestPoolMap.java   |   12 +-
 .../hbase/util/TestProcessBasedCluster.java     |    8 +-
 .../hbase/util/TestRegionSizeCalculator.java    |   48 +-
 .../hbase/util/TestRegionSplitCalculator.java   |    5 +-
 .../hadoop/hbase/util/TestRegionSplitter.java   |   35 +-
 .../apache/hadoop/hbase/util/TestRootPath.java  |    5 +-
 .../hbase/util/TestSortedCopyOnWriteSet.java    |    6 +-
 .../apache/hadoop/hbase/util/TestTableName.java |   96 +-
 .../hadoop/hbase/util/hbck/HbckTestingUtil.java |    3 +-
 .../util/hbck/OfflineMetaRebuildTestCore.java   |   76 +-
 .../util/hbck/TestOfflineMetaRebuildBase.java   |   48 +-
 .../util/hbck/TestOfflineMetaRebuildHole.java   |   12 +-
 .../hbck/TestOfflineMetaRebuildOverlap.java     |   11 +-
 .../hbase/util/test/LoadTestDataGenerator.java  |    2 +-
 .../util/test/LoadTestDataGeneratorWithACL.java |    2 +-
 .../apache/hadoop/hbase/wal/FaultyFSLog.java    |   76 +
 .../apache/hadoop/hbase/wal/IOTestProvider.java |  233 +
 .../wal/TestBoundedRegionGroupingProvider.java  |  183 +
 .../hbase/wal/TestDefaultWALProvider.java       |  363 +
 .../wal/TestDefaultWALProviderWithHLogKey.java  |   34 +
 .../apache/hadoop/hbase/wal/TestSecureWAL.java  |  138 +
 .../apache/hadoop/hbase/wal/TestWALFactory.java |  726 ++
 .../hadoop/hbase/wal/TestWALFiltering.java      |  153 +
 .../apache/hadoop/hbase/wal/TestWALMethods.java |  178 +
 .../hbase/wal/TestWALReaderOnSecureWAL.java     |  218 +
 .../apache/hadoop/hbase/wal/TestWALSplit.java   | 1318 +++
 .../hbase/wal/TestWALSplitCompressed.java       |   36 +
 .../hbase/wal/WALPerformanceEvaluation.java     |  587 ++
 .../hadoop/hbase/zookeeper/TestHQuorumPeer.java |    4 +-
 .../zookeeper/TestRecoverableZooKeeper.java     |    5 +-
 .../hadoop/hbase/zookeeper/TestZKConfig.java    |    5 +-
 .../hbase/zookeeper/TestZKLeaderManager.java    |    4 +-
 .../hadoop/hbase/zookeeper/TestZKMulti.java     |   90 +-
 .../zookeeper/TestZKTableStateManager.java      |  114 -
 .../hbase/zookeeper/TestZooKeeperACL.java       |    6 +-
 .../zookeeper/TestZooKeeperMainServer.java      |    4 +-
 .../zookeeper/TestZooKeeperNodeTracker.java     |    8 +-
 .../lock/TestZKInterProcessReadWriteLock.java   |    5 +-
 .../protobuf/DummyRegionServerEndpoint.proto    |   33 +
 hbase-server/src/test/resources/hbase-site2.xml |  154 +
 hbase-shell/pom.xml                             |   51 +-
 hbase-shell/src/main/asciidoc/.gitignore        |    0
 hbase-shell/src/main/ruby/hbase.rb              |   14 +-
 hbase-shell/src/main/ruby/hbase/admin.rb        |  200 +-
 hbase-shell/src/main/ruby/hbase/hbase.rb        |   15 +-
 hbase-shell/src/main/ruby/hbase/quotas.rb       |  214 +
 .../src/main/ruby/hbase/replication_admin.rb    |   12 +
 hbase-shell/src/main/ruby/hbase/security.rb     |   71 +-
 hbase-shell/src/main/ruby/hbase/table.rb        |   77 +-
 .../src/main/ruby/hbase/visibility_labels.rb    |   19 +-
 hbase-shell/src/main/ruby/shell.rb              |   38 +-
 hbase-shell/src/main/ruby/shell/commands.rb     |    8 +-
 .../src/main/ruby/shell/commands/alter.rb       |   19 +-
 .../ruby/shell/commands/append_peer_tableCFs.rb |   41 +
 .../src/main/ruby/shell/commands/compact_rs.rb  |   43 +
 .../ruby/shell/commands/delete_all_snapshot.rb  |   61 +
 .../src/main/ruby/shell/commands/describe.rb    |   12 +-
 .../src/main/ruby/shell/commands/grant.rb       |   10 +-
 .../src/main/ruby/shell/commands/hlog_roll.rb   |   39 -
 .../src/main/ruby/shell/commands/list_labels.rb |   44 +
 .../src/main/ruby/shell/commands/list_quotas.rb |   52 +
 .../ruby/shell/commands/remove_peer_tableCFs.rb |   42 +
 .../main/ruby/shell/commands/rename_snapshot.rb |   37 -
 .../src/main/ruby/shell/commands/revoke.rb      |    9 +-
 .../src/main/ruby/shell/commands/scan.rb        |    8 +-
 .../src/main/ruby/shell/commands/set_quota.rb   |   70 +
 .../main/ruby/shell/commands/set_visibility.rb  |    8 +-
 .../main/ruby/shell/commands/show_filters.rb    |    5 -
 .../src/main/ruby/shell/commands/trace.rb       |    4 +-
 .../ruby/shell/commands/update_all_config.rb    |   39 +
 .../main/ruby/shell/commands/update_config.rb   |   40 +
 .../main/ruby/shell/commands/user_permission.rb |    8 +-
 .../src/main/ruby/shell/commands/wal_roll.rb    |   42 +
 .../apache/hadoop/hbase/client/TestShell.java   |    9 +-
 hbase-shell/src/test/ruby/hbase/admin_test.rb   |   35 +-
 .../src/test/ruby/hbase/security_admin_test.rb  |    4 +
 hbase-shell/src/test/ruby/hbase/table_test.rb   |   83 +-
 .../ruby/hbase/visibility_labels_admin_test.rb  |    5 +
 hbase-shell/src/test/ruby/test_helper.rb        |    4 +
 hbase-testing-util/pom.xml                      |   16 +-
 hbase-testing-util/src/main/asciidoc/.gitignore |    0
 hbase-thrift/pom.xml                            |  217 +-
 hbase-thrift/src/main/asciidoc/.gitignore       |    0
 .../apache/hadoop/hbase/thrift/CallQueue.java   |    2 +-
 .../thrift/HThreadedSelectorServerArgs.java     |    2 +-
 .../hbase/thrift/HbaseHandlerMetricsProxy.java  |    2 +-
 .../thrift/HttpAuthenticationException.java     |   37 +
 .../hadoop/hbase/thrift/IncrementCoalescer.java |    4 +-
 .../hbase/thrift/TBoundedThreadPoolServer.java  |    2 +-
 .../hadoop/hbase/thrift/ThriftHttpServlet.java  |  202 +
 .../hadoop/hbase/thrift/ThriftMetrics.java      |    2 +-
 .../hadoop/hbase/thrift/ThriftServer.java       |    7 +-
 .../hadoop/hbase/thrift/ThriftServerRunner.java |  130 +-
 .../hadoop/hbase/thrift/ThriftUtilities.java    |    2 +-
 .../hbase/thrift/generated/AlreadyExists.java   |   10 +-
 .../hbase/thrift/generated/BatchMutation.java   |   15 +-
 .../thrift/generated/ColumnDescriptor.java      |   50 +-
 .../hadoop/hbase/thrift/generated/Hbase.java    | 1486 ++-
 .../hadoop/hbase/thrift/generated/IOError.java  |   10 +-
 .../hbase/thrift/generated/IllegalArgument.java |   10 +-
 .../hadoop/hbase/thrift/generated/Mutation.java |   25 +-
 .../hadoop/hbase/thrift/generated/TAppend.java  |   25 +-
 .../hadoop/hbase/thrift/generated/TCell.java    |   15 +-
 .../hadoop/hbase/thrift/generated/TColumn.java  |   15 +-
 .../hbase/thrift/generated/TIncrement.java      |   25 +-
 .../hbase/thrift/generated/TRegionInfo.java     |   40 +-
 .../hbase/thrift/generated/TRowResult.java      |   20 +-
 .../hadoop/hbase/thrift/generated/TScan.java    |   50 +-
 .../apache/hadoop/hbase/thrift2/HTablePool.java |   33 +-
 .../thrift2/ThriftHBaseServiceHandler.java      |   35 +-
 .../hadoop/hbase/thrift2/ThriftServer.java      |   20 +-
 .../hadoop/hbase/thrift2/ThriftUtilities.java   |   13 +-
 .../hadoop/hbase/thrift2/generated/TAppend.java |   30 +-
 .../hbase/thrift2/generated/TAuthorization.java |   10 +-
 .../thrift2/generated/TCellVisibility.java      |   10 +-
 .../hadoop/hbase/thrift2/generated/TColumn.java |   20 +-
 .../thrift2/generated/TColumnIncrement.java     |   20 +-
 .../hbase/thrift2/generated/TColumnValue.java   |   30 +-
 .../hadoop/hbase/thrift2/generated/TDelete.java |  115 +-
 .../hadoop/hbase/thrift2/generated/TGet.java    |  114 +-
 .../hbase/thrift2/generated/THBaseService.java  |  726 +-
 .../hbase/thrift2/generated/TIOError.java       |   10 +-
 .../thrift2/generated/TIllegalArgument.java     |   10 +-
 .../hbase/thrift2/generated/TIncrement.java     |   30 +-
 .../hbase/thrift2/generated/TMutation.java      |   19 +-
 .../hadoop/hbase/thrift2/generated/TPut.java    |   96 +-
 .../hadoop/hbase/thrift2/generated/TResult.java |   15 +-
 .../hbase/thrift2/generated/TRowMutations.java  |   51 +-
 .../hadoop/hbase/thrift2/generated/TScan.java   |  141 +-
 .../hbase/thrift2/generated/TTimeRange.java     |   15 +-
 .../apache/hadoop/hbase/thrift2/package.html    |    6 +-
 .../hadoop/hbase/thrift/TestCallQueue.java      |    8 +-
 .../hbase/thrift/TestThriftHttpServer.java      |  163 +
 .../hadoop/hbase/thrift/TestThriftServer.java   |    5 +-
 .../hbase/thrift/TestThriftServerCmdLine.java   |    9 +-
 .../hadoop/hbase/thrift2/TestHTablePool.java    |  463 +-
 .../thrift2/TestThriftHBaseServiceHandler.java  |    8 +-
 ...TestThriftHBaseServiceHandlerWithLabels.java |    8 +-
 pom.xml                                         |  998 +-
 .../asciidoc/_chapters/appendix_acl_matrix.adoc |  133 +
 .../appendix_contributing_to_documentation.adoc |  395 +
 .../_chapters/appendix_hfile_format.adoc        |  355 +
 src/main/asciidoc/_chapters/architecture.adoc   | 2450 +++++
 src/main/asciidoc/_chapters/asf.adoc            |   47 +
 src/main/asciidoc/_chapters/case_studies.adoc   |  170 +
 src/main/asciidoc/_chapters/community.adoc      |  111 +
 src/main/asciidoc/_chapters/compression.adoc    |  459 +
 src/main/asciidoc/_chapters/configuration.adoc  | 1088 +++
 src/main/asciidoc/_chapters/cp.adoc             |  230 +
 src/main/asciidoc/_chapters/datamodel.adoc      |  562 ++
 src/main/asciidoc/_chapters/developer.adoc      | 1960 ++++
 src/main/asciidoc/_chapters/external_apis.adoc  |   66 +
 src/main/asciidoc/_chapters/faq.adoc            |  141 +
 .../asciidoc/_chapters/getting_started.adoc     |  637 ++
 src/main/asciidoc/_chapters/hbase-default.adoc  | 2254 +++++
 src/main/asciidoc/_chapters/hbase_apis.adoc     |  131 +
 src/main/asciidoc/_chapters/hbase_history.adoc  |   37 +
 src/main/asciidoc/_chapters/hbck_in_depth.adoc  |  212 +
 src/main/asciidoc/_chapters/mapreduce.adoc      |  585 ++
 src/main/asciidoc/_chapters/ops_mgt.adoc        | 2005 ++++
 src/main/asciidoc/_chapters/orca.adoc           |   38 +
 src/main/asciidoc/_chapters/other_info.adoc     |   80 +
 src/main/asciidoc/_chapters/performance.adoc    |  879 ++
 src/main/asciidoc/_chapters/preface.adoc        |   59 +
 src/main/asciidoc/_chapters/rpc.adoc            |  222 +
 src/main/asciidoc/_chapters/schema_design.adoc  |  954 ++
 src/main/asciidoc/_chapters/security.adoc       | 1635 ++++
 src/main/asciidoc/_chapters/shell.adoc          |  390 +
 src/main/asciidoc/_chapters/sql.adoc            |   42 +
 .../_chapters/thrift_filter_language.adoc       |  279 +
 src/main/asciidoc/_chapters/tracing.adoc        |  194 +
 .../asciidoc/_chapters/troubleshooting.adoc     | 1471 +++
 src/main/asciidoc/_chapters/unit_testing.adoc   |  351 +
 src/main/asciidoc/_chapters/upgrading.adoc      |  450 +
 src/main/asciidoc/_chapters/ycsb.adoc           |   42 +
 src/main/asciidoc/_chapters/zookeeper.adoc      |  451 +
 src/main/asciidoc/book.adoc                     |   84 +
 src/main/asciidoc/hbase.css                     |  400 +
 .../appendix_contributing_to_documentation.xml  |  426 -
 src/main/docbkx/appendix_hfile_format.xml       |  653 --
 src/main/docbkx/book.xml                        | 5713 ------------
 src/main/docbkx/case_studies.xml                |  239 -
 src/main/docbkx/community.xml                   |  152 -
 src/main/docbkx/configuration.xml               | 1646 ----
 src/main/docbkx/cp.xml                          |  409 -
 src/main/docbkx/customization.xsl               |   49 -
 src/main/docbkx/developer.xml                   | 2108 -----
 src/main/docbkx/external_apis.xml               |   71 -
 src/main/docbkx/getting_started.xml             |  737 --
 src/main/docbkx/hbase_apis.xml                  |  133 -
 src/main/docbkx/ops_mgt.xml                     | 2228 -----
 src/main/docbkx/performance.xml                 | 1164 ---
 src/main/docbkx/preface.xml                     |   83 -
 src/main/docbkx/rpc.xml                         |  300 -
 src/main/docbkx/schema_design.xml               | 1237 ---
 src/main/docbkx/security.xml                    | 1787 ----
 src/main/docbkx/shell.xml                       |  386 -
 src/main/docbkx/thrift_filter_language.xml      |  757 --
 src/main/docbkx/tracing.xml                     |  187 -
 src/main/docbkx/troubleshooting.xml             | 1651 ----
 src/main/docbkx/unit_testing.xml                |  330 -
 src/main/docbkx/upgrading.xml                   |  517 -
 src/main/docbkx/zookeeper.xml                   |  515 -
 src/main/site/asciidoc/acid-semantics.adoc      |  114 +
 src/main/site/asciidoc/bulk-loads.adoc          |   19 +
 src/main/site/asciidoc/cygwin.adoc              |  193 +
 src/main/site/asciidoc/export_control.adoc      |   40 +
 src/main/site/asciidoc/index.adoc               |   71 +
 src/main/site/asciidoc/metrics.adoc             |   97 +
 src/main/site/asciidoc/old_news.adoc            |  117 +
 src/main/site/asciidoc/pseudo-distributed.adoc  |   19 +
 src/main/site/asciidoc/replication.adoc         |   18 +
 src/main/site/asciidoc/resources.adoc           |   22 +
 src/main/site/asciidoc/sponsors.adoc            |   30 +
 src/main/site/resources/.htaccess               |    8 +
 src/main/site/resources/book/.empty             |    1 +
 .../site/resources/images/coprocessor_stats.png |  Bin 0 -> 297809 bytes
 .../images/hbase_replication_diagram.jpg        |  Bin 0 -> 52298 bytes
 .../resources/images/hbasecon2015.30percent.png |  Bin 0 -> 8684 bytes
 .../site/resources/images/region_states.png     |  Bin 0 -> 99146 bytes
 src/main/site/site.vm                           |  545 --
 src/main/site/site.xml                          |   51 +-
 src/main/site/xdoc/index.xml                    |    7 +
 .../xslt/configuration_to_asciidoc_chapter.xsl  |   92 +
 .../xslt/configuration_to_docbook_section.xsl   |   66 -
 2648 files changed, 184230 insertions(+), 110420 deletions(-)
----------------------------------------------------------------------



[43/50] [abbrv] hbase git commit: Merge branch 'master' (2/11/15) into hbase-11339

Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobExportSnapshot.java
----------------------------------------------------------------------
diff --cc hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobExportSnapshot.java
index fc9d8e7,0000000..9b065e0
mode 100644,000000..100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobExportSnapshot.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobExportSnapshot.java
@@@ -1,439 -1,0 +1,439 @@@
 +/**
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +
 +package org.apache.hadoop.hbase.snapshot;
 +
 +import static org.junit.Assert.assertEquals;
 +import static org.junit.Assert.assertTrue;
 +
 +import java.io.IOException;
 +import java.net.URI;
 +import java.util.ArrayList;
 +import java.util.HashSet;
 +import java.util.List;
 +import java.util.Set;
 +
 +import org.apache.commons.logging.Log;
 +import org.apache.commons.logging.LogFactory;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.FileStatus;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.hbase.HBaseTestingUtility;
 +import org.apache.hadoop.hbase.HConstants;
 +import org.apache.hadoop.hbase.HRegionInfo;
- import org.apache.hadoop.hbase.MediumTests;
 +import org.apache.hadoop.hbase.TableName;
 +import org.apache.hadoop.hbase.client.Admin;
 +import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 +import org.apache.hadoop.hbase.mob.MobConstants;
 +import org.apache.hadoop.hbase.mob.MobUtils;
 +import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 +import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo;
 +import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
 +import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils.SnapshotMock;
++import org.apache.hadoop.hbase.testclassification.MediumTests;
 +import org.apache.hadoop.hbase.util.Bytes;
 +import org.apache.hadoop.hbase.util.FSUtils;
 +import org.apache.hadoop.hbase.util.Pair;
 +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;
 +
 +/**
 + * Test Export Snapshot Tool
 + */
 +@Category(MediumTests.class)
 +public class TestMobExportSnapshot {
 +  private final Log LOG = LogFactory.getLog(getClass());
 +
 +  protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 +
 +  private final static byte[] FAMILY = Bytes.toBytes("cf");
 +
 +  private byte[] emptySnapshotName;
 +  private byte[] snapshotName;
 +  private int tableNumFiles;
 +  private TableName tableName;
 +  private Admin admin;
 +
 +  public static void setUpBaseConf(Configuration conf) {
 +    conf.setBoolean(SnapshotManager.HBASE_SNAPSHOT_ENABLED, true);
 +    conf.setInt("hbase.regionserver.msginterval", 100);
 +    conf.setInt("hbase.client.pause", 250);
 +    conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 6);
 +    conf.setBoolean("hbase.master.enabletable.roundrobin", true);
 +    conf.setInt("mapreduce.map.maxattempts", 10);
 +    conf.setInt(MobConstants.MOB_FILE_CACHE_SIZE_KEY, 0);
 +  }
 +
 +  @BeforeClass
 +  public static void setUpBeforeClass() throws Exception {
 +    setUpBaseConf(TEST_UTIL.getConfiguration());
 +    TEST_UTIL.startMiniCluster(3);
 +    TEST_UTIL.startMiniMapReduceCluster();
 +  }
 +
 +  @AfterClass
 +  public static void tearDownAfterClass() throws Exception {
 +    TEST_UTIL.shutdownMiniMapReduceCluster();
 +    TEST_UTIL.shutdownMiniCluster();
 +  }
 +
 +  /**
 +   * Create a table and take a snapshot of the table used by the export test.
 +   */
 +  @Before
 +  public void setUp() throws Exception {
 +    this.admin = TEST_UTIL.getHBaseAdmin();
 +
 +    long tid = System.currentTimeMillis();
 +    tableName = TableName.valueOf("testtb-" + tid);
 +    snapshotName = Bytes.toBytes("snaptb0-" + tid);
 +    emptySnapshotName = Bytes.toBytes("emptySnaptb0-" + tid);
 +
 +    // create Table
 +    MobSnapshotTestingUtils.createMobTable(TEST_UTIL, tableName, 1, FAMILY);
 +
 +    // Take an empty snapshot
 +    admin.snapshot(emptySnapshotName, tableName);
 +
 +    // Add some rows
 +    SnapshotTestingUtils.loadData(TEST_UTIL, tableName, 50, FAMILY);
 +    tableNumFiles = admin.getTableRegions(tableName).size();
 +
 +    // take a snapshot
 +    admin.snapshot(snapshotName, tableName);
 +  }
 +
 +  @After
 +  public void tearDown() throws Exception {
 +    TEST_UTIL.deleteTable(tableName);
 +    SnapshotTestingUtils.deleteAllSnapshots(TEST_UTIL.getHBaseAdmin());
 +    SnapshotTestingUtils.deleteArchiveDirectory(TEST_UTIL);
 +  }
 +
 +  /**
 +   * Verfy the result of getBalanceSplits() method.
 +   * The result are groups of files, used as input list for the "export" mappers.
 +   * All the groups should have similar amount of data.
 +   *
 +   * The input list is a pair of file path and length.
 +   * The getBalanceSplits() function sort it by length,
 +   * and assign to each group a file, going back and forth through the groups.
 +   */
 +  @Test
 +  public void testBalanceSplit() throws Exception {
 +    // Create a list of files
 +    List<Pair<SnapshotFileInfo, Long>> files = new ArrayList<Pair<SnapshotFileInfo, Long>>();
 +    for (long i = 0; i <= 20; i++) {
 +      SnapshotFileInfo fileInfo = SnapshotFileInfo.newBuilder()
 +        .setType(SnapshotFileInfo.Type.HFILE)
 +        .setHfile("file-" + i)
 +        .build();
 +      files.add(new Pair<SnapshotFileInfo, Long>(fileInfo, i));
 +    }
 +
 +    // Create 5 groups (total size 210)
 +    //    group 0: 20, 11, 10,  1 (total size: 42)
 +    //    group 1: 19, 12,  9,  2 (total size: 42)
 +    //    group 2: 18, 13,  8,  3 (total size: 42)
 +    //    group 3: 17, 12,  7,  4 (total size: 42)
 +    //    group 4: 16, 11,  6,  5 (total size: 42)
 +    List<List<Pair<SnapshotFileInfo, Long>>> splits = ExportSnapshot.getBalancedSplits(files, 5);
 +    assertEquals(5, splits.size());
 +
 +    String[] split0 = new String[] {"file-20", "file-11", "file-10", "file-1", "file-0"};
 +    verifyBalanceSplit(splits.get(0), split0, 42);
 +    String[] split1 = new String[] {"file-19", "file-12", "file-9",  "file-2"};
 +    verifyBalanceSplit(splits.get(1), split1, 42);
 +    String[] split2 = new String[] {"file-18", "file-13", "file-8",  "file-3"};
 +    verifyBalanceSplit(splits.get(2), split2, 42);
 +    String[] split3 = new String[] {"file-17", "file-14", "file-7",  "file-4"};
 +    verifyBalanceSplit(splits.get(3), split3, 42);
 +    String[] split4 = new String[] {"file-16", "file-15", "file-6",  "file-5"};
 +    verifyBalanceSplit(splits.get(4), split4, 42);
 +  }
 +
 +  private void verifyBalanceSplit(final List<Pair<SnapshotFileInfo, Long>> split,
 +      final String[] expected, final long expectedSize) {
 +    assertEquals(expected.length, split.size());
 +    long totalSize = 0;
 +    for (int i = 0; i < expected.length; ++i) {
 +      Pair<SnapshotFileInfo, Long> fileInfo = split.get(i);
 +      assertEquals(expected[i], fileInfo.getFirst().getHfile());
 +      totalSize += fileInfo.getSecond();
 +    }
 +    assertEquals(expectedSize, totalSize);
 +  }
 +
 +  /**
 +   * Verify if exported snapshot and copied files matches the original one.
 +   */
 +  @Test
 +  public void testExportFileSystemState() throws Exception {
 +    testExportFileSystemState(tableName, snapshotName, snapshotName, tableNumFiles);
 +  }
 +
 +  @Test
 +  public void testExportFileSystemStateWithSkipTmp() throws Exception {
 +    TEST_UTIL.getConfiguration().setBoolean(ExportSnapshot.CONF_SKIP_TMP, true);
 +    testExportFileSystemState(tableName, snapshotName, snapshotName, tableNumFiles);
 +  }
 +
 +  @Test
 +  public void testEmptyExportFileSystemState() throws Exception {
 +    testExportFileSystemState(tableName, emptySnapshotName, emptySnapshotName, 0);
 +  }
 +
 +  @Test
 +  public void testConsecutiveExports() throws Exception {
 +    Path copyDir = getLocalDestinationDir();
 +    testExportFileSystemState(tableName, snapshotName, snapshotName, tableNumFiles, copyDir, false);
 +    testExportFileSystemState(tableName, snapshotName, snapshotName, tableNumFiles, copyDir, true);
 +    removeExportDir(copyDir);
 +  }
 +
 +  @Test
 +  public void testExportWithTargetName() throws Exception {
 +    final byte[] targetName = Bytes.toBytes("testExportWithTargetName");
 +    testExportFileSystemState(tableName, snapshotName, targetName, tableNumFiles);
 +  }
 +
 +  /**
 +   * Mock a snapshot with files in the archive dir,
 +   * two regions, and one reference file.
 +   */
 +  @Test
 +  public void testSnapshotWithRefsExportFileSystemState() throws Exception {
 +    Configuration conf = TEST_UTIL.getConfiguration();
 +
 +    Path rootDir = TEST_UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
 +    FileSystem fs = TEST_UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
 +
 +    SnapshotMock snapshotMock = new SnapshotMock(TEST_UTIL.getConfiguration(), fs, rootDir);
 +    SnapshotMock.SnapshotBuilder builder = snapshotMock.createSnapshotV2("tableWithRefsV1");
 +    testSnapshotWithRefsExportFileSystemState(builder);
 +
 +    snapshotMock = new SnapshotMock(TEST_UTIL.getConfiguration(), fs, rootDir);
 +    builder = snapshotMock.createSnapshotV2("tableWithRefsV2");
 +    testSnapshotWithRefsExportFileSystemState(builder);
 +  }
 +
 +  /**
 +   * Generates a couple of regions for the specified SnapshotMock,
 +   * and then it will run the export and verification.
 +   */
 +  private void testSnapshotWithRefsExportFileSystemState(SnapshotMock.SnapshotBuilder builder)
 +      throws Exception {
 +    Path[] r1Files = builder.addRegion();
 +    Path[] r2Files = builder.addRegion();
 +    builder.commit();
 +    int snapshotFilesCount = r1Files.length + r2Files.length;
 +
 +    byte[] snapshotName = Bytes.toBytes(builder.getSnapshotDescription().getName());
 +    TableName tableName = builder.getTableDescriptor().getTableName();
 +    testExportFileSystemState(tableName, snapshotName, snapshotName, snapshotFilesCount);
 +  }
 +
 +  private void testExportFileSystemState(final TableName tableName, final byte[] snapshotName,
 +      final byte[] targetName, int filesExpected) throws Exception {
 +    Path copyDir = getHdfsDestinationDir();
 +    testExportFileSystemState(tableName, snapshotName, targetName, filesExpected, copyDir, false);
 +    removeExportDir(copyDir);
 +  }
 +
 +  /**
 +   * Test ExportSnapshot
 +   */
 +  private void testExportFileSystemState(final TableName tableName, final byte[] snapshotName,
 +      final byte[] targetName, int filesExpected, Path copyDir, boolean overwrite)
 +      throws Exception {
 +    URI hdfsUri = FileSystem.get(TEST_UTIL.getConfiguration()).getUri();
 +    FileSystem fs = FileSystem.get(copyDir.toUri(), new Configuration());
 +    copyDir = copyDir.makeQualified(fs);
 +
 +    List<String> opts = new ArrayList<String>();
 +    opts.add("-snapshot");
 +    opts.add(Bytes.toString(snapshotName));
 +    opts.add("-copy-to");
 +    opts.add(copyDir.toString());
 +    if (targetName != snapshotName) {
 +      opts.add("-target");
 +      opts.add(Bytes.toString(targetName));
 +    }
 +    if (overwrite) opts.add("-overwrite");
 +
 +    // Export Snapshot
 +    int res = ExportSnapshot.innerMain(TEST_UTIL.getConfiguration(),
 +        opts.toArray(new String[opts.size()]));
 +    assertEquals(0, res);
 +
 +    // Verify File-System state
 +    FileStatus[] rootFiles = fs.listStatus(copyDir);
 +    assertEquals(filesExpected > 0 ? 2 : 1, rootFiles.length);
 +    for (FileStatus fileStatus: rootFiles) {
 +      String name = fileStatus.getPath().getName();
 +      assertTrue(fileStatus.isDirectory());
 +      assertTrue(name.equals(HConstants.SNAPSHOT_DIR_NAME) ||
 +                 name.equals(HConstants.HFILE_ARCHIVE_DIRECTORY));
 +    }
 +
 +    // compare the snapshot metadata and verify the hfiles
 +    final FileSystem hdfs = FileSystem.get(hdfsUri, TEST_UTIL.getConfiguration());
 +    final Path snapshotDir = new Path(HConstants.SNAPSHOT_DIR_NAME, Bytes.toString(snapshotName));
 +    final Path targetDir = new Path(HConstants.SNAPSHOT_DIR_NAME, Bytes.toString(targetName));
 +    verifySnapshotDir(hdfs, new Path(TEST_UTIL.getDefaultRootDirPath(), snapshotDir),
 +        fs, new Path(copyDir, targetDir));
 +    Set<String> snapshotFiles = verifySnapshot(fs, copyDir, tableName, Bytes.toString(targetName));
 +    assertEquals(filesExpected, snapshotFiles.size());
 +  }
 +
 +  /**
 +   * Check that ExportSnapshot will return a failure if something fails.
 +   */
 +  @Test
 +  public void testExportFailure() throws Exception {
 +    assertEquals(1, runExportAndInjectFailures(snapshotName, false));
 +  }
 +
 +  /**
 +   * Check that ExportSnapshot will succede if something fails but the retry succede.
 +   */
 +  @Test
 +  public void testExportRetry() throws Exception {
 +    assertEquals(0, runExportAndInjectFailures(snapshotName, true));
 +  }
 +
 +  /*
 +   * Execute the ExportSnapshot job injecting failures
 +   */
 +  private int runExportAndInjectFailures(final byte[] snapshotName, boolean retry)
 +      throws Exception {
 +    Path copyDir = getLocalDestinationDir();
 +    URI hdfsUri = FileSystem.get(TEST_UTIL.getConfiguration()).getUri();
 +    FileSystem fs = FileSystem.get(copyDir.toUri(), new Configuration());
 +    copyDir = copyDir.makeQualified(fs);
 +
 +    Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
 +    conf.setBoolean(ExportSnapshot.CONF_TEST_FAILURE, true);
 +    conf.setBoolean(ExportSnapshot.CONF_TEST_RETRY, retry);
 +
 +    // Export Snapshot
 +    Path sourceDir = TEST_UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
 +    int res = ExportSnapshot.innerMain(conf, new String[] {
 +      "-snapshot", Bytes.toString(snapshotName),
 +      "-copy-from", sourceDir.toString(),
 +      "-copy-to", copyDir.toString()
 +    });
 +    return res;
 +  }
 +
 +  /*
 +   * verify if the snapshot folder on file-system 1 match the one on file-system 2
 +   */
 +  private void verifySnapshotDir(final FileSystem fs1, final Path root1,
 +      final FileSystem fs2, final Path root2) throws IOException {
 +    assertEquals(listFiles(fs1, root1, root1), listFiles(fs2, root2, root2));
 +  }
 +
 +  /*
 +   * Verify if the files exists
 +   */
 +  private Set<String> verifySnapshot(final FileSystem fs, final Path rootDir,
 +      final TableName tableName, final String snapshotName) throws IOException {
 +    final Path exportedSnapshot = new Path(rootDir,
 +      new Path(HConstants.SNAPSHOT_DIR_NAME, snapshotName));
 +    final Set<String> snapshotFiles = new HashSet<String>();
 +    final Path exportedArchive = new Path(rootDir, HConstants.HFILE_ARCHIVE_DIRECTORY);
 +    SnapshotReferenceUtil.visitReferencedFiles(TEST_UTIL.getConfiguration(), fs, exportedSnapshot,
 +          new SnapshotReferenceUtil.SnapshotVisitor() {
 +        @Override
 +        public void storeFile(final HRegionInfo regionInfo, final String family,
 +            final SnapshotRegionManifest.StoreFile storeFile) throws IOException {
 +          if(MobUtils.isMobRegionInfo(regionInfo))
 +            return;
 +          String hfile = storeFile.getName();
 +          snapshotFiles.add(hfile);
 +          if (storeFile.hasReference()) {
 +            // Nothing to do here, we have already the reference embedded
 +          } else {
 +            verifyNonEmptyFile(new Path(exportedArchive,
 +              new Path(FSUtils.getTableDir(new Path("./"), tableName),
 +                  new Path(regionInfo.getEncodedName(), new Path(family, hfile)))));
 +          }
 +        }
 +
 +        @Override
 +        public void logFile (final String server, final String logfile)
 +            throws IOException {
 +          snapshotFiles.add(logfile);
 +          verifyNonEmptyFile(new Path(exportedSnapshot, new Path(server, logfile)));
 +        }
 +
 +        private void verifyNonEmptyFile(final Path path) throws IOException {
 +          assertTrue(path + " should exists", fs.exists(path));
 +          assertTrue(path + " should not be empty", fs.getFileStatus(path).getLen() > 0);
 +        }
 +    });
 +
 +    // Verify Snapshot description
 +    SnapshotDescription desc = SnapshotDescriptionUtils.readSnapshotInfo(fs, exportedSnapshot);
 +    assertTrue(desc.getName().equals(snapshotName));
 +    assertTrue(desc.getTable().equals(tableName.getNameAsString()));
 +    return snapshotFiles;
 +  }
 +
 +  private Set<String> listFiles(final FileSystem fs, final Path root, final Path dir)
 +      throws IOException {
 +    Set<String> files = new HashSet<String>();
 +    int rootPrefix = root.toString().length();
 +    FileStatus[] list = FSUtils.listStatus(fs, dir);
 +    if (list != null) {
 +      for (FileStatus fstat: list) {
 +        LOG.debug(fstat.getPath());
 +        if (fstat.isDirectory()) {
 +          files.addAll(listFiles(fs, root, fstat.getPath()));
 +        } else {
 +          files.add(fstat.getPath().toString().substring(rootPrefix));
 +        }
 +      }
 +    }
 +    return files;
 +  }
 +
 +  private Path getHdfsDestinationDir() {
 +    Path rootDir = TEST_UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
 +    Path path = new Path(new Path(rootDir, "export-test"), "export-" + System.currentTimeMillis());
 +    LOG.info("HDFS export destination path: " + path);
 +    return path;
 +  }
 +
 +  private Path getLocalDestinationDir() {
 +    Path path = TEST_UTIL.getDataTestDir("local-export-" + System.currentTimeMillis());
 +    LOG.info("Local export destination path: " + path);
 +    return path;
 +  }
 +
 +  private void removeExportDir(final Path path) throws IOException {
 +    FileSystem fs = FileSystem.get(path.toUri(), new Configuration());
 +    fs.delete(path, true);
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobFlushSnapshotFromClient.java
----------------------------------------------------------------------
diff --cc hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobFlushSnapshotFromClient.java
index 6fea4dd,0000000..5517f4a
mode 100644,000000..100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobFlushSnapshotFromClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobFlushSnapshotFromClient.java
@@@ -1,551 -1,0 +1,551 @@@
 +/**
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.hadoop.hbase.snapshot;
 +
 +import static org.junit.Assert.*;
 +
 +import java.io.IOException;
 +import java.util.Collections;
 +import java.util.Comparator;
 +import java.util.HashMap;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.concurrent.CountDownLatch;
 +
 +import org.apache.commons.logging.Log;
 +import org.apache.commons.logging.LogFactory;
 +import org.apache.commons.logging.impl.Log4JLogger;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.hbase.HBaseTestingUtility;
 +import org.apache.hadoop.hbase.HConstants;
 +import org.apache.hadoop.hbase.HRegionInfo;
- import org.apache.hadoop.hbase.LargeTests;
 +import org.apache.hadoop.hbase.TableName;
 +import org.apache.hadoop.hbase.TableNotFoundException;
 +import org.apache.hadoop.hbase.client.Admin;
 +import org.apache.hadoop.hbase.client.HTable;
 +import org.apache.hadoop.hbase.client.ScannerCallable;
 +import org.apache.hadoop.hbase.ipc.RpcClient;
 +import org.apache.hadoop.hbase.ipc.RpcServer;
 +import org.apache.hadoop.hbase.master.HMaster;
 +import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 +import org.apache.hadoop.hbase.mob.MobConstants;
 +import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 +import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
++import org.apache.hadoop.hbase.testclassification.ClientTests;
++import org.apache.hadoop.hbase.testclassification.LargeTests;
 +import org.apache.hadoop.hbase.util.Bytes;
 +import org.apache.hadoop.hbase.util.FSUtils;
 +import org.apache.log4j.Level;
 +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;
 +
 +/**
 + * Test creating/using/deleting snapshots from the client
 + * <p>
 + * This is an end-to-end test for the snapshot utility
 + *
 + * TODO This is essentially a clone of TestSnapshotFromClient.  This is worth refactoring this
 + * because there will be a few more flavors of snapshots that need to run these tests.
 + */
- @Category(LargeTests.class)
++@Category({ClientTests.class, LargeTests.class})
 +public class TestMobFlushSnapshotFromClient {
 +  private static final Log LOG = LogFactory.getLog(TestFlushSnapshotFromClient.class);
 +  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
 +  private static final int NUM_RS = 2;
 +  private static final String STRING_TABLE_NAME = "test";
 +  private static final byte[] TEST_FAM = Bytes.toBytes("fam");
 +  private static final TableName TABLE_NAME =
 +      TableName.valueOf(STRING_TABLE_NAME);
 +  private final int DEFAULT_NUM_ROWS = 100;
 +
 +  /**
 +   * Setup the config for the cluster
 +   * @throws Exception on failure
 +   */
 +  @BeforeClass
 +  public static void setupCluster() throws Exception {
 +    ((Log4JLogger)RpcServer.LOG).getLogger().setLevel(Level.ALL);
-     ((Log4JLogger)RpcClient.LOG).getLogger().setLevel(Level.ALL);
 +    ((Log4JLogger)ScannerCallable.LOG).getLogger().setLevel(Level.ALL);
 +    setupConf(UTIL.getConfiguration());
 +    UTIL.startMiniCluster(NUM_RS);
 +  }
 +
 +  private static void setupConf(Configuration conf) {
 +    // disable the ui
 +    conf.setInt("hbase.regionsever.info.port", -1);
 +    // change the flush size to a small amount, regulating number of store files
 +    conf.setInt("hbase.hregion.memstore.flush.size", 25000);
 +    // so make sure we get a compaction when doing a load, but keep around some
 +    // files in the store
 +    conf.setInt("hbase.hstore.compaction.min", 10);
 +    conf.setInt("hbase.hstore.compactionThreshold", 10);
 +    // block writes if we get to 12 store files
 +    conf.setInt("hbase.hstore.blockingStoreFiles", 12);
 +    // Enable snapshot
 +    conf.setBoolean(SnapshotManager.HBASE_SNAPSHOT_ENABLED, true);
 +    conf.set(HConstants.HBASE_REGION_SPLIT_POLICY_KEY,
 +      ConstantSizeRegionSplitPolicy.class.getName());
 +    conf.setInt(MobConstants.MOB_FILE_CACHE_SIZE_KEY, 0);
 +  }
 +
 +  @Before
 +  public void setup() throws Exception {
 +    MobSnapshotTestingUtils.createMobTable(UTIL, TABLE_NAME, 1, TEST_FAM);
 +  }
 +
 +  @After
 +  public void tearDown() throws Exception {
 +    UTIL.deleteTable(TABLE_NAME);
 +
 +    SnapshotTestingUtils.deleteAllSnapshots(UTIL.getHBaseAdmin());
 +    SnapshotTestingUtils.deleteArchiveDirectory(UTIL);
 +  }
 +
 +  @AfterClass
 +  public static void cleanupTest() throws Exception {
 +    try {
 +      UTIL.shutdownMiniCluster();
 +    } catch (Exception e) {
 +      LOG.warn("failure shutting down cluster", e);
 +    }
 +  }
 +
 +  /**
 +   * Test simple flush snapshotting a table that is online
 +   * @throws Exception
 +   */
 +  @Test (timeout=300000)
 +  public void testFlushTableSnapshot() throws Exception {
 +    Admin admin = UTIL.getHBaseAdmin();
 +    // make sure we don't fail on listing snapshots
 +    SnapshotTestingUtils.assertNoSnapshots(admin);
 +
 +    // put some stuff in the table
 +    HTable table = new HTable(UTIL.getConfiguration(), TABLE_NAME);
-     SnapshotTestingUtils.loadData(UTIL, table, DEFAULT_NUM_ROWS, TEST_FAM);
++    SnapshotTestingUtils.loadData(UTIL, TABLE_NAME, DEFAULT_NUM_ROWS, TEST_FAM);
 +
 +    LOG.debug("FS state before snapshot:");
 +    FSUtils.logFileSystemState(UTIL.getTestFileSystem(),
 +        FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
 +
 +    // take a snapshot of the enabled table
 +    String snapshotString = "offlineTableSnapshot";
 +    byte[] snapshot = Bytes.toBytes(snapshotString);
 +    admin.snapshot(snapshotString, TABLE_NAME, SnapshotDescription.Type.FLUSH);
 +    LOG.debug("Snapshot completed.");
 +
 +    // make sure we have the snapshot
 +    List<SnapshotDescription> snapshots = SnapshotTestingUtils.assertOneSnapshotThatMatches(admin,
 +      snapshot, TABLE_NAME);
 +
 +    // make sure its a valid snapshot
 +    FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
 +    Path rootDir = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
 +    LOG.debug("FS state after snapshot:");
 +    FSUtils.logFileSystemState(UTIL.getTestFileSystem(),
 +        FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
 +
 +    SnapshotTestingUtils.confirmSnapshotValid(snapshots.get(0), TABLE_NAME, TEST_FAM, rootDir,
 +        admin, fs);
 +  }
 +
 +   /**
 +   * Test snapshotting a table that is online without flushing
 +   * @throws Exception
 +   */
 +  @Test(timeout=30000)
 +  public void testSkipFlushTableSnapshot() throws Exception {
 +    Admin admin = UTIL.getHBaseAdmin();
 +    // make sure we don't fail on listing snapshots
 +    SnapshotTestingUtils.assertNoSnapshots(admin);
 +
 +    // put some stuff in the table
 +    HTable table = new HTable(UTIL.getConfiguration(), TABLE_NAME);
 +    UTIL.loadTable(table, TEST_FAM);
 +
 +    LOG.debug("FS state before snapshot:");
 +    FSUtils.logFileSystemState(UTIL.getTestFileSystem(),
 +        FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
 +
 +    // take a snapshot of the enabled table
 +    String snapshotString = "skipFlushTableSnapshot";
 +    byte[] snapshot = Bytes.toBytes(snapshotString);
 +    admin.snapshot(snapshotString, TABLE_NAME, SnapshotDescription.Type.SKIPFLUSH);
 +    LOG.debug("Snapshot completed.");
 +
 +    // make sure we have the snapshot
 +    List<SnapshotDescription> snapshots = SnapshotTestingUtils.assertOneSnapshotThatMatches(admin,
 +        snapshot, TABLE_NAME);
 +
 +    // make sure its a valid snapshot
 +    FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
 +    Path rootDir = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
 +    LOG.debug("FS state after snapshot:");
 +    FSUtils.logFileSystemState(UTIL.getTestFileSystem(),
 +        FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
 +
 +    SnapshotTestingUtils.confirmSnapshotValid(snapshots.get(0), TABLE_NAME, TEST_FAM, rootDir,
 +        admin, fs);
 +
 +    admin.deleteSnapshot(snapshot);
 +    snapshots = admin.listSnapshots();
 +    SnapshotTestingUtils.assertNoSnapshots(admin);
 +  }
 +
 +
 +  /**
 +   * Test simple flush snapshotting a table that is online
 +   * @throws Exception
 +   */
 +  @Test (timeout=300000)
 +  public void testFlushTableSnapshotWithProcedure() throws Exception {
 +    Admin admin = UTIL.getHBaseAdmin();
 +    // make sure we don't fail on listing snapshots
 +    SnapshotTestingUtils.assertNoSnapshots(admin);
 +
 +    // put some stuff in the table
 +    HTable table = new HTable(UTIL.getConfiguration(), TABLE_NAME);
-     SnapshotTestingUtils.loadData(UTIL, table, DEFAULT_NUM_ROWS, TEST_FAM);
++    SnapshotTestingUtils.loadData(UTIL, TABLE_NAME, DEFAULT_NUM_ROWS, TEST_FAM);
 +
 +    LOG.debug("FS state before snapshot:");
 +    FSUtils.logFileSystemState(UTIL.getTestFileSystem(),
 +        FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
 +
 +    // take a snapshot of the enabled table
 +    String snapshotString = "offlineTableSnapshot";
 +    byte[] snapshot = Bytes.toBytes(snapshotString);
 +    Map<String, String> props = new HashMap<String, String>();
 +    props.put("table", TABLE_NAME.getNameAsString());
 +    admin.execProcedure(SnapshotManager.ONLINE_SNAPSHOT_CONTROLLER_DESCRIPTION,
 +        snapshotString, props);
 +
 +
 +    LOG.debug("Snapshot completed.");
 +
 +    // make sure we have the snapshot
 +    List<SnapshotDescription> snapshots = SnapshotTestingUtils.assertOneSnapshotThatMatches(admin,
 +      snapshot, TABLE_NAME);
 +
 +    // make sure its a valid snapshot
 +    FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
 +    Path rootDir = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
 +    LOG.debug("FS state after snapshot:");
 +    FSUtils.logFileSystemState(UTIL.getTestFileSystem(),
 +        FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
 +
 +    SnapshotTestingUtils.confirmSnapshotValid(snapshots.get(0), TABLE_NAME, TEST_FAM, rootDir,
 +        admin, fs);
 +  }
 +
 +  @Test (timeout=300000)
 +  public void testSnapshotFailsOnNonExistantTable() throws Exception {
 +    Admin admin = UTIL.getHBaseAdmin();
 +    // make sure we don't fail on listing snapshots
 +    SnapshotTestingUtils.assertNoSnapshots(admin);
 +    TableName tableName = TableName.valueOf("_not_a_table");
 +
 +    // make sure the table doesn't exist
 +    boolean fail = false;
 +    do {
 +    try {
 +      admin.getTableDescriptor(tableName);
 +      fail = true;
 +      LOG.error("Table:" + tableName + " already exists, checking a new name");
 +      tableName = TableName.valueOf(tableName+"!");
 +    } catch (TableNotFoundException e) {
 +      fail = false;
 +      }
 +    } while (fail);
 +
 +    // snapshot the non-existant table
 +    try {
 +      admin.snapshot("fail", tableName, SnapshotDescription.Type.FLUSH);
 +      fail("Snapshot succeeded even though there is not table.");
 +    } catch (SnapshotCreationException e) {
 +      LOG.info("Correctly failed to snapshot a non-existant table:" + e.getMessage());
 +    }
 +  }
 +
 +  @Test(timeout = 300000)
 +  public void testAsyncFlushSnapshot() throws Exception {
 +    Admin admin = UTIL.getHBaseAdmin();
 +    SnapshotDescription snapshot = SnapshotDescription.newBuilder().setName("asyncSnapshot")
 +        .setTable(TABLE_NAME.getNameAsString())
 +        .setType(SnapshotDescription.Type.FLUSH)
 +        .build();
 +
 +    // take the snapshot async
 +    admin.takeSnapshotAsync(snapshot);
 +
 +    // constantly loop, looking for the snapshot to complete
 +    HMaster master = UTIL.getMiniHBaseCluster().getMaster();
 +    SnapshotTestingUtils.waitForSnapshotToComplete(master, snapshot, 200);
 +    LOG.info(" === Async Snapshot Completed ===");
 +    FSUtils.logFileSystemState(UTIL.getTestFileSystem(),
 +      FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
 +    // make sure we get the snapshot
 +    SnapshotTestingUtils.assertOneSnapshotThatMatches(admin, snapshot);
 +  }
 +
 +  @Test (timeout=300000)
 +  public void testSnapshotStateAfterMerge() throws Exception {
 +    int numRows = DEFAULT_NUM_ROWS;
 +    Admin admin = UTIL.getHBaseAdmin();
 +    // make sure we don't fail on listing snapshots
 +    SnapshotTestingUtils.assertNoSnapshots(admin);
 +    // load the table so we have some data
 +    SnapshotTestingUtils.loadData(UTIL, TABLE_NAME, numRows, TEST_FAM);
 +
 +    // Take a snapshot
 +    String snapshotBeforeMergeName = "snapshotBeforeMerge";
 +    admin.snapshot(snapshotBeforeMergeName, TABLE_NAME, SnapshotDescription.Type.FLUSH);
 +
 +    // Clone the table
 +    TableName cloneBeforeMergeName = TableName.valueOf("cloneBeforeMerge");
 +    admin.cloneSnapshot(snapshotBeforeMergeName, cloneBeforeMergeName);
 +    SnapshotTestingUtils.waitForTableToBeOnline(UTIL, cloneBeforeMergeName);
 +
 +    // Merge two regions
 +    List<HRegionInfo> regions = admin.getTableRegions(TABLE_NAME);
 +    Collections.sort(regions, new Comparator<HRegionInfo>() {
 +      public int compare(HRegionInfo r1, HRegionInfo r2) {
 +        return Bytes.compareTo(r1.getStartKey(), r2.getStartKey());
 +      }
 +    });
 +
 +    int numRegions = admin.getTableRegions(TABLE_NAME).size();
 +    int numRegionsAfterMerge = numRegions - 2;
 +    admin.mergeRegions(regions.get(1).getEncodedNameAsBytes(),
 +        regions.get(2).getEncodedNameAsBytes(), true);
 +    admin.mergeRegions(regions.get(5).getEncodedNameAsBytes(),
 +        regions.get(6).getEncodedNameAsBytes(), true);
 +
 +    // Verify that there's one region less
 +    waitRegionsAfterMerge(numRegionsAfterMerge);
 +    assertEquals(numRegionsAfterMerge, admin.getTableRegions(TABLE_NAME).size());
 +
 +    // Clone the table
 +    TableName cloneAfterMergeName = TableName.valueOf("cloneAfterMerge");
 +    admin.cloneSnapshot(snapshotBeforeMergeName, cloneAfterMergeName);
 +    SnapshotTestingUtils.waitForTableToBeOnline(UTIL, cloneAfterMergeName);
 +
 +    MobSnapshotTestingUtils.verifyMobRowCount(UTIL, TABLE_NAME, numRows);
 +    MobSnapshotTestingUtils.verifyMobRowCount(UTIL, cloneBeforeMergeName, numRows);
 +    MobSnapshotTestingUtils.verifyMobRowCount(UTIL, cloneAfterMergeName, numRows);
 +
 +    // test that we can delete the snapshot
 +    UTIL.deleteTable(cloneAfterMergeName);
 +    UTIL.deleteTable(cloneBeforeMergeName);
 +  }
 +
 +  @Test (timeout=300000)
 +  public void testTakeSnapshotAfterMerge() throws Exception {
 +    int numRows = DEFAULT_NUM_ROWS;
 +    Admin admin = UTIL.getHBaseAdmin();
 +    // make sure we don't fail on listing snapshots
 +    SnapshotTestingUtils.assertNoSnapshots(admin);
 +    // load the table so we have some data
 +    SnapshotTestingUtils.loadData(UTIL, TABLE_NAME, numRows, TEST_FAM);
 +
 +    // Merge two regions
 +    List<HRegionInfo> regions = admin.getTableRegions(TABLE_NAME);
 +    Collections.sort(regions, new Comparator<HRegionInfo>() {
 +      public int compare(HRegionInfo r1, HRegionInfo r2) {
 +        return Bytes.compareTo(r1.getStartKey(), r2.getStartKey());
 +      }
 +    });
 +
 +    int numRegions = admin.getTableRegions(TABLE_NAME).size();
 +    int numRegionsAfterMerge = numRegions - 2;
 +    admin.mergeRegions(regions.get(1).getEncodedNameAsBytes(),
 +        regions.get(2).getEncodedNameAsBytes(), true);
 +    admin.mergeRegions(regions.get(5).getEncodedNameAsBytes(),
 +        regions.get(6).getEncodedNameAsBytes(), true);
 +
 +    waitRegionsAfterMerge(numRegionsAfterMerge);
 +    assertEquals(numRegionsAfterMerge, admin.getTableRegions(TABLE_NAME).size());
 +
 +    // Take a snapshot
 +    String snapshotName = "snapshotAfterMerge";
 +    SnapshotTestingUtils.snapshot(admin, snapshotName, TABLE_NAME.getNameAsString(),
 +      SnapshotDescription.Type.FLUSH, 3);
 +
 +    // Clone the table
 +    TableName cloneName = TableName.valueOf("cloneMerge");
 +    admin.cloneSnapshot(snapshotName, cloneName);
 +    SnapshotTestingUtils.waitForTableToBeOnline(UTIL, cloneName);
 +
 +    MobSnapshotTestingUtils.verifyMobRowCount(UTIL, TABLE_NAME, numRows);
 +    MobSnapshotTestingUtils.verifyMobRowCount(UTIL, cloneName, numRows);
 +
 +    // test that we can delete the snapshot
 +    UTIL.deleteTable(cloneName);
 +  }
 +
 +  /**
 +   * Basic end-to-end test of simple-flush-based snapshots
 +   */
 +  @Test (timeout=300000)
 +  public void testFlushCreateListDestroy() throws Exception {
 +    LOG.debug("------- Starting Snapshot test -------------");
 +    Admin admin = UTIL.getHBaseAdmin();
 +    // make sure we don't fail on listing snapshots
 +    SnapshotTestingUtils.assertNoSnapshots(admin);
 +    // load the table so we have some data
 +    SnapshotTestingUtils.loadData(UTIL, TABLE_NAME, DEFAULT_NUM_ROWS, TEST_FAM);
 +
 +    String snapshotName = "flushSnapshotCreateListDestroy";
 +    FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
 +    Path rootDir = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
 +    SnapshotTestingUtils.createSnapshotAndValidate(admin, TABLE_NAME, Bytes.toString(TEST_FAM),
 +      snapshotName, rootDir, fs, true);
 +  }
 +
 +  /**
 +   * Demonstrate that we reject snapshot requests if there is a snapshot already running on the
 +   * same table currently running and that concurrent snapshots on different tables can both
 +   * succeed concurretly.
 +   */
 +  @Test(timeout=300000)
 +  public void testConcurrentSnapshottingAttempts() throws IOException, InterruptedException {
 +    final String STRING_TABLE2_NAME = STRING_TABLE_NAME + "2";
 +    final TableName TABLE2_NAME =
 +        TableName.valueOf(STRING_TABLE2_NAME);
 +
 +    int ssNum = 20;
 +    Admin admin = UTIL.getHBaseAdmin();
 +    // make sure we don't fail on listing snapshots
 +    SnapshotTestingUtils.assertNoSnapshots(admin);
 +    // create second testing table
 +    SnapshotTestingUtils.createTable(UTIL, TABLE2_NAME, TEST_FAM);
 +    // load the table so we have some data
 +    SnapshotTestingUtils.loadData(UTIL, TABLE_NAME, DEFAULT_NUM_ROWS, TEST_FAM);
 +    SnapshotTestingUtils.loadData(UTIL, TABLE2_NAME, DEFAULT_NUM_ROWS, TEST_FAM);
 +
 +    final CountDownLatch toBeSubmitted = new CountDownLatch(ssNum);
 +    // We'll have one of these per thread
 +    class SSRunnable implements Runnable {
 +      SnapshotDescription ss;
 +      SSRunnable(SnapshotDescription ss) {
 +        this.ss = ss;
 +      }
 +
 +      @Override
 +      public void run() {
 +        try {
 +          Admin admin = UTIL.getHBaseAdmin();
 +          LOG.info("Submitting snapshot request: " + ClientSnapshotDescriptionUtils.toString(ss));
 +          admin.takeSnapshotAsync(ss);
 +        } catch (Exception e) {
 +          LOG.info("Exception during snapshot request: " + ClientSnapshotDescriptionUtils.toString(
 +              ss)
 +              + ".  This is ok, we expect some", e);
 +        }
 +        LOG.info("Submitted snapshot request: " + ClientSnapshotDescriptionUtils.toString(ss));
 +        toBeSubmitted.countDown();
 +      }
 +    };
 +
 +    // build descriptions
 +    SnapshotDescription[] descs = new SnapshotDescription[ssNum];
 +    for (int i = 0; i < ssNum; i++) {
 +      SnapshotDescription.Builder builder = SnapshotDescription.newBuilder();
 +      builder.setTable(((i % 2) == 0 ? TABLE_NAME : TABLE2_NAME).getNameAsString());
 +      builder.setName("ss"+i);
 +      builder.setType(SnapshotDescription.Type.FLUSH);
 +      descs[i] = builder.build();
 +    }
 +
 +    // kick each off its own thread
 +    for (int i=0 ; i < ssNum; i++) {
 +      new Thread(new SSRunnable(descs[i])).start();
 +    }
 +
 +    // wait until all have been submitted
 +    toBeSubmitted.await();
 +
 +    // loop until all are done.
 +    while (true) {
 +      int doneCount = 0;
 +      for (SnapshotDescription ss : descs) {
 +        try {
 +          if (admin.isSnapshotFinished(ss)) {
 +            doneCount++;
 +          }
 +        } catch (Exception e) {
 +          LOG.warn("Got an exception when checking for snapshot " + ss.getName(), e);
 +          doneCount++;
 +        }
 +      }
 +      if (doneCount == descs.length) {
 +        break;
 +      }
 +      Thread.sleep(100);
 +    }
 +
 +    // dump for debugging
 +    logFSTree(FSUtils.getRootDir(UTIL.getConfiguration()));
 +
 +    List<SnapshotDescription> taken = admin.listSnapshots();
 +    int takenSize = taken.size();
 +    LOG.info("Taken " + takenSize + " snapshots:  " + taken);
 +    assertTrue("We expect at least 1 request to be rejected because of we concurrently" +
 +        " issued many requests", takenSize < ssNum && takenSize > 0);
 +
 +    // Verify that there's at least one snapshot per table
 +    int t1SnapshotsCount = 0;
 +    int t2SnapshotsCount = 0;
 +    for (SnapshotDescription ss : taken) {
 +      if (TableName.valueOf(ss.getTable()).equals(TABLE_NAME)) {
 +        t1SnapshotsCount++;
 +      } else if (TableName.valueOf(ss.getTable()).equals(TABLE2_NAME)) {
 +        t2SnapshotsCount++;
 +      }
 +    }
 +    assertTrue("We expect at least 1 snapshot of table1 ", t1SnapshotsCount > 0);
 +    assertTrue("We expect at least 1 snapshot of table2 ", t2SnapshotsCount > 0);
 +
 +    UTIL.deleteTable(TABLE2_NAME);
 +  }
 +
 +  private void logFSTree(Path root) throws IOException {
 +    FSUtils.logFileSystemState(UTIL.getDFSCluster().getFileSystem(), root, LOG);
 +  }
 +
 +  private void waitRegionsAfterMerge(final long numRegionsAfterMerge)
 +      throws IOException, InterruptedException {
 +    Admin admin = UTIL.getHBaseAdmin();
 +    // Verify that there's one region less
 +    long startTime = System.currentTimeMillis();
 +    while (admin.getTableRegions(TABLE_NAME).size() != numRegionsAfterMerge) {
 +      // This may be flaky... if after 15sec the merge is not complete give up
 +      // it will fail in the assertEquals(numRegionsAfterMerge).
 +      if ((System.currentTimeMillis() - startTime) > 15000)
 +        break;
 +      Thread.sleep(100);
 +    }
 +    SnapshotTestingUtils.waitForTableToBeOnline(UTIL, TABLE_NAME);
 +  }
 +}
 +

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobRestoreFlushSnapshotFromClient.java
----------------------------------------------------------------------
diff --cc hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobRestoreFlushSnapshotFromClient.java
index 2838d16,0000000..d281763
mode 100644,000000..100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobRestoreFlushSnapshotFromClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobRestoreFlushSnapshotFromClient.java
@@@ -1,209 -1,0 +1,210 @@@
 +/**
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.hadoop.hbase.snapshot;
 +
 +import java.io.IOException;
 +
 +import org.apache.commons.logging.Log;
 +import org.apache.commons.logging.LogFactory;
 +import org.apache.hadoop.hbase.HBaseTestingUtility;
 +import org.apache.hadoop.hbase.HConstants;
- import org.apache.hadoop.hbase.LargeTests;
 +import org.apache.hadoop.hbase.TableName;
 +import org.apache.hadoop.hbase.client.Admin;
 +import org.apache.hadoop.hbase.client.HTable;
 +import org.apache.hadoop.hbase.master.MasterFileSystem;
 +import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 +import org.apache.hadoop.hbase.mob.MobConstants;
 +import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 +import org.apache.hadoop.hbase.regionserver.snapshot.RegionServerSnapshotManager;
++import org.apache.hadoop.hbase.testclassification.ClientTests;
++import org.apache.hadoop.hbase.testclassification.LargeTests;
 +import org.apache.hadoop.hbase.util.Bytes;
 +import org.apache.hadoop.hbase.util.FSUtils;
 +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;
 +
 +/**
 + * Test clone/restore snapshots from the client
 + *
 + * TODO This is essentially a clone of TestRestoreSnapshotFromClient.  This is worth refactoring
 + * this because there will be a few more flavors of snapshots that need to run these tests.
 + */
- @Category(LargeTests.class)
++@Category({ClientTests.class,LargeTests.class})
 +public class TestMobRestoreFlushSnapshotFromClient {
 +  final Log LOG = LogFactory.getLog(getClass());
 +
 +  private final static HBaseTestingUtility UTIL = new HBaseTestingUtility();
 +
 +  private final byte[] FAMILY = Bytes.toBytes("cf");
 +
 +  private byte[] snapshotName0;
 +  private byte[] snapshotName1;
 +  private byte[] snapshotName2;
 +  private int snapshot0Rows;
 +  private int snapshot1Rows;
 +  private TableName tableName;
 +  private Admin admin;
 +
 +  @BeforeClass
 +  public static void setUpBeforeClass() throws Exception {
 +    UTIL.getConfiguration().setBoolean("hbase.online.schema.update.enable", true);
 +    UTIL.getConfiguration().setInt("hbase.regionserver.msginterval", 100);
 +    UTIL.getConfiguration().setInt("hbase.client.pause", 250);
 +    UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 6);
 +    UTIL.getConfiguration().setBoolean(
 +        "hbase.master.enabletable.roundrobin", true);
 +
 +    // Enable snapshot
 +    UTIL.getConfiguration().setBoolean(SnapshotManager.HBASE_SNAPSHOT_ENABLED, true);
 +    UTIL.getConfiguration().setLong(RegionServerSnapshotManager.SNAPSHOT_TIMEOUT_MILLIS_KEY,
 +      RegionServerSnapshotManager.SNAPSHOT_TIMEOUT_MILLIS_DEFAULT * 2);
 +
 +    UTIL.getConfiguration().setInt(MobConstants.MOB_FILE_CACHE_SIZE_KEY, 0);
 +
 +    UTIL.startMiniCluster(3);
 +  }
 +
 +  @AfterClass
 +  public static void tearDownAfterClass() throws Exception {
 +    UTIL.shutdownMiniCluster();
 +  }
 +
 +  /**
 +   * Initialize the tests with a table filled with some data
 +   * and two snapshots (snapshotName0, snapshotName1) of different states.
 +   * The tableName, snapshotNames and the number of rows in the snapshot are initialized.
 +   */
 +  @Before
 +  public void setup() throws Exception {
 +    this.admin = UTIL.getHBaseAdmin();
 +
 +    long tid = System.currentTimeMillis();
 +    tableName = TableName.valueOf("testtb-" + tid);
 +    snapshotName0 = Bytes.toBytes("snaptb0-" + tid);
 +    snapshotName1 = Bytes.toBytes("snaptb1-" + tid);
 +    snapshotName2 = Bytes.toBytes("snaptb2-" + tid);
 +
 +    // create Table
 +    MobSnapshotTestingUtils.createMobTable(UTIL, tableName, 1, FAMILY);
 +
 +    HTable table = new HTable(UTIL.getConfiguration(), tableName);
-     SnapshotTestingUtils.loadData(UTIL, table, 500, FAMILY);
++    SnapshotTestingUtils.loadData(UTIL, tableName, 500, FAMILY);
 +    snapshot0Rows = MobSnapshotTestingUtils.countMobRows(table);
 +    LOG.info("=== before snapshot with 500 rows");
 +    logFSTree();
 +
 +    // take a snapshot
 +    admin.snapshot(Bytes.toString(snapshotName0), tableName,
 +        SnapshotDescription.Type.FLUSH);
 +
 +    LOG.info("=== after snapshot with 500 rows");
 +    logFSTree();
 +
 +    // insert more data
-     SnapshotTestingUtils.loadData(UTIL, table, 500, FAMILY);
++    SnapshotTestingUtils.loadData(UTIL, tableName, 500, FAMILY);
 +    snapshot1Rows = MobSnapshotTestingUtils.countMobRows(table);
 +    LOG.info("=== before snapshot with 1000 rows");
 +    logFSTree();
 +
 +    // take a snapshot of the updated table
 +    admin.snapshot(Bytes.toString(snapshotName1), tableName,
 +        SnapshotDescription.Type.FLUSH);
 +    LOG.info("=== after snapshot with 1000 rows");
 +    logFSTree();
 +    table.close();
 +  }
 +
 +  @After
 +  public void tearDown() throws Exception {
 +    SnapshotTestingUtils.deleteAllSnapshots(UTIL.getHBaseAdmin());
 +    SnapshotTestingUtils.deleteArchiveDirectory(UTIL);
 +  }
 +
 +  @Test
 +  public void testTakeFlushSnapshot() throws IOException {
 +    // taking happens in setup.
 +  }
 +
 +  @Test
 +  public void testRestoreSnapshot() throws IOException {
 +    MobSnapshotTestingUtils.verifyMobRowCount(UTIL, tableName, snapshot1Rows);
 +
 +    // Restore from snapshot-0
 +    admin.disableTable(tableName);
 +    admin.restoreSnapshot(snapshotName0);
 +    logFSTree();
 +    admin.enableTable(tableName);
 +    LOG.info("=== after restore with 500 row snapshot");
 +    logFSTree();
 +    MobSnapshotTestingUtils.verifyMobRowCount(UTIL, tableName, snapshot0Rows);
 +
 +    // Restore from snapshot-1
 +    admin.disableTable(tableName);
 +    admin.restoreSnapshot(snapshotName1);
 +    admin.enableTable(tableName);
 +    MobSnapshotTestingUtils.verifyMobRowCount(UTIL, tableName, snapshot1Rows);
 +  }
 +
 +  @Test(expected=SnapshotDoesNotExistException.class)
 +  public void testCloneNonExistentSnapshot() throws IOException, InterruptedException {
 +    String snapshotName = "random-snapshot-" + System.currentTimeMillis();
 +    TableName tableName = TableName.valueOf("random-table-" + System.currentTimeMillis());
 +    admin.cloneSnapshot(snapshotName, tableName);
 +  }
 +
 +  @Test
 +  public void testCloneSnapshot() throws IOException, InterruptedException {
 +    TableName clonedTableName = TableName.valueOf("clonedtb-" + System.currentTimeMillis());
 +    testCloneSnapshot(clonedTableName, snapshotName0, snapshot0Rows);
 +    testCloneSnapshot(clonedTableName, snapshotName1, snapshot1Rows);
 +  }
 +
 +  private void testCloneSnapshot(final TableName tableName, final byte[] snapshotName,
 +      int snapshotRows) throws IOException, InterruptedException {
 +    // create a new table from snapshot
 +    admin.cloneSnapshot(snapshotName, tableName);
 +    MobSnapshotTestingUtils.verifyMobRowCount(UTIL, tableName, snapshotRows);
 +
 +    UTIL.deleteTable(tableName);
 +  }
 +
 +  @Test
 +  public void testRestoreSnapshotOfCloned() throws IOException, InterruptedException {
 +    TableName clonedTableName = TableName.valueOf("clonedtb-" + System.currentTimeMillis());
 +    admin.cloneSnapshot(snapshotName0, clonedTableName);
 +    MobSnapshotTestingUtils.verifyMobRowCount(UTIL, clonedTableName, snapshot0Rows);
 +    admin.snapshot(Bytes.toString(snapshotName2), clonedTableName, SnapshotDescription.Type.FLUSH);
 +    UTIL.deleteTable(clonedTableName);
 +
 +    admin.cloneSnapshot(snapshotName2, clonedTableName);
 +    MobSnapshotTestingUtils.verifyMobRowCount(UTIL, clonedTableName, snapshot0Rows);
 +    UTIL.deleteTable(clonedTableName);
 +  }
 +
 +  // ==========================================================================
 +  //  Helpers
 +  // ==========================================================================
 +  private void logFSTree() throws IOException {
 +    MasterFileSystem mfs = UTIL.getMiniHBaseCluster().getMaster().getMasterFileSystem();
 +    FSUtils.logFileSystemState(mfs.getFileSystem(), mfs.getRootDir(), LOG);
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobRestoreSnapshotHelper.java
----------------------------------------------------------------------
diff --cc hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobRestoreSnapshotHelper.java
index 3448119,0000000..1893c7a
mode 100644,000000..100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobRestoreSnapshotHelper.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobRestoreSnapshotHelper.java
@@@ -1,163 -1,0 +1,163 @@@
 +/**
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.hadoop.hbase.snapshot;
 +
 +import static org.junit.Assert.assertEquals;
 +import static org.junit.Assert.assertTrue;
 +
 +import java.io.IOException;
 +
 +import org.apache.commons.logging.Log;
 +import org.apache.commons.logging.LogFactory;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.hbase.HBaseTestingUtility;
 +import org.apache.hadoop.hbase.HConstants;
 +import org.apache.hadoop.hbase.HTableDescriptor;
- import org.apache.hadoop.hbase.SmallTests;
++import org.apache.hadoop.hbase.testclassification.SmallTests;
 +import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
 +import org.apache.hadoop.hbase.io.HFileLink;
 +import org.apache.hadoop.hbase.mob.MobConstants;
 +import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 +import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 +import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
 +import org.apache.hadoop.hbase.snapshot.MobSnapshotTestingUtils.SnapshotMock;
 +import org.apache.hadoop.hbase.util.FSTableDescriptors;
 +import org.apache.hadoop.hbase.util.FSUtils;
 +import org.junit.After;
 +import org.junit.Before;
 +import org.junit.Test;
 +import org.junit.experimental.categories.Category;
 +import org.mockito.Mockito;
 +
 +/**
 + * Test the restore/clone operation from a file-system point of view.
 + */
 +@Category(SmallTests.class)
 +public class TestMobRestoreSnapshotHelper {
 +  final Log LOG = LogFactory.getLog(getClass());
 +
 +  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 +  private final static String TEST_HFILE = "abc";
 +
 +  private Configuration conf;
 +  private Path archiveDir;
 +  private FileSystem fs;
 +  private Path rootDir;
 +
 +  @Before
 +  public void setup() throws Exception {
 +    rootDir = TEST_UTIL.getDataTestDir("testRestore");
 +    archiveDir = new Path(rootDir, HConstants.HFILE_ARCHIVE_DIRECTORY);
 +    fs = TEST_UTIL.getTestFileSystem();
 +    TEST_UTIL.getConfiguration().setInt(MobConstants.MOB_FILE_CACHE_SIZE_KEY, 0);
 +    conf = TEST_UTIL.getConfiguration();
 +    FSUtils.setRootDir(conf, rootDir);
 +  }
 +
 +  @After
 +  public void tearDown() throws Exception {
 +    fs.delete(TEST_UTIL.getDataTestDir(), true);
 +  }
 +
 +  @Test
 +  public void testRestore() throws IOException {
 +    // Test Rolling-Upgrade like Snapshot.
 +    // half machines writing using v1 and the others using v2 format.
 +    SnapshotMock snapshotMock = new SnapshotMock(TEST_UTIL.getConfiguration(), fs, rootDir);
 +    SnapshotMock.SnapshotBuilder builder = snapshotMock.createSnapshotV2("snapshot");
 +    builder.addRegionV1();
 +    builder.addRegionV2();
 +    builder.addRegionV2();
 +    builder.addRegionV1();
 +    Path snapshotDir = builder.commit();
 +    HTableDescriptor htd = builder.getTableDescriptor();
 +    SnapshotDescription desc = builder.getSnapshotDescription();
 +
 +    // Test clone a snapshot
 +    HTableDescriptor htdClone = snapshotMock.createHtd("testtb-clone");
 +    testRestore(snapshotDir, desc, htdClone);
 +    verifyRestore(rootDir, htd, htdClone);
 +
 +    // Test clone a clone ("link to link")
 +    SnapshotDescription cloneDesc = SnapshotDescription.newBuilder()
 +        .setName("cloneSnapshot")
 +        .setTable("testtb-clone")
 +        .build();
 +    Path cloneDir = FSUtils.getTableDir(rootDir, htdClone.getTableName());
 +    HTableDescriptor htdClone2 = snapshotMock.createHtd("testtb-clone2");
 +    testRestore(cloneDir, cloneDesc, htdClone2);
 +    verifyRestore(rootDir, htd, htdClone2);
 +  }
 +
 +  private void verifyRestore(final Path rootDir, final HTableDescriptor sourceHtd,
 +      final HTableDescriptor htdClone) throws IOException {
 +    String[] files = SnapshotTestingUtils.listHFileNames(fs,
 +      FSUtils.getTableDir(rootDir, htdClone.getTableName()));
 +    assertEquals(12, files.length);
 +    for (int i = 0; i < files.length; i += 2) {
 +      String linkFile = files[i];
 +      String refFile = files[i+1];
 +      assertTrue(linkFile + " should be a HFileLink", HFileLink.isHFileLink(linkFile));
 +      assertTrue(refFile + " should be a Referene", StoreFileInfo.isReference(refFile));
 +      assertEquals(sourceHtd.getTableName(), HFileLink.getReferencedTableName(linkFile));
 +      Path refPath = getReferredToFile(refFile);
 +      LOG.debug("get reference name for file " + refFile + " = " + refPath);
 +      assertTrue(refPath.getName() + " should be a HFileLink", HFileLink.isHFileLink(refPath.getName()));
 +      assertEquals(linkFile, refPath.getName());
 +    }
 +  }
 +
 +  /**
 +   * Execute the restore operation
 +   * @param snapshotDir The snapshot directory to use as "restore source"
 +   * @param sd The snapshot descriptor
 +   * @param htdClone The HTableDescriptor of the table to restore/clone.
 +   */
 +  public void testRestore(final Path snapshotDir, final SnapshotDescription sd,
 +      final HTableDescriptor htdClone) throws IOException {
 +    LOG.debug("pre-restore table=" + htdClone.getTableName() + " snapshot=" + snapshotDir);
 +    FSUtils.logFileSystemState(fs, rootDir, LOG);
 +
 +    new FSTableDescriptors(conf).createTableDescriptor(htdClone);
 +    RestoreSnapshotHelper helper = getRestoreHelper(rootDir, snapshotDir, sd, htdClone);
 +    helper.restoreHdfsRegions();
 +
 +    LOG.debug("post-restore table=" + htdClone.getTableName() + " snapshot=" + snapshotDir);
 +    FSUtils.logFileSystemState(fs, rootDir, LOG);
 +  }
 +
 +  /**
 +   * Initialize the restore helper, based on the snapshot and table information provided.
 +   */
 +  private RestoreSnapshotHelper getRestoreHelper(final Path rootDir, final Path snapshotDir,
 +      final SnapshotDescription sd, final HTableDescriptor htdClone) throws IOException {
 +    ForeignExceptionDispatcher monitor = Mockito.mock(ForeignExceptionDispatcher.class);
 +    MonitoredTask status = Mockito.mock(MonitoredTask.class);
 +
 +    SnapshotManifest manifest = SnapshotManifest.open(conf, fs, snapshotDir, sd);
 +    return new RestoreSnapshotHelper(conf, fs, manifest,
 +      htdClone, rootDir, monitor, status);
 +  }
 +
 +  private Path getReferredToFile(final String referenceName) {
 +    Path fakeBasePath = new Path(new Path("table", "region"), "cf");
 +    return StoreFileInfo.getReferredToFile(new Path(fakeBasePath, referenceName));
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobSecureExportSnapshot.java
----------------------------------------------------------------------
diff --cc hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobSecureExportSnapshot.java
index 0000000,0000000..4a89db8
new file mode 100644
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobSecureExportSnapshot.java
@@@ -1,0 -1,0 +1,53 @@@
++/**
++ * 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.snapshot;
++
++import org.apache.hadoop.hbase.testclassification.LargeTests;
++import org.apache.hadoop.hbase.mapreduce.HadoopSecurityEnabledUserProviderForTesting;
++import org.apache.hadoop.hbase.security.UserProvider;
++import org.apache.hadoop.hbase.security.access.AccessControlLists;
++import org.apache.hadoop.hbase.security.access.SecureTestUtil;
++
++import org.junit.BeforeClass;
++import org.junit.experimental.categories.Category;
++
++/**
++ * Reruns TestMobExportSnapshot using MobExportSnapshot in secure mode.
++ */
++@Category(LargeTests.class)
++public class TestMobSecureExportSnapshot extends TestMobExportSnapshot {
++  @BeforeClass
++  public static void setUpBeforeClass() throws Exception {
++    setUpBaseConf(TEST_UTIL.getConfiguration());
++
++    // set the always on security provider
++    UserProvider.setUserProviderForTesting(TEST_UTIL.getConfiguration(),
++      HadoopSecurityEnabledUserProviderForTesting.class);
++
++    // setup configuration
++    SecureTestUtil.enableSecurity(TEST_UTIL.getConfiguration());
++
++    TEST_UTIL.startMiniCluster(3);
++    TEST_UTIL.startMiniMapReduceCluster();
++
++    // Wait for the ACL table to become available
++    TEST_UTIL.waitTableEnabled(AccessControlLists.ACL_TABLE_NAME);
++  }
++}

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-shell/src/main/ruby/hbase/admin.rb
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/pom.xml
----------------------------------------------------------------------
diff --cc pom.xml
index 5c946b0,132215d..ff3d4d3
--- a/pom.xml
+++ b/pom.xml
@@@ -521,8 -516,12 +516,8 @@@
              <testFailureIgnore>${surefire.testFailureIgnore}</testFailureIgnore>
              <forkedProcessTimeoutInSeconds>${surefire.timeout}</forkedProcessTimeoutInSeconds>
              <!--Allocate some direct memory for direct memory tests-->
-             <argLine>-enableassertions -XX:MaxDirectMemorySize=1G -Xmx1900m -XX:MaxPermSize=256m -Djava.security.egd=file:/dev/./urandom -Djava.net.preferIPv4Stack=true -Djava.awt.headless=true</argLine>
+             <argLine>${hbase-surefire.argLine} ${argLine}</argLine>
              <redirectTestOutputToFile>${test.output.tofile}</redirectTestOutputToFile>
 -            <systemPropertyVariables>
 -              <jacoco-agent.destfile>target/jacoco.exec</jacoco-agent.destfile>
 -            </systemPropertyVariables>
 -
              <excludes>
  	      <!-- users can add -D option to skip particular test classes
  		   ex: mvn test -Dtest.exclude.pattern=**/TestFoo.java,**/TestBar.java


[19/50] [abbrv] hbase git commit: HBASE-12982 Adding timeouts to TestChoreService

Posted by jm...@apache.org.
HBASE-12982 Adding timeouts to TestChoreService


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

Branch: refs/heads/hbase-11339
Commit: ac175b1bd9ec3878f50458382563810142df032d
Parents: 3f44277
Author: stack <st...@apache.org>
Authored: Fri Feb 6 19:45:25 2015 -0800
Committer: stack <st...@apache.org>
Committed: Fri Feb 6 19:45:39 2015 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/TestChoreService.java   | 60 ++++++++++----------
 1 file changed, 30 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ac175b1b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestChoreService.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestChoreService.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestChoreService.java
index 35cc530..ed5fa60 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestChoreService.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestChoreService.java
@@ -70,7 +70,7 @@ public class TestChoreService {
       public SlowChore(String name, int period) {
         this(name, new SampleStopper(), period);
       }
-      
+
       public SlowChore(String name, Stoppable stopper, int period) {
         super(name, stopper, period);
       }
@@ -102,7 +102,7 @@ public class TestChoreService {
       public DoNothingChore(String name, int period) {
         super(name, new SampleStopper(), period);
       }
-      
+
       public DoNothingChore(String name, Stoppable stopper, int period) {
         super(name, stopper, period);
       }
@@ -125,7 +125,7 @@ public class TestChoreService {
         super(name, stopper, period);
         this.sleepTime = sleepTime;
       }
-      
+
       @Override
       protected boolean initialChore() {
         try {
@@ -153,12 +153,12 @@ public class TestChoreService {
       public CountingChore(String name, int period) {
         this(name, new SampleStopper(), period);
       }
-      
+
       public CountingChore(String name, Stoppable stopper, int period) {
         this(name, stopper, period, false);
       }
 
-      public CountingChore(String name, Stoppable stopper, int period, 
+      public CountingChore(String name, Stoppable stopper, int period,
           final boolean outputOnTicks) {
         super(name, stopper, period);
         this.countOfChoreCalls = 0;
@@ -202,7 +202,7 @@ public class TestChoreService {
     public static class FailInitialChore extends ScheduledChore {
       private int numberOfFailures;
       private int failureThreshold;
-      
+
       /**
        * @param failThreshold Number of times the Chore fails when trying to execute initialChore
        *          before succeeding.
@@ -210,7 +210,7 @@ public class TestChoreService {
       public FailInitialChore(String name, int period, int failThreshold) {
         this(name, new SampleStopper(), period, failThreshold);
       }
-      
+
       public FailInitialChore(String name, Stoppable stopper, int period, int failThreshold) {
         super(name, stopper, period);
         numberOfFailures = 0;
@@ -236,7 +236,7 @@ public class TestChoreService {
     }
   }
 
-  @Test
+  @Test (timeout=20000)
   public void testInitialChorePrecedence() throws InterruptedException {
     ChoreService service = ChoreService.getInstance(TEST_SERVER_NAME);
 
@@ -247,7 +247,7 @@ public class TestChoreService {
 
     int loopCount = 0;
     boolean brokeOutOfLoop = false;
-    
+
     while (!chore.isInitialChoreComplete() && chore.isScheduled()) {
       Thread.sleep(failureThreshold * period);
       loopCount++;
@@ -260,7 +260,7 @@ public class TestChoreService {
     assertFalse(brokeOutOfLoop);
   }
 
-  @Test
+  @Test (timeout=20000)
   public void testCancelChore() throws InterruptedException {
     final int period = 100;
     ScheduledChore chore1 = new DoNothingChore("chore1", period);
@@ -274,7 +274,7 @@ public class TestChoreService {
     assertTrue(service.getNumberOfScheduledChores() == 0);
   }
 
-  @Test
+  @Test (timeout=20000)
   public void testScheduledChoreConstruction() {
     final String NAME = "chore";
     final int PERIOD = 100;
@@ -307,7 +307,7 @@ public class TestChoreService {
       invalidDelayChore.getInitialDelay());
   }
 
-  @Test
+  @Test (timeout=20000)
   public void testChoreServiceConstruction() {
     final int corePoolSize = 10;
     final int defaultCorePoolSize = ChoreService.MIN_CORE_POOL_SIZE;
@@ -322,7 +322,7 @@ public class TestChoreService {
     assertEquals(defaultCorePoolSize, invalidInit.getCorePoolSize());
   }
 
-  @Test 
+  @Test (timeout=20000)
   public void testFrequencyOfChores() throws InterruptedException {
     final int period = 100;
     // Small delta that acts as time buffer (allowing chores to complete if running slowly)
@@ -338,7 +338,7 @@ public class TestChoreService {
     assertTrue(chore.getCountOfChoreCalls() == 21);
   }
 
-  @Test
+  @Test (timeout=20000)
   public void testForceTrigger() throws InterruptedException {
     final int period = 100;
     final int delta = 5;
@@ -369,13 +369,13 @@ public class TestChoreService {
     assertTrue(chore.getCountOfChoreCalls() == 26);
   }
 
-  @Test
+  @Test (timeout=20000)
   public void testCorePoolIncrease() throws InterruptedException {
     final int initialCorePoolSize = 3;
     ChoreService service = new ChoreService(TEST_SERVER_NAME, initialCorePoolSize);
     assertEquals("Should have a core pool of size: " + initialCorePoolSize, initialCorePoolSize,
         service.getCorePoolSize());
-    
+
     final int slowChorePeriod = 100;
     SlowChore slowChore1 = new SlowChore("slowChore1", slowChorePeriod);
     SlowChore slowChore2 = new SlowChore("slowChore2", slowChorePeriod);
@@ -384,9 +384,9 @@ public class TestChoreService {
     service.scheduleChore(slowChore1);
     service.scheduleChore(slowChore2);
     service.scheduleChore(slowChore3);
-    
+
     Thread.sleep(slowChorePeriod * 10);
-    assertEquals("Should not create more pools than scheduled chores", 3, 
+    assertEquals("Should not create more pools than scheduled chores", 3,
       service.getCorePoolSize());
 
     SlowChore slowChore4 = new SlowChore("slowChore4", slowChorePeriod);
@@ -404,7 +404,7 @@ public class TestChoreService {
       service.getCorePoolSize());
   }
 
-  @Test
+  @Test (timeout=20000)
   public void testCorePoolDecrease() throws InterruptedException {
     final int initialCorePoolSize = 3;
     ChoreService service = new ChoreService(TEST_SERVER_NAME, initialCorePoolSize);
@@ -506,7 +506,7 @@ public class TestChoreService {
     assertEquals("Should not change", 3, service.getCorePoolSize());
   }
 
-  @Test
+  @Test (timeout=20000)
   public void testNumberOfRunningChores() throws InterruptedException {
     ChoreService service = new ChoreService(TEST_SERVER_NAME);
 
@@ -543,7 +543,7 @@ public class TestChoreService {
     assertEquals("Scheduled chore mismatch", 0, service.getNumberOfScheduledChores());
   }
 
-  @Test
+  @Test (timeout=20000)
   public void testNumberOfChoresMissingStartTime() throws InterruptedException {
     ChoreService service = new ChoreService(TEST_SERVER_NAME);
 
@@ -587,7 +587,7 @@ public class TestChoreService {
    * been scheduled with the service. For example, if 4 ScheduledChores are scheduled with a
    * ChoreService, the number of threads in the ChoreService's core pool should never exceed 4
    */
-  @Test
+  @Test (timeout=20000)
   public void testMaximumChoreServiceThreads() throws InterruptedException {
     ChoreService service = new ChoreService(TEST_SERVER_NAME);
 
@@ -610,7 +610,7 @@ public class TestChoreService {
     service.scheduleChore(sc3);
     service.scheduleChore(sc4);
     service.scheduleChore(sc5);
-    
+
     Thread.sleep(sleepTime);
     assertTrue(service.getCorePoolSize() <= service.getNumberOfScheduledChores());
 
@@ -630,7 +630,7 @@ public class TestChoreService {
     assertTrue(service.getCorePoolSize() <= service.getNumberOfScheduledChores());
   }
 
-  @Test
+  @Test (timeout=20000)
   public void testScheduledChoreReset() throws InterruptedException {
     final int period = 100;
     ChoreService service = new ChoreService(TEST_SERVER_NAME);
@@ -657,7 +657,7 @@ public class TestChoreService {
     assertTrue(chore.getTimeOfThisRun() == -1);
   }
 
-  @Test
+  @Test (timeout=20000)
   public void testChangingChoreServices() throws InterruptedException {
     final int period = 100;
     final int sleepTime = 10;
@@ -691,7 +691,7 @@ public class TestChoreService {
     assertTrue(chore.getChoreServicer() == null);
   }
 
-  @Test
+  @Test (timeout=20000)
   public void testTriggerNowFailsWhenNotScheduled() throws InterruptedException {
     final int period = 100;
     // Small sleep time buffer to allow CountingChore to complete
@@ -716,7 +716,7 @@ public class TestChoreService {
     assertEquals(5, chore.getCountOfChoreCalls());
   }
 
-  @Test
+  @Test (timeout=20000)
   public void testStopperForScheduledChores() throws InterruptedException {
     ChoreService service = ChoreService.getInstance(TEST_SERVER_NAME);
     Stoppable stopperForGroup1 = new SampleStopper();
@@ -767,7 +767,7 @@ public class TestChoreService {
     assertFalse(chore3_group2.isScheduled());
   }
 
-  @Test
+  @Test (timeout=20000)
   public void testShutdownCancelsScheduledChores() throws InterruptedException {
     final int period = 100;
     ChoreService service = new ChoreService(TEST_SERVER_NAME);
@@ -789,7 +789,7 @@ public class TestChoreService {
     assertFalse(successChore3.isScheduled());
   }
 
-  @Test
+  @Test (timeout=20000)
   public void testShutdownWorksWhileChoresAreExecuting() throws InterruptedException {
     final int period = 100;
     final int sleep = 5 * period;
@@ -814,7 +814,7 @@ public class TestChoreService {
     assertTrue(service.isTerminated());
   }
 
-  @Test
+  @Test (timeout=20000)
   public void testShutdownRejectsNewSchedules() throws InterruptedException {
     final int period = 100;
     ChoreService service = new ChoreService(TEST_SERVER_NAME);


[07/50] [abbrv] hbase git commit: HBASE-12958 SSH doing hbase:meta get but hbase:meta not assigned

Posted by jm...@apache.org.
HBASE-12958 SSH doing hbase:meta get but hbase:meta not assigned


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

Branch: refs/heads/hbase-11339
Commit: 96cdc7987e8894b304a3201f67cb0b9595c68cc3
Parents: 8aeb3ac
Author: stack <st...@apache.org>
Authored: Wed Feb 4 22:56:40 2015 -0800
Committer: stack <st...@apache.org>
Committed: Wed Feb 4 22:56:40 2015 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/MetaTableAccessor.java  |   1 +
 .../hadoop/hbase/master/RegionStates.java       | 122 +++++++++++--------
 .../hadoop/hbase/master/TestRegionStates.java   |  60 ++++++++-
 3 files changed, 130 insertions(+), 53 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/96cdc798/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
index cb031ce..8f3a20e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
@@ -202,6 +202,7 @@ public class MetaTableAccessor {
    * @throws IOException
    */
   private static Result get(final Table t, final Get g) throws IOException {
+    if (t == null) return null;
     try {
       return t.get(g);
     } finally {

http://git-wip-us.apache.org/repos/asf/hbase/blob/96cdc798/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
index 11139f6..221c7a4 100644
--- 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
@@ -571,71 +571,88 @@ public class RegionStates {
   /**
    * A server is offline, all regions on it are dead.
    */
-  public synchronized List<HRegionInfo> serverOffline(final ServerName sn) {
+  public List<HRegionInfo> serverOffline(final ServerName sn) {
     // Offline all regions on this server not already in transition.
     List<HRegionInfo> rits = new ArrayList<HRegionInfo>();
-    Set<HRegionInfo> assignedRegions = serverHoldings.get(sn);
-    if (assignedRegions == null) {
-      assignedRegions = new HashSet<HRegionInfo>();
-    }
+    Set<HRegionInfo> regionsToCleanIfNoMetaEntry = new HashSet<HRegionInfo>();
+    synchronized (this) {
+      Set<HRegionInfo> assignedRegions = serverHoldings.get(sn);
+      if (assignedRegions == null) {
+        assignedRegions = new HashSet<HRegionInfo>();
+      }
 
-    // Offline regions outside the loop to avoid ConcurrentModificationException
-    Set<HRegionInfo> regionsToOffline = new HashSet<HRegionInfo>();
-    for (HRegionInfo region : assignedRegions) {
-      // Offline open regions, no need to offline if SPLIT/MERGED/OFFLINE
-      if (isRegionOnline(region)) {
-        regionsToOffline.add(region);
-      } else if (isRegionInState(region, State.SPLITTING, State.MERGING)) {
-        LOG.debug("Offline splitting/merging region " + getRegionState(region));
-        regionsToOffline.add(region);
+      // Offline regions outside the loop to avoid ConcurrentModificationException
+      Set<HRegionInfo> regionsToOffline = new HashSet<HRegionInfo>();
+      for (HRegionInfo region : assignedRegions) {
+        // Offline open regions, no need to offline if SPLIT/MERGED/OFFLINE
+        if (isRegionOnline(region)) {
+          regionsToOffline.add(region);
+        } else if (isRegionInState(region, State.SPLITTING, State.MERGING)) {
+          LOG.debug("Offline splitting/merging region " + getRegionState(region));
+          regionsToOffline.add(region);
+        }
       }
-    }
 
-    for (RegionState state : regionsInTransition.values()) {
-      HRegionInfo hri = state.getRegion();
-      if (assignedRegions.contains(hri)) {
-        // Region is open on this region server, but in transition.
-        // This region must be moving away from this server, or splitting/merging.
-        // SSH will handle it, either skip assigning, or re-assign.
-        LOG.info("Transitioning " + state + " will be handled by SSH for " + sn);
-      } else if (sn.equals(state.getServerName())) {
-        // Region is in transition on this region server, and this
-        // region is not open on this server. So the region must be
-        // moving to this server from another one (i.e. opening or
-        // pending open on this server, was open on another one.
-        // Offline state is also kind of pending open if the region is in
-        // transition. The region could be in failed_close state too if we have
-        // tried several times to open it while this region server is not reachable)
-        if (isOneOfStates(state, State.OPENING, State.PENDING_OPEN,
-            State.FAILED_OPEN, State.FAILED_CLOSE, State.OFFLINE)) {
-          LOG.info("Found region in " + state + " to be reassigned by SSH for " + sn);
-          rits.add(hri);
-        } else if (isOneOfStates(state, State.SPLITTING_NEW)) {
-          try {
-            if (MetaTableAccessor.getRegion(server.getConnection(), state.getRegion()
-                .getEncodedNameAsBytes()) == null) {
-              regionsToOffline.add(state.getRegion());
-              FSUtils.deleteRegionDir(server.getConfiguration(), state.getRegion());
-            }
-          } catch (IOException e) {
-            LOG.warn("Got exception while deleting " + state.getRegion()
-                + " directories from file system.", e);
+      for (RegionState state : regionsInTransition.values()) {
+        HRegionInfo hri = state.getRegion();
+        if (assignedRegions.contains(hri)) {
+          // Region is open on this region server, but in transition.
+          // This region must be moving away from this server, or splitting/merging.
+          // SSH will handle it, either skip assigning, or re-assign.
+          LOG.info("Transitioning " + state + " will be handled by SSH for " + sn);
+        } else if (sn.equals(state.getServerName())) {
+          // Region is in transition on this region server, and this
+          // region is not open on this server. So the region must be
+          // moving to this server from another one (i.e. opening or
+          // pending open on this server, was open on another one.
+          // Offline state is also kind of pending open if the region is in
+          // transition. The region could be in failed_close state too if we have
+          // tried several times to open it while this region server is not reachable)
+          if (isOneOfStates(state, State.OPENING, State.PENDING_OPEN,
+              State.FAILED_OPEN, State.FAILED_CLOSE, State.OFFLINE)) {
+            LOG.info("Found region in " + state + " to be reassigned by SSH for " + sn);
+            rits.add(hri);
+          } else if (isOneOfStates(state, State.SPLITTING_NEW)) {
+            regionsToCleanIfNoMetaEntry.add(state.getRegion());
+          } else {
+            LOG.warn("THIS SHOULD NOT HAPPEN: unexpected " + state);
           }
-        } else {
-          LOG.warn("THIS SHOULD NOT HAPPEN: unexpected " + state);
         }
       }
-    }
 
-    for (HRegionInfo hri : regionsToOffline) {
-      regionOffline(hri);
-    }
+      for (HRegionInfo hri : regionsToOffline) {
+        regionOffline(hri);
+      }
 
-    this.notifyAll();
+      this.notifyAll();
+    }
+    cleanIfNoMetaEntry(regionsToCleanIfNoMetaEntry);
     return rits;
   }
 
   /**
+   * This method does an RPC to hbase:meta. Do not call this method with a lock/synchronize held.
+   * @param hris The hris to check if empty in hbase:meta and if so, clean them up.
+   */
+  private void cleanIfNoMetaEntry(Set<HRegionInfo> hris) {
+    if (hris.isEmpty()) return;
+    for (HRegionInfo hri: hris) {
+      try {
+        // This is RPC to meta table. It is done while we have a synchronize on
+        // regionstates. No progress will be made if meta is not available at this time.
+        // This is a cleanup task. Not critical.
+        if (MetaTableAccessor.getRegion(server.getConnection(), hri.getEncodedNameAsBytes()) ==
+            null) {
+          regionOffline(hri);
+          FSUtils.deleteRegionDir(server.getConfiguration(), hri);
+        }
+      } catch (IOException e) {
+        LOG.warn("Got exception while deleting " + hri + " directories from file system.", e);
+      }
+    }
+  }
+
+  /**
    * Gets the online regions of the specified table.
    * This method looks at the in-memory state.  It does not go to <code>hbase:meta</code>.
    * Only returns <em>online</em> regions.  If a region on this table has been
@@ -1000,7 +1017,8 @@ public class RegionStates {
   }
 
   /**
-   * Get the HRegionInfo from cache, if not there, from the hbase:meta table
+   * Get the HRegionInfo from cache, if not there, from the hbase:meta table.
+   * Be careful. Does RPC. Do not hold a lock or synchronize when you call this method.
    * @param  regionName
    * @return HRegionInfo for the region
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/96cdc798/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionStates.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionStates.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionStates.java
index c16ac02..ce61e40 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionStates.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionStates.java
@@ -22,6 +22,12 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.master.RegionState.State;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.junit.Test;
@@ -31,7 +37,15 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 import java.util.UUID;
+import java.util.concurrent.BrokenBarrierException;
+import java.util.concurrent.CyclicBarrier;
 
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.io.IOException;
+import static org.junit.Assert.assertTrue;
 import static junit.framework.Assert.assertFalse;
 import static org.mockito.Matchers.isA;
 import static org.mockito.Mockito.mock;
@@ -39,7 +53,51 @@ import static org.mockito.Mockito.when;
 
 @Category({MasterTests.class, SmallTests.class})
 public class TestRegionStates {
+  @Test (timeout=10000)
+  public void testCanMakeProgressThoughMetaIsDown()
+  throws IOException, InterruptedException, BrokenBarrierException {
+    Server server = mock(Server.class);
+    when(server.getServerName()).thenReturn(ServerName.valueOf("master,1,1"));
+    Connection connection = mock(ClusterConnection.class);
+    // Set up a table that gets 'stuck' when we try to fetch a row from the meta table.
+    // It is stuck on a CyclicBarrier latch. We use CyclicBarrier because it will tell us when
+    // thread is waiting on latch.
+    Table metaTable = Mockito.mock(Table.class);
+    final CyclicBarrier latch = new CyclicBarrier(2);
+    when(metaTable.get((Get)Mockito.any())).thenAnswer(new Answer<Result>() {
+      @Override
+      public Result answer(InvocationOnMock invocation) throws Throwable {
+        latch.await();
+        throw new java.net.ConnectException("Connection refused");
+      }
+    });
+    when(connection.getTable(TableName.META_TABLE_NAME)).thenReturn(metaTable);
+    when(server.getConnection()).thenReturn((ClusterConnection)connection);
+    Configuration configuration = mock(Configuration.class);
+    when(server.getConfiguration()).thenReturn(configuration);
+    TableStateManager tsm = mock(TableStateManager.class);
+    ServerManager sm = mock(ServerManager.class);
+    when(sm.isServerOnline(isA(ServerName.class))).thenReturn(true);
 
+    RegionStateStore rss = mock(RegionStateStore.class);
+    final RegionStates regionStates = new RegionStates(server, tsm, sm, rss);
+    final ServerName sn = mockServer("one", 1);
+    regionStates.updateRegionState(HRegionInfo.FIRST_META_REGIONINFO, State.SPLITTING_NEW, sn);
+    Thread backgroundThread = new Thread("Get stuck setting server offline") {
+      @Override
+      public void run() {
+        regionStates.serverOffline(sn);
+      }
+    };
+    assertTrue(latch.getNumberWaiting() == 0);
+    backgroundThread.start();
+    while (latch.getNumberWaiting() == 0);
+    // Verify I can do stuff with synchronized RegionStates methods, that I am not locked out.
+    // Below is a call that is synchronized.  Can I do it and not block?
+    regionStates.getRegionServerOfRegion(HRegionInfo.FIRST_META_REGIONINFO);
+    // Done. Trip the barrier on the background thread.
+    latch.await();
+  }
 
   @Test
   public void testWeDontReturnDrainingServersForOurBalancePlans() throws Exception {
@@ -84,4 +142,4 @@ public class TestRegionStates {
     when(serverName.getPort()).thenReturn(fakePort);
     return serverName;
   }
-}
\ No newline at end of file
+}


[27/50] [abbrv] hbase git commit: HBASE-12997 print wal pipeline on low replication.

Posted by jm...@apache.org.
HBASE-12997 print wal pipeline on low replication.


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

Branch: refs/heads/hbase-11339
Commit: 3d692cf044bc25327269328933299053ba19e2df
Parents: 9283b93
Author: Sean Busbey <bu...@apache.org>
Authored: Mon Feb 9 16:30:05 2015 -0600
Committer: Sean Busbey <bu...@apache.org>
Committed: Mon Feb 9 23:02:16 2015 -0600

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/3d692cf0/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
index 0211a17..443134d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
@@ -1399,7 +1399,8 @@ public class FSHLog implements WAL {
             LOG.warn("HDFS pipeline error detected. " + "Found "
                 + numCurrentReplicas + " replicas but expecting no less than "
                 + this.minTolerableReplication + " replicas. "
-                + " Requesting close of wal.");
+                + " Requesting close of wal. current pipeline: "
+                + Arrays.toString(getPipeLine()));
             logRollNeeded = true;
             // If rollWriter is requested, increase consecutiveLogRolls. Once it
             // is larger than lowReplicationRollLimit, disable the


[09/50] [abbrv] hbase git commit: HBASE-12979 Use setters instead of return values for handing back statistics from HRegion methods

Posted by jm...@apache.org.
HBASE-12979 Use setters instead of return values for handing back statistics from HRegion methods


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

Branch: refs/heads/hbase-11339
Commit: 75148385ee5b2065992aea19a810436196576f20
Parents: 2583e8d
Author: Jesse Yates <jy...@apache.org>
Authored: Fri Feb 6 10:42:03 2015 -0800
Committer: Jesse Yates <jy...@apache.org>
Committed: Fri Feb 6 13:24:20 2015 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/regionserver/HRegion.java    | 11 +++++------
 .../apache/hadoop/hbase/regionserver/RSRpcServices.java  |  3 ++-
 2 files changed, 7 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/75148385/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 d4a6ecf..3312df7 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
@@ -5245,18 +5245,18 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
     return results;
   }
 
-  public ClientProtos.RegionLoadStats mutateRow(RowMutations rm) throws IOException {
+  public void mutateRow(RowMutations rm) throws IOException {
     // Don't need nonces here - RowMutations only supports puts and deletes
-    return mutateRowsWithLocks(rm.getMutations(), Collections.singleton(rm.getRow()));
+    mutateRowsWithLocks(rm.getMutations(), Collections.singleton(rm.getRow()));
   }
 
   /**
    * Perform atomic mutations within the region w/o nonces.
    * See {@link #mutateRowsWithLocks(Collection, Collection, long, long)}
    */
-  public ClientProtos.RegionLoadStats mutateRowsWithLocks(Collection<Mutation> mutations,
+  public void mutateRowsWithLocks(Collection<Mutation> mutations,
       Collection<byte[]> rowsToLock) throws IOException {
-    return mutateRowsWithLocks(mutations, rowsToLock, HConstants.NO_NONCE, HConstants.NO_NONCE);
+    mutateRowsWithLocks(mutations, rowsToLock, HConstants.NO_NONCE, HConstants.NO_NONCE);
   }
 
   /**
@@ -5271,11 +5271,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
    * <code>rowsToLock</code> is sorted in order to avoid deadlocks.
    * @throws IOException
    */
-  public ClientProtos.RegionLoadStats mutateRowsWithLocks(Collection<Mutation> mutations,
+  public void mutateRowsWithLocks(Collection<Mutation> mutations,
       Collection<byte[]> rowsToLock, long nonceGroup, long nonce) throws IOException {
     MultiRowMutationProcessor proc = new MultiRowMutationProcessor(mutations, rowsToLock);
     processRowsWithLocks(proc, -1, nonceGroup, nonce);
-    return getRegionStats();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/75148385/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index 3bdb528..68530d3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -382,7 +382,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
           throw new DoNotRetryIOException("Atomic put and/or delete only, not " + type.name());
       }
     }
-    return region.mutateRow(rm);
+    region.mutateRow(rm);
+    return region.getRegionStats();
   }
 
   /**


[29/50] [abbrv] hbase git commit: HBASE-13004 Addendum fixes timeout value in message (Andrey Stepachev)

Posted by jm...@apache.org.
HBASE-13004 Addendum fixes timeout value in message (Andrey Stepachev)


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

Branch: refs/heads/hbase-11339
Commit: ede20aa2880ce509e6d3df67818ad305ca9f4eb8
Parents: a3f495f
Author: tedyu <yu...@gmail.com>
Authored: Tue Feb 10 09:43:31 2015 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Tue Feb 10 09:43:31 2015 -0800

----------------------------------------------------------------------
 hbase-common/src/test/java/org/apache/hadoop/hbase/Waiter.java | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ede20aa2/hbase-common/src/test/java/org/apache/hadoop/hbase/Waiter.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/Waiter.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/Waiter.java
index 1110f20..cd31ce5 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/Waiter.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/Waiter.java
@@ -207,11 +207,11 @@ public final class Waiter {
         } else if (failIfTimeout) {
           String msg = getExplanation(predicate);
           fail(MessageFormat
-              .format("Waiting timed out after [{0}] msec" + msg, adjustedTimeout));
+              .format("Waiting timed out after [{0}] msec", adjustedTimeout) + msg);
         } else {
           String msg = getExplanation(predicate);
           LOG.warn(
-              MessageFormat.format("Waiting timed out after [{0}] msec" + msg, adjustedTimeout));
+              MessageFormat.format("Waiting timed out after [{0}] msec", adjustedTimeout) + msg);
         }
       }
       return (eval || interrupted) ? (System.currentTimeMillis() - started) : -1;


[28/50] [abbrv] hbase git commit: HBASE-13004 Make possible to explain why HBaseTestingUtility.waitFor fails (Andrey Stepachev)

Posted by jm...@apache.org.
HBASE-13004 Make possible to explain why HBaseTestingUtility.waitFor fails (Andrey Stepachev)


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

Branch: refs/heads/hbase-11339
Commit: a3f495f20d492ab491f2af4759b2aef5d5c3bd74
Parents: 3d692cf
Author: tedyu <yu...@gmail.com>
Authored: Tue Feb 10 07:33:52 2015 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Tue Feb 10 07:33:52 2015 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/Waiter.java    |  40 +++-
 .../hadoop/hbase/HBaseTestingUtility.java       | 206 ++++++++++++-------
 .../security/access/TestAccessController.java   |   8 +-
 .../hbase/snapshot/SnapshotTestingUtils.java    |   8 +-
 4 files changed, 175 insertions(+), 87 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a3f495f2/hbase-common/src/test/java/org/apache/hadoop/hbase/Waiter.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/Waiter.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/Waiter.java
index 3453baf..1110f20 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/Waiter.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/Waiter.java
@@ -21,13 +21,13 @@ package org.apache.hadoop.hbase;
 
 import java.text.MessageFormat;
 
-import junit.framework.Assert;
-
 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 static org.junit.Assert.fail;
+
 /**
  * A class that provides a standard waitFor pattern
  * See details at https://issues.apache.org/jira/browse/HBASE-7384
@@ -98,6 +98,21 @@ public final class Waiter {
   }
 
   /**
+   * A mixin interface, can be used with {@link Waiter} to explain failed state.
+   */
+  @InterfaceAudience.Private
+  public interface ExplainingPredicate<E extends Exception> extends Predicate<E> {
+
+    /**
+     * Perform a predicate evaluation.
+     *
+     * @return explanation of failed state
+     */
+    String explainFailure() throws E;
+
+  }
+
+  /**
    * Makes the current thread sleep for the duration equal to the specified time in milliseconds
    * multiplied by the {@link #getWaitForRatio(Configuration)}.
    * @param conf the configuration
@@ -190,9 +205,13 @@ public final class Waiter {
           LOG.warn(MessageFormat.format("Waiting interrupted after [{0}] msec",
             System.currentTimeMillis() - started));
         } else if (failIfTimeout) {
-          Assert.fail(MessageFormat.format("Waiting timed out after [{0}] msec", adjustedTimeout));
+          String msg = getExplanation(predicate);
+          fail(MessageFormat
+              .format("Waiting timed out after [{0}] msec" + msg, adjustedTimeout));
         } else {
-          LOG.warn(MessageFormat.format("Waiting timed out after [{0}] msec", adjustedTimeout));
+          String msg = getExplanation(predicate);
+          LOG.warn(
+              MessageFormat.format("Waiting timed out after [{0}] msec" + msg, adjustedTimeout));
         }
       }
       return (eval || interrupted) ? (System.currentTimeMillis() - started) : -1;
@@ -201,4 +220,17 @@ public final class Waiter {
     }
   }
 
+  public static String getExplanation(Predicate explain) {
+    if (explain instanceof ExplainingPredicate) {
+      try {
+        return " " + ((ExplainingPredicate) explain).explainFailure();
+      } catch (Exception e) {
+        LOG.error("Failed to get explanation, ", e);
+        return e.getMessage();
+      }
+    } else {
+      return "";
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3f495f2/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index 8613276..37114b9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -17,10 +17,7 @@
  */
 package org.apache.hadoop.hbase;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
+import javax.annotation.Nullable;
 import java.io.File;
 import java.io.IOException;
 import java.io.OutputStream;
@@ -44,6 +41,7 @@ import java.util.Set;
 import java.util.TreeSet;
 import java.util.UUID;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.commons.lang.RandomStringUtils;
 import org.apache.commons.logging.Log;
@@ -53,6 +51,7 @@ import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
 import org.apache.hadoop.hbase.Waiter.Predicate;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
@@ -71,6 +70,7 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.fs.HFileSystem;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
@@ -122,6 +122,10 @@ import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.ZooKeeper.States;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 /**
  * Facility for testing HBase. Replacement for
  * old HBaseTestCase and HBaseClusterTestCase functionality.
@@ -2899,12 +2903,12 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    */
   public void waitTableAvailable(TableName table)
       throws InterruptedException, IOException {
-    waitTableAvailable(getHBaseAdmin(), table.getName(), 30000);
+    waitTableAvailable(table.getName(), 30000);
   }
 
-  public void waitTableAvailable(Admin admin, byte[] table)
+  public void waitTableAvailable(TableName table, long timeoutMillis)
       throws InterruptedException, IOException {
-    waitTableAvailable(admin, table, 30000);
+    waitFor(timeoutMillis, predicateTableAvailable(table));
   }
 
   /**
@@ -2916,23 +2920,54 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    */
   public void waitTableAvailable(byte[] table, long timeoutMillis)
   throws InterruptedException, IOException {
-    waitTableAvailable(getHBaseAdmin(), table, timeoutMillis);
+    waitFor(timeoutMillis, predicateTableAvailable(TableName.valueOf(table)));
+  }
+
+  public String explainTableAvailability(TableName tableName) throws IOException {
+    String msg = explainTableState(tableName, TableState.State.ENABLED) + ", ";
+    if (getHBaseCluster().getMaster().isAlive()) {
+      Map<HRegionInfo, ServerName> assignments =
+          getHBaseCluster().getMaster().getAssignmentManager().getRegionStates()
+              .getRegionAssignments();
+      final List<Pair<HRegionInfo, ServerName>> metaLocations =
+          MetaTableAccessor.getTableRegionsAndLocations(connection, tableName);
+      for (Pair<HRegionInfo, ServerName> metaLocation : metaLocations) {
+        HRegionInfo hri = metaLocation.getFirst();
+        ServerName sn = metaLocation.getSecond();
+        if (!assignments.containsKey(hri)) {
+          msg += ", region " + hri
+              + " not assigned, but found in meta, it expected to be on " + sn;
+
+        } else if (sn == null) {
+          msg += ",  region " + hri
+              + " assigned,  but has no server in meta";
+        } else if (!sn.equals(assignments.get(hri))) {
+          msg += ",  region " + hri
+              + " assigned,  but has different servers in meta and AM ( " +
+              sn + " <> " + assignments.get(hri);
+        }
+      }
+    }
+    return msg;
   }
 
-  public void waitTableAvailable(Admin admin, byte[] table, long timeoutMillis)
-  throws InterruptedException, IOException {
-    long startWait = System.currentTimeMillis();
-    while (!admin.isTableAvailable(TableName.valueOf(table))) {
-      assertTrue("Timed out waiting for table to become available " +
-        Bytes.toStringBinary(table),
-        System.currentTimeMillis() - startWait < timeoutMillis);
-      Thread.sleep(200);
+  public String explainTableState(final TableName table, TableState.State state)
+      throws IOException {
+    TableState.State tableState =
+        getHBaseCluster().getMaster().getTableStateManager().getTableState(table);
+    if (tableState == null) {
+      return "TableState: No state for table " + table;
+    } else if (!tableState.equals(state)) {
+      return "TableState: Not " + state + " state, but " + tableState;
+    } else {
+      return "TableState: OK";
     }
   }
 
   /**
    * Waits for a table to be 'enabled'.  Enabled means that table is set as 'enabled' and the
    * regions have been all assigned.  Will timeout after default period (30 seconds)
+   * Tolerates nonexistent table.
    * @param table Table to wait on.
    * @param table
    * @throws InterruptedException
@@ -2940,12 +2975,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    */
   public void waitTableEnabled(TableName table)
       throws InterruptedException, IOException {
-    waitTableEnabled(getHBaseAdmin(), table.getName(), 30000);
-  }
-
-  public void waitTableEnabled(Admin admin, byte[] table)
-      throws InterruptedException, IOException {
-    waitTableEnabled(admin, table, 30000);
+    waitTableEnabled(table, 30000);
   }
 
   /**
@@ -2959,30 +2989,12 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    */
   public void waitTableEnabled(byte[] table, long timeoutMillis)
   throws InterruptedException, IOException {
-    waitTableEnabled(getHBaseAdmin(), table, timeoutMillis);
+    waitTableEnabled(TableName.valueOf(table), timeoutMillis);
   }
 
-  public void waitTableEnabled(Admin admin, byte[] table, long timeoutMillis)
-  throws InterruptedException, IOException {
-    TableName tableName = TableName.valueOf(table);
-    long startWait = System.currentTimeMillis();
-    waitTableAvailable(admin, table, timeoutMillis);
-    while (!admin.isTableEnabled(tableName)) {
-      assertTrue("Timed out waiting for table to become available and enabled " +
-         Bytes.toStringBinary(table),
-         System.currentTimeMillis() - startWait < timeoutMillis);
-      Thread.sleep(200);
-    }
-    // Finally make sure all regions are fully open and online out on the cluster. Regions may be
-    // in the hbase:meta table and almost open on all regionservers but there setting the region
-    // online in the regionserver is the very last thing done and can take a little while to happen.
-    // Below we do a get.  The get will retry if a NotServeringRegionException or a
-    // RegionOpeningException.  It is crass but when done all will be online.
-    try {
-      Canary.sniff(admin, tableName);
-    } catch (Exception e) {
-      throw new IOException(e);
-    }
+  public void waitTableEnabled(TableName table, long timeoutMillis)
+  throws IOException {
+    waitFor(timeoutMillis, predicateTableEnabled(table));
   }
 
   /**
@@ -2994,12 +3006,12 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    */
   public void waitTableDisabled(byte[] table)
           throws InterruptedException, IOException {
-    waitTableDisabled(getHBaseAdmin(), table, 30000);
+    waitTableDisabled(table, 30000);
   }
 
-  public void waitTableDisabled(Admin admin, byte[] table)
+  public void waitTableDisabled(TableName table, long millisTimeout)
           throws InterruptedException, IOException {
-    waitTableDisabled(admin, table, 30000);
+    waitFor(millisTimeout, predicateTableDisabled(table));
   }
 
   /**
@@ -3011,19 +3023,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    */
   public void waitTableDisabled(byte[] table, long timeoutMillis)
           throws InterruptedException, IOException {
-    waitTableDisabled(getHBaseAdmin(), table, timeoutMillis);
-  }
-
-  public void waitTableDisabled(Admin admin, byte[] table, long timeoutMillis)
-          throws InterruptedException, IOException {
-    TableName tableName = TableName.valueOf(table);
-    long startWait = System.currentTimeMillis();
-    while (!admin.isTableDisabled(tableName)) {
-      assertTrue("Timed out waiting for table to become disabled " +
-                      Bytes.toStringBinary(table),
-              System.currentTimeMillis() - startWait < timeoutMillis);
-      Thread.sleep(200);
-    }
+    waitTableDisabled(TableName.valueOf(table), timeoutMillis);
   }
 
   /**
@@ -3189,7 +3189,12 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
       throws IOException {
     final Table meta = getConnection().getTable(TableName.META_TABLE_NAME);
     try {
-      waitFor(timeout, 200, true, new Predicate<IOException>() {
+      long l = waitFor(timeout, 200, true, new ExplainingPredicate<IOException>() {
+        @Override
+        public String explainFailure() throws IOException {
+          return explainTableAvailability(tableName);
+        }
+
         @Override
         public boolean evaluate() throws IOException {
           boolean allRegionsAssigned = true;
@@ -3199,7 +3204,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
           try {
             Result r;
             while ((r = s.next()) != null) {
-              byte [] b = r.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
+              byte[] b = r.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
               HRegionInfo info = HRegionInfo.parseFromOrNull(b);
               if (info != null && info.getTable().equals(tableName)) {
                 b = r.getValue(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER);
@@ -3222,7 +3227,12 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
       // returing -- sometimes this can lag.
       HMaster master = getHBaseCluster().getMaster();
       final RegionStates states = master.getAssignmentManager().getRegionStates();
-      waitFor(timeout, 200, new Predicate<IOException>() {
+      waitFor(timeout, 200, new ExplainingPredicate<IOException>() {
+        @Override
+        public String explainFailure() throws IOException {
+          return explainTableAvailability(tableName);
+        }
+
         @Override
         public boolean evaluate() throws IOException {
           List<HRegionInfo> hris = states.getRegionsOfTable(tableName);
@@ -3696,10 +3706,17 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
   /**
    * Returns a {@link Predicate} for checking that there are no regions in transition in master
    */
-  public Waiter.Predicate<Exception> predicateNoRegionsInTransition() {
-    return new Waiter.Predicate<Exception>() {
+  public ExplainingPredicate<IOException> predicateNoRegionsInTransition() {
+    return new ExplainingPredicate<IOException>() {
       @Override
-      public boolean evaluate() throws Exception {
+      public String explainFailure() throws IOException {
+        final RegionStates regionStates = getMiniHBaseCluster().getMaster()
+            .getAssignmentManager().getRegionStates();
+        return "found in transition: " + regionStates.getRegionsInTransition().toString();
+      }
+
+      @Override
+      public boolean evaluate() throws IOException {
         final RegionStates regionStates = getMiniHBaseCluster().getMaster()
             .getAssignmentManager().getRegionStates();
         return !regionStates.isRegionsInTransition();
@@ -3710,11 +3727,58 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
   /**
    * Returns a {@link Predicate} for checking that table is enabled
    */
-  public Waiter.Predicate<Exception> predicateTableEnabled(final TableName tableName) {
-    return new Waiter.Predicate<Exception>() {
-     @Override
-     public boolean evaluate() throws Exception {
-       return getHBaseAdmin().isTableEnabled(tableName);
+  public Waiter.Predicate<IOException> predicateTableEnabled(final TableName tableName) {
+    return new ExplainingPredicate<IOException>() {
+      @Override
+      public String explainFailure() throws IOException {
+        return explainTableState(tableName, TableState.State.ENABLED);
+      }
+
+      @Override
+      public boolean evaluate() throws IOException {
+        return getHBaseAdmin().tableExists(tableName) && getHBaseAdmin().isTableEnabled(tableName);
+      }
+    };
+  }
+
+  /**
+   * Returns a {@link Predicate} for checking that table is enabled
+   */
+  public Waiter.Predicate<IOException> predicateTableDisabled(final TableName tableName) {
+    return new ExplainingPredicate<IOException>() {
+      @Override
+      public String explainFailure() throws IOException {
+        return explainTableState(tableName, TableState.State.DISABLED);
+      }
+
+      @Override
+      public boolean evaluate() throws IOException {
+        return getHBaseAdmin().isTableDisabled(tableName);
+      }
+    };
+  }
+
+  /**
+   * Returns a {@link Predicate} for checking that table is enabled
+   */
+  public Waiter.Predicate<IOException> predicateTableAvailable(final TableName tableName) {
+    return new ExplainingPredicate<IOException>() {
+      @Override
+      public String explainFailure() throws IOException {
+        return explainTableAvailability(tableName);
+      }
+
+      @Override
+      public boolean evaluate() throws IOException {
+        boolean tableAvailable = getHBaseAdmin().isTableAvailable(tableName);
+        if (tableAvailable) {
+          try {
+            Canary.sniff(getHBaseAdmin(), tableName);
+          } catch (Exception e) {
+            throw new IOException("Canary sniff failed for table " + tableName, e);
+          }
+        }
+        return tableAvailable;
       }
     };
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3f495f2/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
index 3c8fea5..8296e81 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
@@ -927,11 +927,9 @@ public class TestAccessController extends SecureTestUtil {
       setPermission(loadPath, FsPermission.valueOf("-rwxrwxrwx"));
 
       try (HTable table = (HTable)TEST_UTIL.getConnection().getTable(tableName)) {
-        try (Admin admin = TEST_UTIL.getHBaseAdmin()) {
-          TEST_UTIL.waitTableEnabled(admin, tableName.getName());
-          LoadIncrementalHFiles loader = new LoadIncrementalHFiles(conf);
-          loader.doBulkLoad(loadPath, table);
-        }
+        TEST_UTIL.waitTableEnabled(tableName);
+        LoadIncrementalHFiles loader = new LoadIncrementalHFiles(conf);
+        loader.doBulkLoad(loadPath, table);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3f495f2/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java
index a8588cc..b663a2a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java
@@ -635,13 +635,7 @@ public class SnapshotTestingUtils {
       region.waitForFlushesAndCompactions();
     }
     // Wait up to 60 seconds for a table to be available.
-    final HBaseAdmin hBaseAdmin = util.getHBaseAdmin();
-    util.waitFor(60000, new Waiter.Predicate<IOException>() {
-      @Override
-      public boolean evaluate() throws IOException {
-        return hBaseAdmin.isTableAvailable(tableName);
-      }
-    });
+    util.waitFor(60000, util.predicateTableAvailable(tableName));
   }
 
   public static void createTable(final HBaseTestingUtility util, final TableName tableName,


[34/50] [abbrv] hbase git commit: HBASE-12978 Region goes permanently offline (WAS: hbase:meta has a row missing hregioninfo and it causes my long-running job to fail)

Posted by jm...@apache.org.
HBASE-12978 Region goes permanently offline (WAS: hbase:meta has a row missing hregioninfo and it causes my long-running job to fail)


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

Branch: refs/heads/hbase-11339
Commit: 9c66afbae752b06b058df34c0e40fbbf54426ac3
Parents: f9cf565
Author: stack <st...@apache.org>
Authored: Tue Feb 10 17:03:08 2015 -0800
Committer: stack <st...@apache.org>
Committed: Tue Feb 10 17:03:29 2015 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/CellComparator.java | 20 +++++++++++++--
 .../apache/hadoop/hbase/TestCellComparator.java | 26 ++++++++++++++------
 .../hadoop/hbase/io/hfile/HFileWriterV2.java    |  8 +++---
 3 files changed, 40 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/9c66afba/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java
index d760aa2..cbb7ff3 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java
@@ -178,11 +178,19 @@ public class CellComparator implements Comparator<Cell>, Serializable {
     return compareWithoutRow(left, right);
   }
 
+  /**
+   * Do not use comparing rows from hbase:meta. Meta table Cells have schema (table,startrow,hash)
+   * so can't be treated as plain byte arrays as this method does.
+   */
   public static int compareRows(final Cell left, final Cell right) {
     return Bytes.compareTo(left.getRowArray(), left.getRowOffset(), left.getRowLength(),
         right.getRowArray(), right.getRowOffset(), right.getRowLength());
   }
 
+  /**
+   * Do not use comparing rows from hbase:meta. Meta table Cells have schema (table,startrow,hash)
+   * so can't be treated as plain byte arrays as this method does.
+   */
   public static int compareRows(byte[] left, int loffset, int llength, byte[] right, int roffset,
       int rlength) {
     return Bytes.compareTo(left, loffset, llength, right, roffset, rlength);
@@ -375,14 +383,16 @@ public class CellComparator implements Comparator<Cell>, Serializable {
 
   /**
    * Try to return a Cell that falls between <code>left</code> and <code>right</code> but that is
-   * shorter; i.e. takes up less space. This is trick is used building HFile block index.
+   * shorter; i.e. takes up less space. This trick is used building HFile block index.
    * Its an optimization. It does not always work.  In this case we'll just return the
    * <code>right</code> cell.
+   * @param comparator Comparator to use.
    * @param left
    * @param right
    * @return A cell that sorts between <code>left</code> and <code>right</code>.
    */
-  public static Cell getMidpoint(final Cell left, final Cell right) {
+  public static Cell getMidpoint(final KeyValue.KVComparator comparator, final Cell left,
+      final Cell right) {
     // TODO: Redo so only a single pass over the arrays rather than one to compare and then a
     // second composing midpoint.
     if (right == null) {
@@ -391,6 +401,12 @@ public class CellComparator implements Comparator<Cell>, Serializable {
     if (left == null) {
       return right;
     }
+    // If Cells from meta table, don't mess around. meta table Cells have schema
+    // (table,startrow,hash) so can't be treated as plain byte arrays. Just skip out without
+    // trying to do this optimization.
+    if (comparator != null && comparator instanceof KeyValue.MetaComparator) {
+      return right;
+    }
     int diff = compareRows(left, right);
     if (diff > 0) {
       throw new IllegalArgumentException("Left row sorts after right row; left=" +

http://git-wip-us.apache.org/repos/asf/hbase/blob/9c66afba/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java
index d6a2f72..007f826 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
+import org.apache.hadoop.hbase.KeyValue.KVComparator;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
@@ -79,56 +80,65 @@ public class TestCellComparator {
 
   @Test
   public void testGetShortMidpoint() {
+    KeyValue.KVComparator comparator = new KeyValue.KVComparator();
+
     Cell left = CellUtil.createCell(Bytes.toBytes("a"), Bytes.toBytes("a"), Bytes.toBytes("a"));
     Cell right = CellUtil.createCell(Bytes.toBytes("a"), Bytes.toBytes("a"), Bytes.toBytes("a"));
-    Cell mid = CellComparator.getMidpoint(left, right);
+    Cell mid = CellComparator.getMidpoint(comparator, left, right);
     assertTrue(CellComparator.compare(left, mid, true) <= 0);
     assertTrue(CellComparator.compare(mid, right, true) <= 0);
 
     left = CellUtil.createCell(Bytes.toBytes("a"), Bytes.toBytes("a"), Bytes.toBytes("a"));
     right = CellUtil.createCell(Bytes.toBytes("b"), Bytes.toBytes("a"), Bytes.toBytes("a"));
-    mid = CellComparator.getMidpoint(left, right);
+    mid = CellComparator.getMidpoint(comparator, left, right);
     assertTrue(CellComparator.compare(left, mid, true) < 0);
     assertTrue(CellComparator.compare(mid, right, true) <= 0);
 
     left = CellUtil.createCell(Bytes.toBytes("g"), Bytes.toBytes("a"), Bytes.toBytes("a"));
     right = CellUtil.createCell(Bytes.toBytes("i"), Bytes.toBytes("a"), Bytes.toBytes("a"));
-    mid = CellComparator.getMidpoint(left, right);
+    mid = CellComparator.getMidpoint(comparator, left, right);
     assertTrue(CellComparator.compare(left, mid, true) < 0);
     assertTrue(CellComparator.compare(mid, right, true) <= 0);
 
     left = CellUtil.createCell(Bytes.toBytes("a"), Bytes.toBytes("a"), Bytes.toBytes("a"));
     right = CellUtil.createCell(Bytes.toBytes("bbbbbbb"), Bytes.toBytes("a"), Bytes.toBytes("a"));
-    mid = CellComparator.getMidpoint(left, right);
+    mid = CellComparator.getMidpoint(comparator, left, right);
     assertTrue(CellComparator.compare(left, mid, true) < 0);
     assertTrue(CellComparator.compare(mid, right, true) < 0);
     assertEquals(1, (int)mid.getRowLength());
 
     left = CellUtil.createCell(Bytes.toBytes("a"), Bytes.toBytes("a"), Bytes.toBytes("a"));
     right = CellUtil.createCell(Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("a"));
-    mid = CellComparator.getMidpoint(left, right);
+    mid = CellComparator.getMidpoint(comparator, left, right);
     assertTrue(CellComparator.compare(left, mid, true) < 0);
     assertTrue(CellComparator.compare(mid, right, true) <= 0);
 
     left = CellUtil.createCell(Bytes.toBytes("a"), Bytes.toBytes("a"), Bytes.toBytes("a"));
     right = CellUtil.createCell(Bytes.toBytes("a"), Bytes.toBytes("aaaaaaaa"), Bytes.toBytes("b"));
-    mid = CellComparator.getMidpoint(left, right);
+    mid = CellComparator.getMidpoint(comparator, left, right);
     assertTrue(CellComparator.compare(left, mid, true) < 0);
     assertTrue(CellComparator.compare(mid, right, true) < 0);
     assertEquals(2, (int)mid.getFamilyLength());
 
     left = CellUtil.createCell(Bytes.toBytes("a"), Bytes.toBytes("a"), Bytes.toBytes("a"));
     right = CellUtil.createCell(Bytes.toBytes("a"), Bytes.toBytes("a"), Bytes.toBytes("aaaaaaaaa"));
-    mid = CellComparator.getMidpoint(left, right);
+    mid = CellComparator.getMidpoint(comparator, left, right);
     assertTrue(CellComparator.compare(left, mid, true) < 0);
     assertTrue(CellComparator.compare(mid, right, true) < 0);
     assertEquals(2, (int)mid.getQualifierLength());
 
     left = CellUtil.createCell(Bytes.toBytes("a"), Bytes.toBytes("a"), Bytes.toBytes("a"));
     right = CellUtil.createCell(Bytes.toBytes("a"), Bytes.toBytes("a"), Bytes.toBytes("b"));
-    mid = CellComparator.getMidpoint(left, right);
+    mid = CellComparator.getMidpoint(comparator, left, right);
     assertTrue(CellComparator.compare(left, mid, true) < 0);
     assertTrue(CellComparator.compare(mid, right, true) <= 0);
     assertEquals(1, (int)mid.getQualifierLength());
+
+    // Assert that if meta comparator, it returns the right cell -- i.e.  no optimization done.
+    left = CellUtil.createCell(Bytes.toBytes("g"), Bytes.toBytes("a"), Bytes.toBytes("a"));
+    right = CellUtil.createCell(Bytes.toBytes("i"), Bytes.toBytes("a"), Bytes.toBytes("a"));
+    mid = CellComparator.getMidpoint(new KeyValue.MetaComparator(), left, right);
+    assertTrue(CellComparator.compare(left, mid, true) < 0);
+    assertTrue(CellComparator.compare(mid, right, true) == 0);
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/9c66afba/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV2.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV2.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV2.java
index 1df8bc2..28c4655 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV2.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV2.java
@@ -162,9 +162,8 @@ public class HFileWriterV2 extends AbstractHFileWriter {
     fsBlockWriter.writeHeaderAndData(outputStream);
     int onDiskSize = fsBlockWriter.getOnDiskSizeWithHeader();
 
-    // Rather than CellComparator, we should be making use of an Interface here with the
-    // implementation class serialized out to the HFile metadata. TODO.
-    Cell indexEntry = CellComparator.getMidpoint(lastCellOfPreviousBlock, firstCellInBlock);
+    Cell indexEntry =
+      CellComparator.getMidpoint(this.comparator, lastCellOfPreviousBlock, firstCellInBlock);
     dataBlockIndexWriter.addEntry(CellUtil.getCellKeySerializedAsKeyValueKey(indexEntry),
       lastDataBlockOffset, onDiskSize);
     totalUncompressedBytes += fsBlockWriter.getUncompressedSizeWithHeader();
@@ -264,8 +263,9 @@ public class HFileWriterV2 extends AbstractHFileWriter {
       checkBlockBoundary();
     }
 
-    if (!fsBlockWriter.isWriting())
+    if (!fsBlockWriter.isWriting()) {
       newBlock();
+    }
 
     fsBlockWriter.write(cell);
 


[42/50] [abbrv] hbase git commit: HBASE-13018 WALSplitter should not try to get table states while splitting META (Andrey Stepachev)

Posted by jm...@apache.org.
HBASE-13018 WALSplitter should not try to get table states while splitting META (Andrey Stepachev)


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

Branch: refs/heads/hbase-11339
Commit: 3dd220f8fdf6e34e11ffa309ff182172f6346091
Parents: dc522d0
Author: tedyu <yu...@gmail.com>
Authored: Wed Feb 11 12:45:54 2015 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Wed Feb 11 12:45:54 2015 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/wal/WALSplitter.java    | 37 +++++++++++++-------
 .../master/TestDistributedLogSplitting.java     |  1 +
 2 files changed, 25 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/3dd220f8/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
index c187af1..4d8cc2d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
@@ -65,6 +65,7 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
 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.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
@@ -146,8 +147,8 @@ public class WALSplitter {
   OutputSink outputSink;
   EntryBuffers entryBuffers;
 
-  private Set<TableName> disablingOrDisabledTables =
-      new HashSet<TableName>();
+  private Map<TableName, TableState> tableStatesCache =
+      new ConcurrentHashMap<>();
   private BaseCoordinatedStateManager csm;
   private final WALFactory walFactory;
 
@@ -303,16 +304,6 @@ public class WALSplitter {
         LOG.warn("Nothing to split in log file " + logPath);
         return true;
       }
-      if(csm != null) {
-        HConnection scc = csm.getServer().getConnection();
-        TableName[] tables = scc.listTableNames();
-        for (TableName table : tables) {
-          if (scc.getTableState(table)
-              .inStates(TableState.State.DISABLED, TableState.State.DISABLING)) {
-            disablingOrDisabledTables.add(table);
-          }
-        }
-      }
       int numOpenedFilesBeforeReporting = conf.getInt("hbase.splitlog.report.openedfiles", 3);
       int numOpenedFilesLastCheck = 0;
       outputSink.setReporter(reporter);
@@ -1625,7 +1616,7 @@ public class WALSplitter {
       }
 
       // check if current region in a disabling or disabled table
-      if (disablingOrDisabledTables.contains(buffer.tableName)) {
+      if (isTableDisabledOrDisabling(buffer.tableName)) {
         // need fall back to old way
         logRecoveredEditsOutputSink.append(buffer);
         hasEditsInDisablingOrDisabledTables = true;
@@ -2057,6 +2048,26 @@ public class WALSplitter {
       return this.recoveredRegions.size();
     }
 
+    private boolean isTableDisabledOrDisabling(TableName tableName) {
+      if (csm == null)
+        return false; // we can't get state without CoordinatedStateManager
+      if (tableName.isSystemTable())
+        return false; // assume that system tables never can be disabled
+      TableState tableState = tableStatesCache.get(tableName);
+      if (tableState == null) {
+        try {
+          tableState =
+              MetaTableAccessor.getTableState(csm.getServer().getConnection(), tableName);
+          if (tableState != null)
+            tableStatesCache.put(tableName, tableState);
+        } catch (IOException e) {
+          LOG.warn("State is not accessible for table " + tableName, e);
+        }
+      }
+      return tableState != null && tableState
+          .inStates(TableState.State.DISABLED, TableState.State.DISABLING);
+    }
+
     /**
      * Get a writer and path for a log starting at the given entry. This function is threadsafe so
      * long as multiple threads are always acting on different regions.

http://git-wip-us.apache.org/repos/asf/hbase/blob/3dd220f8/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
index f237631..0038d71 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
@@ -807,6 +807,7 @@ public class TestDistributedLogSplitting {
 
     LOG.info("Disabling table\n");
     TEST_UTIL.getHBaseAdmin().disableTable(TableName.valueOf("disableTable"));
+    TEST_UTIL.waitTableDisabled(TableName.valueOf("disableTable").getName());
 
     // abort RS
     LOG.info("Aborting region server: " + hrs.getServerName());


[32/50] [abbrv] hbase git commit: HBASE-12999 Make foreground_start return the correct exit code

Posted by jm...@apache.org.
HBASE-12999 Make foreground_start return the correct exit code


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

Branch: refs/heads/hbase-11339
Commit: a7d93155f10184942ffc146f0694e5da83c78cd4
Parents: fa852c4
Author: Elliott Clark <ec...@apache.org>
Authored: Mon Feb 9 18:26:10 2015 -0800
Committer: Elliott Clark <ec...@apache.org>
Committed: Tue Feb 10 14:36:29 2015 -0800

----------------------------------------------------------------------
 bin/hbase-daemon.sh | 12 ++++++------
 1 file changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a7d93155/bin/hbase-daemon.sh
----------------------------------------------------------------------
diff --git a/bin/hbase-daemon.sh b/bin/hbase-daemon.sh
index 94ab2d1..e411dad 100755
--- a/bin/hbase-daemon.sh
+++ b/bin/hbase-daemon.sh
@@ -205,15 +205,15 @@ case $startStop in
     # Add to the command log file vital stats on our environment.
     echo "`date` Starting $command on `hostname`" >> ${HBASE_LOGLOG}
     `ulimit -a` >> "$HBASE_LOGLOG" 2>&1
-    nice -n $HBASE_NICENESS "$HBASE_HOME"/bin/hbase \
-        --config "${HBASE_CONF_DIR}" \
-        $command "$@" start >> ${HBASE_LOGOUT} 2>&1 &
-    echo $! > ${HBASE_PID}
     # in case the parent shell gets the kill make sure to trap signals.
     # Only one will get called. Either the trap or the flow will go through.
     trap cleanAfterRun SIGHUP SIGINT SIGTERM EXIT
-    wait
-    cleanAfterRun
+    nice -n $HBASE_NICENESS "$HBASE_HOME"/bin/hbase \
+        --config "${HBASE_CONF_DIR}" \
+        $command "$@" start >> ${HBASE_LOGOUT} 2>&1 &
+    hbase_pid=$!
+    echo $hbase_pid > ${HBASE_PID}
+    wait $hbase_pid
   ;;
 
 (internal_autorestart)


[40/50] [abbrv] hbase git commit: HBASE-12989 region_mover.rb unloadRegions method uses ArrayList concurrently resulting in errors (Abhishek Singh Chouhan)

Posted by jm...@apache.org.
HBASE-12989 region_mover.rb unloadRegions method uses ArrayList concurrently resulting in errors (Abhishek Singh Chouhan)


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

Branch: refs/heads/hbase-11339
Commit: 3f7b18cefce3318b4268985004dead20a26fccf1
Parents: dec0ec2
Author: Andrew Purtell <ap...@apache.org>
Authored: Wed Feb 11 12:12:48 2015 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Feb 11 12:12:48 2015 -0800

----------------------------------------------------------------------
 bin/region_mover.rb | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/3f7b18ce/bin/region_mover.rb
----------------------------------------------------------------------
diff --git a/bin/region_mover.rb b/bin/region_mover.rb
index 78979d9..565b0d5 100644
--- a/bin/region_mover.rb
+++ b/bin/region_mover.rb
@@ -307,7 +307,7 @@ def unloadRegions(options, hostname, port)
     puts "No regions were moved - there was no server available"
     exit 4
   end
-  movedRegions = java.util.ArrayList.new()
+  movedRegions = java.util.Collections.synchronizedList(java.util.ArrayList.new())
   while true
     rs = getRegions(config, servername)
     # Remove those already tried to move


[16/50] [abbrv] hbase git commit: HBASE-11567 Write bulk load COMMIT events to WAL (Alex Newman, Jeffrey Zhong)

Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/3f442773/hbase-protocol/src/main/protobuf/WAL.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/WAL.proto b/hbase-protocol/src/main/protobuf/WAL.proto
index f8a1534..169a9b2 100644
--- a/hbase-protocol/src/main/protobuf/WAL.proto
+++ b/hbase-protocol/src/main/protobuf/WAL.proto
@@ -22,6 +22,7 @@ option java_generate_equals_and_hash = true;
 option optimize_for = SPEED;
 
 import "HBase.proto";
+import "Client.proto";
 
 message WALHeader {
   optional bool has_compression = 1;
@@ -123,6 +124,22 @@ message FlushDescriptor {
   repeated StoreFlushDescriptor store_flushes = 5;
 }
 
+message StoreDescriptor {
+  required bytes family_name = 1;
+  required string store_home_dir = 2; //relative to region dir
+  repeated string store_file = 3; // relative to store dir
+}
+
+/**
+ * Special WAL entry used for writing bulk load events to WAL
+ */
+message BulkLoadDescriptor {
+  required TableName table_name = 1;
+  required bytes encoded_region_name = 2;
+  repeated StoreDescriptor stores = 3;
+  required int64 bulkload_seq_num = 4;
+}
+
 /**
  * Special WAL entry to hold all related to a region event (open/close).
  */
@@ -132,12 +149,6 @@ message RegionEventDescriptor {
     REGION_CLOSE = 1;
   }
 
-  message StoreDescriptor {
-    required bytes family_name = 1;
-    required string store_home_dir = 2; //relative to region dir
-    repeated string store_file = 3; // relative to store dir
-  }
-
   required EventType event_type = 1;
   required bytes table_name = 2;
   required bytes encoded_region_name = 3;

http://git-wip-us.apache.org/repos/asf/hbase/blob/3f442773/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 3312df7..53e732a 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
@@ -61,6 +61,7 @@ import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
+import com.google.protobuf.ByteString;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -128,6 +129,7 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoRespo
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor.FlushAction;
@@ -141,6 +143,7 @@ import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
+import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CancelableProgressable;
 import org.apache.hadoop.hbase.util.ClassSize;
@@ -221,7 +224,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
    * This is the global default value for durability. All tables/mutations not
    * defining a durability or using USE_DEFAULT will default to this value.
    */
-  private static final Durability DEFAULT_DURABLITY = Durability.SYNC_WAL;
+  private static final Durability DEFAULT_DURABILITY = Durability.SYNC_WAL;
 
   final AtomicBoolean closed = new AtomicBoolean(false);
   /* Closing can take some time; use the closing flag if there is stuff we don't
@@ -664,7 +667,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
     this.rowProcessorTimeout = conf.getLong(
         "hbase.hregion.row.processor.timeout", DEFAULT_ROW_PROCESSOR_TIMEOUT);
     this.durability = htd.getDurability() == Durability.USE_DEFAULT
-        ? DEFAULT_DURABLITY
+        ? DEFAULT_DURABILITY
         : htd.getDurability();
     if (rsServices != null) {
       this.rsAccounting = this.rsServices.getRegionServerAccounting();
@@ -753,7 +756,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
   }
 
   private long initializeRegionInternals(final CancelableProgressable reporter,
-      final MonitoredTask status) throws IOException, UnsupportedEncodingException {
+      final MonitoredTask status) throws IOException {
     if (coprocessorHost != null) {
       status.setStatus("Running coprocessor pre-open hook");
       coprocessorHost.preOpen();
@@ -830,7 +833,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
   }
 
   private long initializeRegionStores(final CancelableProgressable reporter, MonitoredTask status)
-      throws IOException, UnsupportedEncodingException {
+      throws IOException {
     // Load in all the HStores.
 
     long maxSeqId = -1;
@@ -2001,8 +2004,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
           long oldestUnflushedSeqId = wal.getEarliestMemstoreSeqNum(encodedRegionName);
           // no oldestUnflushedSeqId means we flushed all stores.
           // or the unflushed stores are all empty.
-          flushedSeqId =
-              oldestUnflushedSeqId == HConstants.NO_SEQNUM ? flushOpSeqId : oldestUnflushedSeqId - 1;
+          flushedSeqId = (oldestUnflushedSeqId == HConstants.NO_SEQNUM) ? flushOpSeqId
+              : oldestUnflushedSeqId - 1;
         } else {
           // use the provided sequence Id as WAL is not being used for this flush.
           flushedSeqId = flushOpSeqId = myseqid;
@@ -2265,7 +2268,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
    return getScanner(scan, null);
   }
 
-  void prepareScanner(Scan scan) throws IOException {
+  void prepareScanner(Scan scan) {
     if(!scan.hasFamilies()) {
       // Adding all families to scanner
       for(byte[] family: this.htableDescriptor.getFamiliesKeys()){
@@ -3241,7 +3244,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
       closeRegionOperation();
     }
   }
-  private void doBatchMutate(Mutation mutation) throws IOException, DoNotRetryIOException {
+  private void doBatchMutate(Mutation mutation) throws IOException {
     // Currently this is only called for puts and deletes, so no nonces.
     OperationStatus[] batchMutate = this.batchMutate(new Mutation[] { mutation },
         HConstants.NO_NONCE, HConstants.NO_NONCE);
@@ -3595,7 +3598,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
   protected long replayRecoveredEditsIfAny(final Path regiondir,
       Map<byte[], Long> maxSeqIdInStores,
       final CancelableProgressable reporter, final MonitoredTask status)
-      throws UnsupportedEncodingException, IOException {
+      throws IOException {
     long minSeqIdForTheRegion = -1;
     for (Long maxSeqIdInStore : maxSeqIdInStores.values()) {
       if (maxSeqIdInStore < minSeqIdForTheRegion || minSeqIdForTheRegion == -1) {
@@ -4101,7 +4104,17 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
     return multipleFamilies;
   }
 
-
+  /**
+   * Bulk load a/many HFiles into this region
+   *
+   * @param familyPaths A list which maps column families to the location of the HFile to load
+   *                    into that column family region.
+   * @param assignSeqId Force a flush, get it's sequenceId to preserve the guarantee that all the
+   *                    edits lower than the highest sequential ID from all the HFiles are flushed
+   *                    on disk.
+   * @return true if successful, false if failed recoverably
+   * @throws IOException if failed unrecoverably.
+   */
   public boolean bulkLoadHFiles(List<Pair<byte[], String>> familyPaths,
                                 boolean assignSeqId) throws IOException {
     return bulkLoadHFiles(familyPaths, assignSeqId, null);
@@ -4111,14 +4124,19 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
    * Attempts to atomically load a group of hfiles.  This is critical for loading
    * rows with multiple column families atomically.
    *
-   * @param familyPaths List of Pair<byte[] column family, String hfilePath>
+   * @param familyPaths      List of Pair<byte[] column family, String hfilePath>
    * @param bulkLoadListener Internal hooks enabling massaging/preparation of a
-   * file about to be bulk loaded
+   *                         file about to be bulk loaded
+   * @param assignSeqId      Force a flush, get it's sequenceId to preserve the guarantee that 
+   *                         all the edits lower than the highest sequential ID from all the 
+   *                         HFiles are flushed on disk.
    * @return true if successful, false if failed recoverably
    * @throws IOException if failed unrecoverably.
    */
   public boolean bulkLoadHFiles(List<Pair<byte[], String>> familyPaths, boolean assignSeqId,
       BulkLoadListener bulkLoadListener) throws IOException {
+    long seqId = -1;
+    Map<byte[], List<Path>> storeFiles = new TreeMap<byte[], List<Path>>(Bytes.BYTES_COMPARATOR);
     Preconditions.checkNotNull(familyPaths);
     // we need writeLock for multi-family bulk load
     startBulkRegionOperation(hasMultipleColumnFamilies(familyPaths));
@@ -4164,7 +4182,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
         StringBuilder list = new StringBuilder();
         for (Pair<byte[], String> p : failures) {
           list.append("\n").append(Bytes.toString(p.getFirst())).append(" : ")
-            .append(p.getSecond());
+              .append(p.getSecond());
         }
         // problem when validating
         LOG.warn("There was a recoverable bulk load failure likely due to a" +
@@ -4172,7 +4190,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
         return false;
       }
 
-      long seqId = -1;
       // We need to assign a sequential ID that's in between two memstores in order to preserve
       // the guarantee that all the edits lower than the highest sequential ID from all the
       // HFiles are flushed on disk. See HBASE-10958.  The sequence id returned when we flush is
@@ -4196,11 +4213,19 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
         Store store = getStore(familyName);
         try {
           String finalPath = path;
-          if(bulkLoadListener != null) {
+          if (bulkLoadListener != null) {
             finalPath = bulkLoadListener.prepareBulkLoad(familyName, path);
           }
           store.bulkLoadHFile(finalPath, seqId);
-          if(bulkLoadListener != null) {
+          
+          if(storeFiles.containsKey(familyName)) {
+            storeFiles.get(familyName).add(new Path(finalPath));
+          } else {
+            List<Path> storeFileNames = new ArrayList<Path>();
+            storeFileNames.add(new Path(finalPath));
+            storeFiles.put(familyName, storeFileNames);
+          }
+          if (bulkLoadListener != null) {
             bulkLoadListener.doneBulkLoad(familyName, path);
           }
         } catch (IOException ioe) {
@@ -4209,20 +4234,38 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
 
           // TODO Need a better story for reverting partial failures due to HDFS.
           LOG.error("There was a partial failure due to IO when attempting to" +
-              " load " + Bytes.toString(p.getFirst()) + " : "+ p.getSecond(), ioe);
-          if(bulkLoadListener != null) {
+              " load " + Bytes.toString(p.getFirst()) + " : " + p.getSecond(), ioe);
+          if (bulkLoadListener != null) {
             try {
               bulkLoadListener.failedBulkLoad(familyName, path);
             } catch (Exception ex) {
-              LOG.error("Error while calling failedBulkLoad for family "+
-                  Bytes.toString(familyName)+" with path "+path, ex);
+              LOG.error("Error while calling failedBulkLoad for family " +
+                  Bytes.toString(familyName) + " with path " + path, ex);
             }
           }
           throw ioe;
         }
       }
+
       return true;
     } finally {
+      if (wal != null && !storeFiles.isEmpty()) {
+        // write a bulk load event when not all hfiles are loaded
+        try {
+          WALProtos.BulkLoadDescriptor loadDescriptor = ProtobufUtil.toBulkLoadDescriptor(
+              this.getRegionInfo().getTable(),
+              ByteStringer.wrap(this.getRegionInfo().getEncodedNameAsBytes()), storeFiles, seqId);
+          WALUtil.writeBulkLoadMarkerAndSync(wal, this.htableDescriptor, getRegionInfo(),
+              loadDescriptor, sequenceId);
+        } catch (IOException ioe) {
+          if (this.rsServices != null) {
+            // Have to abort region server because some hfiles has been loaded but we can't write
+            // the event into WAL
+            this.rsServices.abort("Failed to write bulk load event into WAL.", ioe);
+          }
+        }
+      }
+    
       closeBulkRegionOperation();
     }
   }
@@ -5340,8 +5383,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
         doProcessRowWithTimeout(
             processor, now, this, null, null, timeout);
         processor.postProcess(this, walEdit, true);
-      } catch (IOException e) {
-        throw e;
       } finally {
         closeRegionOperation();
       }
@@ -5460,8 +5501,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
       // 14. Run post-process hook
       processor.postProcess(this, walEdit, walSyncSuccessful);
 
-    } catch (IOException e) {
-      throw e;
     } finally {
       closeRegionOperation();
       if (!mutations.isEmpty() &&
@@ -5621,8 +5660,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
                   }
                 }
                 if (cell.getTagsLength() > 0) {
-                  Iterator<Tag> i  = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(),
-                    cell.getTagsLength());
+                  Iterator<Tag> i  = CellUtil.tagsIterator(cell.getTagsArray(), 
+                    cell.getTagsOffset(), cell.getTagsLength());
                   while (i.hasNext()) {
                     newTags.add(i.next());
                   }
@@ -6534,7 +6573,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
   }
 
   /**
-   * A mocked list implementaion - discards all updates.
+   * A mocked list implementation - discards all updates.
    */
   private static final List<Cell> MOCKED_LIST = new AbstractList<Cell>() {
 
@@ -6785,7 +6824,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
   }
 
   /**
-   * Explictly sync wal
+   * Explicitly sync wal
    * @throws IOException
    */
   public void syncWal() throws IOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/3f442773/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java
index 05cead2..39d0536 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.codec.Codec;
 import org.apache.hadoop.hbase.io.HeapSize;
+import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor;
@@ -90,6 +91,7 @@ public class WALEdit implements Writable, HeapSize {
   static final byte[] COMPACTION = Bytes.toBytes("HBASE::COMPACTION");
   static final byte [] FLUSH = Bytes.toBytes("HBASE::FLUSH");
   static final byte [] REGION_EVENT = Bytes.toBytes("HBASE::REGION_EVENT");
+  public static final byte [] BULK_LOAD = Bytes.toBytes("HBASE::BULK_LOAD");
 
   private final int VERSION_2 = -1;
   private final boolean isReplay;
@@ -294,7 +296,7 @@ public class WALEdit implements Writable, HeapSize {
   }
 
   /**
-   * Create a compacion WALEdit
+   * Create a compaction WALEdit
    * @param c
    * @return A WALEdit that has <code>c</code> serialized as its value
    */
@@ -326,4 +328,33 @@ public class WALEdit implements Writable, HeapSize {
     }
     return null;
   }
-}
+
+  /**
+   * Create a bulk loader WALEdit
+   *
+   * @param hri                The HRegionInfo for the region in which we are bulk loading
+   * @param bulkLoadDescriptor The descriptor for the Bulk Loader
+   * @return The WALEdit for the BulkLoad
+   */
+  public static WALEdit createBulkLoadEvent(HRegionInfo hri,
+                                            WALProtos.BulkLoadDescriptor bulkLoadDescriptor) {
+    KeyValue kv = new KeyValue(getRowForRegion(hri),
+        METAFAMILY,
+        BULK_LOAD,
+        EnvironmentEdgeManager.currentTime(),
+        bulkLoadDescriptor.toByteArray());
+    return new WALEdit().add(kv);
+  }
+  
+  /**
+   * Deserialized and returns a BulkLoadDescriptor from the passed in Cell
+   * @param cell the key value
+   * @return deserialized BulkLoadDescriptor or null.
+   */
+  public static WALProtos.BulkLoadDescriptor getBulkLoadDescriptor(Cell cell) throws IOException {
+    if (CellUtil.matchingColumn(cell, METAFAMILY, BULK_LOAD)) {
+      return WALProtos.BulkLoadDescriptor.parseFrom(cell.getValue());
+    }
+    return null;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/3f442773/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java
index 5f00643..94ef072 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java
@@ -20,14 +20,17 @@
 package org.apache.hadoop.hbase.regionserver.wal;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor;
@@ -97,5 +100,41 @@ public class WALUtil {
     }
     return trx;
   }
+
+  /**
+   * Write a log marker that a bulk load has succeeded and is about to be committed.
+   *
+   * @param wal        The log to write into.
+   * @param htd        A description of the table that we are bulk loading into.
+   * @param info       A description of the region in the table that we are bulk loading into.
+   * @param descriptor A protocol buffers based description of the client's bulk loading request
+   * @param sequenceId The current sequenceId in the log at the time when we were to write the
+   *                   bulk load marker.
+   * @return txid of this transaction or if nothing to do, the last txid
+   * @throws IOException We will throw an IOException if we can not append to the HLog.
+   */
+  public static long writeBulkLoadMarkerAndSync(final WAL wal,
+                                                final HTableDescriptor htd,
+                                                final HRegionInfo info,
+                                                final WALProtos.BulkLoadDescriptor descriptor,
+                                                final AtomicLong sequenceId) throws IOException {
+    TableName tn = info.getTable();
+    WALKey key = new HLogKey(info.getEncodedNameAsBytes(), tn);
+
+    // Add it to the log but the false specifies that we don't need to add it to the memstore
+    long trx = wal.append(htd,
+            info,
+            key,
+            WALEdit.createBulkLoadEvent(info, descriptor),
+            sequenceId,
+            false,
+            new ArrayList<Cell>());
+    wal.sync(trx);
+
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Appended Bulk Load marker " + TextFormat.shortDebugString(descriptor));
+    }
+    return trx;
+  }
   
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/3f442773/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java
new file mode 100644
index 0000000..15dbef5
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java
@@ -0,0 +1,312 @@
+/**
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.io.hfile.HFileContext;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor;
+import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALKey;
+import org.hamcrest.Description;
+import org.hamcrest.Matcher;
+import org.hamcrest.TypeSafeMatcher;
+import org.jmock.Expectations;
+import org.jmock.integration.junit4.JUnitRuleMockery;
+import org.jmock.lib.concurrent.Synchroniser;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TemporaryFolder;
+import org.junit.rules.TestName;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static java.util.Arrays.asList;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * This class attempts to unit test bulk HLog loading.
+ */
+@Category(SmallTests.class)
+public class TestBulkLoad {
+
+  @ClassRule
+  public static TemporaryFolder testFolder = new TemporaryFolder();
+  @Rule
+  public final JUnitRuleMockery context = new JUnitRuleMockery() {{
+    setThreadingPolicy(new Synchroniser());
+  }};
+  private final WAL log = context.mock(WAL.class);
+  private final Configuration conf = HBaseConfiguration.create();
+  private final Random random = new Random();
+  private final byte[] randomBytes = new byte[100];
+  private final byte[] family1 = Bytes.toBytes("family1");
+  private final byte[] family2 = Bytes.toBytes("family2");
+  private final Expectations callOnce;
+  @Rule
+  public TestName name = new TestName();
+  
+  public TestBulkLoad() throws IOException {
+    callOnce = new Expectations() {
+      {
+        oneOf(log).append(with(any(HTableDescriptor.class)), with(any(HRegionInfo.class)),
+                with(any(WALKey.class)), with(bulkLogWalEditType(WALEdit.BULK_LOAD)),
+                with(any(AtomicLong.class)), with(any(boolean.class)), with(any(List.class)));
+        will(returnValue(0l));
+        oneOf(log).sync(with(any(long.class)));
+      }
+    };
+  }
+
+  @Before
+  public void before() throws IOException {
+    random.nextBytes(randomBytes);
+  }
+
+  @Test
+  public void verifyBulkLoadEvent() throws IOException {
+    TableName tableName = TableName.valueOf("test", "test");
+    List<Pair<byte[], String>> familyPaths = withFamilyPathsFor(family1);
+    byte[] familyName = familyPaths.get(0).getFirst();
+    String storeFileName = familyPaths.get(0).getSecond();
+    storeFileName = (new Path(storeFileName)).getName();
+    List<String> storeFileNames = new ArrayList<String>();
+    storeFileNames.add(storeFileName);
+    final Matcher<WALEdit> bulkEventMatcher = bulkLogWalEdit(WALEdit.BULK_LOAD,
+      tableName.toBytes(), familyName, storeFileNames);
+    Expectations expection = new Expectations() {
+      {
+        oneOf(log).append(with(any(HTableDescriptor.class)), with(any(HRegionInfo.class)),
+                with(any(WALKey.class)), with(bulkEventMatcher),
+                with(any(AtomicLong.class)), with(any(boolean.class)), with(any(List.class)));
+        will(returnValue(0l));
+        oneOf(log).sync(with(any(long.class)));
+      }
+    };
+    context.checking(expection);
+    testRegionWithFamiliesAndSpecifiedTableName(tableName, family1)
+        .bulkLoadHFiles(familyPaths, false);
+  }
+
+  @Test
+  public void bulkHLogShouldThrowNoErrorAndWriteMarkerWithBlankInput() throws IOException {
+    testRegionWithFamilies(family1).bulkLoadHFiles(new ArrayList<Pair<byte[], String>>(), false);
+  }
+
+  @Test
+  public void shouldBulkLoadSingleFamilyHLog() throws IOException {
+    context.checking(callOnce);
+    testRegionWithFamilies(family1).bulkLoadHFiles(withFamilyPathsFor(family1), false);
+  }
+
+  @Test
+  public void shouldBulkLoadManyFamilyHLog() throws IOException {
+    context.checking(callOnce);
+    testRegionWithFamilies(family1, family2).bulkLoadHFiles(withFamilyPathsFor(family1, family2),
+        false);
+  }
+
+  @Test
+  public void shouldBulkLoadManyFamilyHLogEvenWhenTableNameNamespaceSpecified() throws IOException {
+    context.checking(callOnce);
+    TableName tableName = TableName.valueOf("test", "test");
+    testRegionWithFamiliesAndSpecifiedTableName(tableName, family1, family2)
+        .bulkLoadHFiles(withFamilyPathsFor(family1, family2), false);
+  }
+
+  @Test(expected = DoNotRetryIOException.class)
+  public void shouldCrashIfBulkLoadFamiliesNotInTable() throws IOException {
+    testRegionWithFamilies(family1).bulkLoadHFiles(withFamilyPathsFor(family1, family2), false);
+  }
+
+  @Test(expected = DoNotRetryIOException.class)
+  public void bulkHLogShouldThrowErrorWhenFamilySpecifiedAndHFileExistsButNotInTableDescriptor()
+      throws IOException {
+    testRegionWithFamilies().bulkLoadHFiles(withFamilyPathsFor(family1), false);
+  }
+
+  @Test(expected = DoNotRetryIOException.class)
+  public void shouldThrowErrorIfBadFamilySpecifiedAsFamilyPath() throws IOException {
+    testRegionWithFamilies()
+        .bulkLoadHFiles(asList(withInvalidColumnFamilyButProperHFileLocation(family1)),
+            false);
+  }
+
+  @Test(expected = FileNotFoundException.class)
+  public void shouldThrowErrorIfHFileDoesNotExist() throws IOException {
+    List<Pair<byte[], String>> list = asList(withMissingHFileForFamily(family1));
+    testRegionWithFamilies(family1).bulkLoadHFiles(list, false);
+  }
+
+  private Pair<byte[], String> withMissingHFileForFamily(byte[] family) {
+    return new Pair<byte[], String>(family, "/tmp/does_not_exist");
+  }
+
+  private Pair<byte[], String> withInvalidColumnFamilyButProperHFileLocation(byte[] family)
+      throws IOException {
+    createHFileForFamilies(family);
+    return new Pair<byte[], String>(new byte[]{0x00, 0x01, 0x02}, "/tmp/does_not_exist");
+  }
+
+
+  private HRegion testRegionWithFamiliesAndSpecifiedTableName(TableName tableName,
+                                                              byte[]... families)
+  throws IOException {
+    HRegionInfo hRegionInfo = new HRegionInfo(tableName);
+    HTableDescriptor hTableDescriptor = new HTableDescriptor(tableName);
+    for (byte[] family : families) {
+      hTableDescriptor.addFamily(new HColumnDescriptor(family));
+    }
+
+    // TODO We need a way to do this without creating files
+    return HRegion.createHRegion(hRegionInfo,
+        new Path(testFolder.newFolder().toURI()),
+        conf,
+        hTableDescriptor,
+        log);
+
+  }
+
+  private HRegion testRegionWithFamilies(byte[]... families) throws IOException {
+    TableName tableName = TableName.valueOf(name.getMethodName());
+    return testRegionWithFamiliesAndSpecifiedTableName(tableName, families);
+  }
+
+  private List<Pair<byte[], String>> getBlankFamilyPaths(){
+    return new ArrayList<Pair<byte[], String>>();
+  }
+
+  private List<Pair<byte[], String>> withFamilyPathsFor(byte[]... families) throws IOException {
+    List<Pair<byte[], String>> familyPaths = getBlankFamilyPaths();
+    for (byte[] family : families) {
+      familyPaths.add(new Pair<byte[], String>(family, createHFileForFamilies(family)));
+    }
+    return familyPaths;
+  }
+
+  private String createHFileForFamilies(byte[] family) throws IOException {
+    HFile.WriterFactory hFileFactory = HFile.getWriterFactoryNoCache(conf);
+    // TODO We need a way to do this without creating files
+    File hFileLocation = testFolder.newFile();
+    hFileFactory.withOutputStream(new FSDataOutputStream(new FileOutputStream(hFileLocation)));
+    hFileFactory.withFileContext(new HFileContext());
+    HFile.Writer writer = hFileFactory.create();
+
+    writer.append(new KeyValue(CellUtil.createCell(randomBytes,
+        family,
+        randomBytes,
+        0l,
+        KeyValue.Type.Put.getCode(),
+        randomBytes)));
+    writer.close();
+    return hFileLocation.getAbsoluteFile().getAbsolutePath();
+  }
+
+  private static Matcher<WALEdit> bulkLogWalEditType(byte[] typeBytes) {
+    return new WalMatcher(typeBytes);
+  }
+
+  private static Matcher<WALEdit> bulkLogWalEdit(byte[] typeBytes, byte[] tableName,
+      byte[] familyName, List<String> storeFileNames) {
+    return new WalMatcher(typeBytes, tableName, familyName, storeFileNames);
+  }
+
+  private static class WalMatcher extends TypeSafeMatcher<WALEdit> {
+    private final byte[] typeBytes;
+    private final byte[] tableName;
+    private final byte[] familyName;
+    private final List<String> storeFileNames;
+
+    public WalMatcher(byte[] typeBytes) {
+      this(typeBytes, null, null, null);
+    }
+
+    public WalMatcher(byte[] typeBytes, byte[] tableName, byte[] familyName,
+        List<String> storeFileNames) {
+      this.typeBytes = typeBytes;
+      this.tableName = tableName;
+      this.familyName = familyName;
+      this.storeFileNames = storeFileNames;
+    }
+
+    @Override
+    protected boolean matchesSafely(WALEdit item) {
+      assertTrue(Arrays.equals(item.getCells().get(0).getQualifier(), typeBytes));
+      BulkLoadDescriptor desc;
+      try {
+        desc = WALEdit.getBulkLoadDescriptor(item.getCells().get(0));
+      } catch (IOException e) {
+        return false;
+      }
+      assertNotNull(desc);
+
+      if (tableName != null) {
+        assertTrue(Bytes.equals(ProtobufUtil.toTableName(desc.getTableName()).getName(), 
+          tableName));
+      }
+
+      if(storeFileNames != null) {
+        int index=0;
+        StoreDescriptor store = desc.getStores(0);
+        assertTrue(Bytes.equals(store.getFamilyName().toByteArray(), familyName));
+        assertTrue(Bytes.equals(Bytes.toBytes(store.getStoreHomeDir()), familyName));
+        assertEquals(storeFileNames.size(), store.getStoreFileCount());
+        for (String storeFile : store.getStoreFileList()) {
+          assertTrue(storeFile.equals(storeFileNames.get(index++)));
+        }
+      }
+      
+      return true;
+    }
+
+    @Override
+    public void describeTo(Description description) {
+
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3f442773/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
index 8b41594..ea06346 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
@@ -124,7 +124,7 @@ import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor.FlushAction;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor;
-import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor;
+import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor;
 import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl;
 import org.apache.hadoop.hbase.regionserver.HRegion.RowLock;
 import org.apache.hadoop.hbase.regionserver.TestStore.FaultyFileSystem;

http://git-wip-us.apache.org/repos/asf/hbase/blob/3f442773/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
index d2bfb52..af49556 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
@@ -17,27 +17,24 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.atomic.AtomicLong;
-
+import com.google.common.collect.Lists;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.MultithreadedTestUtil.RepeatingTestThread;
 import org.apache.hadoop.hbase.MultithreadedTestUtil.TestContext;
 import org.apache.hadoop.hbase.TableExistsException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.RegionServerCallable;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
@@ -55,15 +52,28 @@ import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
+import org.apache.hadoop.hbase.regionserver.wal.TestWALActionsListener;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALKey;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.google.common.collect.Lists;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertThat;
+
 /**
  * Tests bulk loading of HFiles and shows the atomicity or lack of atomicity of
  * the region server's bullkLoad functionality.
@@ -290,7 +300,11 @@ public class TestHRegionServerBulkLoad {
 
     UTIL.startMiniCluster(1);
     try {
+      WAL log = UTIL.getHBaseCluster().getRegionServer(0).getWAL(null);
+      FindBulkHBaseListener listener = new FindBulkHBaseListener();
+      log.registerWALActionsListener(listener);
       runAtomicBulkloadTest(TABLE_NAME, millisToRun, numScanners);
+      assertThat(listener.isFound(), is(true));
     } finally {
       UTIL.shutdownMiniCluster();
     }
@@ -346,5 +360,25 @@ public class TestHRegionServerBulkLoad {
     UTIL = new HBaseTestingUtility(c);
   }
 
+  static class FindBulkHBaseListener extends TestWALActionsListener.DummyWALActionsListener {
+    private boolean found = false;
+
+    @Override
+    public void visitLogEntryBeforeWrite(HTableDescriptor htd, WALKey logKey, WALEdit logEdit) {
+      for (Cell cell : logEdit.getCells()) {
+        KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
+        for (Map.Entry entry : kv.toStringMap().entrySet()) {
+          if (entry.getValue().equals(Bytes.toString(WALEdit.BULK_LOAD))) {
+            found = true;
+          }
+        }
+      }
+    }
+
+    public boolean isFound() {
+      return found;
+    }
+  }
 }
 
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/3f442773/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java
index c8629d0..25c83a8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java
@@ -125,7 +125,7 @@ public class TestWALActionsListener {
   /**
    * Just counts when methods are called
    */
-  static class DummyWALActionsListener extends WALActionsListener.Base {
+  public static class DummyWALActionsListener extends WALActionsListener.Base {
     public int preLogRollCounter = 0;
     public int postLogRollCounter = 0;
     public int closedCount = 0;

http://git-wip-us.apache.org/repos/asf/hbase/blob/3f442773/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 57e23c1..975f1f8 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1034,6 +1034,7 @@
     <jetty.version>6.1.26</jetty.version>
     <jetty.jspapi.version>6.1.14</jetty.jspapi.version>
     <jersey.version>1.9</jersey.version>
+    <jmock-junit4.version>2.6.0</jmock-junit4.version>
     <jruby.version>1.6.8</jruby.version>
     <junit.version>4.11</junit.version>
     <hamcrest.version>1.3</hamcrest.version>
@@ -1540,6 +1541,18 @@
         <artifactId>disruptor</artifactId>
         <version>${disruptor.version}</version>
       </dependency>
+      <dependency>
+        <groupId>org.jmock</groupId>
+        <artifactId>jmock-junit4</artifactId>
+        <version>${jmock-junit4.version}</version>
+        <scope>test</scope>
+        <exclusions>
+          <exclusion>
+            <artifactId>junit-dep</artifactId>
+            <groupId>junit</groupId>
+          </exclusion>
+        </exclusions>
+      </dependency>
     </dependencies>
   </dependencyManagement>
   <!-- Dependencies needed by subprojects -->
@@ -1563,6 +1576,10 @@
       <groupId>org.mockito</groupId>
       <artifactId>mockito-all</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.jmock</groupId>
+      <artifactId>jmock-junit4</artifactId>
+    </dependency>
   </dependencies>
   <!--
   To publish, use the following settings.xml file ( placed in ~/.m2/settings.xml )


[10/50] [abbrv] hbase git commit: HBASE-12980 Delete of a table may not clean all rows from hbase:meta

Posted by jm...@apache.org.
HBASE-12980 Delete of a table may not clean all rows from hbase:meta


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

Branch: refs/heads/hbase-11339
Commit: 57319c536a136d331b925614417a8deba159ad8c
Parents: 7514838
Author: stack <st...@apache.org>
Authored: Fri Feb 6 14:06:42 2015 -0800
Committer: stack <st...@apache.org>
Committed: Fri Feb 6 14:06:59 2015 -0800

----------------------------------------------------------------------
 .../master/handler/DeleteTableHandler.java      | 45 ++++++++++++++---
 .../master/handler/TestEnableTableHandler.java  | 52 ++++++++++++++++++++
 2 files changed, 91 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/57319c53/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java
index 004e2bb..092a17d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java
@@ -18,31 +18,37 @@
  */
 package org.apache.hadoop.hbase.master.handler;
 
-import java.io.InterruptedIOException;
 import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.CoordinatedStateException;
-import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.Server;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.backup.HFileArchiver;
-import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.executor.EventType;
-import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.master.AssignmentManager;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.master.MasterServices;
-import org.apache.hadoop.hbase.master.RegionStates;
 import org.apache.hadoop.hbase.master.RegionState.State;
+import org.apache.hadoop.hbase.master.RegionStates;
+import org.apache.hadoop.hbase.regionserver.HRegion;
 
 @InterfaceAudience.Private
 public class DeleteTableHandler extends TableEventHandler {
@@ -125,6 +131,33 @@ public class DeleteTableHandler extends TableEventHandler {
     // 5. If entry for this table states, remove it.
     LOG.debug("Marking '" + tableName + "' as deleted.");
     am.getTableStateManager().setDeletedTable(tableName);
+
+    // 6.Clean any remaining rows for this table.
+    cleanAnyRemainingRows();
+  }
+
+  /**
+   * There may be items for this table still up in hbase:meta in the case where the
+   * info:regioninfo column was empty because of some write error. Remove ALL rows from hbase:meta
+   * that have to do with this table. See HBASE-12980.
+   * @throws IOException
+   */
+  private void cleanAnyRemainingRows() throws IOException {
+    Scan tableScan = MetaTableAccessor.getScanForTableName(tableName);
+    try (Table metaTable =
+        this.masterServices.getConnection().getTable(TableName.META_TABLE_NAME)) {
+      List<Delete> deletes = new ArrayList<Delete>();
+      try (ResultScanner resScanner = metaTable.getScanner(tableScan)) {
+        for (Result result : resScanner) {
+          deletes.add(new Delete(result.getRow()));
+        }
+      }
+      if (!deletes.isEmpty()) {
+        LOG.warn("Deleting some vestigal " + deletes.size() + " rows of " + this.tableName +
+          " from " + TableName.META_TABLE_NAME);
+        metaTable.delete(deletes);
+      }
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/57319c53/hbase-server/src/test/java/org/apache/hadoop/hbase/master/handler/TestEnableTableHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/handler/TestEnableTableHandler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/handler/TestEnableTableHandler.java
index f5f2cd0..89fd9ba 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/handler/TestEnableTableHandler.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/handler/TestEnableTableHandler.java
@@ -22,6 +22,8 @@ import java.util.List;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
@@ -42,6 +44,13 @@ import org.junit.experimental.categories.Category;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
+import java.io.IOException;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+
 @Category({ MasterTests.class, MediumTests.class })
 public class TestEnableTableHandler {
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
@@ -98,4 +107,47 @@ public class TestEnableTableHandler {
     assertEquals(tableName, onlineRegions.get(0).getTable());
   }
 
+  /**
+   * We were only clearing rows that had a hregioninfo column in hbase:meta.  Mangled rows that
+   * were missing the hregioninfo because of error were being left behind messing up any
+   * subsequent table made with the same name. HBASE-12980
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  @Test(timeout=60000)
+  public void testDeleteForSureClearsAllTableRowsFromMeta()
+  throws IOException, InterruptedException {
+    final TableName tableName = TableName.valueOf("testDeleteForSureClearsAllTableRowsFromMeta");
+    final MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
+    final HMaster m = cluster.getMaster();
+    final HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
+    final HTableDescriptor desc = new HTableDescriptor(tableName);
+    desc.addFamily(new HColumnDescriptor(FAMILYNAME));
+    admin.createTable(desc, HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE);
+    // Now I have a nice table, mangle it by removing the HConstants.REGIONINFO_QUALIFIER_STR
+    // content from a few of the rows.
+    Scan metaScannerForMyTable = MetaTableAccessor.getScanForTableName(tableName);
+    try (Table metaTable = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME)) {
+      try (ResultScanner scanner = metaTable.getScanner(metaScannerForMyTable)) {
+        for (Result result : scanner) {
+          // Just delete one row.
+          Delete d = new Delete(result.getRow());
+          d.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
+          metaTable.delete(d);
+          break;
+        }
+      }
+      admin.disableTable(tableName);
+      TEST_UTIL.waitTableDisabled(tableName.getName());
+      // Presume this synchronous all is.
+      admin.deleteTable(tableName);
+      int rowCount = 0;
+      try (ResultScanner scanner = metaTable.getScanner(metaScannerForMyTable)) {
+        for (Result result : scanner) {
+          rowCount++;
+        }
+      }
+      assertEquals(0, rowCount);
+    }
+  }
 }


[06/50] [abbrv] hbase git commit: HBASE-12966 NPE in HMaster while recovering tables in Enabling state (Andrey Stepachev)

Posted by jm...@apache.org.
HBASE-12966 NPE in HMaster while recovering tables in Enabling state (Andrey Stepachev)


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

Branch: refs/heads/hbase-11339
Commit: 8aeb3acaf959e2905191fd6c92fa56300f7d3597
Parents: 4f47206
Author: tedyu <yu...@gmail.com>
Authored: Wed Feb 4 14:53:55 2015 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Wed Feb 4 14:53:55 2015 -0800

----------------------------------------------------------------------
 .../master/handler/EnableTableHandler.java      |  35 ++++---
 .../master/handler/TestEnableTableHandler.java  | 101 +++++++++++++++++++
 2 files changed, 122 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/8aeb3aca/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java
index 280e3e4..c4969be 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java
@@ -26,14 +26,14 @@ import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotDisabledException;
 import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.executor.EventHandler;
 import org.apache.hadoop.hbase.executor.EventType;
@@ -207,19 +207,26 @@ public class EnableTableHandler extends EventHandler {
     List<ServerName> onlineServers = serverManager.createDestinationServersList();
     Map<ServerName, List<HRegionInfo>> bulkPlan =
         this.assignmentManager.getBalancer().retainAssignment(regionsToAssign, onlineServers);
-    LOG.info("Bulk assigning " + regionsCount + " region(s) across " + bulkPlan.size()
-      + " server(s), retainAssignment=true");
+    if (bulkPlan != null) {
+      LOG.info("Bulk assigning " + regionsCount + " region(s) across " + bulkPlan.size()
+          + " server(s), retainAssignment=true");
 
-    BulkAssigner ba = new GeneralBulkAssigner(this.server, bulkPlan, this.assignmentManager, true);
-    try {
-      if (ba.bulkAssign()) {
-        done = true;
+      BulkAssigner ba =
+          new GeneralBulkAssigner(this.server, bulkPlan, this.assignmentManager, true);
+      try {
+        if (ba.bulkAssign()) {
+          done = true;
+        }
+      } catch (InterruptedException e) {
+        LOG.warn("Enable operation was interrupted when enabling table '"
+            + this.tableName + "'");
+        // Preserve the interrupt.
+        Thread.currentThread().interrupt();
       }
-    } catch (InterruptedException e) {
-      LOG.warn("Enable operation was interrupted when enabling table '"
-        + this.tableName + "'");
-      // Preserve the interrupt.
-      Thread.currentThread().interrupt();
+    } else {
+      LOG.info("Balancer was unable to find suitable servers for table " + tableName
+          + ", leaving unassigned");
+      done = true;
     }
     if (done) {
       // Flip the table to enabled.

http://git-wip-us.apache.org/repos/asf/hbase/blob/8aeb3aca/hbase-server/src/test/java/org/apache/hadoop/hbase/master/handler/TestEnableTableHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/handler/TestEnableTableHandler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/handler/TestEnableTableHandler.java
new file mode 100644
index 0000000..f5f2cd0
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/handler/TestEnableTableHandler.java
@@ -0,0 +1,101 @@
+/**
+ *
+ * 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.handler;
+
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.master.HMaster;
+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.JVMClusterUtil;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+@Category({ MasterTests.class, MediumTests.class })
+public class TestEnableTableHandler {
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static final Log LOG = LogFactory.getLog(TestEnableTableHandler.class);
+  private static final byte[] FAMILYNAME = Bytes.toBytes("fam");
+
+  @Before
+  public void setUp() throws Exception {
+    TEST_UTIL.startMiniCluster(1);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @Test(timeout = 300000)
+  public void testEnableTableWithNoRegionServers() throws Exception {
+    final TableName tableName = TableName.valueOf("testEnableTableWithNoRegionServers");
+    final MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
+    final HMaster m = cluster.getMaster();
+    final HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
+    final HTableDescriptor desc = new HTableDescriptor(tableName);
+    desc.addFamily(new HColumnDescriptor(FAMILYNAME));
+    admin.createTable(desc);
+    admin.disableTable(tableName);
+    TEST_UTIL.waitTableDisabled(tableName.getName());
+
+    admin.enableTable(tableName);
+    TEST_UTIL.waitTableEnabled(tableName);
+
+    // disable once more
+    admin.disableTable(tableName);
+
+    // now stop region servers
+    JVMClusterUtil.RegionServerThread rs = cluster.getRegionServerThreads().get(0);
+    rs.getRegionServer().stop("stop");
+    cluster.waitForRegionServerToStop(rs.getRegionServer().getServerName(), 10000);
+
+    EnableTableHandler handler =
+        new EnableTableHandler(m, tableName, m.getAssignmentManager(), m.getTableLockManager(),
+            true);
+    handler.prepare();
+    handler.process();
+
+    assertTrue(admin.isTableEnabled(tableName));
+
+    JVMClusterUtil.RegionServerThread rs2 = cluster.startRegionServer();
+    m.getAssignmentManager().assign(admin.getTableRegions(tableName));
+    TEST_UTIL.waitUntilAllRegionsAssigned(tableName);
+    List<HRegionInfo> onlineRegions = admin.getOnlineRegions(
+        rs2.getRegionServer().getServerName());
+    assertEquals(1, onlineRegions.size());
+    assertEquals(tableName, onlineRegions.get(0).getTable());
+  }
+
+}


[15/50] [abbrv] hbase git commit: Revert "HBASE-12891 Parallel execution for Hbck checkRegionConsistency"

Posted by jm...@apache.org.
Revert "HBASE-12891 Parallel execution for Hbck checkRegionConsistency"

This reverts commit eddd5739a14ceb5cfc9b9c7d2e357eea96bd9703.


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

Branch: refs/heads/hbase-11339
Commit: 7f4146bf4d4df84041b284a76d917d602b5531da
Parents: 1426f85
Author: Andrew Purtell <ap...@apache.org>
Authored: Fri Feb 6 16:52:04 2015 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Fri Feb 6 16:52:04 2015 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/util/HBaseFsck.java | 28 +-------------------
 1 file changed, 1 insertion(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/7f4146bf/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
index 96bd0f7..8e1d848 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
@@ -1703,19 +1703,9 @@ public class HBaseFsck extends Configured implements Closeable {
   throws IOException, KeeperException, InterruptedException {
     // Divide the checks in two phases. One for default/primary replicas and another
     // for the non-primary ones. Keeps code cleaner this way.
-    List<WorkItemRegionConsistency> workItems =
-        new ArrayList<WorkItemRegionConsistency>(regionInfoMap.size());
     for (java.util.Map.Entry<String, HbckInfo> e: regionInfoMap.entrySet()) {
       if (e.getValue().getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID) {
-        workItems.add(new WorkItemRegionConsistency(e.getKey(), e.getValue()));
-      }
-    }
-    List<Future<Void>> workFutures = executor.invokeAll(workItems);
-    for(Future<Void> f: workFutures) {
-      try {
-        f.get();
-      } catch(ExecutionException e1) {
-        LOG.warn("Could not check region consistency " , e1.getCause());
+        checkRegionConsistency(e.getKey(), e.getValue());
       }
     }
     boolean prevHdfsCheck = shouldCheckHdfs();
@@ -2365,22 +2355,6 @@ public class HBaseFsck extends Configured implements Closeable {
     }
   };
 
-  class WorkItemRegionConsistency implements Callable<Void> {
-    private final String key;
-    private final HbckInfo hbi;
-
-    WorkItemRegionConsistency(String key, HbckInfo hbi) {
-      this.key = key;
-      this.hbi = hbi;
-    }
-
-    @Override
-    public synchronized Void call() throws Exception {
-      checkRegionConsistency(key, hbi);
-      return null;
-    }
-  }
-
 
   /**
    * Maintain information about a particular table.


[24/50] [abbrv] hbase git commit: HBASE-12998 Compilation with Hdfs-2.7.0-SNAPSHOT is broken after HDFS-7647

Posted by jm...@apache.org.
HBASE-12998 Compilation with Hdfs-2.7.0-SNAPSHOT is broken after HDFS-7647


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

Branch: refs/heads/hbase-11339
Commit: f97c00fd99609214830e68f52c1ec48c4e506c1c
Parents: 1e6afa2
Author: Enis Soztutar <en...@apache.org>
Authored: Mon Feb 9 17:14:02 2015 -0800
Committer: Enis Soztutar <en...@apache.org>
Committed: Mon Feb 9 18:17:33 2015 -0800

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/f97c00fd/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java
index db751b2..613d1ea 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java
@@ -175,11 +175,12 @@ public class TestBlockReorder {
           public void reorderBlocks(Configuration c, LocatedBlocks lbs, String src) {
             for (LocatedBlock lb : lbs.getLocatedBlocks()) {
               if (lb.getLocations().length > 1) {
-                if (lb.getLocations()[0].getHostName().equals(lookup)) {
+                DatanodeInfo[] infos = lb.getLocations();
+                if (infos[0].getHostName().equals(lookup)) {
                   LOG.info("HFileSystem bad host, inverting");
-                  DatanodeInfo tmp = lb.getLocations()[0];
-                  lb.getLocations()[0] = lb.getLocations()[1];
-                  lb.getLocations()[1] = tmp;
+                  DatanodeInfo tmp = infos[0];
+                  infos[0] = infos[1];
+                  infos[1] = tmp;
                 }
               }
             }


[37/50] [abbrv] hbase git commit: HBASE-13007 Fix the test timeouts being caused by ChoreService (Jonathan Lawlor)

Posted by jm...@apache.org.
HBASE-13007 Fix the test timeouts being caused by ChoreService (Jonathan Lawlor)


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

Branch: refs/heads/hbase-11339
Commit: a8d325eed889a36266cd131b4d033a4c099fbc79
Parents: f048508
Author: stack <st...@apache.org>
Authored: Tue Feb 10 21:19:14 2015 -0800
Committer: stack <st...@apache.org>
Committed: Tue Feb 10 21:19:14 2015 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/ChoreService.java   |  2 +-
 .../org/apache/hadoop/hbase/ScheduledChore.java |  6 +--
 .../apache/hadoop/hbase/TestChoreService.java   | 41 ++++++++++----------
 3 files changed, 24 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a8d325ee/hbase-common/src/main/java/org/apache/hadoop/hbase/ChoreService.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ChoreService.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ChoreService.java
index fd6cbc9..5e01c39 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ChoreService.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ChoreService.java
@@ -330,7 +330,7 @@ public class ChoreService implements ChoreServicer {
       choresToCancel.add(chore);
     }
     for (ScheduledChore chore : choresToCancel) {
-      chore.cancel(mayInterruptIfRunning);
+      cancelChore(chore, mayInterruptIfRunning);
     }
     choresToCancel.clear();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a8d325ee/hbase-common/src/main/java/org/apache/hadoop/hbase/ScheduledChore.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ScheduledChore.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ScheduledChore.java
index 84002c5..ccedcc7 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ScheduledChore.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ScheduledChore.java
@@ -177,10 +177,10 @@ public abstract class ScheduledChore implements Runnable {
   public synchronized void run() {
     timeOfLastRun = timeOfThisRun;
     timeOfThisRun = System.currentTimeMillis();
-    if (missedStartTime() && choreServicer != null) {
+    if (missedStartTime() && isScheduled()) {
       choreServicer.onChoreMissedStartTime(this);
       if (LOG.isInfoEnabled()) LOG.info("Chore: " + getName() + " missed its start time");
-    } else if (stopper.isStopped()) {
+    } else if (stopper.isStopped() || choreServicer == null || !isScheduled()) {
       cancel();
       cleanup();
       LOG.info("Chore: " + getName() + " was stopped");
@@ -252,7 +252,7 @@ public abstract class ScheduledChore implements Runnable {
   }
 
   public synchronized void cancel(boolean mayInterruptIfRunning) {
-    if (choreServicer != null) choreServicer.cancelChore(this, mayInterruptIfRunning);
+    if (isScheduled()) choreServicer.cancelChore(this, mayInterruptIfRunning);
 
     choreServicer = null;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a8d325ee/hbase-common/src/test/java/org/apache/hadoop/hbase/TestChoreService.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestChoreService.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestChoreService.java
index b1bad3d..811d4d9 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestChoreService.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestChoreService.java
@@ -39,7 +39,6 @@ import org.junit.experimental.categories.Category;
 @Category(SmallTests.class)
 public class TestChoreService {
   private final Log LOG = LogFactory.getLog(this.getClass());
-  private final String TEST_SERVER_NAME = "testServerName";
 
   /**
    * A few ScheduledChore samples that are useful for testing with ChoreService
@@ -238,7 +237,7 @@ public class TestChoreService {
 
   @Test (timeout=20000)
   public void testInitialChorePrecedence() throws InterruptedException {
-    ChoreService service = ChoreService.getInstance(TEST_SERVER_NAME);
+    ChoreService service = ChoreService.getInstance("testInitialChorePrecedence");
 
     final int period = 100;
     final int failureThreshold = 5;
@@ -265,7 +264,7 @@ public class TestChoreService {
   public void testCancelChore() throws InterruptedException {
     final int period = 100;
     ScheduledChore chore1 = new DoNothingChore("chore1", period);
-    ChoreService service = ChoreService.getInstance(TEST_SERVER_NAME);
+    ChoreService service = ChoreService.getInstance("testCancelChore");
 
     service.scheduleChore(chore1);
     assertTrue(chore1.isScheduled());
@@ -314,13 +313,13 @@ public class TestChoreService {
     final int corePoolSize = 10;
     final int defaultCorePoolSize = ChoreService.MIN_CORE_POOL_SIZE;
 
-    ChoreService customInit = new ChoreService(TEST_SERVER_NAME, corePoolSize);
+    ChoreService customInit = new ChoreService("testChoreServiceConstruction_custom", corePoolSize);
     assertEquals(corePoolSize, customInit.getCorePoolSize());
 
-    ChoreService defaultInit = new ChoreService(TEST_SERVER_NAME);
+    ChoreService defaultInit = new ChoreService("testChoreServiceConstruction_default");
     assertEquals(defaultCorePoolSize, defaultInit.getCorePoolSize());
 
-    ChoreService invalidInit = new ChoreService(TEST_SERVER_NAME, -10);
+    ChoreService invalidInit = new ChoreService("testChoreServiceConstruction_invalid", -10);
     assertEquals(defaultCorePoolSize, invalidInit.getCorePoolSize());
 
     shutdownService(customInit);
@@ -334,7 +333,7 @@ public class TestChoreService {
     final int period = 100;
     // Small delta that acts as time buffer (allowing chores to complete if running slowly)
     final int delta = 5;
-    ChoreService service = ChoreService.getInstance(TEST_SERVER_NAME);
+    ChoreService service = ChoreService.getInstance("testFrequencyOfChores");
     CountingChore chore = new CountingChore("countingChore", period);
     service.scheduleChore(chore);
 
@@ -358,7 +357,7 @@ public class TestChoreService {
   public void testForceTrigger() throws InterruptedException {
     final int period = 100;
     final int delta = 5;
-    ChoreService service = ChoreService.getInstance(TEST_SERVER_NAME);
+    ChoreService service = ChoreService.getInstance("testForceTrigger");
     CountingChore chore = new CountingChore("countingChore", period);
     service.scheduleChore(chore);
     Thread.sleep(10 * period + delta);
@@ -390,7 +389,7 @@ public class TestChoreService {
   @Test (timeout=20000)
   public void testCorePoolIncrease() throws InterruptedException {
     final int initialCorePoolSize = 3;
-    ChoreService service = new ChoreService(TEST_SERVER_NAME, initialCorePoolSize);
+    ChoreService service = new ChoreService("testCorePoolIncrease", initialCorePoolSize);
     assertEquals("Should have a core pool of size: " + initialCorePoolSize, initialCorePoolSize,
         service.getCorePoolSize());
 
@@ -427,7 +426,7 @@ public class TestChoreService {
   @Test (timeout=20000)
   public void testCorePoolDecrease() throws InterruptedException {
     final int initialCorePoolSize = 3;
-    ChoreService service = new ChoreService(TEST_SERVER_NAME, initialCorePoolSize);
+    ChoreService service = new ChoreService("testCorePoolDecrease", initialCorePoolSize);
     final int chorePeriod = 100;
 
     // Slow chores always miss their start time and thus the core pool size should be at least as
@@ -530,7 +529,7 @@ public class TestChoreService {
 
   @Test (timeout=20000)
   public void testNumberOfRunningChores() throws InterruptedException {
-    ChoreService service = new ChoreService(TEST_SERVER_NAME);
+    ChoreService service = new ChoreService("testNumberOfRunningChores");
 
     final int period = 100;
     final int sleepTime = 5;
@@ -569,7 +568,7 @@ public class TestChoreService {
 
   @Test (timeout=20000)
   public void testNumberOfChoresMissingStartTime() throws InterruptedException {
-    ChoreService service = new ChoreService(TEST_SERVER_NAME);
+    ChoreService service = new ChoreService("testNumberOfChoresMissingStartTime");
 
     final int period = 100;
     final int sleepTime = 5 * period;
@@ -615,7 +614,7 @@ public class TestChoreService {
    */
   @Test (timeout=20000)
   public void testMaximumChoreServiceThreads() throws InterruptedException {
-    ChoreService service = new ChoreService(TEST_SERVER_NAME);
+    ChoreService service = new ChoreService("testMaximumChoreServiceThreads");
 
     final int period = 100;
     final int sleepTime = 5 * period;
@@ -661,7 +660,7 @@ public class TestChoreService {
   @Test (timeout=20000)
   public void testScheduledChoreReset() throws InterruptedException {
     final int period = 100;
-    ChoreService service = new ChoreService(TEST_SERVER_NAME);
+    ChoreService service = new ChoreService("testScheduledChoreReset");
     ScheduledChore chore = new DoNothingChore("sampleChore", period);
 
     // TRUE
@@ -691,8 +690,8 @@ public class TestChoreService {
   public void testChangingChoreServices() throws InterruptedException {
     final int period = 100;
     final int sleepTime = 10;
-    ChoreService service1 = new ChoreService(TEST_SERVER_NAME);
-    ChoreService service2 = new ChoreService(TEST_SERVER_NAME);
+    ChoreService service1 = new ChoreService("testChangingChoreServices_1");
+    ChoreService service2 = new ChoreService("testChangingChoreServices_2");
     ScheduledChore chore = new DoNothingChore("sample", period);
 
     assertFalse(chore.isScheduled());
@@ -729,7 +728,7 @@ public class TestChoreService {
     final int period = 100;
     // Small sleep time buffer to allow CountingChore to complete
     final int sleep = 5;
-    ChoreService service = new ChoreService(TEST_SERVER_NAME);
+    ChoreService service = new ChoreService("testTriggerNowFailsWhenNotScheduled");
     CountingChore chore = new CountingChore("dn", period);
 
     assertFalse(chore.triggerNow());
@@ -753,7 +752,7 @@ public class TestChoreService {
 
   @Test (timeout=20000)
   public void testStopperForScheduledChores() throws InterruptedException {
-    ChoreService service = ChoreService.getInstance(TEST_SERVER_NAME);
+    ChoreService service = ChoreService.getInstance("testStopperForScheduledChores");
     Stoppable stopperForGroup1 = new SampleStopper();
     Stoppable stopperForGroup2 = new SampleStopper();
     final int period = 100;
@@ -807,7 +806,7 @@ public class TestChoreService {
   @Test (timeout=20000)
   public void testShutdownCancelsScheduledChores() throws InterruptedException {
     final int period = 100;
-    ChoreService service = new ChoreService(TEST_SERVER_NAME);
+    ChoreService service = new ChoreService("testShutdownCancelsScheduledChores");
     ScheduledChore successChore1 = new DoNothingChore("sc1", period);
     ScheduledChore successChore2 = new DoNothingChore("sc2", period);
     ScheduledChore successChore3 = new DoNothingChore("sc3", period);
@@ -830,7 +829,7 @@ public class TestChoreService {
   public void testShutdownWorksWhileChoresAreExecuting() throws InterruptedException {
     final int period = 100;
     final int sleep = 5 * period;
-    ChoreService service = new ChoreService(TEST_SERVER_NAME);
+    ChoreService service = new ChoreService("testShutdownWorksWhileChoresAreExecuting");
     ScheduledChore slowChore1 = new SleepingChore("sc1", period, sleep);
     ScheduledChore slowChore2 = new SleepingChore("sc2", period, sleep);
     ScheduledChore slowChore3 = new SleepingChore("sc3", period, sleep);
@@ -856,7 +855,7 @@ public class TestChoreService {
   @Test (timeout=20000)
   public void testShutdownRejectsNewSchedules() throws InterruptedException {
     final int period = 100;
-    ChoreService service = new ChoreService(TEST_SERVER_NAME);
+    ChoreService service = new ChoreService("testShutdownRejectsNewSchedules");
     ScheduledChore successChore1 = new DoNothingChore("sc1", period);
     ScheduledChore successChore2 = new DoNothingChore("sc2", period);
     ScheduledChore successChore3 = new DoNothingChore("sc3", period);


[46/50] [abbrv] hbase git commit: Merge branch 'master' (2/11/15) into hbase-11339

Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobSnapshotCloneIndependence.java
----------------------------------------------------------------------
diff --cc hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobSnapshotCloneIndependence.java
index bc8a41b,0000000..612b98a
mode 100644,000000..100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobSnapshotCloneIndependence.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobSnapshotCloneIndependence.java
@@@ -1,376 -1,0 +1,376 @@@
 +/**
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +
 +package org.apache.hadoop.hbase.client;
 +
 +import java.util.List;
 +
 +import org.apache.commons.logging.Log;
 +import org.apache.commons.logging.LogFactory;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.hbase.TableName;
 +import org.apache.hadoop.hbase.HBaseTestingUtility;
 +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.LargeTests;
 +import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 +import org.apache.hadoop.hbase.mob.MobConstants;
 +import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
 +import org.apache.hadoop.hbase.snapshot.MobSnapshotTestingUtils;
 +import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
++import org.apache.hadoop.hbase.testclassification.LargeTests;
 +import org.apache.hadoop.hbase.util.Bytes;
 +import org.junit.After;
 +import org.junit.AfterClass;
 +import org.junit.Assert;
 +import org.junit.Before;
 +import org.junit.BeforeClass;
 +import org.junit.Test;
 +import org.junit.experimental.categories.Category;
 +
 +/**
 + * Test to verify that the cloned table is independent of the table from which it was cloned
 + */
 +@Category(LargeTests.class)
 +public class TestMobSnapshotCloneIndependence {
 +  private static final Log LOG = LogFactory.getLog(TestSnapshotCloneIndependence.class);
 +
 +  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
 +
 +  private static final int NUM_RS = 2;
 +  private static final String STRING_TABLE_NAME = "test";
 +  private static final String TEST_FAM_STR = "fam";
 +  private static final byte[] TEST_FAM = Bytes.toBytes(TEST_FAM_STR);
 +  private static final byte[] TABLE_NAME = Bytes.toBytes(STRING_TABLE_NAME);
 +
 +  /**
 +   * Setup the config for the cluster and start it
 +   * @throws Exception on failure
 +   */
 +  @BeforeClass
 +  public static void setupCluster() throws Exception {
 +    setupConf(UTIL.getConfiguration());
 +    UTIL.startMiniCluster(NUM_RS);
 +  }
 +
 +  private static void setupConf(Configuration conf) {
 +    // enable snapshot support
 +    conf.setBoolean(SnapshotManager.HBASE_SNAPSHOT_ENABLED, true);
 +    // disable the ui
 +    conf.setInt("hbase.regionsever.info.port", -1);
 +    // change the flush size to a small amount, regulating number of store files
 +    conf.setInt("hbase.hregion.memstore.flush.size", 25000);
 +    // so make sure we get a compaction when doing a load, but keep around
 +    // some files in the store
 +    conf.setInt("hbase.hstore.compaction.min", 10);
 +    conf.setInt("hbase.hstore.compactionThreshold", 10);
 +    // block writes if we get to 12 store files
 +    conf.setInt("hbase.hstore.blockingStoreFiles", 12);
 +    conf.setInt("hbase.regionserver.msginterval", 100);
 +    conf.setBoolean("hbase.master.enabletable.roundrobin", true);
 +    // Avoid potentially aggressive splitting which would cause snapshot to fail
 +    conf.set(HConstants.HBASE_REGION_SPLIT_POLICY_KEY,
 +      ConstantSizeRegionSplitPolicy.class.getName());
 +    conf.setInt(MobConstants.MOB_FILE_CACHE_SIZE_KEY, 0);
 +  }
 +
 +  @Before
 +  public void setup() throws Exception {
 +    MobSnapshotTestingUtils.createMobTable(UTIL, TableName.valueOf(STRING_TABLE_NAME), TEST_FAM);
 +  }
 +
 +  @After
 +  public void tearDown() throws Exception {
 +    UTIL.deleteTable(TABLE_NAME);
 +    SnapshotTestingUtils.deleteAllSnapshots(UTIL.getHBaseAdmin());
 +    SnapshotTestingUtils.deleteArchiveDirectory(UTIL);
 +  }
 +
 +  @AfterClass
 +  public static void cleanupTest() throws Exception {
 +    try {
 +      UTIL.shutdownMiniCluster();
 +    } catch (Exception e) {
 +      LOG.warn("failure shutting down cluster", e);
 +    }
 +  }
 +
 +  /**
 +   * Verify that adding data to the cloned table will not affect the original, and vice-versa when
 +   * it is taken as an online snapshot.
 +   */
 +  @Test (timeout=300000)
 +  public void testOnlineSnapshotAppendIndependent() throws Exception {
 +    runTestSnapshotAppendIndependent(true);
 +  }
 +
 +  /**
 +   * Verify that adding data to the cloned table will not affect the original, and vice-versa when
 +   * it is taken as an offline snapshot.
 +   */
 +  @Test (timeout=300000)
 +  public void testOfflineSnapshotAppendIndependent() throws Exception {
 +    runTestSnapshotAppendIndependent(false);
 +  }
 +
 +  /**
 +   * Verify that adding metadata to the cloned table will not affect the original, and vice-versa
 +   * when it is taken as an online snapshot.
 +   */
 +  @Test (timeout=300000)
 +  public void testOnlineSnapshotMetadataChangesIndependent() throws Exception {
 +    runTestSnapshotMetadataChangesIndependent(true);
 +  }
 +
 +  /**
 +   * Verify that adding netadata to the cloned table will not affect the original, and vice-versa
 +   * when is taken as an online snapshot.
 +   */
 +  @Test (timeout=300000)
 +  public void testOfflineSnapshotMetadataChangesIndependent() throws Exception {
 +    runTestSnapshotMetadataChangesIndependent(false);
 +  }
 +
 +  /**
 +   * Verify that region operations, in this case splitting a region, are independent between the
 +   * cloned table and the original.
 +   */
 +  @Test (timeout=300000)
 +  public void testOfflineSnapshotRegionOperationsIndependent() throws Exception {
 +    runTestRegionOperationsIndependent(false);
 +  }
 +
 +  /**
 +   * Verify that region operations, in this case splitting a region, are independent between the
 +   * cloned table and the original.
 +   */
 +  @Test (timeout=300000)
 +  public void testOnlineSnapshotRegionOperationsIndependent() throws Exception {
 +    runTestRegionOperationsIndependent(true);
 +  }
 +
 +  private static void waitOnSplit(final HTable t, int originalCount) throws Exception {
 +    for (int i = 0; i < 200; i++) {
 +      try {
 +        Thread.sleep(50);
 +      } catch (InterruptedException e) {
 +        // Restore the interrupted status
 +        Thread.currentThread().interrupt();
 +      }
 +      if (t.getRegionLocations().size() > originalCount) {
 +        return;
 +      }
 +    }
 +    throw new Exception("Split did not increase the number of regions");
 +  }
 +
 +  /*
 +   * Take a snapshot of a table, add data, and verify that this only
 +   * affects one table
 +   * @param online - Whether the table is online or not during the snapshot
 +   */
 +  private void runTestSnapshotAppendIndependent(boolean online) throws Exception {
 +    FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
 +    Path rootDir = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
 +
 +    Admin admin = UTIL.getHBaseAdmin();
 +    final long startTime = System.currentTimeMillis();
 +    final TableName localTableName =
 +        TableName.valueOf(STRING_TABLE_NAME + startTime);
 +
 +    HTable original = MobSnapshotTestingUtils.createMobTable(UTIL, localTableName, TEST_FAM);
 +    try {
 +
-       SnapshotTestingUtils.loadData(UTIL, original, 500, TEST_FAM);
++      SnapshotTestingUtils.loadData(UTIL, localTableName, 500, TEST_FAM);
 +      final int origTableRowCount = MobSnapshotTestingUtils.countMobRows(original);
 +
 +      // Take a snapshot
 +      final String snapshotNameAsString = "snapshot_" + localTableName;
 +      byte[] snapshotName = Bytes.toBytes(snapshotNameAsString);
 +
 +      SnapshotTestingUtils.createSnapshotAndValidate(admin, localTableName, TEST_FAM_STR,
 +        snapshotNameAsString, rootDir, fs, online);
 +
 +      if (!online) {
 +        admin.enableTable(localTableName);
 +      }
 +      TableName cloneTableName = TableName.valueOf("test-clone-" + localTableName);
 +      admin.cloneSnapshot(snapshotName, cloneTableName);
 +
 +      HTable clonedTable = new HTable(UTIL.getConfiguration(), cloneTableName);
 +
 +      try {
 +        final int clonedTableRowCount = MobSnapshotTestingUtils.countMobRows(clonedTable);
 +
 +        Assert.assertEquals(
 +          "The line counts of original and cloned tables do not match after clone. ",
 +          origTableRowCount, clonedTableRowCount);
 +
 +        // Attempt to add data to the test
 +        final String rowKey = "new-row-" + System.currentTimeMillis();
 +
 +        Put p = new Put(Bytes.toBytes(rowKey));
 +        p.add(TEST_FAM, Bytes.toBytes("someQualifier"), Bytes.toBytes("someString"));
 +        original.put(p);
 +        original.flushCommits();
 +
 +        // Verify that it is not present in the original table
 +        Assert.assertEquals("The row count of the original table was not modified by the put",
 +          origTableRowCount + 1, MobSnapshotTestingUtils.countMobRows(original));
 +        Assert.assertEquals(
 +          "The row count of the cloned table changed as a result of addition to the original",
 +          clonedTableRowCount, MobSnapshotTestingUtils.countMobRows(clonedTable));
 +
 +        p = new Put(Bytes.toBytes(rowKey));
 +        p.add(TEST_FAM, Bytes.toBytes("someQualifier"), Bytes.toBytes("someString"));
 +        clonedTable.put(p);
 +        clonedTable.flushCommits();
 +
 +        // Verify that the new family is not in the restored table's description
 +        Assert.assertEquals(
 +          "The row count of the original table was modified by the put to the clone",
 +          origTableRowCount + 1, MobSnapshotTestingUtils.countMobRows(original));
 +        Assert.assertEquals("The row count of the cloned table was not modified by the put",
 +          clonedTableRowCount + 1, MobSnapshotTestingUtils.countMobRows(clonedTable));
 +      } finally {
 +
 +        clonedTable.close();
 +      }
 +    } finally {
 +
 +      original.close();
 +    }
 +  }
 +
 +  /*
 +   * Take a snapshot of a table, do a split, and verify that this only affects one table
 +   * @param online - Whether the table is online or not during the snapshot
 +   */
 +  private void runTestRegionOperationsIndependent(boolean online) throws Exception {
 +    FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
 +    Path rootDir = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
 +
 +    // Create a table
 +    Admin admin = UTIL.getHBaseAdmin();
 +    final long startTime = System.currentTimeMillis();
 +    final TableName localTableName =
 +        TableName.valueOf(STRING_TABLE_NAME + startTime);
 +    HTable original = MobSnapshotTestingUtils.createMobTable(UTIL, localTableName, TEST_FAM);
-     SnapshotTestingUtils.loadData(UTIL, original, 500, TEST_FAM);
++    SnapshotTestingUtils.loadData(UTIL, localTableName, 500, TEST_FAM);
 +    final int loadedTableCount = MobSnapshotTestingUtils.countMobRows(original);
 +    System.out.println("Original table has: " + loadedTableCount + " rows");
 +
 +    final String snapshotNameAsString = "snapshot_" + localTableName;
 +
 +    // Create a snapshot
 +    SnapshotTestingUtils.createSnapshotAndValidate(admin, localTableName, TEST_FAM_STR,
 +      snapshotNameAsString, rootDir, fs, online);
 +
 +    if (!online) {
 +      admin.enableTable(localTableName);
 +    }
 +
 +    TableName cloneTableName = TableName.valueOf("test-clone-" + localTableName);
 +
 +    // Clone the snapshot
 +    byte[] snapshotName = Bytes.toBytes(snapshotNameAsString);
 +    admin.cloneSnapshot(snapshotName, cloneTableName);
 +
 +    // Verify that region information is the same pre-split
 +    original.clearRegionCache();
 +    List<HRegionInfo> originalTableHRegions = admin.getTableRegions(localTableName);
 +
 +    final int originalRegionCount = originalTableHRegions.size();
 +    final int cloneTableRegionCount = admin.getTableRegions(cloneTableName).size();
 +    Assert.assertEquals(
 +      "The number of regions in the cloned table is different than in the original table.",
 +      originalRegionCount, cloneTableRegionCount);
 +
 +    // Split a region on the parent table
 +    admin.splitRegion(originalTableHRegions.get(0).getRegionName());
 +    waitOnSplit(original, originalRegionCount);
 +
 +    // Verify that the cloned table region is not split
 +    final int cloneTableRegionCount2 = admin.getTableRegions(cloneTableName).size();
 +    Assert.assertEquals(
 +      "The number of regions in the cloned table changed though none of its regions were split.",
 +      cloneTableRegionCount, cloneTableRegionCount2);
 +  }
 +
 +  /*
 +   * Take a snapshot of a table, add metadata, and verify that this only
 +   * affects one table
 +   * @param online - Whether the table is online or not during the snapshot
 +   */
 +  private void runTestSnapshotMetadataChangesIndependent(boolean online) throws Exception {
 +    FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
 +    Path rootDir = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
 +
 +    // Create a table
 +    Admin admin = UTIL.getHBaseAdmin();
 +    final long startTime = System.currentTimeMillis();
 +    final TableName localTableName =
 +        TableName.valueOf(STRING_TABLE_NAME + startTime);
 +    HTable original = MobSnapshotTestingUtils.createMobTable(UTIL, localTableName, TEST_FAM);
-     SnapshotTestingUtils.loadData(UTIL, original, 500, TEST_FAM);
++    SnapshotTestingUtils.loadData(UTIL, localTableName, 500, TEST_FAM);
 +
 +    final String snapshotNameAsString = "snapshot_" + localTableName;
 +
 +    // Create a snapshot
 +    SnapshotTestingUtils.createSnapshotAndValidate(admin, localTableName, TEST_FAM_STR,
 +      snapshotNameAsString, rootDir, fs, online);
 +
 +    if (!online) {
 +      admin.enableTable(localTableName);
 +    }
 +    TableName cloneTableName = TableName.valueOf("test-clone-" + localTableName);
 +
 +    // Clone the snapshot
 +    byte[] snapshotName = Bytes.toBytes(snapshotNameAsString);
 +    admin.cloneSnapshot(snapshotName, cloneTableName);
 +
 +    // Add a new column family to the original table
 +    byte[] TEST_FAM_2 = Bytes.toBytes("fam2");
 +    HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAM_2);
 +
 +    admin.disableTable(localTableName);
 +    admin.addColumn(localTableName, hcd);
 +
 +    // Verify that it is not in the snapshot
 +    admin.enableTable(localTableName);
 +
 +    // get a description of the cloned table
 +    // get a list of its families
 +    // assert that the family is there
 +    HTableDescriptor originalTableDescriptor = original.getTableDescriptor();
 +    HTableDescriptor clonedTableDescriptor = admin.getTableDescriptor(cloneTableName);
 +
 +    Assert.assertTrue("The original family was not found. There is something wrong. ",
 +      originalTableDescriptor.hasFamily(TEST_FAM));
 +    Assert.assertTrue("The original family was not found in the clone. There is something wrong. ",
 +      clonedTableDescriptor.hasFamily(TEST_FAM));
 +
 +    Assert.assertTrue("The new family was not found. ",
 +      originalTableDescriptor.hasFamily(TEST_FAM_2));
 +    Assert.assertTrue("The new family was not found. ",
 +      !clonedTableDescriptor.hasFamily(TEST_FAM_2));
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobSnapshotFromClient.java
----------------------------------------------------------------------
diff --cc hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobSnapshotFromClient.java
index 947354f,0000000..5bf5a30
mode 100644,000000..100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobSnapshotFromClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobSnapshotFromClient.java
@@@ -1,304 -1,0 +1,305 @@@
 +/**
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.hadoop.hbase.client;
 +
 +import static org.junit.Assert.assertEquals;
 +import static org.junit.Assert.fail;
 +
 +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.HBaseTestingUtility;
 +import org.apache.hadoop.hbase.HConstants;
- import org.apache.hadoop.hbase.LargeTests;
 +import org.apache.hadoop.hbase.TableName;
 +import org.apache.hadoop.hbase.TableNotFoundException;
 +import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 +import org.apache.hadoop.hbase.mob.MobConstants;
 +import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 +import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
 +import org.apache.hadoop.hbase.snapshot.MobSnapshotTestingUtils;
 +import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
 +import org.apache.hadoop.hbase.snapshot.SnapshotManifestV1;
 +import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
++import org.apache.hadoop.hbase.testclassification.ClientTests;
++import org.apache.hadoop.hbase.testclassification.LargeTests;
 +import org.apache.hadoop.hbase.util.Bytes;
 +import org.apache.hadoop.hbase.util.FSUtils;
 +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;
 +
 +import com.google.common.collect.Lists;
 +
 +/**
 + * Test create/using/deleting snapshots from the client
 + * <p>
 + * This is an end-to-end test for the snapshot utility
 + */
- @Category(LargeTests.class)
++@Category({LargeTests.class, ClientTests.class})
 +public class TestMobSnapshotFromClient {
 +  private static final Log LOG = LogFactory.getLog(TestSnapshotFromClient.class);
 +  protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
 +  private static final int NUM_RS = 2;
 +  private static final String STRING_TABLE_NAME = "test";
 +  protected static final byte[] TEST_FAM = Bytes.toBytes("fam");
 +  protected static final TableName TABLE_NAME =
 +      TableName.valueOf(STRING_TABLE_NAME);
 +
 +  /**
 +   * Setup the config for the cluster
 +   * @throws Exception on failure
 +   */
 +  @BeforeClass
 +  public static void setupCluster() throws Exception {
 +    setupConf(UTIL.getConfiguration());
 +    UTIL.startMiniCluster(NUM_RS);
 +  }
 +
 +  private static void setupConf(Configuration conf) {
 +    // disable the ui
 +    conf.setInt("hbase.regionsever.info.port", -1);
 +    // change the flush size to a small amount, regulating number of store files
 +    conf.setInt("hbase.hregion.memstore.flush.size", 25000);
 +    // so make sure we get a compaction when doing a load, but keep around some
 +    // files in the store
 +    conf.setInt("hbase.hstore.compaction.min", 10);
 +    conf.setInt("hbase.hstore.compactionThreshold", 10);
 +    // block writes if we get to 12 store files
 +    conf.setInt("hbase.hstore.blockingStoreFiles", 12);
 +    // Enable snapshot
 +    conf.setBoolean(SnapshotManager.HBASE_SNAPSHOT_ENABLED, true);
 +    conf.set(HConstants.HBASE_REGION_SPLIT_POLICY_KEY,
 +      ConstantSizeRegionSplitPolicy.class.getName());
 +    conf.setInt(MobConstants.MOB_FILE_CACHE_SIZE_KEY, 0);
 +  }
 +
 +  @Before
 +  public void setup() throws Exception {
 +    MobSnapshotTestingUtils.createMobTable(UTIL, TABLE_NAME, getNumReplicas(), TEST_FAM);
 +  }
 +
 +  protected int getNumReplicas() {
 +    return 1;
 +  }
 +
 +  @After
 +  public void tearDown() throws Exception {
 +    UTIL.deleteTable(TABLE_NAME);
 +    SnapshotTestingUtils.deleteAllSnapshots(UTIL.getHBaseAdmin());
 +    SnapshotTestingUtils.deleteArchiveDirectory(UTIL);
 +  }
 +
 +  @AfterClass
 +  public static void cleanupTest() throws Exception {
 +    try {
 +      UTIL.shutdownMiniCluster();
 +    } catch (Exception e) {
 +      LOG.warn("failure shutting down cluster", e);
 +    }
 +  }
 +
 +  /**
 +   * Test snapshotting not allowed hbase:meta and -ROOT-
 +   * @throws Exception
 +   */
 +  @Test (timeout=300000)
 +  public void testMetaTablesSnapshot() throws Exception {
 +    Admin admin = UTIL.getHBaseAdmin();
 +    byte[] snapshotName = Bytes.toBytes("metaSnapshot");
 +
 +    try {
 +      admin.snapshot(snapshotName, TableName.META_TABLE_NAME);
 +      fail("taking a snapshot of hbase:meta should not be allowed");
 +    } catch (IllegalArgumentException e) {
 +      // expected
 +    }
 +  }
 +
 +  /**
 +   * Test HBaseAdmin#deleteSnapshots(String) which deletes snapshots whose names match the parameter
 +   *
 +   * @throws Exception
 +   */
 +  @Test (timeout=300000)
 +  public void testSnapshotDeletionWithRegex() throws Exception {
 +    Admin admin = UTIL.getHBaseAdmin();
 +    // make sure we don't fail on listing snapshots
 +    SnapshotTestingUtils.assertNoSnapshots(admin);
 +
 +    // put some stuff in the table
 +    HTable table = new HTable(UTIL.getConfiguration(), TABLE_NAME);
 +    UTIL.loadTable(table, TEST_FAM);
 +    table.close();
 +
 +    byte[] snapshot1 = Bytes.toBytes("TableSnapshot1");
 +    admin.snapshot(snapshot1, TABLE_NAME);
 +    LOG.debug("Snapshot1 completed.");
 +
 +    byte[] snapshot2 = Bytes.toBytes("TableSnapshot2");
 +    admin.snapshot(snapshot2, TABLE_NAME);
 +    LOG.debug("Snapshot2 completed.");
 +
 +    String snapshot3 = "3rdTableSnapshot";
 +    admin.snapshot(Bytes.toBytes(snapshot3), TABLE_NAME);
 +    LOG.debug(snapshot3 + " completed.");
 +
 +    // delete the first two snapshots
 +    admin.deleteSnapshots("TableSnapshot.*");
 +    List<SnapshotDescription> snapshots = admin.listSnapshots();
 +    assertEquals(1, snapshots.size());
 +    assertEquals(snapshots.get(0).getName(), snapshot3);
 +
 +    admin.deleteSnapshot(snapshot3);
 +    admin.close();
 +  }
 +  /**
 +   * Test snapshotting a table that is offline
 +   * @throws Exception
 +   */
 +  @Test (timeout=300000)
 +  public void testOfflineTableSnapshot() throws Exception {
 +    Admin admin = UTIL.getHBaseAdmin();
 +    // make sure we don't fail on listing snapshots
 +    SnapshotTestingUtils.assertNoSnapshots(admin);
 +
 +    // put some stuff in the table
 +    HTable table = new HTable(UTIL.getConfiguration(), TABLE_NAME);
 +    UTIL.loadTable(table, TEST_FAM, false);
 +
 +    LOG.debug("FS state before disable:");
 +    FSUtils.logFileSystemState(UTIL.getTestFileSystem(),
 +      FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
 +    // XXX if this is flakey, might want to consider using the async version and looping as
 +    // disableTable can succeed and still timeout.
 +    admin.disableTable(TABLE_NAME);
 +
 +    LOG.debug("FS state before snapshot:");
 +    FSUtils.logFileSystemState(UTIL.getTestFileSystem(),
 +      FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
 +
 +    // take a snapshot of the disabled table
 +    final String SNAPSHOT_NAME = "offlineTableSnapshot";
 +    byte[] snapshot = Bytes.toBytes(SNAPSHOT_NAME);
 +
 +    SnapshotDescription desc = SnapshotDescription.newBuilder()
 +      .setType(SnapshotDescription.Type.DISABLED)
 +      .setTable(STRING_TABLE_NAME)
 +      .setName(SNAPSHOT_NAME)
 +      .setVersion(SnapshotManifestV1.DESCRIPTOR_VERSION)
 +      .build();
 +    admin.snapshot(desc);
 +    LOG.debug("Snapshot completed.");
 +
 +    // make sure we have the snapshot
 +    List<SnapshotDescription> snapshots = SnapshotTestingUtils.assertOneSnapshotThatMatches(admin,
 +      snapshot, TABLE_NAME);
 +
 +    // make sure its a valid snapshot
 +    FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
 +    Path rootDir = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
 +    LOG.debug("FS state after snapshot:");
 +    FSUtils.logFileSystemState(UTIL.getTestFileSystem(),
 +      FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
 +
 +    SnapshotTestingUtils.confirmSnapshotValid(snapshots.get(0), TABLE_NAME, TEST_FAM, rootDir,
 +      admin, fs);
 +
 +    admin.deleteSnapshot(snapshot);
 +    snapshots = admin.listSnapshots();
 +    SnapshotTestingUtils.assertNoSnapshots(admin);
 +  }
 +
 +  @Test (timeout=300000)
 +  public void testSnapshotFailsOnNonExistantTable() throws Exception {
 +    Admin admin = UTIL.getHBaseAdmin();
 +    // make sure we don't fail on listing snapshots
 +    SnapshotTestingUtils.assertNoSnapshots(admin);
 +    String tableName = "_not_a_table";
 +
 +    // make sure the table doesn't exist
 +    boolean fail = false;
 +    do {
 +    try {
 +      admin.getTableDescriptor(TableName.valueOf(tableName));
 +      fail = true;
 +          LOG.error("Table:" + tableName + " already exists, checking a new name");
 +      tableName = tableName+"!";
 +    } catch (TableNotFoundException e) {
 +      fail = false;
 +      }
 +    } while (fail);
 +
 +    // snapshot the non-existant table
 +    try {
 +      admin.snapshot("fail", TableName.valueOf(tableName));
 +      fail("Snapshot succeeded even though there is not table.");
 +    } catch (SnapshotCreationException e) {
 +      LOG.info("Correctly failed to snapshot a non-existant table:" + e.getMessage());
 +    }
 +  }
 +
 +  @Test (timeout=300000)
 +  public void testOfflineTableSnapshotWithEmptyRegions() throws Exception {
 +    // test with an empty table with one region
 +
 +    Admin admin = UTIL.getHBaseAdmin();
 +    // make sure we don't fail on listing snapshots
 +    SnapshotTestingUtils.assertNoSnapshots(admin);
 +
 +    LOG.debug("FS state before disable:");
 +    FSUtils.logFileSystemState(UTIL.getTestFileSystem(),
 +      FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
 +    admin.disableTable(TABLE_NAME);
 +
 +    LOG.debug("FS state before snapshot:");
 +    FSUtils.logFileSystemState(UTIL.getTestFileSystem(),
 +      FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
 +
 +    // take a snapshot of the disabled table
 +    byte[] snapshot = Bytes.toBytes("testOfflineTableSnapshotWithEmptyRegions");
 +    admin.snapshot(snapshot, TABLE_NAME);
 +    LOG.debug("Snapshot completed.");
 +
 +    // make sure we have the snapshot
 +    List<SnapshotDescription> snapshots = SnapshotTestingUtils.assertOneSnapshotThatMatches(admin,
 +      snapshot, TABLE_NAME);
 +
 +    // make sure its a valid snapshot
 +    FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
 +    Path rootDir = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
 +    LOG.debug("FS state after snapshot:");
 +    FSUtils.logFileSystemState(UTIL.getTestFileSystem(),
 +      FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
 +
 +    List<byte[]> emptyCfs = Lists.newArrayList(TEST_FAM); // no file in the region
 +    List<byte[]> nonEmptyCfs = Lists.newArrayList();
 +    SnapshotTestingUtils.confirmSnapshotValid(snapshots.get(0), TABLE_NAME, nonEmptyCfs, emptyCfs,
 +      rootDir, admin, fs);
 +
 +    admin.deleteSnapshot(snapshot);
 +    snapshots = admin.listSnapshots();
 +    SnapshotTestingUtils.assertNoSnapshots(admin);
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestCachedMobFile.java
----------------------------------------------------------------------
diff --cc hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestCachedMobFile.java
index b39dd2a,0000000..e0e9541
mode 100644,000000..100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestCachedMobFile.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestCachedMobFile.java
@@@ -1,154 -1,0 +1,154 @@@
 +/**
 + *
 + * 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.mob;
 +
 +import junit.framework.TestCase;
 +
 +import org.apache.commons.logging.Log;
 +import org.apache.commons.logging.LogFactory;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.hbase.HBaseConfiguration;
 +import org.apache.hadoop.hbase.KeyValue;
 +import org.apache.hadoop.hbase.KeyValue.Type;
 +import org.apache.hadoop.hbase.KeyValueUtil;
- import org.apache.hadoop.hbase.SmallTests;
 +import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 +import org.apache.hadoop.hbase.io.hfile.HFileContext;
 +import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
 +import org.apache.hadoop.hbase.regionserver.StoreFile;
++import org.apache.hadoop.hbase.testclassification.SmallTests;
 +import org.apache.hadoop.hbase.util.Bytes;
 +import org.apache.hadoop.hbase.util.FSUtils;
 +import org.junit.Assert;
 +import org.junit.Test;
 +import org.junit.experimental.categories.Category;
 +
 +@Category(SmallTests.class)
 +public class TestCachedMobFile extends TestCase{
 +  static final Log LOG = LogFactory.getLog(TestCachedMobFile.class);
 +  private Configuration conf = HBaseConfiguration.create();
 +  private CacheConfig cacheConf = new CacheConfig(conf);
 +  private final String TABLE = "tableName";
 +  private final String FAMILY = "familyName";
 +  private final String FAMILY1 = "familyName1";
 +  private final String FAMILY2 = "familyName2";
 +  private final long EXPECTED_REFERENCE_ZERO = 0;
 +  private final long EXPECTED_REFERENCE_ONE = 1;
 +  private final long EXPECTED_REFERENCE_TWO = 2;
 +
 +  @Test
 +  public void testOpenClose() throws Exception {
 +    String caseName = getName();
 +    FileSystem fs = FileSystem.get(conf);
 +    Path testDir = FSUtils.getRootDir(conf);
 +    Path outputDir = new Path(new Path(testDir, TABLE),
 +        FAMILY);
 +    HFileContext meta = new HFileContextBuilder().withBlockSize(8*1024).build();
 +    StoreFile.Writer writer = new StoreFile.WriterBuilder(conf, cacheConf, fs)
 +        .withOutputDir(outputDir).withFileContext(meta).build();
 +    MobTestUtil.writeStoreFile(writer, caseName);
 +    CachedMobFile cachedMobFile = CachedMobFile.create(fs, writer.getPath(), conf, cacheConf);
 +    Assert.assertEquals(EXPECTED_REFERENCE_ZERO, cachedMobFile.getReferenceCount());
 +    cachedMobFile.open();
 +    Assert.assertEquals(EXPECTED_REFERENCE_ONE, cachedMobFile.getReferenceCount());
 +    cachedMobFile.open();
 +    Assert.assertEquals(EXPECTED_REFERENCE_TWO, cachedMobFile.getReferenceCount());
 +    cachedMobFile.close();
 +    Assert.assertEquals(EXPECTED_REFERENCE_ONE, cachedMobFile.getReferenceCount());
 +    cachedMobFile.close();
 +    Assert.assertEquals(EXPECTED_REFERENCE_ZERO, cachedMobFile.getReferenceCount());
 +  }
 +
 +  @Test
 +  public void testCompare() throws Exception {
 +    String caseName = getName();
 +    FileSystem fs = FileSystem.get(conf);
 +    Path testDir = FSUtils.getRootDir(conf);
 +    Path outputDir1 = new Path(new Path(testDir, TABLE),
 +        FAMILY1);
 +    HFileContext meta = new HFileContextBuilder().withBlockSize(8 * 1024).build();
 +    StoreFile.Writer writer1 = new StoreFile.WriterBuilder(conf, cacheConf, fs)
 +        .withOutputDir(outputDir1).withFileContext(meta).build();
 +    MobTestUtil.writeStoreFile(writer1, caseName);
 +    CachedMobFile cachedMobFile1 = CachedMobFile.create(fs, writer1.getPath(), conf, cacheConf);
 +    Path outputDir2 = new Path(new Path(testDir, TABLE),
 +        FAMILY2);
 +    StoreFile.Writer writer2 = new StoreFile.WriterBuilder(conf, cacheConf, fs)
 +    .withOutputDir(outputDir2)
 +    .withFileContext(meta)
 +    .build();
 +    MobTestUtil.writeStoreFile(writer2, caseName);
 +    CachedMobFile cachedMobFile2 = CachedMobFile.create(fs, writer2.getPath(), conf, cacheConf);
 +    cachedMobFile1.access(1);
 +    cachedMobFile2.access(2);
 +    Assert.assertEquals(cachedMobFile1.compareTo(cachedMobFile2), 1);
 +    Assert.assertEquals(cachedMobFile2.compareTo(cachedMobFile1), -1);
 +    Assert.assertEquals(cachedMobFile1.compareTo(cachedMobFile1), 0);
 +  }
 +
 +  @Test
 +  public void testReadKeyValue() throws Exception {
 +    FileSystem fs = FileSystem.get(conf);
 +    Path testDir = FSUtils.getRootDir(conf);
 +    Path outputDir = new Path(new Path(testDir, TABLE), "familyname");
 +    HFileContext meta = new HFileContextBuilder().withBlockSize(8 * 1024).build();
 +    StoreFile.Writer writer = new StoreFile.WriterBuilder(conf, cacheConf, fs)
 +        .withOutputDir(outputDir).withFileContext(meta).build();
 +    String caseName = getName();
 +    MobTestUtil.writeStoreFile(writer, caseName);
 +    CachedMobFile cachedMobFile = CachedMobFile.create(fs, writer.getPath(), conf, cacheConf);
 +    byte[] family = Bytes.toBytes(caseName);
 +    byte[] qualify = Bytes.toBytes(caseName);
 +    // Test the start key
 +    byte[] startKey = Bytes.toBytes("aa");  // The start key bytes
 +    KeyValue expectedKey =
 +        new KeyValue(startKey, family, qualify, Long.MAX_VALUE, Type.Put, startKey);
 +    KeyValue seekKey = expectedKey.createKeyOnly(false);
 +    KeyValue kv = KeyValueUtil.ensureKeyValue(cachedMobFile.readCell(seekKey, false));
 +    MobTestUtil.assertKeyValuesEquals(expectedKey, kv);
 +
 +    // Test the end key
 +    byte[] endKey = Bytes.toBytes("zz");  // The end key bytes
 +    expectedKey = new KeyValue(endKey, family, qualify, Long.MAX_VALUE, Type.Put, endKey);
 +    seekKey = expectedKey.createKeyOnly(false);
 +    kv = KeyValueUtil.ensureKeyValue(cachedMobFile.readCell(seekKey, false));
 +    MobTestUtil.assertKeyValuesEquals(expectedKey, kv);
 +
 +    // Test the random key
 +    byte[] randomKey = Bytes.toBytes(MobTestUtil.generateRandomString(2));
 +    expectedKey = new KeyValue(randomKey, family, qualify, Long.MAX_VALUE, Type.Put, randomKey);
 +    seekKey = expectedKey.createKeyOnly(false);
 +    kv = KeyValueUtil.ensureKeyValue(cachedMobFile.readCell(seekKey, false));
 +    MobTestUtil.assertKeyValuesEquals(expectedKey, kv);
 +
 +    // Test the key which is less than the start key
 +    byte[] lowerKey = Bytes.toBytes("a1"); // Smaller than "aa"
 +    expectedKey = new KeyValue(startKey, family, qualify, Long.MAX_VALUE, Type.Put, startKey);
 +    seekKey = new KeyValue(lowerKey, family, qualify, Long.MAX_VALUE, Type.Put, lowerKey);
 +    kv = KeyValueUtil.ensureKeyValue(cachedMobFile.readCell(seekKey, false));
 +    MobTestUtil.assertKeyValuesEquals(expectedKey, kv);
 +
 +    // Test the key which is more than the end key
 +    byte[] upperKey = Bytes.toBytes("z{"); // Bigger than "zz"
 +    seekKey = new KeyValue(upperKey, family, qualify, Long.MAX_VALUE, Type.Put, upperKey);
 +    kv = KeyValueUtil.ensureKeyValue(cachedMobFile.readCell(seekKey, false));
 +    Assert.assertNull(kv);
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestDefaultMobStoreFlusher.java
----------------------------------------------------------------------
diff --cc hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestDefaultMobStoreFlusher.java
index 3432139,0000000..5e3a695
mode 100644,000000..100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestDefaultMobStoreFlusher.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestDefaultMobStoreFlusher.java
@@@ -1,193 -1,0 +1,193 @@@
 +/**
 +*
 +* 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.mob;
 +
 +import java.io.IOException;
 +import java.util.List;
 +
 +import org.apache.hadoop.hbase.Cell;
 +import org.apache.hadoop.hbase.CellUtil;
 +import org.apache.hadoop.hbase.HBaseTestingUtility;
 +import org.apache.hadoop.hbase.HColumnDescriptor;
 +import org.apache.hadoop.hbase.HTableDescriptor;
- import org.apache.hadoop.hbase.LargeTests;
 +import org.apache.hadoop.hbase.MasterNotRunningException;
 +import org.apache.hadoop.hbase.TableName;
 +import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 +import org.apache.hadoop.hbase.client.HBaseAdmin;
 +import org.apache.hadoop.hbase.client.HTable;
 +import org.apache.hadoop.hbase.client.Put;
 +import org.apache.hadoop.hbase.client.Result;
 +import org.apache.hadoop.hbase.client.ResultScanner;
 +import org.apache.hadoop.hbase.client.Scan;
++import org.apache.hadoop.hbase.testclassification.LargeTests;
 +import org.apache.hadoop.hbase.util.Bytes;
 +import org.junit.AfterClass;
 +import org.junit.Assert;
 +import org.junit.BeforeClass;
 +import org.junit.Test;
 +import org.junit.experimental.categories.Category;
 +
 +@Category(LargeTests.class)
 +public class TestDefaultMobStoreFlusher {
 +
 + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 + private final static byte [] row1 = Bytes.toBytes("row1");
 + private final static byte [] row2 = Bytes.toBytes("row2");
 + private final static byte [] family = Bytes.toBytes("family");
 + private final static byte [] qf1 = Bytes.toBytes("qf1");
 + private final static byte [] qf2 = Bytes.toBytes("qf2");
 + private final static byte [] value1 = Bytes.toBytes("value1");
 + private final static byte [] value2 = Bytes.toBytes("value2");
 +
 + @BeforeClass
 + public static void setUpBeforeClass() throws Exception {
 +   TEST_UTIL.getConfiguration().setInt("hbase.master.info.port", 0);
 +   TEST_UTIL.getConfiguration().setBoolean("hbase.regionserver.info.port.auto", true);
 +
 +   TEST_UTIL.startMiniCluster(1);
 + }
 +
 + @AfterClass
 + public static void tearDownAfterClass() throws Exception {
 +   TEST_UTIL.shutdownMiniCluster();
 + }
 +
 + @Test
 + public void testFlushNonMobFile() throws InterruptedException {
 +   String TN = "testFlushNonMobFile";
 +   HTable table = null;
 +   HBaseAdmin admin = null;
 +
 +   try {
 +     HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(TN));
 +     HColumnDescriptor hcd = new HColumnDescriptor(family);
 +     hcd.setMaxVersions(4);
 +     desc.addFamily(hcd);
 +
 +     admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
 +     admin.createTable(desc);
 +     table = new HTable(TEST_UTIL.getConfiguration(), TN);
 +
 +     //Put data
 +     Put put0 = new Put(row1);
 +     put0.add(family, qf1, 1, value1);
 +     table.put(put0);
 +
 +     //Put more data
 +     Put put1 = new Put(row2);
 +     put1.add(family, qf2, 1, value2);
 +     table.put(put1);
 +
 +     //Flush
 +     table.flushCommits();
 +     admin.flush(TN);
 +
 +     Scan scan = new Scan();
 +     scan.addColumn(family, qf1);
 +     scan.setMaxVersions(4);
 +     ResultScanner scanner = table.getScanner(scan);
 +
 +     //Compare
 +     Result result = scanner.next();
 +     int size = 0;
 +     while (result != null) {
 +       size++;
 +       List<Cell> cells = result.getColumnCells(family, qf1);
 +       // Verify the cell size
 +       Assert.assertEquals(1, cells.size());
 +       // Verify the value
 +       Assert.assertEquals(Bytes.toString(value1),
 +           Bytes.toString(CellUtil.cloneValue(cells.get(0))));
 +       result = scanner.next();
 +     }
 +     scanner.close();
 +     Assert.assertEquals(1, size);
 +     admin.close();
 +   } catch (MasterNotRunningException e1) {
 +     e1.printStackTrace();
 +   } catch (ZooKeeperConnectionException e2) {
 +     e2.printStackTrace();
 +   } catch (IOException e3) {
 +     e3.printStackTrace();
 +   }
 + }
 +
 + @Test
 + public void testFlushMobFile() throws InterruptedException {
 +   String TN = "testFlushMobFile";
 +   HTable table = null;
 +   HBaseAdmin admin = null;
 +
 +   try {
 +     HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(TN));
 +     HColumnDescriptor hcd = new HColumnDescriptor(family);
 +     hcd.setMobEnabled(true);
 +     hcd.setMobThreshold(3L);
 +     hcd.setMaxVersions(4);
 +     desc.addFamily(hcd);
 +
 +     admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
 +     admin.createTable(desc);
 +     table = new HTable(TEST_UTIL.getConfiguration(), TN);
 +
 +     //put data
 +     Put put0 = new Put(row1);
 +     put0.add(family, qf1, 1, value1);
 +     table.put(put0);
 +
 +     //put more data
 +     Put put1 = new Put(row2);
 +     put1.add(family, qf2, 1, value2);
 +     table.put(put1);
 +
 +     //flush
 +     table.flushCommits();
 +     admin.flush(TN);
 +
 +     //Scan
 +     Scan scan = new Scan();
 +     scan.addColumn(family, qf1);
 +     scan.setMaxVersions(4);
 +     ResultScanner scanner = table.getScanner(scan);
 +
 +     //Compare
 +     Result result = scanner.next();
 +     int size = 0;
 +     while (result != null) {
 +       size++;
 +       List<Cell> cells = result.getColumnCells(family, qf1);
 +       // Verify the the cell size
 +       Assert.assertEquals(1, cells.size());
 +       // Verify the value
 +       Assert.assertEquals(Bytes.toString(value1),
 +           Bytes.toString(CellUtil.cloneValue(cells.get(0))));
 +       result = scanner.next();
 +     }
 +     scanner.close();
 +     Assert.assertEquals(1, size);
 +     admin.close();
 +   } catch (MasterNotRunningException e1) {
 +     e1.printStackTrace();
 +   } catch (ZooKeeperConnectionException e2) {
 +     e2.printStackTrace();
 +   } catch (IOException e3) {
 +     e3.printStackTrace();
 +   }
 + }
 +}

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestExpiredMobFileCleaner.java
----------------------------------------------------------------------
diff --cc hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestExpiredMobFileCleaner.java
index ea50075,0000000..f16fa20
mode 100644,000000..100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestExpiredMobFileCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestExpiredMobFileCleaner.java
@@@ -1,179 -1,0 +1,179 @@@
 +/**
 + * 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.mob;
 +
 +import static org.junit.Assert.assertEquals;
 +
 +import java.util.Random;
 +
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.FileStatus;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.hbase.HBaseTestingUtility;
 +import org.apache.hadoop.hbase.HColumnDescriptor;
 +import org.apache.hadoop.hbase.HTableDescriptor;
- import org.apache.hadoop.hbase.MediumTests;
 +import org.apache.hadoop.hbase.TableName;
 +import org.apache.hadoop.hbase.client.Admin;
 +import org.apache.hadoop.hbase.client.HTable;
 +import org.apache.hadoop.hbase.client.Put;
 +import org.apache.hadoop.hbase.mob.ExpiredMobFileCleaner;
 +import org.apache.hadoop.hbase.mob.MobUtils;
++import org.apache.hadoop.hbase.testclassification.MediumTests;
 +import org.apache.hadoop.hbase.util.Bytes;
 +import org.apache.hadoop.util.ToolRunner;
 +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(MediumTests.class)
 +public class TestExpiredMobFileCleaner {
 +
 +  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 +  private final static TableName tableName = TableName.valueOf("TestExpiredMobFileCleaner");
 +  private final static String family = "family";
 +  private final static byte[] row1 = Bytes.toBytes("row1");
 +  private final static byte[] row2 = Bytes.toBytes("row2");
 +  private final static byte[] qf = Bytes.toBytes("qf");
 +
 +  private static HTable table;
 +  private static Admin admin;
 +
 +  @BeforeClass
 +  public static void setUpBeforeClass() throws Exception {
 +    TEST_UTIL.getConfiguration().setInt("hbase.master.info.port", 0);
 +    TEST_UTIL.getConfiguration().setBoolean("hbase.regionserver.info.port.auto", true);
 +
 +    TEST_UTIL.getConfiguration().setInt("hfile.format.version", 3);
 +  }
 +
 +  @AfterClass
 +  public static void tearDownAfterClass() throws Exception {
 +
 +  }
 +
 +  @Before
 +  public void setUp() throws Exception {
 +    TEST_UTIL.startMiniCluster(1);
 +  }
 +
 +  @After
 +  public void tearDown() throws Exception {
 +    admin.disableTable(tableName);
 +    admin.deleteTable(tableName);
 +    admin.close();
 +    TEST_UTIL.shutdownMiniCluster();
 +    TEST_UTIL.getTestFileSystem().delete(TEST_UTIL.getDataTestDir(), true);
 +  }
 +
 +  private void init() throws Exception {
 +    HTableDescriptor desc = new HTableDescriptor(tableName);
 +    HColumnDescriptor hcd = new HColumnDescriptor(family);
 +    hcd.setMobEnabled(true);
 +    hcd.setMobThreshold(3L);
 +    hcd.setMaxVersions(4);
 +    desc.addFamily(hcd);
 +
 +    admin = TEST_UTIL.getHBaseAdmin();
 +    admin.createTable(desc);
 +    table = new HTable(TEST_UTIL.getConfiguration(), tableName);
 +    table.setAutoFlush(false, false);
 +  }
 +
 +  private void modifyColumnExpiryDays(int expireDays) throws Exception {
 +    HColumnDescriptor hcd = new HColumnDescriptor(family);
 +    hcd.setMobEnabled(true);
 +    hcd.setMobThreshold(3L);
 +    // change ttl as expire days to make some row expired
 +    int timeToLive = expireDays * secondsOfDay();
 +    hcd.setTimeToLive(timeToLive);
 +
 +    admin.modifyColumn(tableName, hcd);
 +  }
 +
 +  private void putKVAndFlush(HTable table, byte[] row, byte[] value, long ts)
 +      throws Exception {
 +
 +    Put put = new Put(row, ts);
 +    put.add(Bytes.toBytes(family), qf, value);
 +    table.put(put);
 +
 +    table.flushCommits();
 +    admin.flush(tableName);
 +  }
 +
 +  /**
 +   * Creates a 3 day old hfile and an 1 day old hfile then sets expiry to 2 days.
 +   * Verifies that the 3 day old hfile is removed but the 1 day one is still present
 +   * after the expiry based cleaner is run.
 +   */
 +  @Test
 +  public void testCleaner() throws Exception {
 +    init();
 +
 +    Path mobDirPath = getMobFamilyPath(TEST_UTIL.getConfiguration(), tableName, family);
 +
 +    byte[] dummyData = makeDummyData(600);
 +    long ts = System.currentTimeMillis() - 3 * secondsOfDay() * 1000; // 3 days before
 +    putKVAndFlush(table, row1, dummyData, ts);
 +    FileStatus[] firstFiles = TEST_UTIL.getTestFileSystem().listStatus(mobDirPath);
 +    //the first mob file
 +    assertEquals("Before cleanup without delay 1", 1, firstFiles.length);
 +    String firstFile = firstFiles[0].getPath().getName();
 +
 +    ts = System.currentTimeMillis() - 1 * secondsOfDay() * 1000; // 1 day before
 +    putKVAndFlush(table, row2, dummyData, ts);
 +    FileStatus[] secondFiles = TEST_UTIL.getTestFileSystem().listStatus(mobDirPath);
 +    //now there are 2 mob files
 +    assertEquals("Before cleanup without delay 2", 2, secondFiles.length);
 +    String f1 = secondFiles[0].getPath().getName();
 +    String f2 = secondFiles[1].getPath().getName();
 +    String secondFile = f1.equals(firstFile) ? f2 : f1;
 +
 +    modifyColumnExpiryDays(2); // ttl = 2, make the first row expired
 +
 +    //run the cleaner
 +    String[] args = new String[2];
 +    args[0] = tableName.getNameAsString();
 +    args[1] = family;
 +    ToolRunner.run(TEST_UTIL.getConfiguration(), new ExpiredMobFileCleaner(), args);
 +
 +    FileStatus[] filesAfterClean = TEST_UTIL.getTestFileSystem().listStatus(mobDirPath);
 +    String lastFile = filesAfterClean[0].getPath().getName();
 +    //the first mob fie is removed
 +    assertEquals("After cleanup without delay 1", 1, filesAfterClean.length);
 +    assertEquals("After cleanup without delay 2", secondFile, lastFile);
 +  }
 +
 +  private Path getMobFamilyPath(Configuration conf, TableName tableName, String familyName) {
 +    Path p = new Path(MobUtils.getMobRegionPath(conf, tableName), familyName);
 +    return p;
 +  }
 +
 +  private int secondsOfDay() {
 +    return 24 * 3600;
 +  }
 +
 +  private byte[] makeDummyData(int size) {
 +    byte [] dummyData = new byte[size];
 +    new Random().nextBytes(dummyData);
 +    return dummyData;
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobDataBlockEncoding.java
----------------------------------------------------------------------
diff --cc hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobDataBlockEncoding.java
index 163dd25,0000000..055eac3
mode 100644,000000..100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobDataBlockEncoding.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobDataBlockEncoding.java
@@@ -1,141 -1,0 +1,141 @@@
 +/**
 +*
 +* 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.mob;
 +
 +import java.util.List;
 +import java.util.Random;
 +
 +import org.apache.hadoop.hbase.Cell;
 +import org.apache.hadoop.hbase.CellUtil;
 +import org.apache.hadoop.hbase.HBaseTestingUtility;
 +import org.apache.hadoop.hbase.HColumnDescriptor;
 +import org.apache.hadoop.hbase.HTableDescriptor;
- import org.apache.hadoop.hbase.MediumTests;
 +import org.apache.hadoop.hbase.TableName;
 +import org.apache.hadoop.hbase.client.HBaseAdmin;
 +import org.apache.hadoop.hbase.client.HTable;
 +import org.apache.hadoop.hbase.client.Put;
 +import org.apache.hadoop.hbase.client.Result;
 +import org.apache.hadoop.hbase.client.ResultScanner;
 +import org.apache.hadoop.hbase.client.Scan;
 +import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
++import org.apache.hadoop.hbase.testclassification.MediumTests;
 +import org.apache.hadoop.hbase.util.Bytes;
 +import org.junit.AfterClass;
 +import org.junit.Assert;
 +import org.junit.BeforeClass;
 +import org.junit.Test;
 +import org.junit.experimental.categories.Category;
 +
 +@Category(MediumTests.class)
 +public class TestMobDataBlockEncoding {
 +
 +  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 +  private final static byte [] row1 = Bytes.toBytes("row1");
 +  private final static byte [] family = Bytes.toBytes("family");
 +  private final static byte [] qf1 = Bytes.toBytes("qualifier1");
 +  private final static byte [] qf2 = Bytes.toBytes("qualifier2");
 +  protected final byte[] qf3 = Bytes.toBytes("qualifier3");
 +  private static HTable table;
 +  private static HBaseAdmin admin;
 +  private static HColumnDescriptor hcd;
 +  private static HTableDescriptor desc;
 +  private static Random random = new Random();
 +  private static long defaultThreshold = 10;
 +
 +  @BeforeClass
 +  public static void setUpBeforeClass() throws Exception {
 +    TEST_UTIL.getConfiguration().setInt("hbase.master.info.port", 0);
 +    TEST_UTIL.getConfiguration().setBoolean("hbase.regionserver.info.port.auto", true);
 +
 +    TEST_UTIL.startMiniCluster(1);
 +  }
 +
 +  @AfterClass
 +  public static void tearDownAfterClass() throws Exception {
 +    TEST_UTIL.shutdownMiniCluster();
 +  }
 +
 +  public void setUp(long threshold, String TN, DataBlockEncoding encoding)
 +      throws Exception {
 +    desc = new HTableDescriptor(TableName.valueOf(TN));
 +    hcd = new HColumnDescriptor(family);
 +    hcd.setMobEnabled(true);
 +    hcd.setMobThreshold(threshold);
 +    hcd.setMaxVersions(4);
 +    hcd.setDataBlockEncoding(encoding);
 +    desc.addFamily(hcd);
 +    admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
 +    admin.createTable(desc);
 +    table = new HTable(TEST_UTIL.getConfiguration(), TN);
 +  }
 +
 +  /**
 +   * Generate the mob value.
 +   *
 +   * @param size the size of the value
 +   * @return the mob value generated
 +   */
 +  private static byte[] generateMobValue(int size) {
 +    byte[] mobVal = new byte[size];
 +    random.nextBytes(mobVal);
 +    return mobVal;
 +  }
 +
 +  @Test
 +  public void testDataBlockEncoding() throws Exception {
 +    for (DataBlockEncoding encoding : DataBlockEncoding.values()) {
 +      testDataBlockEncoding(encoding);
 +    }
 +  }
 +
 +  public void testDataBlockEncoding(DataBlockEncoding encoding) throws Exception {
 +    String TN = "testDataBlockEncoding" + encoding;
 +    setUp(defaultThreshold, TN, encoding);
 +    long ts1 = System.currentTimeMillis();
 +    long ts2 = ts1 + 1;
 +    long ts3 = ts1 + 2;
 +    byte[] value = generateMobValue((int) defaultThreshold + 1);
 +
 +    Put put1 = new Put(row1);
 +    put1.add(family, qf1, ts3, value);
 +    put1.add(family, qf2, ts2, value);
 +    put1.add(family, qf3, ts1, value);
 +    table.put(put1);
 +
 +    table.flushCommits();
 +    admin.flush(TN);
 +
 +    Scan scan = new Scan();
 +    scan.setMaxVersions(4);
 +
 +    ResultScanner results = table.getScanner(scan);
 +    int count = 0;
 +    for (Result res : results) {
 +      List<Cell> cells = res.listCells();
 +      for(Cell cell : cells) {
 +        // Verify the value
 +        Assert.assertEquals(Bytes.toString(value),
 +            Bytes.toString(CellUtil.cloneValue(cell)));
 +        count++;
 +      }
 +    }
 +    results.close();
 +    Assert.assertEquals(3, count);
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFile.java
----------------------------------------------------------------------
diff --cc hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFile.java
index f6511f7,0000000..01050ae
mode 100644,000000..100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFile.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFile.java
@@@ -1,124 -1,0 +1,124 @@@
 +/**
 + *
 + * 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.mob;
 +
 +import junit.framework.TestCase;
 +
 +import org.apache.commons.logging.Log;
 +import org.apache.commons.logging.LogFactory;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.hbase.HBaseTestingUtility;
 +import org.apache.hadoop.hbase.KeyValue;
 +import org.apache.hadoop.hbase.KeyValue.Type;
 +import org.apache.hadoop.hbase.KeyValueUtil;
- import org.apache.hadoop.hbase.SmallTests;
 +import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 +import org.apache.hadoop.hbase.io.hfile.HFileContext;
 +import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
 +import org.apache.hadoop.hbase.regionserver.BloomType;
 +import org.apache.hadoop.hbase.regionserver.StoreFile;
 +import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
++import org.apache.hadoop.hbase.testclassification.SmallTests;
 +import org.apache.hadoop.hbase.util.Bytes;
 +import org.apache.hadoop.hbase.util.FSUtils;
 +import org.junit.Test;
 +import org.junit.experimental.categories.Category;
 +
 +@Category(SmallTests.class)
 +public class TestMobFile extends TestCase {
 +  static final Log LOG = LogFactory.getLog(TestMobFile.class);
 +  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 +  private Configuration conf = TEST_UTIL.getConfiguration();
 +  private CacheConfig cacheConf =  new CacheConfig(conf);
 +  private final String TABLE = "tableName";
 +  private final String FAMILY = "familyName";
 +
 +  @Test
 +  public void testReadKeyValue() throws Exception {
 +    FileSystem fs = FileSystem.get(conf);
 +	Path testDir = FSUtils.getRootDir(conf);
 +    Path outputDir = new Path(new Path(testDir, TABLE), FAMILY);
 +    HFileContext meta = new HFileContextBuilder().withBlockSize(8*1024).build();
 +    StoreFile.Writer writer = new StoreFile.WriterBuilder(conf, cacheConf, fs)
 +            .withOutputDir(outputDir)
 +            .withFileContext(meta)
 +            .build();
 +    String caseName = getName();
 +    MobTestUtil.writeStoreFile(writer, caseName);
 +
 +    MobFile mobFile = new MobFile(new StoreFile(fs, writer.getPath(),
 +        conf, cacheConf, BloomType.NONE));
 +    byte[] family = Bytes.toBytes(caseName);
 +    byte[] qualify = Bytes.toBytes(caseName);
 +
 +    // Test the start key
 +    byte[] startKey = Bytes.toBytes("aa");  // The start key bytes
 +    KeyValue expectedKey =
 +        new KeyValue(startKey, family, qualify, Long.MAX_VALUE, Type.Put, startKey);
 +    KeyValue seekKey = expectedKey.createKeyOnly(false);
 +    KeyValue kv = KeyValueUtil.ensureKeyValue(mobFile.readCell(seekKey, false));
 +    MobTestUtil.assertKeyValuesEquals(expectedKey, kv);
 +
 +    // Test the end key
 +    byte[] endKey = Bytes.toBytes("zz");  // The end key bytes
 +    expectedKey = new KeyValue(endKey, family, qualify, Long.MAX_VALUE, Type.Put, endKey);
 +    seekKey = expectedKey.createKeyOnly(false);
 +    kv = KeyValueUtil.ensureKeyValue(mobFile.readCell(seekKey, false));
 +    MobTestUtil.assertKeyValuesEquals(expectedKey, kv);
 +
 +    // Test the random key
 +    byte[] randomKey = Bytes.toBytes(MobTestUtil.generateRandomString(2));
 +    expectedKey = new KeyValue(randomKey, family, qualify, Long.MAX_VALUE, Type.Put, randomKey);
 +    seekKey = expectedKey.createKeyOnly(false);
 +    kv = KeyValueUtil.ensureKeyValue(mobFile.readCell(seekKey, false));
 +    MobTestUtil.assertKeyValuesEquals(expectedKey, kv);
 +
 +    // Test the key which is less than the start key
 +    byte[] lowerKey = Bytes.toBytes("a1"); // Smaller than "aa"
 +    expectedKey = new KeyValue(startKey, family, qualify, Long.MAX_VALUE, Type.Put, startKey);
 +    seekKey = new KeyValue(lowerKey, family, qualify, Long.MAX_VALUE, Type.Put, lowerKey);
 +    kv = KeyValueUtil.ensureKeyValue(mobFile.readCell(seekKey, false));
 +    MobTestUtil.assertKeyValuesEquals(expectedKey, kv);
 +
 +    // Test the key which is more than the end key
 +    byte[] upperKey = Bytes.toBytes("z{"); // Bigger than "zz"
 +    seekKey = new KeyValue(upperKey, family, qualify, Long.MAX_VALUE, Type.Put, upperKey);
 +    kv = KeyValueUtil.ensureKeyValue(mobFile.readCell(seekKey, false));
 +    assertNull(kv);
 +  }
 +
 +  @Test
 +  public void testGetScanner() throws Exception {
 +    FileSystem fs = FileSystem.get(conf);
 +    Path testDir = FSUtils.getRootDir(conf);
 +    Path outputDir = new Path(new Path(testDir, TABLE), FAMILY);
 +    HFileContext meta = new HFileContextBuilder().withBlockSize(8*1024).build();
 +    StoreFile.Writer writer = new StoreFile.WriterBuilder(conf, cacheConf, fs)
 +            .withOutputDir(outputDir)
 +            .withFileContext(meta)
 +            .build();
 +    MobTestUtil.writeStoreFile(writer, getName());
 +
 +    MobFile mobFile = new MobFile(new StoreFile(fs, writer.getPath(),
 +        conf, cacheConf, BloomType.NONE));
 +    assertNotNull(mobFile.getScanner());
 +    assertTrue(mobFile.getScanner() instanceof StoreFileScanner);
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCache.java
----------------------------------------------------------------------
diff --cc hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCache.java
index 154327c,0000000..1a809a1
mode 100644,000000..100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCache.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCache.java
@@@ -1,206 -1,0 +1,207 @@@
 +/**
 + *
 + * 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.mob;
 +
 +import java.io.IOException;
 +import java.util.Date;
 +
 +import junit.framework.TestCase;
 +
 +import org.apache.commons.logging.Log;
 +import org.apache.commons.logging.LogFactory;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.hbase.HBaseConfiguration;
 +import org.apache.hadoop.hbase.HBaseTestingUtility;
 +import org.apache.hadoop.hbase.HColumnDescriptor;
 +import org.apache.hadoop.hbase.HRegionInfo;
 +import org.apache.hadoop.hbase.HTableDescriptor;
 +import org.apache.hadoop.hbase.KeyValue;
- import org.apache.hadoop.hbase.SmallTests;
 +import org.apache.hadoop.hbase.TableName;
 +import org.apache.hadoop.hbase.regionserver.HMobStore;
 +import org.apache.hadoop.hbase.regionserver.HRegion;
 +import org.apache.hadoop.hbase.regionserver.StoreFile;
++import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
++import org.apache.hadoop.hbase.testclassification.SmallTests;
 +import org.apache.hadoop.hbase.util.Bytes;
 +import org.junit.Test;
 +import org.junit.experimental.categories.Category;
 +
 +@Category(SmallTests.class)
 +public class TestMobFileCache extends TestCase {
 +  static final Log LOG = LogFactory.getLog(TestMobFileCache.class);
 +  private HBaseTestingUtility UTIL;
 +  private HRegion region;
 +  private Configuration conf;
 +  private MobCacheConfig mobCacheConf;
 +  private MobFileCache mobFileCache;
 +  private Date currentDate = new Date();
 +  private final String TEST_CACHE_SIZE = "2";
 +  private final int EXPECTED_CACHE_SIZE_ZERO = 0;
 +  private final int EXPECTED_CACHE_SIZE_ONE = 1;
 +  private final int EXPECTED_CACHE_SIZE_TWO = 2;
 +  private final int EXPECTED_CACHE_SIZE_THREE = 3;
 +  private final long EXPECTED_REFERENCE_ONE = 1;
 +  private final long EXPECTED_REFERENCE_TWO = 2;
 +
 +  private final String TABLE = "tableName";
 +  private final String FAMILY1 = "family1";
 +  private final String FAMILY2 = "family2";
 +  private final String FAMILY3 = "family3";
 +
 +  private final byte[] ROW = Bytes.toBytes("row");
 +  private final byte[] ROW2 = Bytes.toBytes("row2");
 +  private final byte[] VALUE = Bytes.toBytes("value");
 +  private final byte[] VALUE2 = Bytes.toBytes("value2");
 +  private final byte[] QF1 = Bytes.toBytes("qf1");
 +  private final byte[] QF2 = Bytes.toBytes("qf2");
 +  private final byte[] QF3 = Bytes.toBytes("qf3");
 +
 +  @Override
 +  public void setUp() throws Exception {
 +    UTIL = HBaseTestingUtility.createLocalHTU();
 +    conf = UTIL.getConfiguration();
 +    HTableDescriptor htd = UTIL.createTableDescriptor("testMobFileCache");
 +    HColumnDescriptor hcd1 = new HColumnDescriptor(FAMILY1);
 +    hcd1.setMobEnabled(true);
 +    hcd1.setMobThreshold(0);
 +    HColumnDescriptor hcd2 = new HColumnDescriptor(FAMILY2);
 +    hcd2.setMobEnabled(true);
 +    hcd2.setMobThreshold(0);
 +    HColumnDescriptor hcd3 = new HColumnDescriptor(FAMILY3);
 +    hcd3.setMobEnabled(true);
 +    hcd3.setMobThreshold(0);
 +    htd.addFamily(hcd1);
 +    htd.addFamily(hcd2);
 +    htd.addFamily(hcd3);
 +    region = UTIL.createLocalHRegion(htd, null, null);
 +  }
 +
 +  @Override
 +  protected void tearDown() throws Exception {
 +    region.close();
 +    region.getFilesystem().delete(UTIL.getDataTestDir(), true);
 +  }
 +
 +  /**
 +   * Create the mob store file.
 +   * @param family
 +   */
 +  private Path createMobStoreFile(String family) throws IOException {
 +    return createMobStoreFile(HBaseConfiguration.create(), family);
 +  }
 +
 +  /**
 +   * Create the mob store file
 +   * @param conf
 +   * @param family
 +   */
 +  private Path createMobStoreFile(Configuration conf, String family) throws IOException {
 +    HColumnDescriptor hcd = new HColumnDescriptor(family);
 +    hcd.setMaxVersions(4);
 +    hcd.setMobEnabled(true);
 +    mobCacheConf = new MobCacheConfig(conf, hcd);
 +    return createMobStoreFile(conf, hcd);
 +  }
 +
 +  /**
 +   * Create the mob store file
 +   * @param conf
 +   * @param hcd
 +   */
 +  private Path createMobStoreFile(Configuration conf, HColumnDescriptor hcd)
 +      throws IOException {
 +    // Setting up a Store
 +    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(TABLE));
 +    htd.addFamily(hcd);
 +    HMobStore mobStore = (HMobStore) region.getStore(hcd.getName());
 +    KeyValue key1 = new KeyValue(ROW, hcd.getName(), QF1, 1, VALUE);
 +    KeyValue key2 = new KeyValue(ROW, hcd.getName(), QF2, 1, VALUE);
 +    KeyValue key3 = new KeyValue(ROW2, hcd.getName(), QF3, 1, VALUE2);
 +    KeyValue[] keys = new KeyValue[] { key1, key2, key3 };
 +    int maxKeyCount = keys.length;
 +    HRegionInfo regionInfo = new HRegionInfo();
 +    StoreFile.Writer mobWriter = mobStore.createWriterInTmp(currentDate,
 +        maxKeyCount, hcd.getCompactionCompression(), regionInfo.getStartKey());
 +    Path mobFilePath = mobWriter.getPath();
 +    String fileName = mobFilePath.getName();
 +    mobWriter.append(key1);
 +    mobWriter.append(key2);
 +    mobWriter.append(key3);
 +    mobWriter.close();
 +    String targetPathName = MobUtils.formatDate(currentDate);
 +    Path targetPath = new Path(mobStore.getPath(), targetPathName);
 +    mobStore.commitFile(mobFilePath, targetPath);
 +    return new Path(targetPath, fileName);
 +  }
 +
 +  @Test
 +  public void testMobFileCache() throws Exception {
 +    FileSystem fs = FileSystem.get(conf);
 +    conf.set(MobConstants.MOB_FILE_CACHE_SIZE_KEY, TEST_CACHE_SIZE);
 +    mobFileCache = new MobFileCache(conf);
 +    Path file1Path = createMobStoreFile(FAMILY1);
 +    Path file2Path = createMobStoreFile(FAMILY2);
 +    Path file3Path = createMobStoreFile(FAMILY3);
 +
 +    // Before open one file by the MobFileCache
 +    assertEquals(EXPECTED_CACHE_SIZE_ZERO, mobFileCache.getCacheSize());
 +    // Open one file by the MobFileCache
 +    CachedMobFile cachedMobFile1 = (CachedMobFile) mobFileCache.openFile(
 +        fs, file1Path, mobCacheConf);
 +    assertEquals(EXPECTED_CACHE_SIZE_ONE, mobFileCache.getCacheSize());
 +    assertNotNull(cachedMobFile1);
 +    assertEquals(EXPECTED_REFERENCE_TWO, cachedMobFile1.getReferenceCount());
 +
 +    // The evict is also managed by a schedule thread pool.
 +    // And its check period is set as 3600 seconds by default.
 +    // This evict should get the lock at the most time
 +    mobFileCache.evict();  // Cache not full, evict it
 +    assertEquals(EXPECTED_CACHE_SIZE_ONE, mobFileCache.getCacheSize());
 +    assertEquals(EXPECTED_REFERENCE_TWO, cachedMobFile1.getReferenceCount());
 +
 +    mobFileCache.evictFile(file1Path.getName());  // Evict one file
 +    assertEquals(EXPECTED_CACHE_SIZE_ZERO, mobFileCache.getCacheSize());
 +    assertEquals(EXPECTED_REFERENCE_ONE, cachedMobFile1.getReferenceCount());
 +
 +    cachedMobFile1.close();  // Close the cached mob file
 +
 +    // Reopen three cached file
 +    cachedMobFile1 = (CachedMobFile) mobFileCache.openFile(
 +        fs, file1Path, mobCacheConf);
 +    assertEquals(EXPECTED_CACHE_SIZE_ONE, mobFileCache.getCacheSize());
 +    CachedMobFile cachedMobFile2 = (CachedMobFile) mobFileCache.openFile(
 +        fs, file2Path, mobCacheConf);
 +    assertEquals(EXPECTED_CACHE_SIZE_TWO, mobFileCache.getCacheSize());
 +    CachedMobFile cachedMobFile3 = (CachedMobFile) mobFileCache.openFile(
 +        fs, file3Path, mobCacheConf);
 +    // Before the evict
 +    // Evict the cache, should clost the first file 1
 +    assertEquals(EXPECTED_CACHE_SIZE_THREE, mobFileCache.getCacheSize());
 +    assertEquals(EXPECTED_REFERENCE_TWO, cachedMobFile1.getReferenceCount());
 +    assertEquals(EXPECTED_REFERENCE_TWO, cachedMobFile2.getReferenceCount());
 +    assertEquals(EXPECTED_REFERENCE_TWO, cachedMobFile3.getReferenceCount());
 +    mobFileCache.evict();
 +    assertEquals(EXPECTED_CACHE_SIZE_ONE, mobFileCache.getCacheSize());
 +    assertEquals(EXPECTED_REFERENCE_ONE, cachedMobFile1.getReferenceCount());
 +    assertEquals(EXPECTED_REFERENCE_ONE, cachedMobFile2.getReferenceCount());
 +    assertEquals(EXPECTED_REFERENCE_TWO, cachedMobFile3.getReferenceCount());
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileName.java
----------------------------------------------------------------------
diff --cc hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileName.java
index 9a6cf7f,0000000..fd78e59
mode 100644,000000..100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileName.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileName.java
@@@ -1,79 -1,0 +1,79 @@@
 +/**
 + *
 + * 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.mob;
 +
 +import java.util.Date;
 +import java.util.Random;
 +import java.util.UUID;
 +
 +import junit.framework.TestCase;
 +
- import org.apache.hadoop.hbase.SmallTests;
++import org.apache.hadoop.hbase.testclassification.SmallTests;
 +import org.apache.hadoop.hbase.util.Bytes;
 +import org.apache.hadoop.hbase.util.MD5Hash;
 +import org.junit.Test;
 +import org.junit.experimental.categories.Category;
 +
 +@Category(SmallTests.class)
 +public class TestMobFileName extends TestCase {
 +
 +  private String uuid;
 +  private Date date;
 +  private String dateStr;
 +  private byte[] startKey;
 +
 +  public void setUp() {
 +    Random random = new Random();
 +    uuid = UUID.randomUUID().toString().replaceAll("-", "");
 +    date = new Date();
 +    dateStr = MobUtils.formatDate(date);
 +    startKey = Bytes.toBytes(random.nextInt());
 +  }
 +
 +  @Test
 +  public void testHashCode() {
 +    assertEquals(MobFileName.create(startKey, dateStr, uuid).hashCode(),
 +        MobFileName.create(startKey, dateStr, uuid).hashCode());
 +    assertNotSame(MobFileName.create(startKey, dateStr, uuid).hashCode(),
 +        MobFileName.create(startKey, dateStr, uuid).hashCode());
 +  }
 +
 +  @Test
 +  public void testCreate() {
 +    MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid);
 +    assertEquals(mobFileName, MobFileName.create(mobFileName.getFileName()));
 +  }
 +
 +  @Test
 +  public void testGet() {
 +    MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid);
 +    assertEquals(MD5Hash.getMD5AsHex(startKey, 0, startKey.length), mobFileName.getStartKey());
 +    assertEquals(dateStr, mobFileName.getDate());
 +    assertEquals(mobFileName.getFileName(), MD5Hash.getMD5AsHex(startKey, 0, startKey.length)
 +        + dateStr + uuid);
 +  }
 +
 +  @Test
 +  public void testEquals() {
 +    MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid);
 +    assertTrue(mobFileName.equals(mobFileName));
 +    assertFalse(mobFileName.equals(this));
 +    assertTrue(mobFileName.equals(MobFileName.create(startKey, dateStr, uuid)));
 +  }
 +}


[14/50] [abbrv] hbase git commit: HBASE-12897 Minimum memstore size is a percentage (Churro Morales)

Posted by jm...@apache.org.
HBASE-12897 Minimum memstore size is a percentage (Churro Morales)


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

Branch: refs/heads/hbase-11339
Commit: 1426f85b15b5b613e0d36693758ce2fc2ade82bf
Parents: 2f2faa6
Author: stack <st...@apache.org>
Authored: Fri Feb 6 16:15:29 2015 -0800
Committer: stack <st...@apache.org>
Committed: Fri Feb 6 16:15:29 2015 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/io/util/HeapMemorySizeUtil.java | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/1426f85b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/util/HeapMemorySizeUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/util/HeapMemorySizeUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/util/HeapMemorySizeUtil.java
index 250a984..7cfa35d 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/util/HeapMemorySizeUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/util/HeapMemorySizeUtil.java
@@ -79,10 +79,10 @@ public class HeapMemorySizeUtil {
   public static float getGlobalMemStorePercent(final Configuration c, final boolean logInvalid) {
     float limit = c.getFloat(MEMSTORE_SIZE_KEY,
         c.getFloat(MEMSTORE_SIZE_OLD_KEY, DEFAULT_MEMSTORE_SIZE));
-    if (limit > 0.8f || limit < 0.05f) {
+    if (limit > 0.8f || limit <= 0.0f) {
       if (logInvalid) {
         LOG.warn("Setting global memstore limit to default of " + DEFAULT_MEMSTORE_SIZE
-            + " because supplied value outside allowed range of 0.05 -> 0.8");
+            + " because supplied value outside allowed range of (0 -> 0.8]");
       }
       limit = DEFAULT_MEMSTORE_SIZE;
     }


[39/50] [abbrv] hbase git commit: HBASE-13016 Clean up remnants of table states stored in table descriptors (Andrey Stepachev)

Posted by jm...@apache.org.
HBASE-13016 Clean up remnants of table states stored in table descriptors (Andrey Stepachev)


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

Branch: refs/heads/hbase-11339
Commit: dec0ec229070465ce5a1f7381366a82278149f06
Parents: a0f2bc0
Author: stack <st...@apache.org>
Authored: Wed Feb 11 11:08:47 2015 -0800
Committer: stack <st...@apache.org>
Committed: Wed Feb 11 11:09:03 2015 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/client/TableState.java  |  37 +-
 .../hbase/protobuf/generated/HBaseProtos.java   | 507 ++-----------------
 hbase-protocol/src/main/protobuf/HBase.proto    |   3 -
 .../apache/hadoop/hbase/TableDescriptor.java    |  58 +--
 .../hadoop/hbase/master/TableStateManager.java  |  17 +-
 .../hadoop/hbase/TestTableDescriptor.java       |   3 +-
 6 files changed, 61 insertions(+), 564 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/dec0ec22/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableState.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableState.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableState.java
index 77c90f5..c69cdfc 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableState.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableState.java
@@ -22,7 +22,6 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 
 /**
@@ -94,28 +93,17 @@ public class TableState {
 
   }
 
-  private final long timestamp;
   private final TableName tableName;
   private final State state;
 
   /**
    * Create instance of TableState.
+   * @param tableName name of the table
    * @param state table state
    */
-  public TableState(TableName tableName, State state, long timestamp) {
+  public TableState(TableName tableName, State state) {
     this.tableName = tableName;
     this.state = state;
-    this.timestamp = timestamp;
-  }
-
-  /**
-   * Create instance of TableState with current timestamp
-   *
-   * @param tableName table for which state is created
-   * @param state     state of the table
-   */
-  public TableState(TableName tableName, State state) {
-    this(tableName, state, System.currentTimeMillis());
   }
 
   /**
@@ -126,15 +114,6 @@ public class TableState {
   }
 
   /**
-   * Timestamp of table state
-   *
-   * @return milliseconds
-   */
-  public long getTimestamp() {
-    return timestamp;
-  }
-
-  /**
    * Table name for state
    *
    * @return milliseconds
@@ -172,10 +151,7 @@ public class TableState {
    */
   public HBaseProtos.TableState convert() {
     return HBaseProtos.TableState.newBuilder()
-        .setState(this.state.convert())
-        .setTable(ProtobufUtil.toProtoTableName(this.tableName)) // set for backward compatibility
-        .setTimestamp(this.timestamp)
-            .build();
+        .setState(this.state.convert()).build();
   }
 
   /**
@@ -187,7 +163,7 @@ public class TableState {
    */
   public static TableState convert(TableName tableName, HBaseProtos.TableState tableState) {
     TableState.State state = State.convert(tableState.getState());
-    return new TableState(tableName, state, tableState.getTimestamp());
+    return new TableState(tableName, state);
   }
 
   public static TableState parseFrom(TableName tableName, byte[] bytes)
@@ -220,7 +196,6 @@ public class TableState {
 
     TableState that = (TableState) o;
 
-    if (timestamp != that.timestamp) return false;
     if (state != that.state) return false;
     if (tableName != null ? !tableName.equals(that.tableName) : that.tableName != null)
       return false;
@@ -230,8 +205,7 @@ public class TableState {
 
   @Override
   public int hashCode() {
-    int result = (int) (timestamp ^ (timestamp >>> 32));
-    result = 31 * result + (tableName != null ? tableName.hashCode() : 0);
+    int result = (tableName != null ? tableName.hashCode() : 0);
     result = 31 * result + (state != null ? state.hashCode() : 0);
     return result;
   }
@@ -239,7 +213,6 @@ public class TableState {
   @Override
   public String toString() {
     return "TableState{" +
-        "timestamp=" + timestamp +
         ", tableName=" + tableName +
         ", state=" + state +
         '}';

http://git-wip-us.apache.org/repos/asf/hbase/blob/dec0ec22/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java
index a96ef17..3007d25 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java
@@ -2551,30 +2551,6 @@ public final class HBaseProtos {
      * </pre>
      */
     org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State getState();
-
-    // required .TableName table = 2;
-    /**
-     * <code>required .TableName table = 2;</code>
-     */
-    boolean hasTable();
-    /**
-     * <code>required .TableName table = 2;</code>
-     */
-    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTable();
-    /**
-     * <code>required .TableName table = 2;</code>
-     */
-    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableOrBuilder();
-
-    // optional uint64 timestamp = 3;
-    /**
-     * <code>optional uint64 timestamp = 3;</code>
-     */
-    boolean hasTimestamp();
-    /**
-     * <code>optional uint64 timestamp = 3;</code>
-     */
-    long getTimestamp();
   }
   /**
    * Protobuf type {@code TableState}
@@ -2642,24 +2618,6 @@ public final class HBaseProtos {
               }
               break;
             }
-            case 18: {
-              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder subBuilder = null;
-              if (((bitField0_ & 0x00000002) == 0x00000002)) {
-                subBuilder = table_.toBuilder();
-              }
-              table_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.PARSER, extensionRegistry);
-              if (subBuilder != null) {
-                subBuilder.mergeFrom(table_);
-                table_ = subBuilder.buildPartial();
-              }
-              bitField0_ |= 0x00000002;
-              break;
-            }
-            case 24: {
-              bitField0_ |= 0x00000004;
-              timestamp_ = input.readUInt64();
-              break;
-            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -2828,48 +2786,8 @@ public final class HBaseProtos {
       return state_;
     }
 
-    // required .TableName table = 2;
-    public static final int TABLE_FIELD_NUMBER = 2;
-    private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName table_;
-    /**
-     * <code>required .TableName table = 2;</code>
-     */
-    public boolean hasTable() {
-      return ((bitField0_ & 0x00000002) == 0x00000002);
-    }
-    /**
-     * <code>required .TableName table = 2;</code>
-     */
-    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTable() {
-      return table_;
-    }
-    /**
-     * <code>required .TableName table = 2;</code>
-     */
-    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableOrBuilder() {
-      return table_;
-    }
-
-    // optional uint64 timestamp = 3;
-    public static final int TIMESTAMP_FIELD_NUMBER = 3;
-    private long timestamp_;
-    /**
-     * <code>optional uint64 timestamp = 3;</code>
-     */
-    public boolean hasTimestamp() {
-      return ((bitField0_ & 0x00000004) == 0x00000004);
-    }
-    /**
-     * <code>optional uint64 timestamp = 3;</code>
-     */
-    public long getTimestamp() {
-      return timestamp_;
-    }
-
     private void initFields() {
       state_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State.ENABLED;
-      table_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
-      timestamp_ = 0L;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -2880,14 +2798,6 @@ public final class HBaseProtos {
         memoizedIsInitialized = 0;
         return false;
       }
-      if (!hasTable()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      if (!getTable().isInitialized()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
       memoizedIsInitialized = 1;
       return true;
     }
@@ -2898,12 +2808,6 @@ public final class HBaseProtos {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeEnum(1, state_.getNumber());
       }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeMessage(2, table_);
-      }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        output.writeUInt64(3, timestamp_);
-      }
       getUnknownFields().writeTo(output);
     }
 
@@ -2917,14 +2821,6 @@ public final class HBaseProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeEnumSize(1, state_.getNumber());
       }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(2, table_);
-      }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeUInt64Size(3, timestamp_);
-      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -2953,16 +2849,6 @@ public final class HBaseProtos {
         result = result &&
             (getState() == other.getState());
       }
-      result = result && (hasTable() == other.hasTable());
-      if (hasTable()) {
-        result = result && getTable()
-            .equals(other.getTable());
-      }
-      result = result && (hasTimestamp() == other.hasTimestamp());
-      if (hasTimestamp()) {
-        result = result && (getTimestamp()
-            == other.getTimestamp());
-      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -2980,14 +2866,6 @@ public final class HBaseProtos {
         hash = (37 * hash) + STATE_FIELD_NUMBER;
         hash = (53 * hash) + hashEnum(getState());
       }
-      if (hasTable()) {
-        hash = (37 * hash) + TABLE_FIELD_NUMBER;
-        hash = (53 * hash) + getTable().hashCode();
-      }
-      if (hasTimestamp()) {
-        hash = (37 * hash) + TIMESTAMP_FIELD_NUMBER;
-        hash = (53 * hash) + hashLong(getTimestamp());
-      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -3093,7 +2971,6 @@ public final class HBaseProtos {
       }
       private void maybeForceBuilderInitialization() {
         if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
-          getTableFieldBuilder();
         }
       }
       private static Builder create() {
@@ -3104,14 +2981,6 @@ public final class HBaseProtos {
         super.clear();
         state_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State.ENABLED;
         bitField0_ = (bitField0_ & ~0x00000001);
-        if (tableBuilder_ == null) {
-          table_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
-        } else {
-          tableBuilder_.clear();
-        }
-        bitField0_ = (bitField0_ & ~0x00000002);
-        timestamp_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000004);
         return this;
       }
 
@@ -3144,18 +3013,6 @@ public final class HBaseProtos {
           to_bitField0_ |= 0x00000001;
         }
         result.state_ = state_;
-        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
-          to_bitField0_ |= 0x00000002;
-        }
-        if (tableBuilder_ == null) {
-          result.table_ = table_;
-        } else {
-          result.table_ = tableBuilder_.build();
-        }
-        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
-          to_bitField0_ |= 0x00000004;
-        }
-        result.timestamp_ = timestamp_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -3175,12 +3032,6 @@ public final class HBaseProtos {
         if (other.hasState()) {
           setState(other.getState());
         }
-        if (other.hasTable()) {
-          mergeTable(other.getTable());
-        }
-        if (other.hasTimestamp()) {
-          setTimestamp(other.getTimestamp());
-        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -3190,14 +3041,6 @@ public final class HBaseProtos {
           
           return false;
         }
-        if (!hasTable()) {
-          
-          return false;
-        }
-        if (!getTable().isInitialized()) {
-          
-          return false;
-        }
         return true;
       }
 
@@ -3272,156 +3115,6 @@ public final class HBaseProtos {
         return this;
       }
 
-      // required .TableName table = 2;
-      private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName table_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
-      private com.google.protobuf.SingleFieldBuilder<
-          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableBuilder_;
-      /**
-       * <code>required .TableName table = 2;</code>
-       */
-      public boolean hasTable() {
-        return ((bitField0_ & 0x00000002) == 0x00000002);
-      }
-      /**
-       * <code>required .TableName table = 2;</code>
-       */
-      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTable() {
-        if (tableBuilder_ == null) {
-          return table_;
-        } else {
-          return tableBuilder_.getMessage();
-        }
-      }
-      /**
-       * <code>required .TableName table = 2;</code>
-       */
-      public Builder setTable(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
-        if (tableBuilder_ == null) {
-          if (value == null) {
-            throw new NullPointerException();
-          }
-          table_ = value;
-          onChanged();
-        } else {
-          tableBuilder_.setMessage(value);
-        }
-        bitField0_ |= 0x00000002;
-        return this;
-      }
-      /**
-       * <code>required .TableName table = 2;</code>
-       */
-      public Builder setTable(
-          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) {
-        if (tableBuilder_ == null) {
-          table_ = builderForValue.build();
-          onChanged();
-        } else {
-          tableBuilder_.setMessage(builderForValue.build());
-        }
-        bitField0_ |= 0x00000002;
-        return this;
-      }
-      /**
-       * <code>required .TableName table = 2;</code>
-       */
-      public Builder mergeTable(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
-        if (tableBuilder_ == null) {
-          if (((bitField0_ & 0x00000002) == 0x00000002) &&
-              table_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance()) {
-            table_ =
-              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.newBuilder(table_).mergeFrom(value).buildPartial();
-          } else {
-            table_ = value;
-          }
-          onChanged();
-        } else {
-          tableBuilder_.mergeFrom(value);
-        }
-        bitField0_ |= 0x00000002;
-        return this;
-      }
-      /**
-       * <code>required .TableName table = 2;</code>
-       */
-      public Builder clearTable() {
-        if (tableBuilder_ == null) {
-          table_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
-          onChanged();
-        } else {
-          tableBuilder_.clear();
-        }
-        bitField0_ = (bitField0_ & ~0x00000002);
-        return this;
-      }
-      /**
-       * <code>required .TableName table = 2;</code>
-       */
-      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder getTableBuilder() {
-        bitField0_ |= 0x00000002;
-        onChanged();
-        return getTableFieldBuilder().getBuilder();
-      }
-      /**
-       * <code>required .TableName table = 2;</code>
-       */
-      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableOrBuilder() {
-        if (tableBuilder_ != null) {
-          return tableBuilder_.getMessageOrBuilder();
-        } else {
-          return table_;
-        }
-      }
-      /**
-       * <code>required .TableName table = 2;</code>
-       */
-      private com.google.protobuf.SingleFieldBuilder<
-          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
-          getTableFieldBuilder() {
-        if (tableBuilder_ == null) {
-          tableBuilder_ = new com.google.protobuf.SingleFieldBuilder<
-              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
-                  table_,
-                  getParentForChildren(),
-                  isClean());
-          table_ = null;
-        }
-        return tableBuilder_;
-      }
-
-      // optional uint64 timestamp = 3;
-      private long timestamp_ ;
-      /**
-       * <code>optional uint64 timestamp = 3;</code>
-       */
-      public boolean hasTimestamp() {
-        return ((bitField0_ & 0x00000004) == 0x00000004);
-      }
-      /**
-       * <code>optional uint64 timestamp = 3;</code>
-       */
-      public long getTimestamp() {
-        return timestamp_;
-      }
-      /**
-       * <code>optional uint64 timestamp = 3;</code>
-       */
-      public Builder setTimestamp(long value) {
-        bitField0_ |= 0x00000004;
-        timestamp_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional uint64 timestamp = 3;</code>
-       */
-      public Builder clearTimestamp() {
-        bitField0_ = (bitField0_ & ~0x00000004);
-        timestamp_ = 0L;
-        onChanged();
-        return this;
-      }
-
       // @@protoc_insertion_point(builder_scope:TableState)
     }
 
@@ -3449,16 +3142,6 @@ public final class HBaseProtos {
      * <code>required .TableSchema schema = 1;</code>
      */
     org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getSchemaOrBuilder();
-
-    // optional .TableState.State state = 2 [default = ENABLED, deprecated = true];
-    /**
-     * <code>optional .TableState.State state = 2 [default = ENABLED, deprecated = true];</code>
-     */
-    @java.lang.Deprecated boolean hasState();
-    /**
-     * <code>optional .TableState.State state = 2 [default = ENABLED, deprecated = true];</code>
-     */
-    @java.lang.Deprecated org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State getState();
   }
   /**
    * Protobuf type {@code TableDescriptor}
@@ -3528,17 +3211,6 @@ public final class HBaseProtos {
               bitField0_ |= 0x00000001;
               break;
             }
-            case 16: {
-              int rawValue = input.readEnum();
-              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State value = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State.valueOf(rawValue);
-              if (value == null) {
-                unknownFields.mergeVarintField(2, rawValue);
-              } else {
-                bitField0_ |= 0x00000002;
-                state_ = value;
-              }
-              break;
-            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -3601,25 +3273,8 @@ public final class HBaseProtos {
       return schema_;
     }
 
-    // optional .TableState.State state = 2 [default = ENABLED, deprecated = true];
-    public static final int STATE_FIELD_NUMBER = 2;
-    private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State state_;
-    /**
-     * <code>optional .TableState.State state = 2 [default = ENABLED, deprecated = true];</code>
-     */
-    @java.lang.Deprecated public boolean hasState() {
-      return ((bitField0_ & 0x00000002) == 0x00000002);
-    }
-    /**
-     * <code>optional .TableState.State state = 2 [default = ENABLED, deprecated = true];</code>
-     */
-    @java.lang.Deprecated public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State getState() {
-      return state_;
-    }
-
     private void initFields() {
       schema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance();
-      state_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State.ENABLED;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -3644,9 +3299,6 @@ public final class HBaseProtos {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, schema_);
       }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeEnum(2, state_.getNumber());
-      }
       getUnknownFields().writeTo(output);
     }
 
@@ -3660,10 +3312,6 @@ public final class HBaseProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, schema_);
       }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeEnumSize(2, state_.getNumber());
-      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -3692,11 +3340,6 @@ public final class HBaseProtos {
         result = result && getSchema()
             .equals(other.getSchema());
       }
-      result = result && (hasState() == other.hasState());
-      if (hasState()) {
-        result = result &&
-            (getState() == other.getState());
-      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -3714,10 +3357,6 @@ public final class HBaseProtos {
         hash = (37 * hash) + SCHEMA_FIELD_NUMBER;
         hash = (53 * hash) + getSchema().hashCode();
       }
-      if (hasState()) {
-        hash = (37 * hash) + STATE_FIELD_NUMBER;
-        hash = (53 * hash) + hashEnum(getState());
-      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -3838,8 +3477,6 @@ public final class HBaseProtos {
           schemaBuilder_.clear();
         }
         bitField0_ = (bitField0_ & ~0x00000001);
-        state_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State.ENABLED;
-        bitField0_ = (bitField0_ & ~0x00000002);
         return this;
       }
 
@@ -3876,10 +3513,6 @@ public final class HBaseProtos {
         } else {
           result.schema_ = schemaBuilder_.build();
         }
-        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
-          to_bitField0_ |= 0x00000002;
-        }
-        result.state_ = state_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -3899,9 +3532,6 @@ public final class HBaseProtos {
         if (other.hasSchema()) {
           mergeSchema(other.getSchema());
         }
-        if (other.hasState()) {
-          setState(other.getState());
-        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -4054,42 +3684,6 @@ public final class HBaseProtos {
         return schemaBuilder_;
       }
 
-      // optional .TableState.State state = 2 [default = ENABLED, deprecated = true];
-      private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State state_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State.ENABLED;
-      /**
-       * <code>optional .TableState.State state = 2 [default = ENABLED, deprecated = true];</code>
-       */
-      @java.lang.Deprecated public boolean hasState() {
-        return ((bitField0_ & 0x00000002) == 0x00000002);
-      }
-      /**
-       * <code>optional .TableState.State state = 2 [default = ENABLED, deprecated = true];</code>
-       */
-      @java.lang.Deprecated public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State getState() {
-        return state_;
-      }
-      /**
-       * <code>optional .TableState.State state = 2 [default = ENABLED, deprecated = true];</code>
-       */
-      @java.lang.Deprecated public Builder setState(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State value) {
-        if (value == null) {
-          throw new NullPointerException();
-        }
-        bitField0_ |= 0x00000002;
-        state_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional .TableState.State state = 2 [default = ENABLED, deprecated = true];</code>
-       */
-      @java.lang.Deprecated public Builder clearState() {
-        bitField0_ = (bitField0_ & ~0x00000002);
-        state_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State.ENABLED;
-        onChanged();
-        return this;
-      }
-
       // @@protoc_insertion_point(builder_scope:TableDescriptor)
     }
 
@@ -18193,56 +17787,53 @@ public final class HBaseProtos {
       "Name\022#\n\nattributes\030\002 \003(\0132\017.BytesBytesPai" +
       "r\022,\n\017column_families\030\003 \003(\0132\023.ColumnFamil" +
       "ySchema\022&\n\rconfiguration\030\004 \003(\0132\017.NameStr" +
-      "ingPair\"\235\001\n\nTableState\022 \n\005state\030\001 \002(\0162\021." +
-      "TableState.State\022\031\n\005table\030\002 \002(\0132\n.TableN" +
-      "ame\022\021\n\ttimestamp\030\003 \001(\004\"?\n\005State\022\013\n\007ENABL" +
-      "ED\020\000\022\014\n\010DISABLED\020\001\022\r\n\tDISABLING\020\002\022\014\n\010ENA",
-      "BLING\020\003\"^\n\017TableDescriptor\022\034\n\006schema\030\001 \002" +
-      "(\0132\014.TableSchema\022-\n\005state\030\002 \001(\0162\021.TableS" +
-      "tate.State:\007ENABLEDB\002\030\001\"o\n\022ColumnFamilyS" +
-      "chema\022\014\n\004name\030\001 \002(\014\022#\n\nattributes\030\002 \003(\0132" +
-      "\017.BytesBytesPair\022&\n\rconfiguration\030\003 \003(\0132" +
-      "\017.NameStringPair\"\232\001\n\nRegionInfo\022\021\n\tregio" +
-      "n_id\030\001 \002(\004\022\036\n\ntable_name\030\002 \002(\0132\n.TableNa" +
-      "me\022\021\n\tstart_key\030\003 \001(\014\022\017\n\007end_key\030\004 \001(\014\022\017" +
-      "\n\007offline\030\005 \001(\010\022\r\n\005split\030\006 \001(\010\022\025\n\nreplic" +
-      "a_id\030\007 \001(\005:\0010\"1\n\014FavoredNodes\022!\n\014favored",
-      "_node\030\001 \003(\0132\013.ServerName\"\225\001\n\017RegionSpeci" +
-      "fier\0222\n\004type\030\001 \002(\0162$.RegionSpecifier.Reg" +
-      "ionSpecifierType\022\r\n\005value\030\002 \002(\014\"?\n\023Regio" +
-      "nSpecifierType\022\017\n\013REGION_NAME\020\001\022\027\n\023ENCOD" +
-      "ED_REGION_NAME\020\002\"%\n\tTimeRange\022\014\n\004from\030\001 " +
-      "\001(\004\022\n\n\002to\030\002 \001(\004\"A\n\nServerName\022\021\n\thost_na" +
-      "me\030\001 \002(\t\022\014\n\004port\030\002 \001(\r\022\022\n\nstart_code\030\003 \001" +
-      "(\004\"\033\n\013Coprocessor\022\014\n\004name\030\001 \002(\t\"-\n\016NameS" +
-      "tringPair\022\014\n\004name\030\001 \002(\t\022\r\n\005value\030\002 \002(\t\"," +
-      "\n\rNameBytesPair\022\014\n\004name\030\001 \002(\t\022\r\n\005value\030\002",
-      " \001(\014\"/\n\016BytesBytesPair\022\r\n\005first\030\001 \002(\014\022\016\n" +
-      "\006second\030\002 \002(\014\",\n\rNameInt64Pair\022\014\n\004name\030\001" +
-      " \001(\t\022\r\n\005value\030\002 \001(\003\"\314\001\n\023SnapshotDescript" +
-      "ion\022\014\n\004name\030\001 \002(\t\022\r\n\005table\030\002 \001(\t\022\030\n\rcrea" +
-      "tion_time\030\003 \001(\003:\0010\022.\n\004type\030\004 \001(\0162\031.Snaps" +
-      "hotDescription.Type:\005FLUSH\022\017\n\007version\030\005 " +
-      "\001(\005\022\r\n\005owner\030\006 \001(\t\".\n\004Type\022\014\n\010DISABLED\020\000" +
-      "\022\t\n\005FLUSH\020\001\022\r\n\tSKIPFLUSH\020\002\"}\n\024ProcedureD" +
-      "escription\022\021\n\tsignature\030\001 \002(\t\022\020\n\010instanc" +
-      "e\030\002 \001(\t\022\030\n\rcreation_time\030\003 \001(\003:\0010\022&\n\rcon",
-      "figuration\030\004 \003(\0132\017.NameStringPair\"\n\n\010Emp" +
-      "tyMsg\"\033\n\007LongMsg\022\020\n\010long_msg\030\001 \002(\003\"\037\n\tDo" +
-      "ubleMsg\022\022\n\ndouble_msg\030\001 \002(\001\"\'\n\rBigDecima" +
-      "lMsg\022\026\n\016bigdecimal_msg\030\001 \002(\014\"5\n\004UUID\022\026\n\016" +
-      "least_sig_bits\030\001 \002(\004\022\025\n\rmost_sig_bits\030\002 " +
-      "\002(\004\"K\n\023NamespaceDescriptor\022\014\n\004name\030\001 \002(\014" +
-      "\022&\n\rconfiguration\030\002 \003(\0132\017.NameStringPair" +
-      "\"$\n\020RegionServerInfo\022\020\n\010infoPort\030\001 \001(\005*r" +
-      "\n\013CompareType\022\010\n\004LESS\020\000\022\021\n\rLESS_OR_EQUAL" +
-      "\020\001\022\t\n\005EQUAL\020\002\022\r\n\tNOT_EQUAL\020\003\022\024\n\020GREATER_",
-      "OR_EQUAL\020\004\022\013\n\007GREATER\020\005\022\t\n\005NO_OP\020\006*n\n\010Ti" +
-      "meUnit\022\017\n\013NANOSECONDS\020\001\022\020\n\014MICROSECONDS\020" +
-      "\002\022\020\n\014MILLISECONDS\020\003\022\013\n\007SECONDS\020\004\022\013\n\007MINU" +
-      "TES\020\005\022\t\n\005HOURS\020\006\022\010\n\004DAYS\020\007B>\n*org.apache" +
-      ".hadoop.hbase.protobuf.generatedB\013HBaseP" +
-      "rotosH\001\240\001\001"
+      "ingPair\"o\n\nTableState\022 \n\005state\030\001 \002(\0162\021.T" +
+      "ableState.State\"?\n\005State\022\013\n\007ENABLED\020\000\022\014\n" +
+      "\010DISABLED\020\001\022\r\n\tDISABLING\020\002\022\014\n\010ENABLING\020\003" +
+      "\"/\n\017TableDescriptor\022\034\n\006schema\030\001 \002(\0132\014.Ta",
+      "bleSchema\"o\n\022ColumnFamilySchema\022\014\n\004name\030" +
+      "\001 \002(\014\022#\n\nattributes\030\002 \003(\0132\017.BytesBytesPa" +
+      "ir\022&\n\rconfiguration\030\003 \003(\0132\017.NameStringPa" +
+      "ir\"\232\001\n\nRegionInfo\022\021\n\tregion_id\030\001 \002(\004\022\036\n\n" +
+      "table_name\030\002 \002(\0132\n.TableName\022\021\n\tstart_ke" +
+      "y\030\003 \001(\014\022\017\n\007end_key\030\004 \001(\014\022\017\n\007offline\030\005 \001(" +
+      "\010\022\r\n\005split\030\006 \001(\010\022\025\n\nreplica_id\030\007 \001(\005:\0010\"" +
+      "1\n\014FavoredNodes\022!\n\014favored_node\030\001 \003(\0132\013." +
+      "ServerName\"\225\001\n\017RegionSpecifier\0222\n\004type\030\001" +
+      " \002(\0162$.RegionSpecifier.RegionSpecifierTy",
+      "pe\022\r\n\005value\030\002 \002(\014\"?\n\023RegionSpecifierType" +
+      "\022\017\n\013REGION_NAME\020\001\022\027\n\023ENCODED_REGION_NAME" +
+      "\020\002\"%\n\tTimeRange\022\014\n\004from\030\001 \001(\004\022\n\n\002to\030\002 \001(" +
+      "\004\"A\n\nServerName\022\021\n\thost_name\030\001 \002(\t\022\014\n\004po" +
+      "rt\030\002 \001(\r\022\022\n\nstart_code\030\003 \001(\004\"\033\n\013Coproces" +
+      "sor\022\014\n\004name\030\001 \002(\t\"-\n\016NameStringPair\022\014\n\004n" +
+      "ame\030\001 \002(\t\022\r\n\005value\030\002 \002(\t\",\n\rNameBytesPai" +
+      "r\022\014\n\004name\030\001 \002(\t\022\r\n\005value\030\002 \001(\014\"/\n\016BytesB" +
+      "ytesPair\022\r\n\005first\030\001 \002(\014\022\016\n\006second\030\002 \002(\014\"" +
+      ",\n\rNameInt64Pair\022\014\n\004name\030\001 \001(\t\022\r\n\005value\030",
+      "\002 \001(\003\"\314\001\n\023SnapshotDescription\022\014\n\004name\030\001 " +
+      "\002(\t\022\r\n\005table\030\002 \001(\t\022\030\n\rcreation_time\030\003 \001(" +
+      "\003:\0010\022.\n\004type\030\004 \001(\0162\031.SnapshotDescription" +
+      ".Type:\005FLUSH\022\017\n\007version\030\005 \001(\005\022\r\n\005owner\030\006" +
+      " \001(\t\".\n\004Type\022\014\n\010DISABLED\020\000\022\t\n\005FLUSH\020\001\022\r\n" +
+      "\tSKIPFLUSH\020\002\"}\n\024ProcedureDescription\022\021\n\t" +
+      "signature\030\001 \002(\t\022\020\n\010instance\030\002 \001(\t\022\030\n\rcre" +
+      "ation_time\030\003 \001(\003:\0010\022&\n\rconfiguration\030\004 \003" +
+      "(\0132\017.NameStringPair\"\n\n\010EmptyMsg\"\033\n\007LongM" +
+      "sg\022\020\n\010long_msg\030\001 \002(\003\"\037\n\tDoubleMsg\022\022\n\ndou",
+      "ble_msg\030\001 \002(\001\"\'\n\rBigDecimalMsg\022\026\n\016bigdec" +
+      "imal_msg\030\001 \002(\014\"5\n\004UUID\022\026\n\016least_sig_bits" +
+      "\030\001 \002(\004\022\025\n\rmost_sig_bits\030\002 \002(\004\"K\n\023Namespa" +
+      "ceDescriptor\022\014\n\004name\030\001 \002(\014\022&\n\rconfigurat" +
+      "ion\030\002 \003(\0132\017.NameStringPair\"$\n\020RegionServ" +
+      "erInfo\022\020\n\010infoPort\030\001 \001(\005*r\n\013CompareType\022" +
+      "\010\n\004LESS\020\000\022\021\n\rLESS_OR_EQUAL\020\001\022\t\n\005EQUAL\020\002\022" +
+      "\r\n\tNOT_EQUAL\020\003\022\024\n\020GREATER_OR_EQUAL\020\004\022\013\n\007" +
+      "GREATER\020\005\022\t\n\005NO_OP\020\006*n\n\010TimeUnit\022\017\n\013NANO" +
+      "SECONDS\020\001\022\020\n\014MICROSECONDS\020\002\022\020\n\014MILLISECO",
+      "NDS\020\003\022\013\n\007SECONDS\020\004\022\013\n\007MINUTES\020\005\022\t\n\005HOURS" +
+      "\020\006\022\010\n\004DAYS\020\007B>\n*org.apache.hadoop.hbase." +
+      "protobuf.generatedB\013HBaseProtosH\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -18266,13 +17857,13 @@ public final class HBaseProtos {
           internal_static_TableState_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_TableState_descriptor,
-              new java.lang.String[] { "State", "Table", "Timestamp", });
+              new java.lang.String[] { "State", });
           internal_static_TableDescriptor_descriptor =
             getDescriptor().getMessageTypes().get(3);
           internal_static_TableDescriptor_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_TableDescriptor_descriptor,
-              new java.lang.String[] { "Schema", "State", });
+              new java.lang.String[] { "Schema", });
           internal_static_ColumnFamilySchema_descriptor =
             getDescriptor().getMessageTypes().get(4);
           internal_static_ColumnFamilySchema_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/hbase/blob/dec0ec22/hbase-protocol/src/main/protobuf/HBase.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/HBase.proto b/hbase-protocol/src/main/protobuf/HBase.proto
index 1566846..00e2850 100644
--- a/hbase-protocol/src/main/protobuf/HBase.proto
+++ b/hbase-protocol/src/main/protobuf/HBase.proto
@@ -55,14 +55,11 @@ message TableState {
   }
   // This is the table's state.
   required State state = 1;
-  required TableName table = 2 [deprecated = true];
-  optional uint64 timestamp = 3;
 }
 
 /** On HDFS representation of table state. */
 message TableDescriptor {
   required TableSchema schema = 1;
-  optional TableState.State state = 2 [ default = ENABLED, deprecated = true ];
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/dec0ec22/hbase-server/src/main/java/org/apache/hadoop/hbase/TableDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/TableDescriptor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/TableDescriptor.java
index d1935db..642d108 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/TableDescriptor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/TableDescriptor.java
@@ -17,14 +17,12 @@
  */
 package org.apache.hadoop.hbase;
 
-import javax.annotation.Nullable;
 import java.io.IOException;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.protobuf.InvalidProtocolBufferException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
@@ -36,26 +34,6 @@ import org.apache.hadoop.hbase.regionserver.BloomType;
 @InterfaceAudience.Private
 public class TableDescriptor {
   private HTableDescriptor hTableDescriptor;
-  /**
-   * Don't use, state was moved to meta, use MetaTableAccessor instead
-   * @deprecated state was moved to meta
-   */
-  @Deprecated
-  @Nullable
-  private TableState.State tableState;
-
-  /**
-   * Creates TableDescriptor with all fields.
-   * @param hTableDescriptor HTableDescriptor to use
-   * @param tableState table state
-   * @deprecated state was moved to meta
-   */
-  @Deprecated
-  public TableDescriptor(HTableDescriptor hTableDescriptor,
-      @Nullable TableState.State tableState) {
-    this.hTableDescriptor = hTableDescriptor;
-    this.tableState = tableState;
-  }
 
   /**
    * Creates TableDescriptor with Enabled table.
@@ -63,7 +41,7 @@ public class TableDescriptor {
    */
   @VisibleForTesting
   public TableDescriptor(HTableDescriptor hTableDescriptor) {
-    this(hTableDescriptor, TableState.State.ENABLED);
+    this.hTableDescriptor = hTableDescriptor;
   }
 
   /**
@@ -79,33 +57,12 @@ public class TableDescriptor {
   }
 
   /**
-   * @return table state
-   * @deprecated state was moved to meta
-   */
-  @Deprecated
-  @Nullable
-  public TableState.State getTableState() {
-    return tableState;
-  }
-
-  /**
-   * @param tableState state to set for table
-   * @deprecated state was moved to meta
-   */
-  @Deprecated
-  public void setTableState(@Nullable TableState.State tableState) {
-    this.tableState = tableState;
-  }
-
-  /**
    * Convert to PB.
    */
   @SuppressWarnings("deprecation")
   public HBaseProtos.TableDescriptor convert() {
     HBaseProtos.TableDescriptor.Builder builder = HBaseProtos.TableDescriptor.newBuilder()
         .setSchema(hTableDescriptor.convert());
-    if (tableState!= null)
-      builder.setState(tableState.convert());
     return builder.build();
   }
 
@@ -113,11 +70,7 @@ public class TableDescriptor {
    * Convert from PB
    */
   public static TableDescriptor convert(HBaseProtos.TableDescriptor proto) {
-    HTableDescriptor hTableDescriptor = HTableDescriptor.convert(proto.getSchema());
-    TableState.State state = proto.hasState()?
-        TableState.State.convert(proto.getState())
-        :null;
-    return new TableDescriptor(hTableDescriptor, state);
+    return new TableDescriptor(HTableDescriptor.convert(proto.getSchema()));
   }
 
   /**
@@ -158,23 +111,18 @@ public class TableDescriptor {
     if (hTableDescriptor != null ?
         !hTableDescriptor.equals(that.hTableDescriptor) :
         that.hTableDescriptor != null) return false;
-    if (tableState != that.tableState) return false;
-
     return true;
   }
 
   @Override
   public int hashCode() {
-    int result = hTableDescriptor != null ? hTableDescriptor.hashCode() : 0;
-    result = 31 * result + (tableState != null ? tableState.hashCode() : 0);
-    return result;
+    return hTableDescriptor != null ? hTableDescriptor.hashCode() : 0;
   }
 
   @Override
   public String toString() {
     return "TableDescriptor{" +
         "hTableDescriptor=" + hTableDescriptor +
-        ", tableState=" + tableState +
         '}';
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/dec0ec22/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
index 5d1e638..5daa823 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
@@ -214,20 +214,9 @@ public class TableStateManager {
       if (table.equals(TableName.META_TABLE_NAME.getNameAsString()))
         continue;
       if (!states.containsKey(table)) {
-        LOG.warn("Found table without state " + table);
-        TableDescriptor td = entry.getValue();
-        TableState.State tds = td.getTableState();
-        if (tds != null) {
-          LOG.warn("Found table with state in descriptor, using that state");
-          MetaTableAccessor.updateTableState(connection, TableName.valueOf(table), tds);
-          LOG.warn("Updating table descriptor");
-          td.setTableState(null);
-          tableDescriptors.add(td);
-        } else {
-          LOG.warn("Found table with no state in descriptor, assuming ENABLED");
-          MetaTableAccessor.updateTableState(connection, TableName.valueOf(table),
-              TableState.State.ENABLED);
-        }
+        LOG.warn("Found table with no state, assuming ENABLED");
+        MetaTableAccessor.updateTableState(connection, TableName.valueOf(table),
+            TableState.State.ENABLED);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/dec0ec22/hbase-server/src/test/java/org/apache/hadoop/hbase/TestTableDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestTableDescriptor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestTableDescriptor.java
index a179c47..89029b9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestTableDescriptor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestTableDescriptor.java
@@ -47,11 +47,10 @@ public class TestTableDescriptor {
     htd.setDurability(Durability.ASYNC_WAL);
     htd.setReadOnly(true);
     htd.setRegionReplication(2);
-    TableDescriptor td = new TableDescriptor(htd, TableState.State.ENABLED);
+    TableDescriptor td = new TableDescriptor(htd);
     byte[] bytes = td.toByteArray();
     TableDescriptor deserializedTd = TableDescriptor.parseFrom(bytes);
     assertEquals(td, deserializedTd);
     assertEquals(td.getHTableDescriptor(), deserializedTd.getHTableDescriptor());
-    assertEquals(td.getTableState(), deserializedTd.getTableState());
   }
 }


[03/50] [abbrv] hbase git commit: HBASE-12957 Revert accidental checkin of unrelated test

Posted by jm...@apache.org.
HBASE-12957  Revert accidental checkin of unrelated test


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

Branch: refs/heads/hbase-11339
Commit: fd0bb89fdf67d996e8cc678d81c6acb799c2cc49
Parents: 4388fed
Author: tedyu <yu...@gmail.com>
Authored: Tue Feb 3 21:20:42 2015 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Tue Feb 3 21:20:42 2015 -0800

----------------------------------------------------------------------
 .../regionserver/TestRegionServerHostname.java  | 98 --------------------
 1 file changed, 98 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/fd0bb89f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerHostname.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerHostname.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerHostname.java
deleted file mode 100644
index 523911b..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerHostname.java
+++ /dev/null
@@ -1,98 +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.regionserver;
-
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.NetworkInterface;
-import java.util.Enumeration;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.testclassification.RegionServerTests;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-/**
- * Tests for the hostname specification by region server
- */
-@Category({RegionServerTests.class, MediumTests.class})
-public class TestRegionServerHostname {
-  private static final Log LOG = LogFactory.getLog(TestRegionServerHostname.class);
-  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-
-  @Test (timeout=30000)
-  public void testInvalidRegionServerHostname() throws Exception {
-    final int NUM_MASTERS = 1;
-    final int NUM_RS = 1;
-    String invalidHostname = "hostAddr";
-    TEST_UTIL.getConfiguration().set(HRegionServer.HOSTNAME_KEY, invalidHostname);
-    try {
-      TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS);
-    } catch (IOException ioe) {
-      Throwable t1 = ioe.getCause();
-      Throwable t2 = t1.getCause();
-      assertTrue(t2.getMessage().contains("Failed resolve of " + invalidHostname));
-      return;
-    } finally {
-      TEST_UTIL.shutdownMiniCluster();
-    }
-    assertTrue("Failed to validate against invalid hostname", false);
-  }
-
-  @Test(timeout=120000)
-  public void testRegionServerHostname() throws Exception {
-    final int NUM_MASTERS = 1;
-    final int NUM_RS = 1;
-    Enumeration<NetworkInterface> netInterfaceList = NetworkInterface.getNetworkInterfaces();
-
-    while (netInterfaceList.hasMoreElements()) {
-      NetworkInterface ni = netInterfaceList.nextElement();
-      Enumeration<InetAddress> addrList = ni.getInetAddresses();
-      // iterate through host addresses and use each as hostname
-      while (addrList.hasMoreElements()) {
-        InetAddress addr = addrList.nextElement();
-        if (addr.isLoopbackAddress() || addr.isLinkLocalAddress()) continue;
-        if (addr.isMulticastAddress()) continue;
-        String hostAddr = addr.getHostAddress();
-        LOG.info("Found " + hostAddr + " on " + ni);
-        
-        TEST_UTIL.getConfiguration().set(HRegionServer.HOSTNAME_KEY, hostAddr);
-        TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS);
-        try {
-          ZooKeeperWatcher zkw = TEST_UTIL.getZooKeeperWatcher();
-          List<String> servers = ZKUtil.listChildrenNoWatch(zkw, zkw.rsZNode);
-          assertTrue(servers.size() > 0);
-          for (String server : servers) {
-            assertTrue(server.startsWith(hostAddr+","));
-          }
-          zkw.close();
-        } finally {
-          TEST_UTIL.shutdownMiniCluster();
-        }
-      }
-    }
-  }
-}


[36/50] [abbrv] hbase git commit: HBASE-13003 - Get tests in TestHFileBlockIndex back (Zhangduo)

Posted by jm...@apache.org.
HBASE-13003 - Get tests in TestHFileBlockIndex back (Zhangduo)


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

Branch: refs/heads/hbase-11339
Commit: f04850810e9b5ec97368435a97b947fc0023447d
Parents: 8a6e982
Author: Ramkrishna <ra...@intel.com>
Authored: Wed Feb 11 10:38:54 2015 +0530
Committer: Ramkrishna <ra...@intel.com>
Committed: Wed Feb 11 10:38:54 2015 +0530

----------------------------------------------------------------------
 .../hbase/io/hfile/TestHFileBlockIndex.java     | 45 ++++++++++++--------
 1 file changed, 27 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/f0485081/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java
index 4e79381..939c019 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java
@@ -42,9 +42,9 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.fs.HFileSystem;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
@@ -54,6 +54,7 @@ import org.apache.hadoop.hbase.testclassification.IOTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ClassSize;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -138,7 +139,7 @@ public class TestHFileBlockIndex {
     fs = HFileSystem.get(conf);
   }
 
-  protected void testBlockIndexInternals(boolean useTags) throws IOException {
+  private void testBlockIndexInternals(boolean useTags) throws IOException {
     path = new Path(TEST_UTIL.getDataTestDir(), "block_index_" + compr + useTags);
     writeWholeIndex(useTags);
     readIndex(useTags);
@@ -186,7 +187,7 @@ public class TestHFileBlockIndex {
     }
   }
 
-  public void readIndex(boolean useTags) throws IOException {
+  private void readIndex(boolean useTags) throws IOException {
     long fileSize = fs.getFileStatus(path).getLen();
     LOG.info("Size of " + path + ": " + fileSize);
 
@@ -216,10 +217,10 @@ public class TestHFileBlockIndex {
     for (byte[] key : keys) {
       assertTrue(key != null);
       assertTrue(indexReader != null);
-      HFileBlock b = indexReader.seekToDataBlock(new KeyValue.KeyOnlyKeyValue(key, 0, key.length),
-          null,
-          true, true, false, null);
-      if (Bytes.BYTES_RAWCOMPARATOR.compare(key, firstKeyInFile) < 0) {
+      HFileBlock b =
+          indexReader.seekToDataBlock(new KeyValue.KeyOnlyKeyValue(key, 0, key.length), null, true,
+            true, false, null);
+      if (KeyValue.COMPARATOR.compareFlatKey(key, firstKeyInFile) < 0) {
         assertTrue(b == null);
         ++i;
         continue;
@@ -260,21 +261,26 @@ public class TestHFileBlockIndex {
         new HFileBlockIndex.BlockIndexWriter(hbw, null, null);
 
     for (int i = 0; i < NUM_DATA_BLOCKS; ++i) {
-      hbw.startWriting(BlockType.DATA).write(
-          String.valueOf(rand.nextInt(1000)).getBytes());
+      hbw.startWriting(BlockType.DATA).write(String.valueOf(rand.nextInt(1000)).getBytes());
       long blockOffset = outputStream.getPos();
       hbw.writeHeaderAndData(outputStream);
 
       byte[] firstKey = null;
+      byte[] family = Bytes.toBytes("f");
+      byte[] qualifier = Bytes.toBytes("q");
       for (int j = 0; j < 16; ++j) {
-        byte[] k = TestHFileWriterV2.randomOrderedKey(rand, i * 16 + j);
+        byte[] k =
+            new KeyValue(TestHFileWriterV2.randomOrderedKey(rand, i * 16 + j), family, qualifier,
+                EnvironmentEdgeManager.currentTime(), KeyValue.Type.Put).getKey();
         keys.add(k);
-        if (j == 8)
+        if (j == 8) {
           firstKey = k;
+        }
       }
       assertTrue(firstKey != null);
-      if (firstKeyInFile == null)
+      if (firstKeyInFile == null) {
         firstKeyInFile = firstKey;
+      }
       biw.addEntry(firstKey, blockOffset, hbw.getOnDiskSizeWithHeader());
 
       writeInlineBlocks(hbw, outputStream, biw, false);
@@ -439,7 +445,7 @@ public class TestHFileBlockIndex {
 
   }
 
-  //@Test
+  @Test
   public void testBlockIndexChunk() throws IOException {
     BlockIndexChunk c = new BlockIndexChunk();
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
@@ -476,7 +482,7 @@ public class TestHFileBlockIndex {
   }
 
   /** Checks if the HeapSize calculator is within reason */
-  //@Test
+  @Test
   public void testHeapSizeForBlockIndex() throws IOException {
     Class<HFileBlockIndex.BlockIndexReader> cl =
         HFileBlockIndex.BlockIndexReader.class;
@@ -504,7 +510,7 @@ public class TestHFileBlockIndex {
    *
    * @throws IOException
    */
-  //@Test
+  @Test
   public void testHFileWriterAndReader() throws IOException {
     Path hfilePath = new Path(TEST_UTIL.getDataTestDir(),
         "hfile_for_block_index");
@@ -536,16 +542,19 @@ public class TestHFileBlockIndex {
                 .withFileContext(meta)
                 .create();
         Random rand = new Random(19231737);
-
+        byte[] family = Bytes.toBytes("f");
+        byte[] qualifier = Bytes.toBytes("q");
         for (int i = 0; i < NUM_KV; ++i) {
           byte[] row = TestHFileWriterV2.randomOrderedKey(rand, i);
 
           // Key will be interpreted by KeyValue.KEY_COMPARATOR
-          KeyValue kv = KeyValueUtil.createFirstOnRow(row, 0, row.length, row, 0, 0,
-              row, 0, 0);
+          KeyValue kv =
+              new KeyValue(row, family, qualifier, EnvironmentEdgeManager.currentTime(),
+                  TestHFileWriterV2.randomValue(rand));
           byte[] k = kv.getKey();
           writer.append(kv);
           keys[i] = k;
+          values[i] = CellUtil.cloneValue(kv);
           keyStrSet.add(Bytes.toStringBinary(k));
           if (i > 0) {
             assertTrue(KeyValue.COMPARATOR.compareFlatKey(keys[i - 1],


[13/50] [abbrv] hbase git commit: HBASE-12966 NPE in HMaster while recovering tables in Enabling state; ADDENDUM

Posted by jm...@apache.org.
HBASE-12966 NPE in HMaster while recovering tables in Enabling state; ADDENDUM


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

Branch: refs/heads/hbase-11339
Commit: 2f2faa68d1b158f3d44b1ea4f2ef832705933f3f
Parents: eddd573
Author: stack <st...@apache.org>
Authored: Fri Feb 6 16:01:43 2015 -0800
Committer: stack <st...@apache.org>
Committed: Fri Feb 6 16:01:43 2015 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/master/handler/TestEnableTableHandler.java  | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2f2faa68/hbase-server/src/test/java/org/apache/hadoop/hbase/master/handler/TestEnableTableHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/handler/TestEnableTableHandler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/handler/TestEnableTableHandler.java
index 89fd9ba..d3d6239 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/handler/TestEnableTableHandler.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/handler/TestEnableTableHandler.java
@@ -85,6 +85,7 @@ public class TestEnableTableHandler {
     // disable once more
     admin.disableTable(tableName);
 
+    TEST_UTIL.waitUntilNoRegionsInTransition(60000);
     // now stop region servers
     JVMClusterUtil.RegionServerThread rs = cluster.getRegionServerThreads().get(0);
     rs.getRegionServer().stop("stop");


[49/50] [abbrv] hbase git commit: Merge branch 'master' (2/11/15) into hbase-11339

Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
----------------------------------------------------------------------
diff --cc hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
index 40c5651,5335bef..13f8163
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
@@@ -262,12 -252,10 +258,12 @@@ public class HColumnDescriptor implemen
        DEFAULT_VALUES.put(EVICT_BLOCKS_ON_CLOSE, String.valueOf(DEFAULT_EVICT_BLOCKS_ON_CLOSE));
        DEFAULT_VALUES.put(PREFETCH_BLOCKS_ON_OPEN, String.valueOf(DEFAULT_PREFETCH_BLOCKS_ON_OPEN));
        for (String s : DEFAULT_VALUES.keySet()) {
-         RESERVED_KEYWORDS.add(new ImmutableBytesWritable(Bytes.toBytes(s)));
+         RESERVED_KEYWORDS.add(new Bytes(Bytes.toBytes(s)));
        }
-       RESERVED_KEYWORDS.add(new ImmutableBytesWritable(Bytes.toBytes(ENCRYPTION)));
-       RESERVED_KEYWORDS.add(new ImmutableBytesWritable(Bytes.toBytes(ENCRYPTION_KEY)));
-       RESERVED_KEYWORDS.add(new ImmutableBytesWritable(IS_MOB_BYTES));
-       RESERVED_KEYWORDS.add(new ImmutableBytesWritable(MOB_THRESHOLD_BYTES));
 -    RESERVED_KEYWORDS.add(new Bytes(Bytes.toBytes(ENCRYPTION)));
 -    RESERVED_KEYWORDS.add(new Bytes(Bytes.toBytes(ENCRYPTION_KEY)));
++      RESERVED_KEYWORDS.add(new Bytes(Bytes.toBytes(ENCRYPTION)));
++      RESERVED_KEYWORDS.add(new Bytes(Bytes.toBytes(ENCRYPTION_KEY)));
++      RESERVED_KEYWORDS.add(new Bytes(IS_MOB_BYTES));
++      RESERVED_KEYWORDS.add(new Bytes(MOB_THRESHOLD_BYTES));
    }
  
    private static final int UNINITIALIZED = -1;

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-client/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptor.java
----------------------------------------------------------------------
diff --cc hbase-client/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptor.java
index 0000000,976876cf..1180954
mode 000000,100644..100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptor.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptor.java
@@@ -1,0 -1,120 +1,138 @@@
+ /**
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  *     http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ package org.apache.hadoop.hbase;
+ 
+ import static org.junit.Assert.assertEquals;
+ import static org.junit.Assert.assertTrue;
+ 
+ import org.apache.hadoop.hbase.exceptions.DeserializationException;
+ import org.apache.hadoop.hbase.io.compress.Compression;
+ import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
+ import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
+ import org.apache.hadoop.hbase.regionserver.BloomType;
++import org.apache.hadoop.hbase.util.Bytes;
++import org.apache.hadoop.hbase.util.PrettyPrinter;
+ import org.apache.hadoop.hbase.testclassification.MiscTests;
+ import org.apache.hadoop.hbase.testclassification.SmallTests;
+ import org.apache.hadoop.hbase.util.BuilderStyleTest;
+ import org.junit.Test;
+ import org.junit.experimental.categories.Category;
+ 
+ /** Tests the HColumnDescriptor with appropriate arguments */
+ @Category({MiscTests.class, SmallTests.class})
+ public class TestHColumnDescriptor {
+   @Test
+   public void testPb() throws DeserializationException {
+     HColumnDescriptor hcd = new HColumnDescriptor(
+         new HColumnDescriptor(HConstants.CATALOG_FAMILY)
+             .setInMemory(true)
+             .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+             .setBloomFilterType(BloomType.NONE)
+             .setCacheDataInL1(true));
+     final int v = 123;
+     hcd.setBlocksize(v);
+     hcd.setTimeToLive(v);
+     hcd.setBlockCacheEnabled(!HColumnDescriptor.DEFAULT_BLOCKCACHE);
+     hcd.setValue("a", "b");
+     hcd.setMaxVersions(v);
+     assertEquals(v, hcd.getMaxVersions());
+     hcd.setMinVersions(v);
+     assertEquals(v, hcd.getMinVersions());
+     hcd.setKeepDeletedCells(KeepDeletedCells.TRUE);
+     hcd.setInMemory(!HColumnDescriptor.DEFAULT_IN_MEMORY);
+     boolean inmemory = hcd.isInMemory();
+     hcd.setScope(v);
+     hcd.setDataBlockEncoding(DataBlockEncoding.FAST_DIFF);
+     hcd.setBloomFilterType(BloomType.ROW);
+     hcd.setCompressionType(Algorithm.SNAPPY);
++    hcd.setMobEnabled(true);
++    hcd.setMobThreshold(1000L);
+ 
+ 
+     byte [] bytes = hcd.toByteArray();
+     HColumnDescriptor deserializedHcd = HColumnDescriptor.parseFrom(bytes);
+     assertTrue(hcd.equals(deserializedHcd));
+     assertEquals(v, hcd.getBlocksize());
+     assertEquals(v, hcd.getTimeToLive());
+     assertEquals(hcd.getValue("a"), deserializedHcd.getValue("a"));
+     assertEquals(hcd.getMaxVersions(), deserializedHcd.getMaxVersions());
+     assertEquals(hcd.getMinVersions(), deserializedHcd.getMinVersions());
+     assertEquals(hcd.getKeepDeletedCells(), deserializedHcd.getKeepDeletedCells());
+     assertEquals(inmemory, deserializedHcd.isInMemory());
+     assertEquals(hcd.getScope(), deserializedHcd.getScope());
+     assertTrue(deserializedHcd.getCompressionType().equals(Compression.Algorithm.SNAPPY));
+     assertTrue(deserializedHcd.getDataBlockEncoding().equals(DataBlockEncoding.FAST_DIFF));
+     assertTrue(deserializedHcd.getBloomFilterType().equals(BloomType.ROW));
++    assertEquals(hcd.isMobEnabled(), deserializedHcd.isMobEnabled());
++    assertEquals(hcd.getMobThreshold(), deserializedHcd.getMobThreshold());
+   }
+ 
+   @Test
+   /** Tests HColumnDescriptor with empty familyName*/
+   public void testHColumnDescriptorShouldThrowIAEWhenFamiliyNameEmpty()
+       throws Exception {
+     try {
+       new HColumnDescriptor("".getBytes());
+     } catch (IllegalArgumentException e) {
+       assertEquals("Family name can not be empty", e.getLocalizedMessage());
+     }
+   }
+ 
+   /**
+    * Test that we add and remove strings from configuration properly.
+    */
+   @Test
+   public void testAddGetRemoveConfiguration() throws Exception {
+     HColumnDescriptor desc = new HColumnDescriptor("foo");
+     String key = "Some";
+     String value = "value";
+     desc.setConfiguration(key, value);
+     assertEquals(value, desc.getConfigurationValue(key));
+     desc.removeConfiguration(key);
+     assertEquals(null, desc.getConfigurationValue(key));
+   }
+ 
+   @Test
++  public void testMobValuesInHColumnDescriptorShouldReadable() {
++    boolean isMob = true;
++    long threshold = 1000;
++    String isMobString = PrettyPrinter.format(Bytes.toStringBinary(Bytes.toBytes(isMob)),
++            HColumnDescriptor.getUnit(HColumnDescriptor.IS_MOB));
++    String thresholdString = PrettyPrinter.format(Bytes.toStringBinary(Bytes.toBytes(threshold)),
++            HColumnDescriptor.getUnit(HColumnDescriptor.MOB_THRESHOLD));
++    assertEquals(String.valueOf(isMob), isMobString);
++    assertEquals(String.valueOf(threshold), thresholdString);
++  }
++
++  @Test
+   public void testClassMethodsAreBuilderStyle() {
+     /* HColumnDescriptor should have a builder style setup where setXXX/addXXX methods
+      * can be chainable together:
+      * . For example:
+      * HColumnDescriptor hcd
+      *   = new HColumnDescriptor()
+      *     .setFoo(foo)
+      *     .setBar(bar)
+      *     .setBuz(buz)
+      *
+      * This test ensures that all methods starting with "set" returns the declaring object
+      */
+ 
+     BuilderStyleTest.assertClassesAreBuilderStyle(HColumnDescriptor.class);
+   }
+ }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-common/src/main/java/org/apache/hadoop/hbase/TagType.java
----------------------------------------------------------------------
diff --cc hbase-common/src/main/java/org/apache/hadoop/hbase/TagType.java
index 8613a35,2095b7a..7e200df
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/TagType.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/TagType.java
@@@ -26,10 -26,9 +26,14 @@@ public final class TagType 
    // Please declare new Tag Types here to avoid step on pre-existing tag types.
    public static final byte ACL_TAG_TYPE = (byte) 1;
    public static final byte VISIBILITY_TAG_TYPE = (byte) 2;
-   public static final byte LOG_REPLAY_TAG_TYPE = (byte) 3;
+   // public static final byte LOG_REPLAY_TAG_TYPE = (byte) 3; // deprecated
    public static final byte VISIBILITY_EXP_SERIALIZATION_FORMAT_TAG_TYPE = (byte)4;
 +
 +  // mob tags
 +  public static final byte MOB_REFERENCE_TAG_TYPE = (byte) 5;
 +  public static final byte MOB_TABLE_NAME_TAG_TYPE = (byte) 6;
++
+   // String based tag type used in replication
+   public static final byte STRING_VIS_TAG_TYPE = (byte) 7;
+   public static final byte TTL_TAG_TYPE = (byte)8;
  }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PrettyPrinter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-common/src/main/resources/hbase-default.xml
----------------------------------------------------------------------
diff --cc hbase-common/src/main/resources/hbase-default.xml
index d1429ad,2985685..4ba6d69
--- a/hbase-common/src/main/resources/hbase-default.xml
+++ b/hbase-common/src/main/resources/hbase-default.xml
@@@ -1454,118 -1462,12 +1462,126 @@@ possible configurations would overwhel
      <name>hbase.http.staticuser.user</name>
      <value>dr.stack</value>
    </property>
 +  <!-- Mob properties. -->
 +  <property>
 +    <name>hbase.mob.file.cache.size</name>
 +    <value>1000</value>
 +    <description>
 +      Number of opened file handlers to cache.
 +      A larger value will benefit reads by providing more file handlers per mob
 +      file cache and would reduce frequent file opening and closing.
 +      However, if this is set too high, this could lead to a "too many opened file handlers"
 +      The default value is 1000.
 +    </description>
 +  </property>
 +  <property>
 +    <name>hbase.mob.cache.evict.period</name>
 +    <value>3600</value>
 +    <description>
 +      The amount of time in seconds before the mob cache evicts cached mob files.
 +      The default value is 3600 seconds.
 +    </description>
 +  </property>
 +  <property>
 +    <name>hbase.mob.cache.evict.remain.ratio</name>
 +    <value>0.5f</value>
 +    <description>
 +      The ratio (between 0.0 and 1.0) of files that remains cached after an eviction
 +      is triggered when the number of cached mob files exceeds the hbase.mob.file.cache.size.
 +      The default value is 0.5f.
 +    </description>
 +  </property>
 +  <property>
 +    <name>hbase.mob.sweep.tool.compaction.ratio</name>
 +    <value>0.5f</value>
 +    <description>
 +      If there're too many cells deleted in a mob file, it's regarded
 +      as an invalid file and needs to be merged.
 +      If existingCellsSize/mobFileSize is less than ratio, it's regarded
 +      as an invalid file. The default value is 0.5f.
 +    </description>
 +  </property>
 +  <property>
 +    <name>hbase.mob.sweep.tool.compaction.mergeable.size</name>
 +    <value>134217728</value>
 +    <description>
 +      If the size of a mob file is less than this value, it's regarded as a small
 +      file and needs to be merged. The default value is 128MB.
 +    </description>
 +  </property>
 +  <property>
 +    <name>hbase.mob.sweep.tool.compaction.memstore.flush.size</name>
 +    <value>134217728</value>
 +    <description>
 +      The flush size for the memstore used by sweep job. Each sweep reducer owns such a memstore.
 +      The default value is 128MB.
 +    </description>
 +  </property>
 +  <property>
 +    <name>hbase.master.mob.ttl.cleaner.period</name>
 +    <value>86400000</value>
 +    <description>
 +      The period that ExpiredMobFileCleanerChore runs. The unit is millisecond.
 +      The default value is one day.
 +    </description>
 +  </property>
 +  <property>
 +    <name>hbase.mob.file.compaction.mergeable.threshold</name>
 +    <value>201326592</value>
 +    <description>
 +      If the size of a mob file is less than this value, it's regarded as a small
 +      file and needs to be merged in mob file compaction. The default value is 192MB.
 +    </description>
 +  </property>
 +  <property>
 +    <name>hbase.mob.delfile.max.count</name>
 +    <value>3</value>
 +    <description>
 +      The max number of del files that is allowed in the mob file compaction.
 +      In the mob file compaction, when the number of existing del files is larger than
 +      this value, they are merged until number of del files is not larger this value.
 +      The default value is 3.
 +    </description>
 +  </property>
 +  <property>
 +    <name>hbase.mob.file.compaction.batch.size</name>
 +    <value>100</value>
 +    <description>
 +      The max number of the mob files that is allowed in a batch of the mob file compaction.
 +      The mob file compaction merges the small mob files to bigger ones. If the number of the
 +      small files is very large, it could lead to a "too many opened file handlers" in the merge.
 +      And the merge has to be split into batches. This value limits the number of mob files
 +      that are selected in a batch of the mob file compaction. The default value is 100.
 +    </description>
 +  </property>
 +  <property>
 +    <name>hbase.master.mob.file.compaction.chore.period</name>
 +    <value>604800000</value>
 +    <description>
 +      The period that MobFileCompactionChore runs. The unit is millisecond.
 +      The default value is one week.
 +    </description>
 +  </property>
 +  <property>
 +    <name>hbase.mob.file.compactor.class</name>
 +    <value>org.apache.hadoop.hbase.mob.filecompactions.PartitionedMobFileCompactor</value>
 +    <description>
 +      Implementation of mob file compactor, the default one is PartitionedMobFileCompactor.
 +    </description>
 +  </property>
 +  <property>
 +    <name>hbase.master.mob.file.compaction.chore.threads.max</name>
 +    <value>1</value>
 +    <description>
 +      The max number of threads used in MobFileCompactionChore.
 +    </description>
 +  </property>
+   <property>
+     <name>hbase.regionserver.handler.abort.on.error.percent</name>
+     <value>0.5</value>
+     <description>The percent of region server RPC threads failed to abort RS.
+     -1 Disable aborting; 0 Abort if even a single handler has died;
+     0.x Abort only when this percent of handlers have died;
+     1 Abort only all of the handers have died.</description>
+   </property>
  </configuration>

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java
----------------------------------------------------------------------
diff --cc hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java
index ee2dcbb,b86a8eb..90a9a09
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java
@@@ -233,37 -257,21 +257,53 @@@ public interface MetricsRegionServerSou
    String MAJOR_COMPACTED_CELLS_SIZE = "majorCompactedCellsSize";
    String MAJOR_COMPACTED_CELLS_SIZE_DESC =
        "The total amount of data processed during major compactions, in bytes";
 +  String MOB_COMPACTED_INTO_MOB_CELLS_COUNT = "mobCompactedIntoMobCellsCount";
 +  String MOB_COMPACTED_INTO_MOB_CELLS_COUNT_DESC =
 +      "The number of cells moved to mob during compaction";
 +  String MOB_COMPACTED_FROM_MOB_CELLS_COUNT = "mobCompactedFromMobCellsCount";
 +  String MOB_COMPACTED_FROM_MOB_CELLS_COUNT_DESC =
 +      "The number of cells moved from mob during compaction";
 +  String MOB_COMPACTED_INTO_MOB_CELLS_SIZE = "mobCompactedIntoMobCellsSize";
 +  String MOB_COMPACTED_INTO_MOB_CELLS_SIZE_DESC =
 +      "The total amount of cells move to mob during compaction, in bytes";
 +  String MOB_COMPACTED_FROM_MOB_CELLS_SIZE = "mobCompactedFromMobCellsSize";
 +  String MOB_COMPACTED_FROM_MOB_CELLS_SIZE_DESC =
 +      "The total amount of cells move from mob during compaction, in bytes";
 +  String MOB_FLUSH_COUNT = "mobFlushCount";
 +  String MOB_FLUSH_COUNT_DESC = "The number of the flushes in mob-enabled stores";
 +  String MOB_FLUSHED_CELLS_COUNT = "mobFlushedCellsCount";
 +  String MOB_FLUSHED_CELLS_COUNT_DESC = "The number of mob cells flushed to disk";
 +  String MOB_FLUSHED_CELLS_SIZE = "mobFlushedCellsSize";
 +  String MOB_FLUSHED_CELLS_SIZE_DESC = "The total amount of mob cells flushed to disk, in bytes";
 +  String MOB_SCAN_CELLS_COUNT = "mobScanCellsCount";
 +  String MOB_SCAN_CELLS_COUNT_DESC = "The number of scanned mob cells";
 +  String MOB_SCAN_CELLS_SIZE = "mobScanCellsSize";
 +  String MOB_SCAN_CELLS_SIZE_DESC = "The total amount of scanned mob cells, in bytes";
 +  String MOB_FILE_CACHE_ACCESS_COUNT = "mobFileCacheAccessCount";
 +  String MOB_FILE_CACHE_ACCESS_COUNT_DESC = "The count of accesses to the mob file cache";
 +  String MOB_FILE_CACHE_MISS_COUNT = "mobFileCacheMissCount";
 +  String MOB_FILE_CACHE_MISS_COUNT_DESC = "The count of misses to the mob file cache";
 +  String MOB_FILE_CACHE_HIT_PERCENT = "mobFileCacheHitPercent";
 +  String MOB_FILE_CACHE_HIT_PERCENT_DESC = "The hit percent to the mob file cache";
 +  String MOB_FILE_CACHE_EVICTED_COUNT = "mobFileCacheEvictedCount";
 +  String MOB_FILE_CACHE_EVICTED_COUNT_DESC = "The number of items evicted from the mob file cache";
 +  String MOB_FILE_CACHE_COUNT = "mobFileCacheCount";
 +  String MOB_FILE_CACHE_COUNT_DESC = "The count of cached mob files";
  
+   String HEDGED_READS = "hedgedReads";
+   String HEDGED_READS_DESC = "The number of times we started a hedged read";
+   String HEDGED_READ_WINS = "hedgedReadWins";
+   String HEDGED_READ_WINS_DESC =
+       "The number of times we started a hedged read and a hedged read won";
+ 
+   String BLOCKED_REQUESTS_COUNT = "blockedRequestCount";
+   String BLOCKED_REQUESTS_COUNT_DESC = "The number of blocked requests because of memstore size is "
+       + "larger than blockingMemStoreSize";
+ 
+   String SPLIT_KEY = "splitTime";
+   String SPLIT_REQUEST_KEY = "splitRequestCount";
+   String SPLIT_REQUEST_DESC = "Number of splits requested";
+   String SPLIT_SUCCESS_KEY = "splitSuccessCounnt";
+   String SPLIT_SUCCESS_DESC = "Number of successfully executed splits";
+   String FLUSH_KEY = "flushTime";
  }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java
----------------------------------------------------------------------
diff --cc hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java
index ea55fe8,9634be7..2aad115
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java
@@@ -248,72 -253,17 +253,88 @@@ public interface MetricsRegionServerWra
    long getMajorCompactedCellsSize();
  
    /**
++<<<<<<< HEAD
 +   * Gets the number of cells move to mob during compaction.
 +   */
 +  long getMobCompactedIntoMobCellsCount();
 +
 +  /**
 +   * Gets the number of cells move from mob during compaction.
 +   */
 +  long getMobCompactedFromMobCellsCount();
 +
 +  /**
 +   * Gets the total amount of cells move to mob during compaction, in bytes.
 +   */
 +  long getMobCompactedIntoMobCellsSize();
 +
 +  /**
 +   * Gets the total amount of cells move from mob during compaction, in bytes.
 +   */
 +  long getMobCompactedFromMobCellsSize();
 +
 +  /**
 +   * Gets the number of the flushes in mob-enabled stores.
 +   */
 +  long getMobFlushCount();
 +
 +  /**
 +   * Gets the number of mob cells flushed to disk.
 +   */
 +  long getMobFlushedCellsCount();
 +
 +  /**
 +   * Gets the total amount of mob cells flushed to disk, in bytes.
 +   */
 +  long getMobFlushedCellsSize();
 +
 +  /**
 +   * Gets the number of scanned mob cells.
 +   */
 +  long getMobScanCellsCount();
 +
 +  /**
 +   * Gets the total amount of scanned mob cells, in bytes.
 +   */
 +  long getMobScanCellsSize();
 +
 +  /**
 +   * Gets the count of accesses to the mob file cache.
 +   */
 +  long getMobFileCacheAccessCount();
 +
 +  /**
 +   * Gets the count of misses to the mob file cache.
 +   */
 +  long getMobFileCacheMissCount();
 +
 +  /**
 +   * Gets the number of items evicted from the mob file cache.
 +   */
 +  long getMobFileCacheEvictedCount();
 +
 +  /**
 +   * Gets the count of cached mob files.
 +   */
 +  long getMobFileCacheCount();
 +
 +  /**
 +   * Gets the hit percent to the mob file cache.
 +   */
 +  int getMobFileCacheHitPercent();
++
++  /**
+    * @return Count of hedged read operations
+    */
+   public long getHedgedReadOps();
+ 
+   /**
+    * @return Count of times a hedged read beat out the primary read.
+    */
+   public long getHedgedReadWins();
+ 
+   /**
+    * @return Count of requests blocked because the memstore size is larger than blockingMemStoreSize
+    */
+   public long getBlockedRequestsCount();
  }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
----------------------------------------------------------------------
diff --cc hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
index 27d5437,2c1dcd3..45e2699
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
@@@ -219,35 -249,14 +249,44 @@@ public class MetricsRegionServerSourceI
                rsWrap.getCompactedCellsSize())
            .addCounter(Interns.info(MAJOR_COMPACTED_CELLS_SIZE, MAJOR_COMPACTED_CELLS_SIZE_DESC),
                rsWrap.getMajorCompactedCellsSize())
+ 
 +          .addCounter(Interns.info(MOB_COMPACTED_FROM_MOB_CELLS_COUNT, MOB_COMPACTED_FROM_MOB_CELLS_COUNT_DESC),
 +              rsWrap.getMobCompactedFromMobCellsCount())
 +          .addCounter(Interns.info(MOB_COMPACTED_INTO_MOB_CELLS_COUNT, MOB_COMPACTED_INTO_MOB_CELLS_COUNT_DESC),
 +              rsWrap.getMobCompactedIntoMobCellsCount())
 +          .addCounter(Interns.info(MOB_COMPACTED_FROM_MOB_CELLS_SIZE, MOB_COMPACTED_FROM_MOB_CELLS_SIZE_DESC),
 +              rsWrap.getMobCompactedFromMobCellsSize())
 +          .addCounter(Interns.info(MOB_COMPACTED_INTO_MOB_CELLS_SIZE, MOB_COMPACTED_INTO_MOB_CELLS_SIZE_DESC),
 +              rsWrap.getMobCompactedIntoMobCellsSize())
 +          .addCounter(Interns.info(MOB_FLUSH_COUNT, MOB_FLUSH_COUNT_DESC),
 +              rsWrap.getMobFlushCount())
 +          .addCounter(Interns.info(MOB_FLUSHED_CELLS_COUNT, MOB_FLUSHED_CELLS_COUNT_DESC),
 +              rsWrap.getMobFlushedCellsCount())
 +          .addCounter(Interns.info(MOB_FLUSHED_CELLS_SIZE, MOB_FLUSHED_CELLS_SIZE_DESC),
 +              rsWrap.getMobFlushedCellsSize())
 +          .addCounter(Interns.info(MOB_SCAN_CELLS_COUNT, MOB_SCAN_CELLS_COUNT_DESC),
 +              rsWrap.getMobScanCellsCount())
 +          .addCounter(Interns.info(MOB_SCAN_CELLS_SIZE, MOB_SCAN_CELLS_SIZE_DESC),
 +              rsWrap.getMobScanCellsSize())
 +          .addGauge(Interns.info(MOB_FILE_CACHE_COUNT, MOB_FILE_CACHE_COUNT_DESC),
 +              rsWrap.getMobFileCacheCount())
 +          .addCounter(Interns.info(MOB_FILE_CACHE_ACCESS_COUNT, MOB_FILE_CACHE_ACCESS_COUNT_DESC),
 +              rsWrap.getMobFileCacheAccessCount())
 +          .addCounter(Interns.info(MOB_FILE_CACHE_MISS_COUNT, MOB_FILE_CACHE_MISS_COUNT_DESC),
 +              rsWrap.getMobFileCacheMissCount())
 +          .addCounter(
 +              Interns.info(MOB_FILE_CACHE_EVICTED_COUNT, MOB_FILE_CACHE_EVICTED_COUNT_DESC),
 +              rsWrap.getMobFileCacheEvictedCount())
 +          .addGauge(Interns.info(MOB_FILE_CACHE_HIT_PERCENT, MOB_FILE_CACHE_HIT_PERCENT_DESC),
 +              rsWrap.getMobFileCacheHitPercent())
++
+           .addCounter(Interns.info(HEDGED_READS, HEDGED_READS_DESC), rsWrap.getHedgedReadOps())
+           .addCounter(Interns.info(HEDGED_READ_WINS, HEDGED_READ_WINS_DESC),
+               rsWrap.getHedgedReadWins())
+ 
+           .addCounter(Interns.info(BLOCKED_REQUESTS_COUNT, BLOCKED_REQUESTS_COUNT_DESC),
+             rsWrap.getBlockedRequestsCount())
+ 
            .tag(Interns.info(ZOOKEEPER_QUORUM_NAME, ZOOKEEPER_QUORUM_DESC),
                rsWrap.getZookeeperQuorum())
            .tag(Interns.info(SERVER_NAME_NAME, SERVER_NAME_DESC), rsWrap.getServerName())

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithMOB.java
----------------------------------------------------------------------
diff --cc hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithMOB.java
index f8a4412,0000000..85c01cc
mode 100644,000000..100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithMOB.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithMOB.java
@@@ -1,154 -1,0 +1,155 @@@
 +/**
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.hadoop.hbase;
 +
 +import java.io.IOException;
 +import java.util.ArrayList;
 +import java.util.Arrays;
 +import java.util.List;
 +
 +import org.apache.commons.cli.CommandLine;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.hbase.client.HBaseAdmin;
++import org.apache.hadoop.hbase.testclassification.IntegrationTests;
 +import org.apache.hadoop.hbase.util.Bytes;
 +import org.apache.hadoop.hbase.util.LoadTestDataGeneratorWithMOB;
 +import org.apache.hadoop.hbase.util.LoadTestTool;
 +import org.apache.hadoop.util.ToolRunner;
 +import org.junit.Test;
 +import org.junit.experimental.categories.Category;
 +
 +/**
 + * Integration Test for MOB ingest.
 + */
 +@Category(IntegrationTests.class)
 +public class IntegrationTestIngestWithMOB extends IntegrationTestIngest {
 +  private static final char COLON = ':';
 +
 +  private byte[] mobColumnFamily = LoadTestTool.COLUMN_FAMILY;
 +  public static final String THRESHOLD = "threshold";
 +  public static final String MIN_MOB_DATA_SIZE = "minMobDataSize";
 +  public static final String MAX_MOB_DATA_SIZE = "maxMobDataSize";
 +  private int threshold = 1024; // 1KB
 +  private int minMobDataSize = 512; // 512B
 +  private int maxMobDataSize = threshold * 5; // 5KB
 +  private static final long JUNIT_RUN_TIME = 2 * 60 * 1000; // 2 minutes
 +
 +  //similar to LOAD_TEST_TOOL_INIT_ARGS except OPT_IN_MEMORY is removed
 +  protected String[] LOAD_TEST_TOOL_MOB_INIT_ARGS = {
 +      LoadTestTool.OPT_COMPRESSION,
 +      LoadTestTool.OPT_DATA_BLOCK_ENCODING,
 +      LoadTestTool.OPT_ENCRYPTION,
 +      LoadTestTool.OPT_NUM_REGIONS_PER_SERVER,
 +      LoadTestTool.OPT_REGION_REPLICATION,
 +  };
 +
 +  @Override
 +  protected String[] getArgsForLoadTestToolInitTable() {
 +    List<String> args = new ArrayList<String>();
 +    args.add("-tn");
 +    args.add(getTablename().getNameAsString());
 +    // pass all remaining args from conf with keys <test class name>.<load test tool arg>
 +    String clazz = this.getClass().getSimpleName();
 +    for (String arg : LOAD_TEST_TOOL_MOB_INIT_ARGS) {
 +      String val = conf.get(String.format("%s.%s", clazz, arg));
 +      if (val != null) {
 +        args.add("-" + arg);
 +        args.add(val);
 +      }
 +    }
 +    args.add("-init_only");
 +    return args.toArray(new String[args.size()]);
 +  }
 +
 +  @Override
 +  protected void addOptions() {
 +    super.addOptions();
 +    super.addOptWithArg(THRESHOLD, "The threshold to classify cells to mob data");
 +    super.addOptWithArg(MIN_MOB_DATA_SIZE, "Minimum value size for mob data");
 +    super.addOptWithArg(MAX_MOB_DATA_SIZE, "Maximum value size for mob data");
 +  }
 +
 +  @Override
 +  protected void processOptions(CommandLine cmd) {
 +    super.processOptions(cmd);
 +    if (cmd.hasOption(THRESHOLD)) {
 +      threshold = Integer.parseInt(cmd.getOptionValue(THRESHOLD));
 +    }
 +    if (cmd.hasOption(MIN_MOB_DATA_SIZE)) {
 +      minMobDataSize = Integer.parseInt(cmd.getOptionValue(MIN_MOB_DATA_SIZE));
 +    }
 +    if (cmd.hasOption(MAX_MOB_DATA_SIZE)) {
 +      maxMobDataSize = Integer.parseInt(cmd.getOptionValue(MAX_MOB_DATA_SIZE));
 +    }
 +    if (minMobDataSize > maxMobDataSize) {
 +      throw new IllegalArgumentException(
 +          "The minMobDataSize should not be larger than minMobDataSize");
 +    }
 +  }
 +
 +  @Test
 +  public void testIngest() throws Exception {
 +    runIngestTest(JUNIT_RUN_TIME, 100, 10, 1024, 10, 20);
 +  };
 +
 +  @Override
 +  protected void initTable() throws IOException {
 +    super.initTable();
 +
 +    byte[] tableName = getTablename().getName();
 +    HBaseAdmin admin = new HBaseAdmin(conf);
 +    HTableDescriptor tableDesc = admin.getTableDescriptor(tableName);
 +    LOG.info("Disabling table " + getTablename());
 +    admin.disableTable(tableName);
 +    for (HColumnDescriptor columnDescriptor : tableDesc.getFamilies()) {
 +      if(Arrays.equals(columnDescriptor.getName(), mobColumnFamily)) {
 +        columnDescriptor.setMobEnabled(true);
 +        columnDescriptor.setMobThreshold((long) threshold);
 +        admin.modifyColumn(tableName, columnDescriptor);
 +      }
 +    }
 +    LOG.info("Enabling table " + getTablename());
 +    admin.enableTable(tableName);
 +    admin.close();
 +  }
 +
 +  @Override
 +  protected String[] getArgsForLoadTestTool(String mode, String modeSpecificArg, long startKey,
 +      long numKeys) {
 +    String[] args = super.getArgsForLoadTestTool(mode, modeSpecificArg, startKey, numKeys);
 +    List<String> tmp = new ArrayList<String>(Arrays.asList(args));
 +    // LoadTestDataGeneratorMOB:mobColumnFamily:minMobDataSize:maxMobDataSize
 +    tmp.add(HIPHEN + LoadTestTool.OPT_GENERATOR);
 +    StringBuilder sb = new StringBuilder(LoadTestDataGeneratorWithMOB.class.getName());
 +    sb.append(COLON);
 +    sb.append(Bytes.toString(mobColumnFamily));
 +    sb.append(COLON);
 +    sb.append(minMobDataSize);
 +    sb.append(COLON);
 +    sb.append(maxMobDataSize);
 +    tmp.add(sb.toString());
 +    return tmp.toArray(new String[tmp.size()]);
 +  }
 +
 +  public static void main(String[] args) throws Exception {
 +    Configuration conf = HBaseConfiguration.create();
 +    IntegrationTestingUtility.setUseDistributedCluster(conf);
 +    int ret = ToolRunner.run(conf, new IntegrationTestIngestWithMOB(), args);
 +    System.exit(ret);
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HFileLink.java
----------------------------------------------------------------------
diff --cc hbase-server/src/main/java/org/apache/hadoop/hbase/io/HFileLink.java
index 6e97a76,ff33951..a950dce
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HFileLink.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HFileLink.java
@@@ -94,37 -92,41 +94,45 @@@ public class HFileLink extends FileLin
    private final Path tempPath;
  
    /**
-    * @param conf {@link Configuration} from which to extract specific archive locations
-    * @param path The path of the HFile Link.
-    * @throws IOException on unexpected error.
+    * Dead simple hfile link constructor
     */
-   public HFileLink(Configuration conf, Path path) throws IOException {
-     this(FSUtils.getRootDir(conf), HFileArchiveUtil.getArchivePath(conf), path);
 -  public HFileLink(final Path originPath, final Path tempPath,
++  public HFileLink(final Path originPath, final Path tempPath, final Path mobPath,
+                    final Path archivePath) {
 -    this.tempPath  = tempPath;
++    this.tempPath = tempPath;
+     this.originPath = originPath;
++    this.mobPath = mobPath;
+     this.archivePath = archivePath;
 -
 -    setLocations(originPath, tempPath, archivePath);
++    setLocations(originPath, tempPath, mobPath, archivePath);
    }
  
++
    /**
-    * @param rootDir Path to the root directory where hbase files are stored
-    * @param archiveDir Path to the hbase archive directory
-    * @param mobDir path to the hbase mob directory
-    * @param path The path of the HFile Link.
+    * @param conf {@link Configuration} from which to extract specific archive locations
+    * @param hFileLinkPattern The path ending with a HFileLink pattern. (table=region-hfile)
+    * @throws IOException on unexpected error.
     */
-     public HFileLink(final Path rootDir, final Path archiveDir, final Path mobDir, final Path path) {
-         Path hfilePath = getRelativeTablePath(path);
-         this.tempPath = new Path(new Path(rootDir, HConstants.HBASE_TEMP_DIRECTORY), hfilePath);
-         this.originPath = new Path(rootDir, hfilePath);
-         this.mobPath = new Path(mobDir, hfilePath);
-         this.archivePath = new Path(archiveDir, hfilePath);
-         setLocations(originPath, mobPath, tempPath, archivePath);
-     }
+   public static final HFileLink buildFromHFileLinkPattern(Configuration conf, Path hFileLinkPattern)
+           throws IOException {
+     return buildFromHFileLinkPattern(FSUtils.getRootDir(conf),
+             HFileArchiveUtil.getArchivePath(conf), hFileLinkPattern);
+   }
  
 +
-     /**
++
+   /**
     * @param rootDir Path to the root directory where hbase files are stored
     * @param archiveDir Path to the hbase archive directory
-    * @param path The path of the HFile Link.
+    * @param hFileLinkPattern The path of the HFile Link.
     */
-   public HFileLink(final Path rootDir, final Path archiveDir, final Path path) {
-     this(rootDir, archiveDir, new Path(rootDir, MobConstants.MOB_DIR_NAME), path);
+   public final static HFileLink buildFromHFileLinkPattern(final Path rootDir,
+                                                           final Path archiveDir,
+                                                           final Path hFileLinkPattern) {
+     Path hfilePath = getHFileLinkPatternRelativePath(hFileLinkPattern);
+     Path tempPath = new Path(new Path(rootDir, HConstants.HBASE_TEMP_DIRECTORY), hfilePath);
+     Path originPath = new Path(rootDir, hfilePath);
++    Path mobPath = new Path(new Path(rootDir, MobConstants.MOB_DIR_NAME), hfilePath);
+     Path archivePath = new Path(archiveDir, hfilePath);
 -    return new HFileLink(originPath, tempPath, archivePath);
++    return new HFileLink(originPath, tempPath, mobPath, archivePath);
    }
  
    /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java
----------------------------------------------------------------------
diff --cc hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java
index cc0d0aa,0000000..427f2cd
mode 100644,000000..100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java
@@@ -1,103 -1,0 +1,104 @@@
 +/**
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +
 +package org.apache.hadoop.hbase.master;
 +
 +import java.io.IOException;
 +import java.util.Map;
 +
 +import org.apache.commons.logging.Log;
 +import org.apache.commons.logging.LogFactory;
 +import org.apache.hadoop.classification.InterfaceAudience;
- import org.apache.hadoop.hbase.Chore;
 +import org.apache.hadoop.hbase.HColumnDescriptor;
 +import org.apache.hadoop.hbase.HTableDescriptor;
++import org.apache.hadoop.hbase.ScheduledChore;
 +import org.apache.hadoop.hbase.TableDescriptors;
 +import org.apache.hadoop.hbase.exceptions.LockTimeoutException;
 +import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
 +import org.apache.hadoop.hbase.mob.ExpiredMobFileCleaner;
 +import org.apache.hadoop.hbase.mob.MobConstants;
 +import org.apache.hadoop.hbase.mob.MobUtils;
 +
 +/**
 + * The Class ExpiredMobFileCleanerChore for running cleaner regularly to remove the expired
 + * mob files.
 + */
 +@InterfaceAudience.Private
- public class ExpiredMobFileCleanerChore extends Chore {
++public class ExpiredMobFileCleanerChore extends ScheduledChore {
 +
 +  private static final Log LOG = LogFactory.getLog(ExpiredMobFileCleanerChore.class);
 +  private final HMaster master;
 +  private TableLockManager tableLockManager;
 +  private ExpiredMobFileCleaner cleaner;
 +
 +  public ExpiredMobFileCleanerChore(HMaster master) {
-     super(master.getServerName() + "-ExpiredMobFileCleanerChore", master.getConfiguration().getInt(
-         MobConstants.MOB_CLEANER_PERIOD, MobConstants.DEFAULT_MOB_CLEANER_PERIOD), master);
++    super(master.getServerName() + "-ExpiredMobFileCleanerChore", master,
++        master.getConfiguration().getInt(MobConstants.MOB_CLEANER_PERIOD,
++                MobConstants.DEFAULT_MOB_CLEANER_PERIOD));
 +    this.master = master;
 +    this.tableLockManager = master.getTableLockManager();
 +    cleaner = new ExpiredMobFileCleaner();
 +  }
 +
 +  @Override
 +  protected void chore() {
 +    try {
 +      TableDescriptors htds = master.getTableDescriptors();
 +      Map<String, HTableDescriptor> map = htds.getAll();
 +      for (HTableDescriptor htd : map.values()) {
 +        for (HColumnDescriptor hcd : htd.getColumnFamilies()) {
 +          if (hcd.isMobEnabled() && hcd.getMinVersions() == 0) {
 +            // clean only for mob-enabled column.
 +            // obtain a read table lock before cleaning, synchronize with MobFileCompactionChore.
 +            boolean tableLocked = false;
 +            TableLock lock = null;
 +            try {
 +              // the tableLockManager might be null in testing. In that case, it is lock-free.
 +              if (tableLockManager != null) {
 +                lock = tableLockManager.readLock(MobUtils.getTableLockName(htd.getTableName()),
 +                  "Run ExpiredMobFileCleanerChore");
 +                lock.acquire();
 +              }
 +              tableLocked = true;
 +              cleaner.cleanExpiredMobFiles(htd.getTableName().getNameAsString(), hcd);
 +            } catch (LockTimeoutException e) {
 +              LOG.info("Fail to acquire the lock because of timeout, maybe a"
 +                + " MobFileCompactor is running", e);
 +            } catch (Exception e) {
 +              LOG.error(
 +                "Fail to clean the expired mob files for the column " + hcd.getNameAsString()
 +                  + " in the table " + htd.getNameAsString(), e);
 +            } finally {
 +              if (lock != null && tableLocked) {
 +                try {
 +                  lock.release();
 +                } catch (IOException e) {
 +                  LOG.error(
 +                    "Fail to release the write lock for the table " + htd.getNameAsString(), e);
 +                }
 +              }
 +            }
 +          }
 +        }
 +      }
 +    } catch (Exception e) {
 +      LOG.error("Fail to clean the expired mob files", e);
 +    }
 +  }
 +
 +}

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --cc hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 7ad49a3,61a1c66..aea7e7f
--- 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
@@@ -612,11 -767,6 +769,12 @@@ public class HMaster extends HRegionSer
      // master initialization. See HBASE-5916.
      this.serverManager.clearDeadServersWithSameHostNameAndPortOfOnlineServer();
  
 +    this.expiredMobFileCleanerChore = new ExpiredMobFileCleanerChore(this);
-     Threads.setDaemonThreadRunning(expiredMobFileCleanerChore.getThread());
++    getChoreService().scheduleChore(expiredMobFileCleanerChore);
++
 +    this.mobFileCompactChore = new MobFileCompactionChore(this);
-     Threads.setDaemonThreadRunning(mobFileCompactChore.getThread());
++    getChoreService().scheduleChore(mobFileCompactChore);
 +
      if (this.cpHost != null) {
        // don't let cp initialization errors kill the master
        try {
@@@ -863,23 -1061,17 +1069,23 @@@
    }
  
    private void stopChores() {
 +    if (this.expiredMobFileCleanerChore != null) {
-       this.expiredMobFileCleanerChore.interrupt();
++      this.expiredMobFileCleanerChore.cancel(true);
 +    }
 +    if (this.mobFileCompactChore != null) {
-       this.mobFileCompactChore.interrupt();
++      this.mobFileCompactChore.cancel(true);
 +    }
      if (this.balancerChore != null) {
-       this.balancerChore.interrupt();
+       this.balancerChore.cancel(true);
      }
      if (this.clusterStatusChore != null) {
-       this.clusterStatusChore.interrupt();
+       this.clusterStatusChore.cancel(true);
      }
      if (this.catalogJanitorChore != null) {
-       this.catalogJanitorChore.interrupt();
+       this.catalogJanitorChore.cancel(true);
      }
      if (this.clusterStatusPublisherChore != null){
-       clusterStatusPublisherChore.interrupt();
+       clusterStatusPublisherChore.cancel(true);
      }
    }
  

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCompactionChore.java
----------------------------------------------------------------------
diff --cc hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCompactionChore.java
index 12491af,0000000..ce2df81
mode 100644,000000..100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCompactionChore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCompactionChore.java
@@@ -1,166 -1,0 +1,166 @@@
 +/**
 + *
 + * 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.Map;
 +import java.util.concurrent.ExecutorService;
 +import java.util.concurrent.RejectedExecutionException;
 +import java.util.concurrent.RejectedExecutionHandler;
 +import java.util.concurrent.SynchronousQueue;
 +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.classification.InterfaceAudience;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.FileSystem;
- import org.apache.hadoop.hbase.Chore;
++import org.apache.hadoop.hbase.ScheduledChore;
 +import org.apache.hadoop.hbase.HColumnDescriptor;
 +import org.apache.hadoop.hbase.HTableDescriptor;
 +import org.apache.hadoop.hbase.TableDescriptors;
 +import org.apache.hadoop.hbase.TableName;
 +import org.apache.hadoop.hbase.exceptions.LockTimeoutException;
 +import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
 +import org.apache.hadoop.hbase.mob.MobConstants;
 +import org.apache.hadoop.hbase.mob.MobUtils;
 +import org.apache.hadoop.hbase.mob.filecompactions.MobFileCompactor;
 +import org.apache.hadoop.hbase.mob.filecompactions.PartitionedMobFileCompactor;
 +import org.apache.hadoop.hbase.util.ReflectionUtils;
 +import org.apache.hadoop.hbase.util.Threads;
 +
 +/**
 + * The Class MobFileCompactChore for running compaction regularly to merge small mob files.
 + */
 +@InterfaceAudience.Private
- public class MobFileCompactionChore extends Chore{
++public class MobFileCompactionChore extends ScheduledChore {
 +
 +  private static final Log LOG = LogFactory.getLog(MobFileCompactionChore.class);
 +  private HMaster master;
 +  private TableLockManager tableLockManager;
 +  private ExecutorService pool;
 +
 +  public MobFileCompactionChore(HMaster master) {
-     super(master.getServerName() + "-MobFileCompactChore", master.getConfiguration().getInt(
-       MobConstants.MOB_FILE_COMPACTION_CHORE_PERIOD,
-       MobConstants.DEFAULT_MOB_FILE_COMPACTION_CHORE_PERIOD), master);
++    super(master.getServerName() + "-MobFileCompactChore", master,
++        master.getConfiguration().getInt(MobConstants.MOB_FILE_COMPACTION_CHORE_PERIOD,
++      MobConstants.DEFAULT_MOB_FILE_COMPACTION_CHORE_PERIOD));
 +    this.master = master;
 +    this.tableLockManager = master.getTableLockManager();
 +    this.pool = createThreadPool();
 +  }
 +
 +  @Override
 +  protected void chore() {
 +    try {
 +      String className = master.getConfiguration().get(MobConstants.MOB_FILE_COMPACTOR_CLASS_KEY,
 +        PartitionedMobFileCompactor.class.getName());
 +      TableDescriptors htds = master.getTableDescriptors();
 +      Map<String, HTableDescriptor> map = htds.getAll();
 +      for (HTableDescriptor htd : map.values()) {
 +        for (HColumnDescriptor hcd : htd.getColumnFamilies()) {
 +          if (!hcd.isMobEnabled()) {
 +            continue;
 +          }
 +          // instantiate the mob file compactor.
 +          MobFileCompactor compactor = null;
 +          try {
 +            compactor = ReflectionUtils.instantiateWithCustomCtor(className, new Class[] {
 +              Configuration.class, FileSystem.class, TableName.class, HColumnDescriptor.class,
 +              ExecutorService.class },
 +              new Object[] { master.getConfiguration(), master.getFileSystem(), htd.getTableName(),
 +                hcd, pool });
 +          } catch (Exception e) {
 +            throw new IOException("Unable to load configured mob file compactor '" + className
 +              + "'", e);
 +          }
 +          // compact only for mob-enabled column.
 +          // obtain a write table lock before performing compaction to avoid race condition
 +          // with major compaction in mob-enabled column.
 +          boolean tableLocked = false;
 +          TableLock lock = null;
 +          try {
 +            // the tableLockManager might be null in testing. In that case, it is lock-free.
 +            if (tableLockManager != null) {
 +              lock = tableLockManager.writeLock(MobUtils.getTableLockName(htd.getTableName()),
 +                "Run MobFileCompactChore");
 +              lock.acquire();
 +            }
 +            tableLocked = true;
 +            compactor.compact();
 +          } catch (LockTimeoutException e) {
 +            LOG.info("Fail to acquire the lock because of timeout, maybe a major compaction or an"
 +              + " ExpiredMobFileCleanerChore is running", e);
 +          } catch (Exception e) {
 +            LOG.error("Fail to compact the mob files for the column " + hcd.getNameAsString()
 +              + " in the table " + htd.getNameAsString(), e);
 +          } finally {
 +            if (lock != null && tableLocked) {
 +              try {
 +                lock.release();
 +              } catch (IOException e) {
 +                LOG.error(
 +                  "Fail to release the write lock for the table " + htd.getNameAsString(), e);
 +              }
 +            }
 +          }
 +        }
 +      }
 +    } catch (Exception e) {
 +      LOG.error("Fail to clean the expired mob files", e);
 +    }
 +  }
 +
 +  @Override
 +  protected void cleanup() {
 +    super.cleanup();
 +    pool.shutdown();
 +  }
 +
 +  /**
 +   * Creates a thread pool.
 +   * @return A thread pool.
 +   */
 +  private ExecutorService createThreadPool() {
 +    Configuration conf = master.getConfiguration();
 +    int maxThreads = conf.getInt(MobConstants.MOB_FILE_COMPACTION_CHORE_THREADS_MAX,
 +      MobConstants.DEFAULT_MOB_FILE_COMPACTION_CHORE_THREADS_MAX);
 +    if (maxThreads == 0) {
 +      maxThreads = 1;
 +    }
 +    long keepAliveTime = conf.getLong(MobConstants.MOB_FILE_COMPACTION_CHORE_THREADS_KEEPALIVETIME,
 +      MobConstants.DEFAULT_MOB_FILE_COMPACTION_CHORE_THREADS_KEEPALIVETIME);
 +    final SynchronousQueue<Runnable> queue = new SynchronousQueue<Runnable>();
 +    ThreadPoolExecutor pool = new ThreadPoolExecutor(1, maxThreads, keepAliveTime,
 +      TimeUnit.SECONDS, queue, Threads.newDaemonThreadFactory("MobFileCompactionChore"),
 +      new RejectedExecutionHandler() {
 +        @Override
 +        public void rejectedExecution(Runnable r, ThreadPoolExecutor executor) {
 +          try {
 +            // waiting for a thread to pick up instead of throwing exceptions.
 +            queue.put(r);
 +          } catch (InterruptedException e) {
 +            throw new RejectedExecutionException(e);
 +          }
 +        }
 +    });
 +    ((ThreadPoolExecutor) pool).allowCoreThreadTimeOut(true);
 +    return pool;
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java
----------------------------------------------------------------------
diff --cc hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java
index 95e89d2,092a17d..0664a55
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java
@@@ -24,29 -25,30 +25,28 @@@ import java.util.List
  
  import org.apache.commons.logging.Log;
  import org.apache.commons.logging.LogFactory;
- import org.apache.hadoop.classification.InterfaceAudience;
  import org.apache.hadoop.fs.FileSystem;
  import org.apache.hadoop.fs.Path;
--import org.apache.hadoop.hbase.CoordinatedStateException;
- import org.apache.hadoop.hbase.HColumnDescriptor;
- import org.apache.hadoop.hbase.TableName;
--import org.apache.hadoop.hbase.HRegionInfo;
--import org.apache.hadoop.hbase.HTableDescriptor;
 -import org.apache.hadoop.hbase.MetaTableAccessor;
--import org.apache.hadoop.hbase.Server;
 -import org.apache.hadoop.hbase.TableName;
++import org.apache.hadoop.hbase.*;
  import org.apache.hadoop.hbase.backup.HFileArchiver;
- import org.apache.hadoop.hbase.MetaTableAccessor;
+ import org.apache.hadoop.hbase.classification.InterfaceAudience;
+ import org.apache.hadoop.hbase.client.Delete;
+ import org.apache.hadoop.hbase.client.Result;
+ import org.apache.hadoop.hbase.client.ResultScanner;
+ import org.apache.hadoop.hbase.client.Scan;
+ import org.apache.hadoop.hbase.client.Table;
  import org.apache.hadoop.hbase.executor.EventType;
- import org.apache.hadoop.hbase.regionserver.HRegion;
- import org.apache.hadoop.hbase.util.FSUtils;
  import org.apache.hadoop.hbase.master.AssignmentManager;
  import org.apache.hadoop.hbase.master.HMaster;
  import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
  import org.apache.hadoop.hbase.master.MasterFileSystem;
  import org.apache.hadoop.hbase.master.MasterServices;
- import org.apache.hadoop.hbase.master.RegionStates;
  import org.apache.hadoop.hbase.master.RegionState.State;
 +import org.apache.hadoop.hbase.mob.MobConstants;
 +import org.apache.hadoop.hbase.mob.MobUtils;
+ import org.apache.hadoop.hbase.master.RegionStates;
+ import org.apache.hadoop.hbase.regionserver.HRegion;
++import org.apache.hadoop.hbase.util.FSUtils;
  
  @InterfaceAudience.Private
  public class DeleteTableHandler extends TableEventHandler {
@@@ -135,59 -165,37 +163,63 @@@
     */
    protected void removeTableData(final List<HRegionInfo> regions)
        throws IOException, CoordinatedStateException {
-     // 1. Remove regions from META
-     LOG.debug("Deleting regions from META");
-     MetaTableAccessor.deleteRegions(this.server.getShortCircuitConnection(), regions);
- 
-     // -----------------------------------------------------------------------
-     // NOTE: At this point we still have data on disk, but nothing in hbase:meta
-     //       if the rename below fails, hbck will report an inconsistency.
-     // -----------------------------------------------------------------------
- 
-     // 2. Move the table in /hbase/.tmp
-     MasterFileSystem mfs = this.masterServices.getMasterFileSystem();
-     Path tempTableDir = mfs.moveTableToTemp(tableName);
- 
-     // 3. Archive regions from FS (temp directory)
-     FileSystem fs = mfs.getFileSystem();
-     for (HRegionInfo hri: regions) {
-       LOG.debug("Archiving region " + hri.getRegionNameAsString() + " from FS");
-       HFileArchiver.archiveRegion(fs, mfs.getRootDir(),
-           tempTableDir, HRegion.getRegionDir(tempTableDir, hri.getEncodedName()));
-     }
+     try {
+       // 1. Remove regions from META
+       LOG.debug("Deleting regions from META");
+       MetaTableAccessor.deleteRegions(this.server.getConnection(), regions);
+ 
+       // -----------------------------------------------------------------------
+       // NOTE: At this point we still have data on disk, but nothing in hbase:meta
+       //       if the rename below fails, hbck will report an inconsistency.
+       // -----------------------------------------------------------------------
+ 
+       // 2. Move the table in /hbase/.tmp
+       MasterFileSystem mfs = this.masterServices.getMasterFileSystem();
+       Path tempTableDir = mfs.moveTableToTemp(tableName);
+ 
+       // 3. Archive regions from FS (temp directory)
+       FileSystem fs = mfs.getFileSystem();
+       for (HRegionInfo hri : regions) {
+         LOG.debug("Archiving region " + hri.getRegionNameAsString() + " from FS");
+         HFileArchiver.archiveRegion(fs, mfs.getRootDir(),
+             tempTableDir, HRegion.getRegionDir(tempTableDir, hri.getEncodedName()));
+       }
  
-     // Archive the mob data if there is a mob-enabled column
-     HColumnDescriptor[] hcds = hTableDescriptor.getColumnFamilies();
-     boolean hasMob = false;
-     for (HColumnDescriptor hcd : hcds) {
-       if (hcd.isMobEnabled()) {
-         hasMob = true;
-         break;
++      // Archive the mob data if there is a mob-enabled column
++      HColumnDescriptor[] hcds = hTableDescriptor.getColumnFamilies();
++      boolean hasMob = false;
++      for (HColumnDescriptor hcd : hcds) {
++        if (hcd.isMobEnabled()) {
++          hasMob = true;
++          break;
++        }
 +      }
-     }
-     Path mobTableDir = null;
-     if (hasMob) {
-       // Archive mob data
-       mobTableDir = FSUtils.getTableDir(new Path(mfs.getRootDir(), MobConstants.MOB_DIR_NAME),
-           tableName);
-       Path regionDir =
-           new Path(mobTableDir, MobUtils.getMobRegionInfo(tableName).getEncodedName());
-       if (fs.exists(regionDir)) {
-         HFileArchiver.archiveRegion(fs, mfs.getRootDir(), mobTableDir, regionDir);
++      Path mobTableDir = null;
++      if (hasMob) {
++        // Archive mob data
++        mobTableDir = FSUtils.getTableDir(new Path(mfs.getRootDir(), MobConstants.MOB_DIR_NAME),
++                tableName);
++        Path regionDir =
++                new Path(mobTableDir, MobUtils.getMobRegionInfo(tableName).getEncodedName());
++        if (fs.exists(regionDir)) {
++          HFileArchiver.archiveRegion(fs, mfs.getRootDir(), mobTableDir, regionDir);
++        }
 +      }
-     }
-     // 4. Delete table directory from FS (temp directory)
-     if (!fs.delete(tempTableDir, true)) {
-       LOG.error("Couldn't delete " + tempTableDir);
-     }
-     // Delete the table directory where the mob files are saved
-     if (hasMob && mobTableDir != null && fs.exists(mobTableDir)) {
-       if (!fs.delete(mobTableDir, true)) {
-         LOG.error("Couldn't delete " + mobTableDir);
+       // 4. Delete table directory from FS (temp directory)
+       if (!fs.delete(tempTableDir, true)) {
+         LOG.error("Couldn't delete " + tempTableDir);
+       }
++      // Delete the table directory where the mob files are saved
++      if (hasMob && mobTableDir != null && fs.exists(mobTableDir)) {
++        if (!fs.delete(mobTableDir, true)) {
++          LOG.error("Couldn't delete " + mobTableDir);
++        }
 +      }
-     }
  
-     LOG.debug("Table '" + tableName + "' archived!");
+       LOG.debug("Table '" + tableName + "' archived!");
+     } finally {
+       cleanupTableState();
+     }
    }
  
    @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/DisabledTableSnapshotHandler.java
----------------------------------------------------------------------
diff --cc hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/DisabledTableSnapshotHandler.java
index d91d825,e70e1c8..1f2ed0a
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/DisabledTableSnapshotHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/DisabledTableSnapshotHandler.java
@@@ -25,10 -25,9 +25,10 @@@ import java.util.concurrent.ThreadPoolE
  
  import org.apache.commons.logging.Log;
  import org.apache.commons.logging.LogFactory;
- import org.apache.hadoop.classification.InterfaceAudience;
- import org.apache.hadoop.classification.InterfaceStability;
- import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.hbase.HConstants;
+ import org.apache.hadoop.conf.Configuration;
+ import org.apache.hadoop.hbase.classification.InterfaceAudience;
+ import org.apache.hadoop.hbase.classification.InterfaceStability;
  import org.apache.hadoop.hbase.HRegionInfo;
  import org.apache.hadoop.hbase.ServerName;
  import org.apache.hadoop.hbase.client.RegionReplicaUtil;

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/MasterSnapshotVerifier.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobCompactor.java
----------------------------------------------------------------------
diff --cc hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobCompactor.java
index ec0cfe5,0000000..c2abc7c
mode 100644,000000..100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobCompactor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobCompactor.java
@@@ -1,304 -1,0 +1,308 @@@
 +/**
 + * 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.mob;
 +
 +import java.io.IOException;
 +import java.util.ArrayList;
 +import java.util.Date;
 +import java.util.List;
 +
 +import org.apache.commons.logging.Log;
 +import org.apache.commons.logging.LogFactory;
 +import org.apache.hadoop.classification.InterfaceAudience;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.hbase.Cell;
 +import org.apache.hadoop.hbase.CellUtil;
 +import org.apache.hadoop.hbase.KeyValue;
 +import org.apache.hadoop.hbase.KeyValueUtil;
 +import org.apache.hadoop.hbase.Tag;
 +import org.apache.hadoop.hbase.TagType;
 +import org.apache.hadoop.hbase.client.Scan;
 +import org.apache.hadoop.hbase.regionserver.HMobStore;
 +import org.apache.hadoop.hbase.regionserver.HStore;
 +import org.apache.hadoop.hbase.regionserver.InternalScanner;
 +import org.apache.hadoop.hbase.regionserver.MobCompactionStoreScanner;
 +import org.apache.hadoop.hbase.regionserver.ScanType;
 +import org.apache.hadoop.hbase.regionserver.Store;
 +import org.apache.hadoop.hbase.regionserver.StoreFile;
 +import org.apache.hadoop.hbase.regionserver.StoreFile.Writer;
 +import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
++import org.apache.hadoop.hbase.regionserver.compactions.CompactionThroughputController;
 +import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor;
 +import org.apache.hadoop.hbase.util.Bytes;
 +
 +/**
 + * Compact passed set of files in the mob-enabled column family.
 + */
 +@InterfaceAudience.Private
 +public class DefaultMobCompactor extends DefaultCompactor {
 +
 +  private static final Log LOG = LogFactory.getLog(DefaultMobCompactor.class);
 +  private long mobSizeThreshold;
 +  private HMobStore mobStore;
 +  public DefaultMobCompactor(Configuration conf, Store store) {
 +    super(conf, store);
 +    // The mob cells reside in the mob-enabled column family which is held by HMobStore.
 +    // During the compaction, the compactor reads the cells from the mob files and
 +    // probably creates new mob files. All of these operations are included in HMobStore,
 +    // so we need to cast the Store to HMobStore.
 +    if (!(store instanceof HMobStore)) {
 +      throw new IllegalArgumentException("The store " + store + " is not a HMobStore");
 +    }
 +    mobStore = (HMobStore) store;
 +    mobSizeThreshold = store.getFamily().getMobThreshold();
 +  }
 +
 +  /**
 +   * Creates a writer for a new file in a temporary directory.
 +   * @param fd The file details.
 +   * @param smallestReadPoint The smallest mvcc readPoint across all the scanners in this region.
 +   * @return Writer for a new StoreFile in the tmp dir.
 +   * @throws IOException
 +   */
 +  @Override
 +  protected Writer createTmpWriter(FileDetails fd, long smallestReadPoint) throws IOException {
 +    // make this writer with tags always because of possible new cells with tags.
 +    StoreFile.Writer writer = store.createWriterInTmp(fd.maxKeyCount, this.compactionCompression,
 +        true, fd.maxMVCCReadpoint >= smallestReadPoint, true);
 +    return writer;
 +  }
 +
 +  @Override
 +  protected InternalScanner createScanner(Store store, List<StoreFileScanner> scanners,
 +      ScanType scanType, long smallestReadPoint, long earliestPutTs) throws IOException {
 +    Scan scan = new Scan();
 +    scan.setMaxVersions(store.getFamily().getMaxVersions());
 +    if (scanType == ScanType.COMPACT_DROP_DELETES) {
 +      scanType = ScanType.COMPACT_RETAIN_DELETES;
 +      return new MobCompactionStoreScanner(store, store.getScanInfo(), scan, scanners,
 +          scanType, smallestReadPoint, earliestPutTs, true);
 +    } else {
 +      return new MobCompactionStoreScanner(store, store.getScanInfo(), scan, scanners,
 +          scanType, smallestReadPoint, earliestPutTs, false);
 +    }
 +  }
 +
++  // TODO refactor to take advantage of the throughput controller.
++
 +  /**
 +   * Performs compaction on a column family with the mob flag enabled.
 +   * This is for when the mob threshold size has changed or if the mob
 +   * column family mode has been toggled via an alter table statement.
 +   * Compacts the files by the following rules.
 +   * 1. If the cell has a mob reference tag, the cell's value is the path of the mob file.
 +   * <ol>
 +   * <li>
 +   * If the value size of a cell is larger than the threshold, this cell is regarded as a mob,
 +   * directly copy the (with mob tag) cell into the new store file.
 +   * </li>
 +   * <li>
 +   * Otherwise, retrieve the mob cell from the mob file, and writes a copy of the cell into
 +   * the new store file.
 +   * </li>
 +   * </ol>
 +   * 2. If the cell doesn't have a reference tag.
 +   * <ol>
 +   * <li>
 +   * If the value size of a cell is larger than the threshold, this cell is regarded as a mob,
 +   * write this cell to a mob file, and write the path of this mob file to the store file.
 +   * </li>
 +   * <li>
 +   * Otherwise, directly write this cell into the store file.
 +   * </li>
 +   * </ol>
 +   * In the mob compaction, the {@link MobCompactionStoreScanner} is used as a scanner
 +   * which could output the normal cells and delete markers together when required.
 +   * After the major compaction on the normal hfiles, we have a guarantee that we have purged all
 +   * deleted or old version mob refs, and the delete markers are written to a del file with the
 +   * suffix _del. Because of this, it is safe to use the del file in the mob compaction.
 +   * The mob compaction doesn't take place in the normal hfiles, it occurs directly in the
 +   * mob files. When the small mob files are merged into bigger ones, the del file is added into
 +   * the scanner to filter the deleted cells.
 +   * @param fd File details
 +   * @param scanner Where to read from.
 +   * @param writer Where to write to.
 +   * @param smallestReadPoint Smallest read point.
 +   * @param cleanSeqId When true, remove seqId(used to be mvcc) value which is <= smallestReadPoint
 +   * @param major Is a major compaction.
 +   * @return Whether compaction ended; false if it was interrupted for any reason.
 +   */
 +  @Override
 +  protected boolean performCompaction(FileDetails fd, InternalScanner scanner, CellSink writer,
-       long smallestReadPoint, boolean cleanSeqId, boolean major) throws IOException {
++      long smallestReadPoint, boolean cleanSeqId,
++      CompactionThroughputController throughputController,  boolean major) throws IOException {
 +    if (!(scanner instanceof MobCompactionStoreScanner)) {
 +      throw new IllegalArgumentException(
 +          "The scanner should be an instance of MobCompactionStoreScanner");
 +    }
 +    MobCompactionStoreScanner compactionScanner = (MobCompactionStoreScanner) scanner;
 +    int bytesWritten = 0;
 +    // Since scanner.next() can return 'false' but still be delivering data,
 +    // we have to use a do/while loop.
 +    List<Cell> cells = new ArrayList<Cell>();
 +    // Limit to "hbase.hstore.compaction.kv.max" (default 10) to avoid OOME
 +    int closeCheckInterval = HStore.getCloseCheckInterval();
 +    boolean hasMore;
 +    Path path = MobUtils.getMobFamilyPath(conf, store.getTableName(), store.getColumnFamilyName());
 +    byte[] fileName = null;
 +    StoreFile.Writer mobFileWriter = null;
 +    StoreFile.Writer delFileWriter = null;
 +    long mobCells = 0;
 +    long deleteMarkersCount = 0;
 +    Tag tableNameTag = new Tag(TagType.MOB_TABLE_NAME_TAG_TYPE, store.getTableName()
 +            .getName());
 +    long mobCompactedIntoMobCellsCount = 0;
 +    long mobCompactedFromMobCellsCount = 0;
 +    long mobCompactedIntoMobCellsSize = 0;
 +    long mobCompactedFromMobCellsSize = 0;
 +    try {
 +      try {
 +        // If the mob file writer could not be created, directly write the cell to the store file.
 +        mobFileWriter = mobStore.createWriterInTmp(new Date(fd.latestPutTs), fd.maxKeyCount,
 +            store.getFamily().getCompression(), store.getRegionInfo().getStartKey());
 +        fileName = Bytes.toBytes(mobFileWriter.getPath().getName());
 +      } catch (IOException e) {
 +        LOG.error(
 +            "Fail to create mob writer, "
 +                + "we will continue the compaction by writing MOB cells directly in store files",
 +            e);
 +      }
 +      delFileWriter = mobStore.createDelFileWriterInTmp(new Date(fd.latestPutTs), fd.maxKeyCount,
 +          store.getFamily().getCompression(), store.getRegionInfo().getStartKey());
 +      do {
 +        hasMore = compactionScanner.next(cells, compactionKVMax);
 +        // output to writer:
 +        for (Cell c : cells) {
-           // TODO remove the KeyValueUtil.ensureKeyValue before merging back to trunk.
-           KeyValue kv = KeyValueUtil.ensureKeyValue(c);
-           resetSeqId(smallestReadPoint, cleanSeqId, kv);
++          if (cleanSeqId && c.getSequenceId() <= smallestReadPoint) {
++            CellUtil.setSequenceId(c, 0);
++          }
 +          if (compactionScanner.isOutputDeleteMarkers() && CellUtil.isDelete(c)) {
-             delFileWriter.append(kv);
++            delFileWriter.append(c);
 +            deleteMarkersCount++;
-           } else if (mobFileWriter == null || kv.getTypeByte() != KeyValue.Type.Put.getCode()) {
++          } else if (mobFileWriter == null || c.getTypeByte() != KeyValue.Type.Put.getCode()) {
 +            // If the mob file writer is null or the kv type is not put, directly write the cell
 +            // to the store file.
-             writer.append(kv);
-           } else if (MobUtils.isMobReferenceCell(kv)) {
-             if (MobUtils.hasValidMobRefCellValue(kv)) {
-               int size = MobUtils.getMobValueLength(kv);
++            writer.append(c);
++          } else if (MobUtils.isMobReferenceCell(c)) {
++            if (MobUtils.hasValidMobRefCellValue(c)) {
++              int size = MobUtils.getMobValueLength(c);
 +              if (size > mobSizeThreshold) {
 +                // If the value size is larger than the threshold, it's regarded as a mob. Since
 +                // its value is already in the mob file, directly write this cell to the store file
-                 writer.append(kv);
++                writer.append(c);
 +              } else {
 +                // If the value is not larger than the threshold, it's not regarded a mob. Retrieve
 +                // the mob cell from the mob file, and write it back to the store file.
-                 Cell cell = mobStore.resolve(kv, false);
-                 if (cell.getValueLength() != 0) {
++                Cell mobCell = mobStore.resolve(c, false);
++                if (mobCell.getValueLength() != 0) {
 +                  // put the mob data back to the store file
-                   KeyValue mobKv = KeyValueUtil.ensureKeyValue(cell);
-                   mobKv.setSequenceId(kv.getSequenceId());
-                   writer.append(mobKv);
++                  // KeyValue mobKv = KeyValueUtil.ensureKeyValue(cell);
++                  CellUtil.setSequenceId(mobCell, c.getSequenceId());
++                  writer.append(mobCell);
 +                  mobCompactedFromMobCellsCount++;
-                   mobCompactedFromMobCellsSize += cell.getValueLength();
++                  mobCompactedFromMobCellsSize += mobCell.getValueLength();
 +                } else {
 +                  // If the value of a file is empty, there might be issues when retrieving,
 +                  // directly write the cell to the store file, and leave it to be handled by the
 +                  // next compaction.
-                   writer.append(kv);
++                  writer.append(c);
 +                }
 +              }
 +            } else {
-               LOG.warn("The value format of the KeyValue " + kv
++              LOG.warn("The value format of the KeyValue " + c
 +                  + " is wrong, its length is less than " + Bytes.SIZEOF_INT);
-               writer.append(kv);
++              writer.append(c);
 +            }
-           } else if (kv.getValueLength() <= mobSizeThreshold) {
++          } else if (c.getValueLength() <= mobSizeThreshold) {
 +            // If the value size of a cell is not larger than the threshold, directly write it to
 +            // the store file.
-             writer.append(kv);
++            writer.append(c);
 +          } else {
 +            // If the value size of a cell is larger than the threshold, it's regarded as a mob,
 +            // write this cell to a mob file, and write the path to the store file.
 +            mobCells++;
 +            // append the original keyValue in the mob file.
-             mobFileWriter.append(kv);
-             KeyValue reference = MobUtils.createMobRefKeyValue(kv, fileName, tableNameTag);
++            mobFileWriter.append(c);
++            KeyValue reference = MobUtils.createMobRefKeyValue(c, fileName, tableNameTag);
 +            // write the cell whose value is the path of a mob file to the store file.
 +            writer.append(reference);
 +            mobCompactedIntoMobCellsCount++;
-             mobCompactedIntoMobCellsSize += kv.getValueLength();
++            mobCompactedIntoMobCellsSize += c.getValueLength();
 +          }
 +          ++progress.currentCompactedKVs;
 +
 +          // check periodically to see if a system stop is requested
 +          if (closeCheckInterval > 0) {
-             bytesWritten += kv.getLength();
++            bytesWritten += KeyValueUtil.length(c);
 +            if (bytesWritten > closeCheckInterval) {
 +              bytesWritten = 0;
 +              if (!store.areWritesEnabled()) {
 +                progress.cancel();
 +                return false;
 +              }
 +            }
 +          }
 +        }
 +        cells.clear();
 +      } while (hasMore);
 +    } finally {
 +      if (mobFileWriter != null) {
 +        mobFileWriter.appendMetadata(fd.maxSeqId, major, mobCells);
 +        mobFileWriter.close();
 +      }
 +      if (delFileWriter != null) {
 +        delFileWriter.appendMetadata(fd.maxSeqId, major, deleteMarkersCount);
 +        delFileWriter.close();
 +      }
 +    }
 +    if (mobFileWriter != null) {
 +      if (mobCells > 0) {
 +        // If the mob file is not empty, commit it.
 +        mobStore.commitFile(mobFileWriter.getPath(), path);
 +      } else {
 +        try {
 +          // If the mob file is empty, delete it instead of committing.
 +          store.getFileSystem().delete(mobFileWriter.getPath(), true);
 +        } catch (IOException e) {
 +          LOG.error("Fail to delete the temp mob file", e);
 +        }
 +      }
 +    }
 +    if (delFileWriter != null) {
 +      if (deleteMarkersCount > 0) {
 +        // If the del file is not empty, commit it.
 +        // If the commit fails, the compaction is re-performed again.
 +        mobStore.commitFile(delFileWriter.getPath(), path);
 +      } else {
 +        try {
 +          // If the del file is empty, delete it instead of committing.
 +          store.getFileSystem().delete(delFileWriter.getPath(), true);
 +        } catch (IOException e) {
 +          LOG.error("Fail to delete the temp del file", e);
 +        }
 +      }
 +    }
 +    mobStore.updateMobCompactedFromMobCellsCount(mobCompactedFromMobCellsCount);
 +    mobStore.updateMobCompactedIntoMobCellsCount(mobCompactedIntoMobCellsCount);
 +    mobStore.updateMobCompactedFromMobCellsSize(mobCompactedFromMobCellsSize);
 +    mobStore.updateMobCompactedIntoMobCellsSize(mobCompactedIntoMobCellsSize);
 +    progress.complete();
 +    return true;
 +  }
 +}


[08/50] [abbrv] hbase git commit: HBASE-12976 Set default value for hbase.client.scanner.max.result.size.

Posted by jm...@apache.org.
HBASE-12976 Set default value for hbase.client.scanner.max.result.size.


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

Branch: refs/heads/hbase-11339
Commit: 2583e8de574ae4b002c5dbc80b0da666b42dd699
Parents: 96cdc79
Author: Lars Hofhansl <la...@apache.org>
Authored: Thu Feb 5 13:29:23 2015 -0800
Committer: Lars Hofhansl <la...@apache.org>
Committed: Thu Feb 5 13:29:23 2015 -0800

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hbase/HConstants.java |  4 ++--
 hbase-common/src/main/resources/hbase-default.xml         | 10 ++++++++++
 2 files changed, 12 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2583e8de/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
index 49bb7e1..2ee55f7 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
@@ -611,9 +611,9 @@ public final class HConstants {
    * Note that when a single row is larger than this limit the row is still
    * returned completely.
    *
-   * The default value is unlimited.
+   * The default value is 2MB.
    */
-  public static final long DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE = Long.MAX_VALUE;
+  public static final long DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE = 2 * 1024 * 1024;
 
   /**
    * Parameter name for client pause value, used mostly as value to wait

http://git-wip-us.apache.org/repos/asf/hbase/blob/2583e8de/hbase-common/src/main/resources/hbase-default.xml
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/resources/hbase-default.xml b/hbase-common/src/main/resources/hbase-default.xml
index e412b8f..2985685 100644
--- a/hbase-common/src/main/resources/hbase-default.xml
+++ b/hbase-common/src/main/resources/hbase-default.xml
@@ -1259,6 +1259,16 @@ possible configurations would overwhelm and obscure the important.
   </property>
 
   <property>
+    <name>hbase.client.scanner.max.result.size</name>
+    <value>2097152</value>
+    <description>Maximum number of bytes returned when calling a scanner's next method.
+    Note that when a single row is larger than this limit the row is still returned completely.
+    The default value is 2MB, which is good for 1ge networks.
+    With faster and/or high latency networks this value should be increased.
+    </description>
+  </property>
+
+  <property>
     <name>hbase.status.published</name>
     <value>false</value>
     <description>


[48/50] [abbrv] hbase git commit: Merge branch 'master' (2/11/15) into hbase-11339

Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java
----------------------------------------------------------------------
diff --cc hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java
index d8b1376,0000000..4e8ccc1
mode 100644,000000..100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java
@@@ -1,648 -1,0 +1,648 @@@
 +/**
 + *
 + * 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.mob;
 +
 +import java.io.FileNotFoundException;
 +import java.io.IOException;
 +import java.text.ParseException;
 +import java.text.SimpleDateFormat;
 +import java.util.ArrayList;
 +import java.util.Collection;
 +import java.util.Date;
 +import java.util.List;
 +import java.util.UUID;
 +
 +import org.apache.commons.logging.Log;
 +import org.apache.commons.logging.LogFactory;
 +import org.apache.hadoop.classification.InterfaceAudience;
 +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.Cell;
 +import org.apache.hadoop.hbase.HColumnDescriptor;
 +import org.apache.hadoop.hbase.HConstants;
 +import org.apache.hadoop.hbase.HRegionInfo;
 +import org.apache.hadoop.hbase.KeyValue;
 +import org.apache.hadoop.hbase.TableName;
 +import org.apache.hadoop.hbase.Tag;
 +import org.apache.hadoop.hbase.TagType;
 +import org.apache.hadoop.hbase.backup.HFileArchiver;
 +import org.apache.hadoop.hbase.client.Scan;
 +import org.apache.hadoop.hbase.io.HFileLink;
 +import org.apache.hadoop.hbase.io.compress.Compression;
 +import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 +import org.apache.hadoop.hbase.io.hfile.HFile;
 +import org.apache.hadoop.hbase.io.hfile.HFileContext;
 +import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
 +import org.apache.hadoop.hbase.regionserver.BloomType;
 +import org.apache.hadoop.hbase.regionserver.HStore;
 +import org.apache.hadoop.hbase.regionserver.StoreFile;
 +import org.apache.hadoop.hbase.util.Bytes;
 +import org.apache.hadoop.hbase.util.FSUtils;
 +
 +/**
 + * The mob utilities
 + */
 +@InterfaceAudience.Private
 +public class MobUtils {
 +
 +  private static final Log LOG = LogFactory.getLog(MobUtils.class);
 +
 +  private static final ThreadLocal<SimpleDateFormat> LOCAL_FORMAT =
 +      new ThreadLocal<SimpleDateFormat>() {
 +    @Override
 +    protected SimpleDateFormat initialValue() {
 +      return new SimpleDateFormat("yyyyMMdd");
 +    }
 +  };
 +
 +  /**
 +   * Formats a date to a string.
 +   * @param date The date.
 +   * @return The string format of the date, it's yyyymmdd.
 +   */
 +  public static String formatDate(Date date) {
 +    return LOCAL_FORMAT.get().format(date);
 +  }
 +
 +  /**
 +   * Parses the string to a date.
 +   * @param dateString The string format of a date, it's yyyymmdd.
 +   * @return A date.
 +   * @throws ParseException
 +   */
 +  public static Date parseDate(String dateString) throws ParseException {
 +    return LOCAL_FORMAT.get().parse(dateString);
 +  }
 +
 +  /**
 +   * Whether the current cell is a mob reference cell.
 +   * @param cell The current cell.
 +   * @return True if the cell has a mob reference tag, false if it doesn't.
 +   */
 +  public static boolean isMobReferenceCell(Cell cell) {
 +    if (cell.getTagsLength() > 0) {
 +      Tag tag = Tag.getTag(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength(),
 +          TagType.MOB_REFERENCE_TAG_TYPE);
 +      return tag != null;
 +    }
 +    return false;
 +  }
 +
 +  /**
 +   * Gets the table name tag.
 +   * @param cell The current cell.
 +   * @return The table name tag.
 +   */
 +  public static Tag getTableNameTag(Cell cell) {
 +    if (cell.getTagsLength() > 0) {
 +      Tag tag = Tag.getTag(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength(),
 +          TagType.MOB_TABLE_NAME_TAG_TYPE);
 +      return tag;
 +    }
 +    return null;
 +  }
 +
 +  /**
 +   * Whether the tag list has a mob reference tag.
 +   * @param tags The tag list.
 +   * @return True if the list has a mob reference tag, false if it doesn't.
 +   */
 +  public static boolean hasMobReferenceTag(List<Tag> tags) {
 +    if (!tags.isEmpty()) {
 +      for (Tag tag : tags) {
 +        if (tag.getType() == TagType.MOB_REFERENCE_TAG_TYPE) {
 +          return true;
 +        }
 +      }
 +    }
 +    return false;
 +  }
 +
 +  /**
 +   * Indicates whether it's a raw scan.
 +   * The information is set in the attribute "hbase.mob.scan.raw" of scan.
 +   * For a mob cell, in a normal scan the scanners retrieves the mob cell from the mob file.
 +   * In a raw scan, the scanner directly returns cell in HBase without retrieve the one in
 +   * the mob file.
 +   * @param scan The current scan.
 +   * @return True if it's a raw scan.
 +   */
 +  public static boolean isRawMobScan(Scan scan) {
 +    byte[] raw = scan.getAttribute(MobConstants.MOB_SCAN_RAW);
 +    try {
 +      return raw != null && Bytes.toBoolean(raw);
 +    } catch (IllegalArgumentException e) {
 +      return false;
 +    }
 +  }
 +
 +  /**
 +   * Indicates whether it's a reference only scan.
 +   * The information is set in the attribute "hbase.mob.scan.ref.only" of scan.
 +   * If it's a ref only scan, only the cells with ref tag are returned.
 +   * @param scan The current scan.
 +   * @return True if it's a ref only scan.
 +   */
 +  public static boolean isRefOnlyScan(Scan scan) {
 +    byte[] refOnly = scan.getAttribute(MobConstants.MOB_SCAN_REF_ONLY);
 +    try {
 +      return refOnly != null && Bytes.toBoolean(refOnly);
 +    } catch (IllegalArgumentException e) {
 +      return false;
 +    }
 +  }
 +
 +  /**
 +   * Indicates whether the scan contains the information of caching blocks.
 +   * The information is set in the attribute "hbase.mob.cache.blocks" of scan.
 +   * @param scan The current scan.
 +   * @return True when the Scan attribute specifies to cache the MOB blocks.
 +   */
 +  public static boolean isCacheMobBlocks(Scan scan) {
 +    byte[] cache = scan.getAttribute(MobConstants.MOB_CACHE_BLOCKS);
 +    try {
 +      return cache != null && Bytes.toBoolean(cache);
 +    } catch (IllegalArgumentException e) {
 +      return false;
 +    }
 +  }
 +
 +  /**
 +   * Sets the attribute of caching blocks in the scan.
 +   *
 +   * @param scan
 +   *          The current scan.
 +   * @param cacheBlocks
 +   *          True, set the attribute of caching blocks into the scan, the scanner with this scan
 +   *          caches blocks.
 +   *          False, the scanner doesn't cache blocks for this scan.
 +   */
 +  public static void setCacheMobBlocks(Scan scan, boolean cacheBlocks) {
 +    scan.setAttribute(MobConstants.MOB_CACHE_BLOCKS, Bytes.toBytes(cacheBlocks));
 +  }
 +
 +  /**
 +   * Cleans the expired mob files.
 +   * Cleans the files whose creation date is older than (current - columnFamily.ttl), and
 +   * the minVersions of that column family is 0.
 +   * @param fs The current file system.
 +   * @param conf The current configuration.
 +   * @param tableName The current table name.
 +   * @param columnDescriptor The descriptor of the current column family.
 +   * @param cacheConfig The cacheConfig that disables the block cache.
 +   * @param current The current time.
 +   * @throws IOException
 +   */
 +  public static void cleanExpiredMobFiles(FileSystem fs, Configuration conf, TableName tableName,
 +      HColumnDescriptor columnDescriptor, CacheConfig cacheConfig, long current)
 +      throws IOException {
 +    long timeToLive = columnDescriptor.getTimeToLive();
 +    if (Integer.MAX_VALUE == timeToLive) {
 +      // no need to clean, because the TTL is not set.
 +      return;
 +    }
 +
 +    Date expireDate = new Date(current - timeToLive * 1000);
 +    expireDate = new Date(expireDate.getYear(), expireDate.getMonth(), expireDate.getDate());
 +    LOG.info("MOB HFiles older than " + expireDate.toGMTString() + " will be deleted!");
 +
 +    FileStatus[] stats = null;
 +    Path mobTableDir = FSUtils.getTableDir(getMobHome(conf), tableName);
 +    Path path = getMobFamilyPath(conf, tableName, columnDescriptor.getNameAsString());
 +    try {
 +      stats = fs.listStatus(path);
 +    } catch (FileNotFoundException e) {
 +      LOG.warn("Fail to find the mob file " + path, e);
 +    }
 +    if (null == stats) {
 +      // no file found
 +      return;
 +    }
 +    List<StoreFile> filesToClean = new ArrayList<StoreFile>();
 +    int deletedFileCount = 0;
 +    for (FileStatus file : stats) {
 +      String fileName = file.getPath().getName();
 +      try {
 +        MobFileName mobFileName = null;
 +        if (!HFileLink.isHFileLink(file.getPath())) {
 +          mobFileName = MobFileName.create(fileName);
 +        } else {
-           HFileLink hfileLink = new HFileLink(conf, file.getPath());
++          HFileLink hfileLink = HFileLink.buildFromHFileLinkPattern(conf, file.getPath());
 +          mobFileName = MobFileName.create(hfileLink.getOriginPath().getName());
 +        }
 +        Date fileDate = parseDate(mobFileName.getDate());
 +        if (LOG.isDebugEnabled()) {
 +          LOG.debug("Checking file " + fileName);
 +        }
 +        if (fileDate.getTime() < expireDate.getTime()) {
 +          if (LOG.isDebugEnabled()) {
 +            LOG.debug(fileName + " is an expired file");
 +          }
 +          filesToClean.add(new StoreFile(fs, file.getPath(), conf, cacheConfig, BloomType.NONE));
 +        }
 +      } catch (Exception e) {
 +        LOG.error("Cannot parse the fileName " + fileName, e);
 +      }
 +    }
 +    if (!filesToClean.isEmpty()) {
 +      try {
 +        removeMobFiles(conf, fs, tableName, mobTableDir, columnDescriptor.getName(),
 +            filesToClean);
 +        deletedFileCount = filesToClean.size();
 +      } catch (IOException e) {
 +        LOG.error("Fail to delete the mob files " + filesToClean, e);
 +      }
 +    }
 +    LOG.info(deletedFileCount + " expired mob files are deleted");
 +  }
 +
 +  /**
 +   * Gets the root dir of the mob files.
 +   * It's {HBASE_DIR}/mobdir.
 +   * @param conf The current configuration.
 +   * @return the root dir of the mob file.
 +   */
 +  public static Path getMobHome(Configuration conf) {
 +    Path hbaseDir = new Path(conf.get(HConstants.HBASE_DIR));
 +    return new Path(hbaseDir, MobConstants.MOB_DIR_NAME);
 +  }
 +
 +  /**
 +   * Gets the qualified root dir of the mob files.
 +   * @param conf The current configuration.
 +   * @return The qualified root dir.
 +   * @throws IOException
 +   */
 +  public static Path getQualifiedMobRootDir(Configuration conf) throws IOException {
 +    Path hbaseDir = new Path(conf.get(HConstants.HBASE_DIR));
 +    Path mobRootDir = new Path(hbaseDir, MobConstants.MOB_DIR_NAME);
 +    FileSystem fs = mobRootDir.getFileSystem(conf);
 +    return mobRootDir.makeQualified(fs);
 +  }
 +
 +  /**
 +   * Gets the region dir of the mob files.
 +   * It's {HBASE_DIR}/mobdir/{namespace}/{tableName}/{regionEncodedName}.
 +   * @param conf The current configuration.
 +   * @param tableName The current table name.
 +   * @return The region dir of the mob files.
 +   */
 +  public static Path getMobRegionPath(Configuration conf, TableName tableName) {
 +    Path tablePath = FSUtils.getTableDir(getMobHome(conf), tableName);
 +    HRegionInfo regionInfo = getMobRegionInfo(tableName);
 +    return new Path(tablePath, regionInfo.getEncodedName());
 +  }
 +
 +  /**
 +   * Gets the family dir of the mob files.
 +   * It's {HBASE_DIR}/mobdir/{namespace}/{tableName}/{regionEncodedName}/{columnFamilyName}.
 +   * @param conf The current configuration.
 +   * @param tableName The current table name.
 +   * @param familyName The current family name.
 +   * @return The family dir of the mob files.
 +   */
 +  public static Path getMobFamilyPath(Configuration conf, TableName tableName, String familyName) {
 +    return new Path(getMobRegionPath(conf, tableName), familyName);
 +  }
 +
 +  /**
 +   * Gets the family dir of the mob files.
 +   * It's {HBASE_DIR}/mobdir/{namespace}/{tableName}/{regionEncodedName}/{columnFamilyName}.
 +   * @param regionPath The path of mob region which is a dummy one.
 +   * @param familyName The current family name.
 +   * @return The family dir of the mob files.
 +   */
 +  public static Path getMobFamilyPath(Path regionPath, String familyName) {
 +    return new Path(regionPath, familyName);
 +  }
 +
 +  /**
 +   * Gets the HRegionInfo of the mob files.
 +   * This is a dummy region. The mob files are not saved in a region in HBase.
 +   * This is only used in mob snapshot. It's internally used only.
 +   * @param tableName
 +   * @return A dummy mob region info.
 +   */
 +  public static HRegionInfo getMobRegionInfo(TableName tableName) {
 +    HRegionInfo info = new HRegionInfo(tableName, MobConstants.MOB_REGION_NAME_BYTES,
 +        HConstants.EMPTY_END_ROW, false, 0);
 +    return info;
 +  }
 +
 +  /**
 +   * Gets whether the current HRegionInfo is a mob one.
 +   * @param regionInfo The current HRegionInfo.
 +   * @return If true, the current HRegionInfo is a mob one.
 +   */
 +  public static boolean isMobRegionInfo(HRegionInfo regionInfo) {
 +    return regionInfo == null ? false : getMobRegionInfo(regionInfo.getTable()).getEncodedName()
 +        .equals(regionInfo.getEncodedName());
 +  }
 +
 +  /**
 +   * Gets the working directory of the mob compaction.
 +   * @param root The root directory of the mob compaction.
 +   * @param jobName The current job name.
 +   * @return The directory of the mob compaction for the current job.
 +   */
 +  public static Path getCompactionWorkingPath(Path root, String jobName) {
 +    return new Path(root, jobName);
 +  }
 +
 +  /**
 +   * Archives the mob files.
 +   * @param conf The current configuration.
 +   * @param fs The current file system.
 +   * @param tableName The table name.
 +   * @param tableDir The table directory.
 +   * @param family The name of the column family.
 +   * @param storeFiles The files to be deleted.
 +   * @throws IOException
 +   */
 +  public static void removeMobFiles(Configuration conf, FileSystem fs, TableName tableName,
 +      Path tableDir, byte[] family, Collection<StoreFile> storeFiles) throws IOException {
 +    HFileArchiver.archiveStoreFiles(conf, fs, getMobRegionInfo(tableName), tableDir, family,
 +        storeFiles);
 +  }
 +
 +  /**
 +   * Creates a mob reference KeyValue.
 +   * The value of the mob reference KeyValue is mobCellValueSize + mobFileName.
 +   * @param cell The original Cell.
 +   * @param fileName The mob file name where the mob reference KeyValue is written.
 +   * @param tableNameTag The tag of the current table name. It's very important in
 +   *                        cloning the snapshot.
 +   * @return The mob reference KeyValue.
 +   */
 +  public static KeyValue createMobRefKeyValue(Cell cell, byte[] fileName, Tag tableNameTag) {
 +    // Append the tags to the KeyValue.
 +    // The key is same, the value is the filename of the mob file
 +    List<Tag> tags = new ArrayList<Tag>();
 +    // Add the ref tag as the 1st one.
 +    tags.add(MobConstants.MOB_REF_TAG);
 +    // Add the tag of the source table name, this table is where this mob file is flushed
 +    // from.
 +    // It's very useful in cloning the snapshot. When reading from the cloning table, we need to
 +    // find the original mob files by this table name. For details please see cloning
 +    // snapshot for mob files.
 +    tags.add(tableNameTag);
 +    // Add the existing tags.
 +    tags.addAll(Tag.asList(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength()));
 +    int valueLength = cell.getValueLength();
 +    byte[] refValue = Bytes.add(Bytes.toBytes(valueLength), fileName);
 +    KeyValue reference = new KeyValue(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
 +        cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
 +        cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength(),
 +        cell.getTimestamp(), KeyValue.Type.Put, refValue, 0, refValue.length, tags);
 +    reference.setSequenceId(cell.getSequenceId());
 +    return reference;
 +  }
 +
 +  /**
 +   * Creates a writer for the mob file in temp directory.
 +   * @param conf The current configuration.
 +   * @param fs The current file system.
 +   * @param family The descriptor of the current column family.
 +   * @param date The date string, its format is yyyymmmdd.
 +   * @param basePath The basic path for a temp directory.
 +   * @param maxKeyCount The key count.
 +   * @param compression The compression algorithm.
 +   * @param startKey The hex string of the start key.
 +   * @param cacheConfig The current cache config.
 +   * @return The writer for the mob file.
 +   * @throws IOException
 +   */
 +  public static StoreFile.Writer createWriter(Configuration conf, FileSystem fs,
 +      HColumnDescriptor family, String date, Path basePath, long maxKeyCount,
 +      Compression.Algorithm compression, String startKey, CacheConfig cacheConfig)
 +      throws IOException {
 +    MobFileName mobFileName = MobFileName.create(startKey, date, UUID.randomUUID().toString()
 +        .replaceAll("-", ""));
 +    return createWriter(conf, fs, family, mobFileName, basePath, maxKeyCount, compression,
 +      cacheConfig);
 +  }
 +
 +  /**
 +   * Creates a writer for the ref file in temp directory.
 +   * @param conf The current configuration.
 +   * @param fs The current file system.
 +   * @param family The descriptor of the current column family.
 +   * @param basePath The basic path for a temp directory.
 +   * @param maxKeyCount The key count.
 +   * @param cacheConfig The current cache config.
 +   * @return The writer for the mob file.
 +   * @throws IOException
 +   */
 +  public static StoreFile.Writer createRefFileWriter(Configuration conf, FileSystem fs,
 +    HColumnDescriptor family, Path basePath, long maxKeyCount, CacheConfig cacheConfig)
 +    throws IOException {
 +    HFileContext hFileContext = new HFileContextBuilder().withIncludesMvcc(true)
 +      .withIncludesTags(true).withCompression(family.getCompactionCompression())
 +      .withCompressTags(family.shouldCompressTags()).withChecksumType(HStore.getChecksumType(conf))
 +      .withBytesPerCheckSum(HStore.getBytesPerChecksum(conf)).withBlockSize(family.getBlocksize())
 +      .withHBaseCheckSum(true).withDataBlockEncoding(family.getDataBlockEncoding()).build();
 +    Path tempPath = new Path(basePath, UUID.randomUUID().toString().replaceAll("-", ""));
 +    StoreFile.Writer w = new StoreFile.WriterBuilder(conf, cacheConfig, fs).withFilePath(tempPath)
 +      .withComparator(KeyValue.COMPARATOR).withBloomType(family.getBloomFilterType())
 +      .withMaxKeyCount(maxKeyCount).withFileContext(hFileContext).build();
 +    return w;
 +  }
 +
 +  /**
 +   * Creates a writer for the mob file in temp directory.
 +   * @param conf The current configuration.
 +   * @param fs The current file system.
 +   * @param family The descriptor of the current column family.
 +   * @param date The date string, its format is yyyymmmdd.
 +   * @param basePath The basic path for a temp directory.
 +   * @param maxKeyCount The key count.
 +   * @param compression The compression algorithm.
 +   * @param startKey The start key.
 +   * @param cacheConfig The current cache config.
 +   * @return The writer for the mob file.
 +   * @throws IOException
 +   */
 +  public static StoreFile.Writer createWriter(Configuration conf, FileSystem fs,
 +      HColumnDescriptor family, String date, Path basePath, long maxKeyCount,
 +      Compression.Algorithm compression, byte[] startKey, CacheConfig cacheConfig)
 +      throws IOException {
 +    MobFileName mobFileName = MobFileName.create(startKey, date, UUID.randomUUID().toString()
 +        .replaceAll("-", ""));
 +    return createWriter(conf, fs, family, mobFileName, basePath, maxKeyCount, compression,
 +      cacheConfig);
 +  }
 +
 +  /**
 +   * Creates a writer for the del file in temp directory.
 +   * @param conf The current configuration.
 +   * @param fs The current file system.
 +   * @param family The descriptor of the current column family.
 +   * @param date The date string, its format is yyyymmmdd.
 +   * @param basePath The basic path for a temp directory.
 +   * @param maxKeyCount The key count.
 +   * @param compression The compression algorithm.
 +   * @param startKey The start key.
 +   * @param cacheConfig The current cache config.
 +   * @return The writer for the del file.
 +   * @throws IOException
 +   */
 +  public static StoreFile.Writer createDelFileWriter(Configuration conf, FileSystem fs,
 +      HColumnDescriptor family, String date, Path basePath, long maxKeyCount,
 +      Compression.Algorithm compression, byte[] startKey, CacheConfig cacheConfig)
 +      throws IOException {
 +    String suffix = UUID
 +      .randomUUID().toString().replaceAll("-", "") + "_del";
 +    MobFileName mobFileName = MobFileName.create(startKey, date, suffix);
 +    return createWriter(conf, fs, family, mobFileName, basePath, maxKeyCount, compression,
 +      cacheConfig);
 +  }
 +
 +  /**
 +   * Creates a writer for the del file in temp directory.
 +   * @param conf The current configuration.
 +   * @param fs The current file system.
 +   * @param family The descriptor of the current column family.
 +   * @param mobFileName The mob file name.
 +   * @param basePath The basic path for a temp directory.
 +   * @param maxKeyCount The key count.
 +   * @param compression The compression algorithm.
 +   * @param cacheConfig The current cache config.
 +   * @return The writer for the mob file.
 +   * @throws IOException
 +   */
 +  private static StoreFile.Writer createWriter(Configuration conf, FileSystem fs,
 +    HColumnDescriptor family, MobFileName mobFileName, Path basePath, long maxKeyCount,
 +    Compression.Algorithm compression, CacheConfig cacheConfig) throws IOException {
 +    HFileContext hFileContext = new HFileContextBuilder().withCompression(compression)
 +      .withIncludesMvcc(false).withIncludesTags(true).withChecksumType(HFile.DEFAULT_CHECKSUM_TYPE)
 +      .withBytesPerCheckSum(HFile.DEFAULT_BYTES_PER_CHECKSUM).withBlockSize(family.getBlocksize())
 +      .withHBaseCheckSum(true).withDataBlockEncoding(family.getDataBlockEncoding()).build();
 +
 +    StoreFile.Writer w = new StoreFile.WriterBuilder(conf, cacheConfig, fs)
 +      .withFilePath(new Path(basePath, mobFileName.getFileName()))
 +      .withComparator(KeyValue.COMPARATOR).withBloomType(BloomType.NONE)
 +      .withMaxKeyCount(maxKeyCount).withFileContext(hFileContext).build();
 +    return w;
 +  }
 +
 +  /**
 +   * Commits the mob file.
 +   * @param @param conf The current configuration.
 +   * @param fs The current file system.
 +   * @param path The path where the mob file is saved.
 +   * @param targetPath The directory path where the source file is renamed to.
 +   * @param cacheConfig The current cache config.
 +   * @return The target file path the source file is renamed to.
 +   * @throws IOException
 +   */
 +  public static Path commitFile(Configuration conf, FileSystem fs, final Path sourceFile,
 +      Path targetPath, CacheConfig cacheConfig) throws IOException {
 +    if (sourceFile == null) {
 +      return null;
 +    }
 +    Path dstPath = new Path(targetPath, sourceFile.getName());
 +    validateMobFile(conf, fs, sourceFile, cacheConfig);
 +    String msg = "Renaming flushed file from " + sourceFile + " to " + dstPath;
 +    LOG.info(msg);
 +    Path parent = dstPath.getParent();
 +    if (!fs.exists(parent)) {
 +      fs.mkdirs(parent);
 +    }
 +    if (!fs.rename(sourceFile, dstPath)) {
 +      throw new IOException("Failed rename of " + sourceFile + " to " + dstPath);
 +    }
 +    return dstPath;
 +  }
 +
 +  /**
 +   * Validates a mob file by opening and closing it.
 +   * @param conf The current configuration.
 +   * @param fs The current file system.
 +   * @param path The path where the mob file is saved.
 +   * @param cacheConfig The current cache config.
 +   */
 +  private static void validateMobFile(Configuration conf, FileSystem fs, Path path,
 +      CacheConfig cacheConfig) throws IOException {
 +    StoreFile storeFile = null;
 +    try {
 +      storeFile = new StoreFile(fs, path, conf, cacheConfig, BloomType.NONE);
 +      storeFile.createReader();
 +    } catch (IOException e) {
 +      LOG.error("Fail to open mob file[" + path + "], keep it in temp directory.", e);
 +      throw e;
 +    } finally {
 +      if (storeFile != null) {
 +        storeFile.closeReader(false);
 +      }
 +    }
 +  }
 +
 +  /**
 +   * Indicates whether the current mob ref cell has a valid value.
 +   * A mob ref cell has a mob reference tag.
 +   * The value of a mob ref cell consists of two parts, real mob value length and mob file name.
 +   * The real mob value length takes 4 bytes.
 +   * The remaining part is the mob file name.
 +   * @param cell The mob ref cell.
 +   * @return True if the cell has a valid value.
 +   */
 +  public static boolean hasValidMobRefCellValue(Cell cell) {
 +    return cell.getValueLength() > Bytes.SIZEOF_INT;
 +  }
 +
 +  /**
 +   * Gets the mob value length from the mob ref cell.
 +   * A mob ref cell has a mob reference tag.
 +   * The value of a mob ref cell consists of two parts, real mob value length and mob file name.
 +   * The real mob value length takes 4 bytes.
 +   * The remaining part is the mob file name.
 +   * @param cell The mob ref cell.
 +   * @return The real mob value length.
 +   */
 +  public static int getMobValueLength(Cell cell) {
 +    return Bytes.toInt(cell.getValueArray(), cell.getValueOffset(), Bytes.SIZEOF_INT);
 +  }
 +
 +  /**
 +   * Gets the mob file name from the mob ref cell.
 +   * A mob ref cell has a mob reference tag.
 +   * The value of a mob ref cell consists of two parts, real mob value length and mob file name.
 +   * The real mob value length takes 4 bytes.
 +   * The remaining part is the mob file name.
 +   * @param cell The mob ref cell.
 +   * @return The mob file name.
 +   */
 +  public static String getMobFileName(Cell cell) {
 +    return Bytes.toString(cell.getValueArray(), cell.getValueOffset() + Bytes.SIZEOF_INT,
 +        cell.getValueLength() - Bytes.SIZEOF_INT);
 +  }
 +
 +  /**
 +   * Gets the table name used in the table lock.
 +   * The table lock name is a dummy one, it's not a table name. It's tableName + ".mobLock".
 +   * @param tn The table name.
 +   * @return The table name used in table lock.
 +   */
 +  public static TableName getTableLockName(TableName tn) {
 +    byte[] tableName = tn.getName();
 +    return TableName.valueOf(Bytes.add(tableName, MobConstants.MOB_TABLE_LOCK_SUFFIX));
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/filecompactions/PartitionedMobFileCompactor.java
----------------------------------------------------------------------
diff --cc hbase-server/src/main/java/org/apache/hadoop/hbase/mob/filecompactions/PartitionedMobFileCompactor.java
index 6cd3172,0000000..d6ad143
mode 100644,000000..100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/filecompactions/PartitionedMobFileCompactor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/filecompactions/PartitionedMobFileCompactor.java
@@@ -1,631 -1,0 +1,631 @@@
 +/**
 + *
 + * 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.mob.filecompactions;
 +
 +import java.io.FileNotFoundException;
 +import java.io.IOException;
 +import java.util.ArrayList;
 +import java.util.Collection;
 +import java.util.Collections;
 +import java.util.Date;
 +import java.util.HashMap;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Map.Entry;
 +import java.util.concurrent.Callable;
 +import java.util.concurrent.ExecutorService;
 +import java.util.concurrent.Future;
 +
 +import org.apache.commons.logging.Log;
 +import org.apache.commons.logging.LogFactory;
 +import org.apache.hadoop.classification.InterfaceAudience;
 +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.Cell;
 +import org.apache.hadoop.hbase.HColumnDescriptor;
 +import org.apache.hadoop.hbase.HConstants;
 +import org.apache.hadoop.hbase.KeyValue;
 +import org.apache.hadoop.hbase.KeyValueUtil;
 +import org.apache.hadoop.hbase.TableName;
 +import org.apache.hadoop.hbase.Tag;
 +import org.apache.hadoop.hbase.TagType;
 +import org.apache.hadoop.hbase.client.HTable;
 +import org.apache.hadoop.hbase.client.Scan;
 +import org.apache.hadoop.hbase.io.HFileLink;
 +import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 +import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
 +import org.apache.hadoop.hbase.mob.MobConstants;
 +import org.apache.hadoop.hbase.mob.MobFileName;
 +import org.apache.hadoop.hbase.mob.MobUtils;
 +import org.apache.hadoop.hbase.mob.filecompactions.MobFileCompactionRequest.CompactionType;
 +import org.apache.hadoop.hbase.mob.filecompactions.PartitionedMobFileCompactionRequest.CompactionPartition;
 +import org.apache.hadoop.hbase.mob.filecompactions.PartitionedMobFileCompactionRequest.CompactionPartitionId;
 +import org.apache.hadoop.hbase.regionserver.BloomType;
 +import org.apache.hadoop.hbase.regionserver.HStore;
 +import org.apache.hadoop.hbase.regionserver.ScanInfo;
 +import org.apache.hadoop.hbase.regionserver.ScanType;
 +import org.apache.hadoop.hbase.regionserver.StoreFile;
 +import org.apache.hadoop.hbase.regionserver.StoreFile.Writer;
 +import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
 +import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
 +import org.apache.hadoop.hbase.regionserver.StoreScanner;
 +import org.apache.hadoop.hbase.util.Bytes;
 +import org.apache.hadoop.hbase.util.Pair;
 +
 +/**
 + * An implementation of {@link MobFileCompactor} that compacts the mob files in partitions.
 + */
 +@InterfaceAudience.Private
 +public class PartitionedMobFileCompactor extends MobFileCompactor {
 +
 +  private static final Log LOG = LogFactory.getLog(PartitionedMobFileCompactor.class);
 +  protected long mergeableSize;
 +  protected int delFileMaxCount;
 +  /** The number of files compacted in a batch */
 +  protected int compactionBatchSize;
 +  protected int compactionKVMax;
 +
 +  private Path tempPath;
 +  private Path bulkloadPath;
 +  private CacheConfig compactionCacheConfig;
 +  private Tag tableNameTag;
 +
 +  public PartitionedMobFileCompactor(Configuration conf, FileSystem fs, TableName tableName,
 +    HColumnDescriptor column, ExecutorService pool) {
 +    super(conf, fs, tableName, column, pool);
 +    mergeableSize = conf.getLong(MobConstants.MOB_FILE_COMPACTION_MERGEABLE_THRESHOLD,
 +      MobConstants.DEFAULT_MOB_FILE_COMPACTION_MERGEABLE_THRESHOLD);
 +    delFileMaxCount = conf.getInt(MobConstants.MOB_DELFILE_MAX_COUNT,
 +      MobConstants.DEFAULT_MOB_DELFILE_MAX_COUNT);
 +    // default is 100
 +    compactionBatchSize = conf.getInt(MobConstants.MOB_FILE_COMPACTION_BATCH_SIZE,
 +      MobConstants.DEFAULT_MOB_FILE_COMPACTION_BATCH_SIZE);
 +    tempPath = new Path(MobUtils.getMobHome(conf), MobConstants.TEMP_DIR_NAME);
 +    bulkloadPath = new Path(tempPath, new Path(MobConstants.BULKLOAD_DIR_NAME,
 +      tableName.getNameAsString()));
 +    compactionKVMax = this.conf.getInt(HConstants.COMPACTION_KV_MAX,
 +      HConstants.COMPACTION_KV_MAX_DEFAULT);
 +    Configuration copyOfConf = new Configuration(conf);
 +    copyOfConf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0f);
 +    compactionCacheConfig = new CacheConfig(copyOfConf);
 +    tableNameTag = new Tag(TagType.MOB_TABLE_NAME_TAG_TYPE, tableName.getName());
 +  }
 +
 +  @Override
 +  public List<Path> compact(List<FileStatus> files) throws IOException {
 +    if (files == null || files.isEmpty()) {
 +      return null;
 +    }
 +    // find the files to compact.
 +    PartitionedMobFileCompactionRequest request = select(files);
 +    // compact the files.
 +    return performCompaction(request);
 +  }
 +
 +  /**
 +   * Selects the compacted mob/del files.
 +   * Iterates the candidates to find out all the del files and small mob files.
 +   * @param candidates All the candidates.
 +   * @return A compaction request.
 +   * @throws IOException
 +   */
 +  protected PartitionedMobFileCompactionRequest select(List<FileStatus> candidates)
 +    throws IOException {
 +    Collection<FileStatus> allDelFiles = new ArrayList<FileStatus>();
 +    Map<CompactionPartitionId, CompactionPartition> filesToCompact =
 +      new HashMap<CompactionPartitionId, CompactionPartition>();
 +    int selectedFileCount = 0;
 +    int irrelevantFileCount = 0;
 +    for (FileStatus file : candidates) {
 +      if (!file.isFile()) {
 +        irrelevantFileCount++;
 +        continue;
 +      }
 +      // group the del files and small files.
 +      FileStatus linkedFile = file;
 +      if (HFileLink.isHFileLink(file.getPath())) {
-         HFileLink link = new HFileLink(conf, file.getPath());
++        HFileLink link = HFileLink.buildFromHFileLinkPattern(conf, file.getPath());
 +        linkedFile = getLinkedFileStatus(link);
 +        if (linkedFile == null) {
 +          // If the linked file cannot be found, regard it as an irrelevantFileCount file
 +          irrelevantFileCount++;
 +          continue;
 +        }
 +      }
 +      if (StoreFileInfo.isDelFile(linkedFile.getPath())) {
 +        allDelFiles.add(file);
 +      } else if (linkedFile.getLen() < mergeableSize) {
 +        // add the small files to the merge pool
 +        MobFileName fileName = MobFileName.create(linkedFile.getPath().getName());
 +        CompactionPartitionId id = new CompactionPartitionId(fileName.getStartKey(),
 +          fileName.getDate());
 +        CompactionPartition compactionPartition = filesToCompact.get(id);
 +        if (compactionPartition == null) {
 +          compactionPartition = new CompactionPartition(id);
 +          compactionPartition.addFile(file);
 +          filesToCompact.put(id, compactionPartition);
 +        } else {
 +          compactionPartition.addFile(file);
 +        }
 +        selectedFileCount++;
 +      }
 +    }
 +    PartitionedMobFileCompactionRequest request = new PartitionedMobFileCompactionRequest(
 +      filesToCompact.values(), allDelFiles);
 +    if (candidates.size() == (allDelFiles.size() + selectedFileCount + irrelevantFileCount)) {
 +      // all the files are selected
 +      request.setCompactionType(CompactionType.ALL_FILES);
 +    }
 +    return request;
 +  }
 +
 +  /**
 +   * Performs the compaction on the selected files.
 +   * <ol>
 +   * <li>Compacts the del files.</li>
 +   * <li>Compacts the selected small mob files and all the del files.</li>
 +   * <li>If all the candidates are selected, delete the del files.</li>
 +   * </ol>
 +   * @param request The compaction request.
 +   * @return The paths of new mob files generated in the compaction.
 +   * @throws IOException
 +   */
 +  protected List<Path> performCompaction(PartitionedMobFileCompactionRequest request)
 +    throws IOException {
 +    // merge the del files
 +    List<Path> delFilePaths = new ArrayList<Path>();
 +    for (FileStatus delFile : request.delFiles) {
 +      delFilePaths.add(delFile.getPath());
 +    }
 +    List<Path> newDelPaths = compactDelFiles(request, delFilePaths);
 +    List<StoreFile> newDelFiles = new ArrayList<StoreFile>();
 +    for (Path newDelPath : newDelPaths) {
 +      StoreFile sf = new StoreFile(fs, newDelPath, conf, compactionCacheConfig, BloomType.NONE);
 +      newDelFiles.add(sf);
 +    }
 +    // compact the mob files by partitions.
 +    List<Path> paths = compactMobFiles(request, newDelFiles);
 +    // archive the del files if all the mob files are selected.
 +    if (request.type == CompactionType.ALL_FILES && !newDelPaths.isEmpty()) {
 +      try {
 +        MobUtils.removeMobFiles(conf, fs, tableName, mobTableDir, column.getName(), newDelFiles);
 +      } catch (IOException e) {
 +        LOG.error("Failed to archive the del files " + newDelFiles, e);
 +      }
 +    }
 +    return paths;
 +  }
 +
 +  /**
 +   * Compacts the selected small mob files and all the del files.
 +   * @param request The compaction request.
 +   * @param delFiles The del files.
 +   * @return The paths of new mob files after compactions.
 +   * @throws IOException
 +   */
 +  protected List<Path> compactMobFiles(final PartitionedMobFileCompactionRequest request,
 +    final List<StoreFile> delFiles) throws IOException {
 +    Collection<CompactionPartition> partitions = request.compactionPartitions;
 +    if (partitions == null || partitions.isEmpty()) {
 +      return Collections.emptyList();
 +    }
 +    List<Path> paths = new ArrayList<Path>();
 +    final HTable table = new HTable(conf, tableName);
 +    try {
 +      Map<CompactionPartitionId, Future<List<Path>>> results =
 +        new HashMap<CompactionPartitionId, Future<List<Path>>>();
 +      // compact the mob files by partitions in parallel.
 +      for (final CompactionPartition partition : partitions) {
 +        results.put(partition.getPartitionId(), pool.submit(new Callable<List<Path>>() {
 +          @Override
 +          public List<Path> call() throws Exception {
 +            return compactMobFilePartition(request, partition, delFiles, table);
 +          }
 +        }));
 +      }
 +      // compact the partitions in parallel.
 +      boolean hasFailure = false;
 +      for (Entry<CompactionPartitionId, Future<List<Path>>> result : results.entrySet()) {
 +        try {
 +          paths.addAll(result.getValue().get());
 +        } catch (Exception e) {
 +          // just log the error
 +          LOG.error("Failed to compact the partition " + result.getKey(), e);
 +          hasFailure = true;
 +        }
 +      }
 +      if (hasFailure) {
 +        // if any partition fails in the compaction, directly throw an exception.
 +        throw new IOException("Failed to compact the partitions");
 +      }
 +    } finally {
 +      try {
 +        table.close();
 +      } catch (IOException e) {
 +        LOG.error("Failed to close the HTable", e);
 +      }
 +    }
 +    return paths;
 +  }
 +
 +  /**
 +   * Compacts a partition of selected small mob files and all the del files.
 +   * @param request The compaction request.
 +   * @param partition A compaction partition.
 +   * @param delFiles The del files.
 +   * @param table The current table.
 +   * @return The paths of new mob files after compactions.
 +   * @throws IOException
 +   */
 +  private List<Path> compactMobFilePartition(PartitionedMobFileCompactionRequest request,
 +    CompactionPartition partition, List<StoreFile> delFiles, HTable table) throws IOException {
 +    List<Path> newFiles = new ArrayList<Path>();
 +    List<FileStatus> files = partition.listFiles();
 +    int offset = 0;
 +    Path bulkloadPathOfPartition = new Path(bulkloadPath, partition.getPartitionId().toString());
 +    Path bulkloadColumnPath = new Path(bulkloadPathOfPartition, column.getNameAsString());
 +    while (offset < files.size()) {
 +      int batch = compactionBatchSize;
 +      if (files.size() - offset < compactionBatchSize) {
 +        batch = files.size() - offset;
 +      }
 +      if (batch == 1 && delFiles.isEmpty()) {
 +        // only one file left and no del files, do not compact it,
 +        // and directly add it to the new files.
 +        newFiles.add(files.get(offset).getPath());
 +        offset++;
 +        continue;
 +      }
 +      // clean the bulkload directory to avoid loading old files.
 +      fs.delete(bulkloadPathOfPartition, true);
 +      // add the selected mob files and del files into filesToCompact
 +      List<StoreFile> filesToCompact = new ArrayList<StoreFile>();
 +      for (int i = offset; i < batch + offset; i++) {
 +        StoreFile sf = new StoreFile(fs, files.get(i).getPath(), conf, compactionCacheConfig,
 +          BloomType.NONE);
 +        filesToCompact.add(sf);
 +      }
 +      filesToCompact.addAll(delFiles);
 +      // compact the mob files in a batch.
 +      compactMobFilesInBatch(request, partition, table, filesToCompact, batch,
 +        bulkloadPathOfPartition, bulkloadColumnPath, newFiles);
 +      // move to the next batch.
 +      offset += batch;
 +    }
 +    return newFiles;
 +  }
 +
 +  /**
 +   * Compacts a partition of selected small mob files and all the del files in a batch.
 +   * @param request The compaction request.
 +   * @param partition A compaction partition.
 +   * @param table The current table.
 +   * @param filesToCompact The files to be compacted.
 +   * @param batch The number of mob files to be compacted in a batch.
 +   * @param bulkloadPathOfPartition The directory where the bulkload column of the current
 +   *        partition is saved.
 +   * @param bulkloadColumnPath The directory where the bulkload files of current partition
 +   *        are saved.
 +   * @param newFiles The paths of new mob files after compactions.
 +   * @throws IOException
 +   */
 +  private void compactMobFilesInBatch(PartitionedMobFileCompactionRequest request,
 +    CompactionPartition partition, HTable table, List<StoreFile> filesToCompact, int batch,
 +    Path bulkloadPathOfPartition, Path bulkloadColumnPath, List<Path> newFiles)
 +    throws IOException {
 +    // open scanner to the selected mob files and del files.
 +    StoreScanner scanner = createScanner(filesToCompact, ScanType.COMPACT_DROP_DELETES);
 +    // the mob files to be compacted, not include the del files.
 +    List<StoreFile> mobFilesToCompact = filesToCompact.subList(0, batch);
 +    // Pair(maxSeqId, cellsCount)
 +    Pair<Long, Long> fileInfo = getFileInfo(mobFilesToCompact);
 +    // open writers for the mob files and new ref store files.
 +    Writer writer = null;
 +    Writer refFileWriter = null;
 +    Path filePath = null;
 +    Path refFilePath = null;
 +    long mobCells = 0;
 +    try {
 +      writer = MobUtils.createWriter(conf, fs, column, partition.getPartitionId().getDate(),
 +        tempPath, Long.MAX_VALUE, column.getCompactionCompression(), partition.getPartitionId()
 +          .getStartKey(), compactionCacheConfig);
 +      filePath = writer.getPath();
 +      byte[] fileName = Bytes.toBytes(filePath.getName());
 +      // create a temp file and open a writer for it in the bulkloadPath
 +      refFileWriter = MobUtils.createRefFileWriter(conf, fs, column, bulkloadColumnPath, fileInfo
 +        .getSecond().longValue(), compactionCacheConfig);
 +      refFilePath = refFileWriter.getPath();
 +      List<Cell> cells = new ArrayList<Cell>();
 +      boolean hasMore = false;
 +      do {
 +        hasMore = scanner.next(cells, compactionKVMax);
 +        for (Cell cell : cells) {
 +          // TODO remove this after the new code are introduced.
 +          KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
 +          // write the mob cell to the mob file.
 +          writer.append(kv);
 +          // write the new reference cell to the store file.
 +          KeyValue reference = MobUtils.createMobRefKeyValue(kv, fileName, tableNameTag);
 +          refFileWriter.append(reference);
 +          mobCells++;
 +        }
 +        cells.clear();
 +      } while (hasMore);
 +    } finally {
 +      // close the scanner.
 +      scanner.close();
 +      // append metadata to the mob file, and close the mob file writer.
 +      closeMobFileWriter(writer, fileInfo.getFirst(), mobCells);
 +      // append metadata and bulkload info to the ref mob file, and close the writer.
 +      closeRefFileWriter(refFileWriter, fileInfo.getFirst(), request.selectionTime);
 +    }
 +    if (mobCells > 0) {
 +      // commit mob file
 +      MobUtils.commitFile(conf, fs, filePath, mobFamilyDir, compactionCacheConfig);
 +      // bulkload the ref file
 +      bulkloadRefFile(table, bulkloadPathOfPartition, filePath.getName());
 +      newFiles.add(new Path(mobFamilyDir, filePath.getName()));
 +    } else {
 +      // remove the new files
 +      // the mob file is empty, delete it instead of committing.
 +      deletePath(filePath);
 +      // the ref file is empty, delete it instead of committing.
 +      deletePath(refFilePath);
 +    }
 +    // archive the old mob files, do not archive the del files.
 +    try {
 +      MobUtils
 +        .removeMobFiles(conf, fs, tableName, mobTableDir, column.getName(), mobFilesToCompact);
 +    } catch (IOException e) {
 +      LOG.error("Failed to archive the files " + mobFilesToCompact, e);
 +    }
 +  }
 +
 +  /**
 +   * Compacts the del files in batches which avoids opening too many files.
 +   * @param request The compaction request.
 +   * @param delFilePaths
 +   * @return The paths of new del files after merging or the original files if no merging
 +   *         is necessary.
 +   * @throws IOException
 +   */
 +  protected List<Path> compactDelFiles(PartitionedMobFileCompactionRequest request,
 +    List<Path> delFilePaths) throws IOException {
 +    if (delFilePaths.size() <= delFileMaxCount) {
 +      return delFilePaths;
 +    }
 +    // when there are more del files than the number that is allowed, merge it firstly.
 +    int offset = 0;
 +    List<Path> paths = new ArrayList<Path>();
 +    while (offset < delFilePaths.size()) {
 +      // get the batch
 +      int batch = compactionBatchSize;
 +      if (delFilePaths.size() - offset < compactionBatchSize) {
 +        batch = delFilePaths.size() - offset;
 +      }
 +      List<StoreFile> batchedDelFiles = new ArrayList<StoreFile>();
 +      if (batch == 1) {
 +        // only one file left, do not compact it, directly add it to the new files.
 +        paths.add(delFilePaths.get(offset));
 +        offset++;
 +        continue;
 +      }
 +      for (int i = offset; i < batch + offset; i++) {
 +        batchedDelFiles.add(new StoreFile(fs, delFilePaths.get(i), conf, compactionCacheConfig,
 +          BloomType.NONE));
 +      }
 +      // compact the del files in a batch.
 +      paths.add(compactDelFilesInBatch(request, batchedDelFiles));
 +      // move to the next batch.
 +      offset += batch;
 +    }
 +    return compactDelFiles(request, paths);
 +  }
 +
 +  /**
 +   * Compacts the del file in a batch.
 +   * @param request The compaction request.
 +   * @param delFiles The del files.
 +   * @return The path of new del file after merging.
 +   * @throws IOException
 +   */
 +  private Path compactDelFilesInBatch(PartitionedMobFileCompactionRequest request,
 +    List<StoreFile> delFiles) throws IOException {
 +    // create a scanner for the del files.
 +    StoreScanner scanner = createScanner(delFiles, ScanType.COMPACT_RETAIN_DELETES);
 +    Writer writer = null;
 +    Path filePath = null;
 +    try {
 +      writer = MobUtils.createDelFileWriter(conf, fs, column,
 +        MobUtils.formatDate(new Date(request.selectionTime)), tempPath, Long.MAX_VALUE,
 +        column.getCompactionCompression(), HConstants.EMPTY_START_ROW, compactionCacheConfig);
 +      filePath = writer.getPath();
 +      List<Cell> cells = new ArrayList<Cell>();
 +      boolean hasMore = false;
 +      do {
 +        hasMore = scanner.next(cells, compactionKVMax);
 +        for (Cell cell : cells) {
 +          // TODO remove this after the new code are introduced.
 +          KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
 +          writer.append(kv);
 +        }
 +        cells.clear();
 +      } while (hasMore);
 +    } finally {
 +      scanner.close();
 +      if (writer != null) {
 +        try {
 +          writer.close();
 +        } catch (IOException e) {
 +          LOG.error("Failed to close the writer of the file " + filePath, e);
 +        }
 +      }
 +    }
 +    // commit the new del file
 +    Path path = MobUtils.commitFile(conf, fs, filePath, mobFamilyDir, compactionCacheConfig);
 +    // archive the old del files
 +    try {
 +      MobUtils.removeMobFiles(conf, fs, tableName, mobTableDir, column.getName(), delFiles);
 +    } catch (IOException e) {
 +      LOG.error("Failed to archive the old del files " + delFiles, e);
 +    }
 +    return path;
 +  }
 +
 +  /**
 +   * Creates a store scanner.
 +   * @param filesToCompact The files to be compacted.
 +   * @param scanType The scan type.
 +   * @return The store scanner.
 +   * @throws IOException
 +   */
 +  private StoreScanner createScanner(List<StoreFile> filesToCompact, ScanType scanType)
 +    throws IOException {
 +    List scanners = StoreFileScanner.getScannersForStoreFiles(filesToCompact, false, true, false,
 +      null, HConstants.LATEST_TIMESTAMP);
 +    Scan scan = new Scan();
 +    scan.setMaxVersions(column.getMaxVersions());
 +    long ttl = HStore.determineTTLFromFamily(column);
 +    ScanInfo scanInfo = new ScanInfo(column, ttl, 0, KeyValue.COMPARATOR);
 +    StoreScanner scanner = new StoreScanner(scan, scanInfo, scanType, null, scanners, 0L,
 +      HConstants.LATEST_TIMESTAMP);
 +    return scanner;
 +  }
 +
 +  /**
 +   * Bulkloads the current file.
 +   * @param table The current table.
 +   * @param bulkloadDirectory The path of bulkload directory.
 +   * @param fileName The current file name.
 +   * @throws IOException
 +   */
 +  private void bulkloadRefFile(HTable table, Path bulkloadDirectory, String fileName)
 +    throws IOException {
 +    // bulkload the ref file
 +    try {
 +      LoadIncrementalHFiles bulkload = new LoadIncrementalHFiles(conf);
 +      bulkload.doBulkLoad(bulkloadDirectory, table);
 +    } catch (Exception e) {
 +      // delete the committed mob file
 +      deletePath(new Path(mobFamilyDir, fileName));
 +      throw new IOException(e);
 +    } finally {
 +      // delete the bulkload files in bulkloadPath
 +      deletePath(bulkloadDirectory);
 +    }
 +  }
 +
 +  /**
 +   * Closes the mob file writer.
 +   * @param writer The mob file writer.
 +   * @param maxSeqId Maximum sequence id.
 +   * @param mobCellsCount The number of mob cells.
 +   * @throws IOException
 +   */
 +  private void closeMobFileWriter(Writer writer, long maxSeqId, long mobCellsCount)
 +    throws IOException {
 +    if (writer != null) {
 +      writer.appendMetadata(maxSeqId, false, mobCellsCount);
 +      try {
 +        writer.close();
 +      } catch (IOException e) {
 +        LOG.error("Failed to close the writer of the file " + writer.getPath(), e);
 +      }
 +    }
 +  }
 +
 +  /**
 +   * Closes the ref file writer.
 +   * @param writer The ref file writer.
 +   * @param maxSeqId Maximum sequence id.
 +   * @param bulkloadTime The timestamp at which the bulk load file is created.
 +   * @throws IOException
 +   */
 +  private void closeRefFileWriter(Writer writer, long maxSeqId, long bulkloadTime)
 +    throws IOException {
 +    if (writer != null) {
 +      writer.appendMetadata(maxSeqId, false);
 +      writer.appendFileInfo(StoreFile.BULKLOAD_TIME_KEY, Bytes.toBytes(bulkloadTime));
 +      try {
 +        writer.close();
 +      } catch (IOException e) {
 +        LOG.error("Failed to close the writer of the ref file " + writer.getPath(), e);
 +      }
 +    }
 +  }
 +
 +  /**
 +   * Gets the max seqId and number of cells of the store files.
 +   * @param storeFiles The store files.
 +   * @return The pair of the max seqId and number of cells of the store files.
 +   * @throws IOException
 +   */
 +  private Pair<Long, Long> getFileInfo(List<StoreFile> storeFiles) throws IOException {
 +    long maxSeqId = 0;
 +    long maxKeyCount = 0;
 +    for (StoreFile sf : storeFiles) {
 +      // the readers will be closed later after the merge.
 +      maxSeqId = Math.max(maxSeqId, sf.getMaxSequenceId());
 +      byte[] count = sf.createReader().loadFileInfo().get(StoreFile.MOB_CELLS_COUNT);
 +      if (count != null) {
 +        maxKeyCount += Bytes.toLong(count);
 +      }
 +    }
 +    return new Pair<Long, Long>(Long.valueOf(maxSeqId), Long.valueOf(maxKeyCount));
 +  }
 +
 +  /**
 +   * Deletes a file.
 +   * @param path The path of the file to be deleted.
 +   */
 +  private void deletePath(Path path) {
 +    try {
 +      if (path != null) {
 +        fs.delete(path, true);
 +      }
 +    } catch (IOException e) {
 +      LOG.error("Failed to delete the file " + path, e);
 +    }
 +  }
 +
 +  private FileStatus getLinkedFileStatus(HFileLink link) throws IOException {
 +    Path[] locations = link.getLocations();
 +    for (Path location : locations) {
 +      FileStatus file = getFileStatus(location);
 +      if (file != null) {
 +        return file;
 +      }
 +    }
 +    return null;
 +  }
 +
 +  private FileStatus getFileStatus(Path path) throws IOException {
 +    try {
 +      if (path != null) {
 +        FileStatus file = fs.getFileStatus(path);
 +        return file;
 +      }
 +    } catch (FileNotFoundException e) {
 +      LOG.warn("The file " + path + " can not be found", e);
 +    }
 +    return null;
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/SweepMapper.java
----------------------------------------------------------------------
diff --cc hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/SweepMapper.java
index 56e5726,0000000..559d6db
mode 100644,000000..100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/SweepMapper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/SweepMapper.java
@@@ -1,85 -1,0 +1,87 @@@
 +/**
 + *
 + * 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.mob.mapreduce;
 +
 +import java.io.IOException;
 +
 +import org.apache.hadoop.classification.InterfaceAudience;
++import org.apache.hadoop.hbase.Cell;
 +import org.apache.hadoop.hbase.KeyValue;
++import org.apache.hadoop.hbase.KeyValueUtil;
 +import org.apache.hadoop.hbase.client.Result;
 +import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 +import org.apache.hadoop.hbase.mapreduce.TableMapper;
 +import org.apache.hadoop.hbase.mob.MobUtils;
 +import org.apache.hadoop.hbase.mob.mapreduce.SweepJob.DummyMobAbortable;
 +import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 +import org.apache.hadoop.io.Text;
 +import org.apache.zookeeper.KeeperException;
 +
 +/**
 + * The mapper of a sweep job.
 + * Takes the rows from the table and their results and map to <filename:Text, mobValue:KeyValue>
 + * where mobValue is the actual cell in HBase.
 + */
 +@InterfaceAudience.Private
 +public class SweepMapper extends TableMapper<Text, KeyValue> {
 +
 +  private ZooKeeperWatcher zkw = null;
 +
 +  @Override
 +  protected void setup(Context context) throws IOException,
 +      InterruptedException {
 +    String id = context.getConfiguration().get(SweepJob.SWEEP_JOB_ID);
 +    String owner = context.getConfiguration().get(SweepJob.SWEEP_JOB_SERVERNAME);
 +    String sweeperNode = context.getConfiguration().get(SweepJob.SWEEP_JOB_TABLE_NODE);
 +    zkw = new ZooKeeperWatcher(context.getConfiguration(), id,
 +        new DummyMobAbortable());
 +    try {
 +      SweepJobNodeTracker tracker = new SweepJobNodeTracker(zkw, sweeperNode, owner);
 +      tracker.start();
 +    } catch (KeeperException e) {
 +      throw new IOException(e);
 +    }
 +  }
 +
 +  @Override
 +  protected void cleanup(Context context) throws IOException,
 +      InterruptedException {
 +    if (zkw != null) {
 +      zkw.close();
 +    }
 +  }
 +
 +  @Override
 +  public void map(ImmutableBytesWritable r, Result columns, Context context) throws IOException,
 +      InterruptedException {
 +    if (columns == null) {
 +      return;
 +    }
-     KeyValue[] kvList = columns.raw();
-     if (kvList == null || kvList.length == 0) {
++    Cell[] cells = columns.rawCells();
++    if (cells == null || cells.length == 0) {
 +      return;
 +    }
-     for (KeyValue kv : kvList) {
-       if (MobUtils.hasValidMobRefCellValue(kv)) {
-         String fileName = MobUtils.getMobFileName(kv);
-         context.write(new Text(fileName), kv);
++    for (Cell c : cells) {
++      if (MobUtils.hasValidMobRefCellValue(c)) {
++        String fileName = MobUtils.getMobFileName(c);
++        context.write(new Text(fileName), KeyValueUtil.ensureKeyValue(c));
 +      }
 +    }
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionStateListener.java~HEAD
----------------------------------------------------------------------
diff --cc hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionStateListener.java~HEAD
index 0000000,0000000..6b954ac
new file mode 100644
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionStateListener.java~HEAD
@@@ -1,0 -1,0 +1,54 @@@
++/**
++ * 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.quotas;
++
++import java.io.IOException;
++
++import org.apache.hadoop.classification.InterfaceAudience;
++import org.apache.hadoop.hbase.HRegionInfo;
++
++/**
++ * The listener interface for receiving region state events.
++ */
++@InterfaceAudience.Private
++public interface RegionStateListener {
++
++  /**
++   * Process region split event.
++   *
++   * @param hri An instance of HRegionInfo
++   * @throws IOException
++   */
++  void onRegionSplit(HRegionInfo hri) throws IOException;
++
++  /**
++   * Process region split reverted event.
++   *
++   * @param hri An instance of HRegionInfo
++   * @throws IOException Signals that an I/O exception has occurred.
++   */
++  void onRegionSplitReverted(HRegionInfo hri) throws IOException;
++
++  /**
++   * Process region merge event.
++   *
++   * @param hri An instance of HRegionInfo
++   * @throws IOException
++   */
++  void onRegionMerged(HRegionInfo hri) throws IOException;
++}

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionStateListener.java~HEAD_0
----------------------------------------------------------------------
diff --cc hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionStateListener.java~HEAD_0
index 0000000,0000000..6b954ac
new file mode 100644
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionStateListener.java~HEAD_0
@@@ -1,0 -1,0 +1,54 @@@
++/**
++ * 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.quotas;
++
++import java.io.IOException;
++
++import org.apache.hadoop.classification.InterfaceAudience;
++import org.apache.hadoop.hbase.HRegionInfo;
++
++/**
++ * The listener interface for receiving region state events.
++ */
++@InterfaceAudience.Private
++public interface RegionStateListener {
++
++  /**
++   * Process region split event.
++   *
++   * @param hri An instance of HRegionInfo
++   * @throws IOException
++   */
++  void onRegionSplit(HRegionInfo hri) throws IOException;
++
++  /**
++   * Process region split reverted event.
++   *
++   * @param hri An instance of HRegionInfo
++   * @throws IOException Signals that an I/O exception has occurred.
++   */
++  void onRegionSplitReverted(HRegionInfo hri) throws IOException;
++
++  /**
++   * Process region merge event.
++   *
++   * @param hri An instance of HRegionInfo
++   * @throws IOException
++   */
++  void onRegionMerged(HRegionInfo hri) throws IOException;
++}

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionStateListener.java~jon_master
----------------------------------------------------------------------
diff --cc hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionStateListener.java~jon_master
index 0000000,0000000..6b954ac
new file mode 100644
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionStateListener.java~jon_master
@@@ -1,0 -1,0 +1,54 @@@
++/**
++ * 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.quotas;
++
++import java.io.IOException;
++
++import org.apache.hadoop.classification.InterfaceAudience;
++import org.apache.hadoop.hbase.HRegionInfo;
++
++/**
++ * The listener interface for receiving region state events.
++ */
++@InterfaceAudience.Private
++public interface RegionStateListener {
++
++  /**
++   * Process region split event.
++   *
++   * @param hri An instance of HRegionInfo
++   * @throws IOException
++   */
++  void onRegionSplit(HRegionInfo hri) throws IOException;
++
++  /**
++   * Process region split reverted event.
++   *
++   * @param hri An instance of HRegionInfo
++   * @throws IOException Signals that an I/O exception has occurred.
++   */
++  void onRegionSplitReverted(HRegionInfo hri) throws IOException;
++
++  /**
++   * Process region merge event.
++   *
++   * @param hri An instance of HRegionInfo
++   * @throws IOException
++   */
++  void onRegionMerged(HRegionInfo hri) throws IOException;
++}

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionStateListener.java~master
----------------------------------------------------------------------
diff --cc hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionStateListener.java~master
index 0000000,0000000..6b954ac
new file mode 100644
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionStateListener.java~master
@@@ -1,0 -1,0 +1,54 @@@
++/**
++ * 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.quotas;
++
++import java.io.IOException;
++
++import org.apache.hadoop.classification.InterfaceAudience;
++import org.apache.hadoop.hbase.HRegionInfo;
++
++/**
++ * The listener interface for receiving region state events.
++ */
++@InterfaceAudience.Private
++public interface RegionStateListener {
++
++  /**
++   * Process region split event.
++   *
++   * @param hri An instance of HRegionInfo
++   * @throws IOException
++   */
++  void onRegionSplit(HRegionInfo hri) throws IOException;
++
++  /**
++   * Process region split reverted event.
++   *
++   * @param hri An instance of HRegionInfo
++   * @throws IOException Signals that an I/O exception has occurred.
++   */
++  void onRegionSplitReverted(HRegionInfo hri) throws IOException;
++
++  /**
++   * Process region merge event.
++   *
++   * @param hri An instance of HRegionInfo
++   * @throws IOException
++   */
++  void onRegionMerged(HRegionInfo hri) throws IOException;
++}

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreEngine.java
----------------------------------------------------------------------
diff --cc hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreEngine.java
index 4afa80c,3c1345d..d55822d
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreEngine.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreEngine.java
@@@ -62,29 -64,19 +64,30 @@@ public class DefaultStoreEngine extend
    @Override
    protected void createComponents(
        Configuration conf, Store store, KVComparator kvComparator) throws IOException {
-     storeFileManager = new DefaultStoreFileManager(kvComparator, conf);
 +    createCompactor(conf, store);
 +    createCompactionPolicy(conf, store);
 +    createStoreFlusher(conf, store);
++    storeFileManager = new DefaultStoreFileManager(kvComparator, conf, compactionPolicy.getConf());
++
 +  }
 +
 +  protected void createCompactor(Configuration conf, Store store) throws IOException {
      String className = conf.get(DEFAULT_COMPACTOR_CLASS_KEY, DEFAULT_COMPACTOR_CLASS.getName());
      try {
        compactor = ReflectionUtils.instantiateWithCustomCtor(className,
--          new Class[] { Configuration.class, Store.class }, new Object[] { conf, store });
++              new Class[]{Configuration.class, Store.class}, new Object[]{conf, store});
      } catch (Exception e) {
        throw new IOException("Unable to load configured compactor '" + className + "'", e);
      }
 -    className = conf.get(
 -        DEFAULT_COMPACTION_POLICY_CLASS_KEY, DEFAULT_COMPACTION_POLICY_CLASS.getName());
 +  }
 +
 +  protected void createCompactionPolicy(Configuration conf, Store store) throws IOException {
 +    String className = conf.get(
-         DEFAULT_COMPACTION_POLICY_CLASS_KEY, DEFAULT_COMPACTION_POLICY_CLASS.getName());
++            DEFAULT_COMPACTION_POLICY_CLASS_KEY, DEFAULT_COMPACTION_POLICY_CLASS.getName());
      try {
        compactionPolicy = ReflectionUtils.instantiateWithCustomCtor(className,
--          new Class[] { Configuration.class, StoreConfigInformation.class },
--          new Object[] { conf, store });
++              new Class[]{Configuration.class, StoreConfigInformation.class},
++              new Object[]{conf, store});
      } catch (Exception e) {
        throw new IOException("Unable to load configured compaction policy '" + className + "'", e);
      }
@@@ -101,7 -91,7 +104,6 @@@
      }
    }
  
--
    @Override
    public CompactionContext createCompaction() {
      return new DefaultCompactionContext();


[17/50] [abbrv] hbase git commit: HBASE-11567 Write bulk load COMMIT events to WAL (Alex Newman, Jeffrey Zhong)

Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/3f442773/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/WALProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/WALProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/WALProtos.java
index 977db42..c9fa854 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/WALProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/WALProtos.java
@@ -7604,124 +7604,101 @@ public final class WALProtos {
     // @@protoc_insertion_point(class_scope:FlushDescriptor)
   }
 
-  public interface RegionEventDescriptorOrBuilder
+  public interface StoreDescriptorOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
 
-    // required .RegionEventDescriptor.EventType event_type = 1;
-    /**
-     * <code>required .RegionEventDescriptor.EventType event_type = 1;</code>
-     */
-    boolean hasEventType();
-    /**
-     * <code>required .RegionEventDescriptor.EventType event_type = 1;</code>
-     */
-    org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.EventType getEventType();
-
-    // required bytes table_name = 2;
+    // required bytes family_name = 1;
     /**
-     * <code>required bytes table_name = 2;</code>
+     * <code>required bytes family_name = 1;</code>
      */
-    boolean hasTableName();
+    boolean hasFamilyName();
     /**
-     * <code>required bytes table_name = 2;</code>
+     * <code>required bytes family_name = 1;</code>
      */
-    com.google.protobuf.ByteString getTableName();
+    com.google.protobuf.ByteString getFamilyName();
 
-    // required bytes encoded_region_name = 3;
-    /**
-     * <code>required bytes encoded_region_name = 3;</code>
-     */
-    boolean hasEncodedRegionName();
+    // required string store_home_dir = 2;
     /**
-     * <code>required bytes encoded_region_name = 3;</code>
+     * <code>required string store_home_dir = 2;</code>
+     *
+     * <pre>
+     *relative to region dir
+     * </pre>
      */
-    com.google.protobuf.ByteString getEncodedRegionName();
-
-    // optional uint64 log_sequence_number = 4;
+    boolean hasStoreHomeDir();
     /**
-     * <code>optional uint64 log_sequence_number = 4;</code>
+     * <code>required string store_home_dir = 2;</code>
+     *
+     * <pre>
+     *relative to region dir
+     * </pre>
      */
-    boolean hasLogSequenceNumber();
+    java.lang.String getStoreHomeDir();
     /**
-     * <code>optional uint64 log_sequence_number = 4;</code>
+     * <code>required string store_home_dir = 2;</code>
+     *
+     * <pre>
+     *relative to region dir
+     * </pre>
      */
-    long getLogSequenceNumber();
+    com.google.protobuf.ByteString
+        getStoreHomeDirBytes();
 
-    // repeated .RegionEventDescriptor.StoreDescriptor stores = 5;
-    /**
-     * <code>repeated .RegionEventDescriptor.StoreDescriptor stores = 5;</code>
-     */
-    java.util.List<org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor> 
-        getStoresList();
-    /**
-     * <code>repeated .RegionEventDescriptor.StoreDescriptor stores = 5;</code>
-     */
-    org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor getStores(int index);
-    /**
-     * <code>repeated .RegionEventDescriptor.StoreDescriptor stores = 5;</code>
-     */
-    int getStoresCount();
+    // repeated string store_file = 3;
     /**
-     * <code>repeated .RegionEventDescriptor.StoreDescriptor stores = 5;</code>
-     */
-    java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptorOrBuilder> 
-        getStoresOrBuilderList();
-    /**
-     * <code>repeated .RegionEventDescriptor.StoreDescriptor stores = 5;</code>
+     * <code>repeated string store_file = 3;</code>
+     *
+     * <pre>
+     * relative to store dir
+     * </pre>
      */
-    org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptorOrBuilder getStoresOrBuilder(
-        int index);
-
-    // optional .ServerName server = 6;
+    java.util.List<java.lang.String>
+    getStoreFileList();
     /**
-     * <code>optional .ServerName server = 6;</code>
+     * <code>repeated string store_file = 3;</code>
      *
      * <pre>
-     * Server who opened the region
+     * relative to store dir
      * </pre>
      */
-    boolean hasServer();
+    int getStoreFileCount();
     /**
-     * <code>optional .ServerName server = 6;</code>
+     * <code>repeated string store_file = 3;</code>
      *
      * <pre>
-     * Server who opened the region
+     * relative to store dir
      * </pre>
      */
-    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName getServer();
+    java.lang.String getStoreFile(int index);
     /**
-     * <code>optional .ServerName server = 6;</code>
+     * <code>repeated string store_file = 3;</code>
      *
      * <pre>
-     * Server who opened the region
+     * relative to store dir
      * </pre>
      */
-    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServerOrBuilder();
+    com.google.protobuf.ByteString
+        getStoreFileBytes(int index);
   }
   /**
-   * Protobuf type {@code RegionEventDescriptor}
-   *
-   * <pre>
-   **
-   * Special WAL entry to hold all related to a region event (open/close).
-   * </pre>
+   * Protobuf type {@code StoreDescriptor}
    */
-  public static final class RegionEventDescriptor extends
+  public static final class StoreDescriptor extends
       com.google.protobuf.GeneratedMessage
-      implements RegionEventDescriptorOrBuilder {
-    // Use RegionEventDescriptor.newBuilder() to construct.
-    private RegionEventDescriptor(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      implements StoreDescriptorOrBuilder {
+    // Use StoreDescriptor.newBuilder() to construct.
+    private StoreDescriptor(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
       super(builder);
       this.unknownFields = builder.getUnknownFields();
     }
-    private RegionEventDescriptor(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+    private StoreDescriptor(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
 
-    private static final RegionEventDescriptor defaultInstance;
-    public static RegionEventDescriptor getDefaultInstance() {
+    private static final StoreDescriptor defaultInstance;
+    public static StoreDescriptor getDefaultInstance() {
       return defaultInstance;
     }
 
-    public RegionEventDescriptor getDefaultInstanceForType() {
+    public StoreDescriptor getDefaultInstanceForType() {
       return defaultInstance;
     }
 
@@ -7731,7 +7708,7 @@ public final class WALProtos {
         getUnknownFields() {
       return this.unknownFields;
     }
-    private RegionEventDescriptor(
+    private StoreDescriptor(
         com.google.protobuf.CodedInputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
@@ -7754,51 +7731,22 @@ public final class WALProtos {
               }
               break;
             }
-            case 8: {
-              int rawValue = input.readEnum();
-              org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.EventType value = org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.EventType.valueOf(rawValue);
-              if (value == null) {
-                unknownFields.mergeVarintField(1, rawValue);
-              } else {
-                bitField0_ |= 0x00000001;
-                eventType_ = value;
-              }
+            case 10: {
+              bitField0_ |= 0x00000001;
+              familyName_ = input.readBytes();
               break;
             }
             case 18: {
               bitField0_ |= 0x00000002;
-              tableName_ = input.readBytes();
+              storeHomeDir_ = input.readBytes();
               break;
             }
             case 26: {
-              bitField0_ |= 0x00000004;
-              encodedRegionName_ = input.readBytes();
-              break;
-            }
-            case 32: {
-              bitField0_ |= 0x00000008;
-              logSequenceNumber_ = input.readUInt64();
-              break;
-            }
-            case 42: {
-              if (!((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
-                stores_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor>();
-                mutable_bitField0_ |= 0x00000010;
-              }
-              stores_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor.PARSER, extensionRegistry));
-              break;
-            }
-            case 50: {
-              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder subBuilder = null;
-              if (((bitField0_ & 0x00000010) == 0x00000010)) {
-                subBuilder = server_.toBuilder();
-              }
-              server_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.PARSER, extensionRegistry);
-              if (subBuilder != null) {
-                subBuilder.mergeFrom(server_);
-                server_ = subBuilder.buildPartial();
+              if (!((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
+                storeFile_ = new com.google.protobuf.LazyStringArrayList();
+                mutable_bitField0_ |= 0x00000004;
               }
-              bitField0_ |= 0x00000010;
+              storeFile_.add(input.readBytes());
               break;
             }
           }
@@ -7809,8 +7757,8 @@ public final class WALProtos {
         throw new com.google.protobuf.InvalidProtocolBufferException(
             e.getMessage()).setUnfinishedMessage(this);
       } finally {
-        if (((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
-          stores_ = java.util.Collections.unmodifiableList(stores_);
+        if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
+          storeFile_ = new com.google.protobuf.UnmodifiableLazyStringList(storeFile_);
         }
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
@@ -7818,1036 +7766,2238 @@ public final class WALProtos {
     }
     public static final com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
-      return org.apache.hadoop.hbase.protobuf.generated.WALProtos.internal_static_RegionEventDescriptor_descriptor;
+      return org.apache.hadoop.hbase.protobuf.generated.WALProtos.internal_static_StoreDescriptor_descriptor;
     }
 
     protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
         internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hbase.protobuf.generated.WALProtos.internal_static_RegionEventDescriptor_fieldAccessorTable
+      return org.apache.hadoop.hbase.protobuf.generated.WALProtos.internal_static_StoreDescriptor_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
-              org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.class, org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.Builder.class);
+              org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor.class, org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor.Builder.class);
     }
 
-    public static com.google.protobuf.Parser<RegionEventDescriptor> PARSER =
-        new com.google.protobuf.AbstractParser<RegionEventDescriptor>() {
-      public RegionEventDescriptor parsePartialFrom(
+    public static com.google.protobuf.Parser<StoreDescriptor> PARSER =
+        new com.google.protobuf.AbstractParser<StoreDescriptor>() {
+      public StoreDescriptor parsePartialFrom(
           com.google.protobuf.CodedInputStream input,
           com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws com.google.protobuf.InvalidProtocolBufferException {
-        return new RegionEventDescriptor(input, extensionRegistry);
+        return new StoreDescriptor(input, extensionRegistry);
       }
     };
 
     @java.lang.Override
-    public com.google.protobuf.Parser<RegionEventDescriptor> getParserForType() {
+    public com.google.protobuf.Parser<StoreDescriptor> getParserForType() {
       return PARSER;
     }
 
+    private int bitField0_;
+    // required bytes family_name = 1;
+    public static final int FAMILY_NAME_FIELD_NUMBER = 1;
+    private com.google.protobuf.ByteString familyName_;
     /**
-     * Protobuf enum {@code RegionEventDescriptor.EventType}
+     * <code>required bytes family_name = 1;</code>
      */
-    public enum EventType
-        implements com.google.protobuf.ProtocolMessageEnum {
-      /**
-       * <code>REGION_OPEN = 0;</code>
-       */
-      REGION_OPEN(0, 0),
-      /**
-       * <code>REGION_CLOSE = 1;</code>
-       */
-      REGION_CLOSE(1, 1),
-      ;
-
-      /**
-       * <code>REGION_OPEN = 0;</code>
-       */
-      public static final int REGION_OPEN_VALUE = 0;
-      /**
-       * <code>REGION_CLOSE = 1;</code>
-       */
-      public static final int REGION_CLOSE_VALUE = 1;
-
-
-      public final int getNumber() { return value; }
+    public boolean hasFamilyName() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required bytes family_name = 1;</code>
+     */
+    public com.google.protobuf.ByteString getFamilyName() {
+      return familyName_;
+    }
 
-      public static EventType valueOf(int value) {
-        switch (value) {
-          case 0: return REGION_OPEN;
-          case 1: return REGION_CLOSE;
-          default: return null;
+    // required string store_home_dir = 2;
+    public static final int STORE_HOME_DIR_FIELD_NUMBER = 2;
+    private java.lang.Object storeHomeDir_;
+    /**
+     * <code>required string store_home_dir = 2;</code>
+     *
+     * <pre>
+     *relative to region dir
+     * </pre>
+     */
+    public boolean hasStoreHomeDir() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>required string store_home_dir = 2;</code>
+     *
+     * <pre>
+     *relative to region dir
+     * </pre>
+     */
+    public java.lang.String getStoreHomeDir() {
+      java.lang.Object ref = storeHomeDir_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          storeHomeDir_ = s;
         }
+        return s;
       }
-
-      public static com.google.protobuf.Internal.EnumLiteMap<EventType>
-          internalGetValueMap() {
-        return internalValueMap;
-      }
-      private static com.google.protobuf.Internal.EnumLiteMap<EventType>
-          internalValueMap =
-            new com.google.protobuf.Internal.EnumLiteMap<EventType>() {
-              public EventType findValueByNumber(int number) {
-                return EventType.valueOf(number);
-              }
-            };
-
-      public final com.google.protobuf.Descriptors.EnumValueDescriptor
-          getValueDescriptor() {
-        return getDescriptor().getValues().get(index);
-      }
-      public final com.google.protobuf.Descriptors.EnumDescriptor
-          getDescriptorForType() {
-        return getDescriptor();
-      }
-      public static final com.google.protobuf.Descriptors.EnumDescriptor
-          getDescriptor() {
-        return org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.getDescriptor().getEnumTypes().get(0);
+    }
+    /**
+     * <code>required string store_home_dir = 2;</code>
+     *
+     * <pre>
+     *relative to region dir
+     * </pre>
+     */
+    public com.google.protobuf.ByteString
+        getStoreHomeDirBytes() {
+      java.lang.Object ref = storeHomeDir_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        storeHomeDir_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
       }
+    }
 
-      private static final EventType[] VALUES = values();
-
-      public static EventType valueOf(
-          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 index;
-      private final int value;
-
-      private EventType(int index, int value) {
-        this.index = index;
-        this.value = value;
-      }
-
-      // @@protoc_insertion_point(enum_scope:RegionEventDescriptor.EventType)
+    // repeated string store_file = 3;
+    public static final int STORE_FILE_FIELD_NUMBER = 3;
+    private com.google.protobuf.LazyStringList storeFile_;
+    /**
+     * <code>repeated string store_file = 3;</code>
+     *
+     * <pre>
+     * relative to store dir
+     * </pre>
+     */
+    public java.util.List<java.lang.String>
+        getStoreFileList() {
+      return storeFile_;
     }
-
-    public interface StoreDescriptorOrBuilder
-        extends com.google.protobuf.MessageOrBuilder {
-
-      // required bytes family_name = 1;
-      /**
-       * <code>required bytes family_name = 1;</code>
-       */
-      boolean hasFamilyName();
-      /**
-       * <code>required bytes family_name = 1;</code>
-       */
-      com.google.protobuf.ByteString getFamilyName();
-
-      // required string store_home_dir = 2;
-      /**
-       * <code>required string store_home_dir = 2;</code>
-       *
-       * <pre>
-       *relative to region dir
-       * </pre>
-       */
-      boolean hasStoreHomeDir();
-      /**
-       * <code>required string store_home_dir = 2;</code>
-       *
-       * <pre>
-       *relative to region dir
-       * </pre>
-       */
-      java.lang.String getStoreHomeDir();
-      /**
-       * <code>required string store_home_dir = 2;</code>
-       *
-       * <pre>
-       *relative to region dir
-       * </pre>
-       */
-      com.google.protobuf.ByteString
-          getStoreHomeDirBytes();
-
-      // repeated string store_file = 3;
-      /**
-       * <code>repeated string store_file = 3;</code>
-       *
-       * <pre>
-       * relative to store dir
-       * </pre>
-       */
-      java.util.List<java.lang.String>
-      getStoreFileList();
-      /**
-       * <code>repeated string store_file = 3;</code>
-       *
-       * <pre>
-       * relative to store dir
-       * </pre>
-       */
-      int getStoreFileCount();
-      /**
-       * <code>repeated string store_file = 3;</code>
-       *
-       * <pre>
-       * relative to store dir
-       * </pre>
-       */
-      java.lang.String getStoreFile(int index);
-      /**
-       * <code>repeated string store_file = 3;</code>
-       *
-       * <pre>
-       * relative to store dir
-       * </pre>
-       */
-      com.google.protobuf.ByteString
-          getStoreFileBytes(int index);
+    /**
+     * <code>repeated string store_file = 3;</code>
+     *
+     * <pre>
+     * relative to store dir
+     * </pre>
+     */
+    public int getStoreFileCount() {
+      return storeFile_.size();
+    }
+    /**
+     * <code>repeated string store_file = 3;</code>
+     *
+     * <pre>
+     * relative to store dir
+     * </pre>
+     */
+    public java.lang.String getStoreFile(int index) {
+      return storeFile_.get(index);
     }
     /**
-     * Protobuf type {@code RegionEventDescriptor.StoreDescriptor}
+     * <code>repeated string store_file = 3;</code>
+     *
+     * <pre>
+     * relative to store dir
+     * </pre>
      */
-    public static final class StoreDescriptor extends
-        com.google.protobuf.GeneratedMessage
-        implements StoreDescriptorOrBuilder {
-      // Use StoreDescriptor.newBuilder() to construct.
-      private StoreDescriptor(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
-        super(builder);
-        this.unknownFields = builder.getUnknownFields();
-      }
-      private StoreDescriptor(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+    public com.google.protobuf.ByteString
+        getStoreFileBytes(int index) {
+      return storeFile_.getByteString(index);
+    }
 
-      private static final StoreDescriptor defaultInstance;
-      public static StoreDescriptor getDefaultInstance() {
-        return defaultInstance;
-      }
+    private void initFields() {
+      familyName_ = com.google.protobuf.ByteString.EMPTY;
+      storeHomeDir_ = "";
+      storeFile_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
 
-      public StoreDescriptor getDefaultInstanceForType() {
-        return defaultInstance;
+      if (!hasFamilyName()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasStoreHomeDir()) {
+        memoizedIsInitialized = 0;
+        return false;
       }
+      memoizedIsInitialized = 1;
+      return true;
+    }
 
-      private final com.google.protobuf.UnknownFieldSet unknownFields;
-      @java.lang.Override
-      public final com.google.protobuf.UnknownFieldSet
-          getUnknownFields() {
-        return this.unknownFields;
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeBytes(1, familyName_);
       }
-      private StoreDescriptor(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
-        initFields();
-        int mutable_bitField0_ = 0;
-        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-            com.google.protobuf.UnknownFieldSet.newBuilder();
-        try {
-          boolean done = false;
-          while (!done) {
-            int tag = input.readTag();
-            switch (tag) {
-              case 0:
-                done = true;
-                break;
-              default: {
-                if (!parseUnknownField(input, unknownFields,
-                                       extensionRegistry, tag)) {
-                  done = true;
-                }
-                break;
-              }
-              case 10: {
-                bitField0_ |= 0x00000001;
-                familyName_ = input.readBytes();
-                break;
-              }
-              case 18: {
-                bitField0_ |= 0x00000002;
-                storeHomeDir_ = input.readBytes();
-                break;
-              }
-              case 26: {
-                if (!((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
-                  storeFile_ = new com.google.protobuf.LazyStringArrayList();
-                  mutable_bitField0_ |= 0x00000004;
-                }
-                storeFile_.add(input.readBytes());
-                break;
-              }
-            }
-          }
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-          throw e.setUnfinishedMessage(this);
-        } catch (java.io.IOException e) {
-          throw new com.google.protobuf.InvalidProtocolBufferException(
-              e.getMessage()).setUnfinishedMessage(this);
-        } finally {
-          if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
-            storeFile_ = new com.google.protobuf.UnmodifiableLazyStringList(storeFile_);
-          }
-          this.unknownFields = unknownFields.build();
-          makeExtensionsImmutable();
-        }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeBytes(2, getStoreHomeDirBytes());
       }
-      public static final com.google.protobuf.Descriptors.Descriptor
-          getDescriptor() {
-        return org.apache.hadoop.hbase.protobuf.generated.WALProtos.internal_static_RegionEventDescriptor_StoreDescriptor_descriptor;
+      for (int i = 0; i < storeFile_.size(); i++) {
+        output.writeBytes(3, storeFile_.getByteString(i));
       }
+      getUnknownFields().writeTo(output);
+    }
 
-      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
-          internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hbase.protobuf.generated.WALProtos.internal_static_RegionEventDescriptor_StoreDescriptor_fieldAccessorTable
-            .ensureFieldAccessorsInitialized(
-                org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor.class, org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor.Builder.class);
-      }
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
 
-      public static com.google.protobuf.Parser<StoreDescriptor> PARSER =
-          new com.google.protobuf.AbstractParser<StoreDescriptor>() {
-        public StoreDescriptor parsePartialFrom(
-            com.google.protobuf.CodedInputStream input,
-            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-            throws com.google.protobuf.InvalidProtocolBufferException {
-          return new StoreDescriptor(input, extensionRegistry);
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(1, familyName_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(2, getStoreHomeDirBytes());
+      }
+      {
+        int dataSize = 0;
+        for (int i = 0; i < storeFile_.size(); i++) {
+          dataSize += com.google.protobuf.CodedOutputStream
+            .computeBytesSizeNoTag(storeFile_.getByteString(i));
         }
-      };
-
-      @java.lang.Override
-      public com.google.protobuf.Parser<StoreDescriptor> getParserForType() {
-        return PARSER;
+        size += dataSize;
+        size += 1 * getStoreFileList().size();
       }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
 
-      private int bitField0_;
-      // required bytes family_name = 1;
-      public static final int FAMILY_NAME_FIELD_NUMBER = 1;
-      private com.google.protobuf.ByteString familyName_;
-      /**
-       * <code>required bytes family_name = 1;</code>
-       */
-      public boolean hasFamilyName() {
-        return ((bitField0_ & 0x00000001) == 0x00000001);
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
       }
-      /**
-       * <code>required bytes family_name = 1;</code>
-       */
-      public com.google.protobuf.ByteString getFamilyName() {
-        return familyName_;
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor)) {
+        return super.equals(obj);
       }
+      org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor other = (org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor) obj;
 
-      // required string store_home_dir = 2;
-      public static final int STORE_HOME_DIR_FIELD_NUMBER = 2;
-      private java.lang.Object storeHomeDir_;
-      /**
-       * <code>required string store_home_dir = 2;</code>
-       *
-       * <pre>
-       *relative to region dir
-       * </pre>
-       */
-      public boolean hasStoreHomeDir() {
-        return ((bitField0_ & 0x00000002) == 0x00000002);
+      boolean result = true;
+      result = result && (hasFamilyName() == other.hasFamilyName());
+      if (hasFamilyName()) {
+        result = result && getFamilyName()
+            .equals(other.getFamilyName());
       }
-      /**
-       * <code>required string store_home_dir = 2;</code>
-       *
-       * <pre>
-       *relative to region dir
-       * </pre>
-       */
-      public java.lang.String getStoreHomeDir() {
-        java.lang.Object ref = storeHomeDir_;
-        if (ref instanceof java.lang.String) {
-          return (java.lang.String) ref;
-        } else {
-          com.google.protobuf.ByteString bs = 
-              (com.google.protobuf.ByteString) ref;
-          java.lang.String s = bs.toStringUtf8();
-          if (bs.isValidUtf8()) {
-            storeHomeDir_ = s;
-          }
-          return s;
-        }
+      result = result && (hasStoreHomeDir() == other.hasStoreHomeDir());
+      if (hasStoreHomeDir()) {
+        result = result && getStoreHomeDir()
+            .equals(other.getStoreHomeDir());
       }
-      /**
-       * <code>required string store_home_dir = 2;</code>
-       *
-       * <pre>
-       *relative to region dir
-       * </pre>
-       */
-      public com.google.protobuf.ByteString
-          getStoreHomeDirBytes() {
-        java.lang.Object ref = storeHomeDir_;
-        if (ref instanceof java.lang.String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
-                  (java.lang.String) ref);
-          storeHomeDir_ = b;
-          return b;
-        } else {
-          return (com.google.protobuf.ByteString) ref;
-        }
+      result = result && getStoreFileList()
+          .equals(other.getStoreFileList());
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasFamilyName()) {
+        hash = (37 * hash) + FAMILY_NAME_FIELD_NUMBER;
+        hash = (53 * hash) + getFamilyName().hashCode();
+      }
+      if (hasStoreHomeDir()) {
+        hash = (37 * hash) + STORE_HOME_DIR_FIELD_NUMBER;
+        hash = (53 * hash) + getStoreHomeDir().hashCode();
       }
+      if (getStoreFileCount() > 0) {
+        hash = (37 * hash) + STORE_FILE_FIELD_NUMBER;
+        hash = (53 * hash) + getStoreFileList().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
 
-      // repeated string store_file = 3;
-      public static final int STORE_FILE_FIELD_NUMBER = 3;
-      private com.google.protobuf.LazyStringList storeFile_;
-      /**
-       * <code>repeated string store_file = 3;</code>
-       *
-       * <pre>
-       * relative to store dir
-       * </pre>
-       */
-      public java.util.List<java.lang.String>
-          getStoreFileList() {
-        return storeFile_;
+    public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code StoreDescriptor}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptorOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.WALProtos.internal_static_StoreDescriptor_descriptor;
       }
-      /**
-       * <code>repeated string store_file = 3;</code>
-       *
-       * <pre>
-       * relative to store dir
-       * </pre>
-       */
-      public int getStoreFileCount() {
-        return storeFile_.size();
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.WALProtos.internal_static_StoreDescriptor_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor.class, org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor.Builder.class);
       }
-      /**
-       * <code>repeated string store_file = 3;</code>
-       *
-       * <pre>
-       * relative to store dir
-       * </pre>
-       */
-      public java.lang.String getStoreFile(int index) {
-        return storeFile_.get(index);
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
       }
-      /**
-       * <code>repeated string store_file = 3;</code>
-       *
-       * <pre>
-       * relative to store dir
-       * </pre>
-       */
-      public com.google.protobuf.ByteString
-          getStoreFileBytes(int index) {
-        return storeFile_.getByteString(index);
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
       }
 
-      private void initFields() {
+      public Builder clear() {
+        super.clear();
         familyName_ = com.google.protobuf.ByteString.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000001);
         storeHomeDir_ = "";
+        bitField0_ = (bitField0_ & ~0x00000002);
         storeFile_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000004);
+        return this;
       }
-      private byte memoizedIsInitialized = -1;
-      public final boolean isInitialized() {
-        byte isInitialized = memoizedIsInitialized;
-        if (isInitialized != -1) return isInitialized == 1;
 
-        if (!hasFamilyName()) {
-          memoizedIsInitialized = 0;
-          return false;
-        }
-        if (!hasStoreHomeDir()) {
-          memoizedIsInitialized = 0;
-          return false;
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.WALProtos.internal_static_StoreDescriptor_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor build() {
+        org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
         }
-        memoizedIsInitialized = 1;
-        return true;
+        return result;
       }
 
-      public void writeTo(com.google.protobuf.CodedOutputStream output)
-                          throws java.io.IOException {
-        getSerializedSize();
-        if (((bitField0_ & 0x00000001) == 0x00000001)) {
-          output.writeBytes(1, familyName_);
+      public org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor result = new org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
         }
-        if (((bitField0_ & 0x00000002) == 0x00000002)) {
-          output.writeBytes(2, getStoreHomeDirBytes());
+        result.familyName_ = familyName_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
         }
-        for (int i = 0; i < storeFile_.size(); i++) {
-          output.writeBytes(3, storeFile_.getByteString(i));
+        result.storeHomeDir_ = storeHomeDir_;
+        if (((bitField0_ & 0x00000004) == 0x00000004)) {
+          storeFile_ = new com.google.protobuf.UnmodifiableLazyStringList(
+              storeFile_);
+          bitField0_ = (bitField0_ & ~0x00000004);
         }
-        getUnknownFields().writeTo(output);
+        result.storeFile_ = storeFile_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
       }
 
-      private int memoizedSerializedSize = -1;
-      public int getSerializedSize() {
-        int size = memoizedSerializedSize;
-        if (size != -1) return size;
-
-        size = 0;
-        if (((bitField0_ & 0x00000001) == 0x00000001)) {
-          size += com.google.protobuf.CodedOutputStream
-            .computeBytesSize(1, familyName_);
-        }
-        if (((bitField0_ & 0x00000002) == 0x00000002)) {
-          size += com.google.protobuf.CodedOutputStream
-            .computeBytesSize(2, getStoreHomeDirBytes());
-        }
-        {
-          int dataSize = 0;
-          for (int i = 0; i < storeFile_.size(); i++) {
-            dataSize += com.google.protobuf.CodedOutputStream
-              .computeBytesSizeNoTag(storeFile_.getByteString(i));
-          }
-          size += dataSize;
-          size += 1 * getStoreFileList().size();
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
         }
-        size += getUnknownFields().getSerializedSize();
-        memoizedSerializedSize = size;
-        return size;
-      }
-
-      private static final long serialVersionUID = 0L;
-      @java.lang.Override
-      protected java.lang.Object writeReplace()
-          throws java.io.ObjectStreamException {
-        return super.writeReplace();
       }
 
-      @java.lang.Override
-      public boolean equals(final java.lang.Object obj) {
-        if (obj == this) {
-         return true;
-        }
-        if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor)) {
-          return super.equals(obj);
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor.getDefaultInstance()) return this;
+        if (other.hasFamilyName()) {
+          setFamilyName(other.getFamilyName());
         }
-        org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor other = (org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor) obj;
-
-        boolean result = true;
-        result = result && (hasFamilyName() == other.hasFamilyName());
-        if (hasFamilyName()) {
-          result = result && getFamilyName()
-              .equals(other.getFamilyName());
+        if (other.hasStoreHomeDir()) {
+          bitField0_ |= 0x00000002;
+          storeHomeDir_ = other.storeHomeDir_;
+          onChanged();
         }
-        result = result && (hasStoreHomeDir() == other.hasStoreHomeDir());
-        if (hasStoreHomeDir()) {
-          result = result && getStoreHomeDir()
-              .equals(other.getStoreHomeDir());
+        if (!other.storeFile_.isEmpty()) {
+          if (storeFile_.isEmpty()) {
+            storeFile_ = other.storeFile_;
+            bitField0_ = (bitField0_ & ~0x00000004);
+          } else {
+            ensureStoreFileIsMutable();
+            storeFile_.addAll(other.storeFile_);
+          }
+          onChanged();
         }
-        result = result && getStoreFileList()
-            .equals(other.getStoreFileList());
-        result = result &&
-            getUnknownFields().equals(other.getUnknownFields());
-        return result;
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
       }
 
-      private int memoizedHashCode = 0;
-      @java.lang.Override
-      public int hashCode() {
-        if (memoizedHashCode != 0) {
-          return memoizedHashCode;
-        }
-        int hash = 41;
-        hash = (19 * hash) + getDescriptorForType().hashCode();
-        if (hasFamilyName()) {
-          hash = (37 * hash) + FAMILY_NAME_FIELD_NUMBER;
-          hash = (53 * hash) + getFamilyName().hashCode();
-        }
-        if (hasStoreHomeDir()) {
-          hash = (37 * hash) + STORE_HOME_DIR_FIELD_NUMBER;
-          hash = (53 * hash) + getStoreHomeDir().hashCode();
+      public final boolean isInitialized() {
+        if (!hasFamilyName()) {
+          
+          return false;
         }
-        if (getStoreFileCount() > 0) {
-          hash = (37 * hash) + STORE_FILE_FIELD_NUMBER;
-          hash = (53 * hash) + getStoreFileList().hashCode();
+        if (!hasStoreHomeDir()) {
+          
+          return false;
         }
-        hash = (29 * hash) + getUnknownFields().hashCode();
-        memoizedHashCode = hash;
-        return hash;
+        return true;
       }
 
-      public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor parseFrom(
-          com.google.protobuf.ByteString data)
-          throws com.google.protobuf.InvalidProtocolBufferException {
-        return PARSER.parseFrom(data);
-      }
-      public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor parseFrom(
-          com.google.protobuf.ByteString data,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
-        return PARSER.parseFrom(data, extensionRegistry);
-      }
-      public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor parseFrom(byte[] data)
-          throws com.google.protobuf.InvalidProtocolBufferException {
-        return PARSER.parseFrom(data);
-      }
-      public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor parseFrom(
-          byte[] data,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
-        return PARSER.parseFrom(data, extensionRegistry);
-      }
-      public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor parseFrom(java.io.InputStream input)
-          throws java.io.IOException {
-        return PARSER.parseFrom(input);
-      }
-      public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor parseFrom(
-          java.io.InputStream input,
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
           com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        return PARSER.parseFrom(input, extensionRegistry);
-      }
-      public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor parseDelimitedFrom(java.io.InputStream input)
-          throws java.io.IOException {
-        return PARSER.parseDelimitedFrom(input);
+        org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
       }
-      public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor parseDelimitedFrom(
-          java.io.InputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws java.io.IOException {
-        return PARSER.parseDelimitedFrom(input, extensionRegistry);
+      private int bitField0_;
+
+      // required bytes family_name = 1;
+      private com.google.protobuf.ByteString familyName_ = com.google.protobuf.ByteString.EMPTY;
+      /**
+       * <code>required bytes family_name = 1;</code>
+       */
+      public boolean hasFamilyName() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
       }
-      public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor parseFrom(
-          com.google.protobuf.CodedInputStream input)
-          throws java.io.IOException {
-        return PARSER.parseFrom(input);
+      /**
+       * <code>required bytes family_name = 1;</code>
+       */
+      public com.google.protobuf.ByteString getFamilyName() {
+        return familyName_;
       }
-      public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor parseFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws java.io.IOException {
-        return PARSER.parseFrom(input, extensionRegistry);
+      /**
+       * <code>required bytes family_name = 1;</code>
+       */
+      public Builder setFamilyName(com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        familyName_ = value;
+        onChanged();
+        return this;
       }
-
-      public static Builder newBuilder() { return Builder.create(); }
-      public Builder newBuilderForType() { return newBuilder(); }
-      public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor prototype) {
-        return newBuilder().mergeFrom(prototype);
+      /**
+       * <code>required bytes family_name = 1;</code>
+       */
+      public Builder clearFamilyName() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        familyName_ = getDefaultInstance().getFamilyName();
+        onChanged();
+        return this;
       }
-      public Builder toBuilder() { return newBuilder(this); }
 
-      @java.lang.Override
-      protected Builder newBuilderForType(
-          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-        Builder builder = new Builder(parent);
-        return builder;
+      // required string store_home_dir = 2;
+      private java.lang.Object storeHomeDir_ = "";
+      /**
+       * <code>required string store_home_dir = 2;</code>
+       *
+       * <pre>
+       *relative to region dir
+       * </pre>
+       */
+      public boolean hasStoreHomeDir() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
       }
       /**
-       * Protobuf type {@code RegionEventDescriptor.StoreDescriptor}
+       * <code>required string store_home_dir = 2;</code>
+       *
+       * <pre>
+       *relative to region dir
+       * </pre>
        */
-      public static final class Builder extends
-          com.google.protobuf.GeneratedMessage.Builder<Builder>
-         implements org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptorOrBuilder {
-        public static final com.google.protobuf.Descriptors.Descriptor
-            getDescriptor() {
-          return org.apache.hadoop.hbase.protobuf.generated.WALProtos.internal_static_RegionEventDescriptor_StoreDescriptor_descriptor;
-        }
-
-        protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
-            internalGetFieldAccessorTable() {
-          return org.apache.hadoop.hbase.protobuf.generated.WALProtos.internal_static_RegionEventDescriptor_StoreDescriptor_fieldAccessorTable
-              .ensureFieldAccessorsInitialized(
-                  org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor.class, org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor.Builder.class);
-        }
-
-        // Construct using org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor.newBuilder()
-        private Builder() {
-          maybeForceBuilderInitialization();
-        }
-
-        private Builder(
-            com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-          super(parent);
-          maybeForceBuilderInitialization();
-        }
-        private void maybeForceBuilderInitialization() {
-          if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
-          }
+      public java.lang.String getStoreHomeDir() {
+        java.lang.Object ref = storeHomeDir_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          storeHomeDir_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
         }
-        private static Builder create() {
-          return new Builder();
-        }
-
-        public Builder clear() {
-          super.clear();
-          familyName_ = com.google.protobuf.ByteString.EMPTY;
-          bitField0_ = (bitField0_ & ~0x00000001);
-          storeHomeDir_ = "";
-          bitField0_ = (bitField0_ & ~0x00000002);
-          storeFile_ = com.google.protobuf.LazyStringArrayList.EMPTY;
-          bitField0_ = (bitField0_ & ~0x00000004);
-          return this;
-        }
-
-        public Builder clone() {
-          return create().mergeFrom(buildPartial());
+      }
+      /**
+       * <code>required string store_home_dir = 2;</code>
+       *
+       * <pre>
+       *relative to region dir
+       * </pre>
+       */
+      public com.google.protobuf.ByteString
+          getStoreHomeDirBytes() {
+        java.lang.Object ref = storeHomeDir_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          storeHomeDir_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
         }
+      }
+      /**
+       * <code>required string store_home_dir = 2;</code>
+       *
+       * <pre>
+       *relative to region dir
+       * </pre>
+       */
+      public Builder setStoreHomeDir(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+        storeHomeDir_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string store_home_dir = 2;</code>
+       *
+       * <pre>
+       *relative to region dir
+       * </pre>
+       */
+      public Builder clearStoreHomeDir() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        storeHomeDir_ = getDefaultInstance().getStoreHomeDir();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string store_home_dir = 2;</code>
+       *
+       * <pre>
+       *relative to region dir
+       * </pre>
+       */
+      public Builder setStoreHomeDirBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+        storeHomeDir_ = value;
+        onChanged();
+        return this;
+      }
 
-        public com.google.protobuf.Descriptors.Descriptor
-            getDescriptorForType() {
-          return org.apache.hadoop.hbase.protobuf.generated.WALProtos.internal_static_RegionEventDescriptor_StoreDescriptor_descriptor;
-        }
+      // repeated string store_file = 3;
+      private com.google.protobuf.LazyStringList storeFile_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+      private void ensureStoreFileIsMutable() {
+        if (!((bitField0_ & 0x00000004) == 0x00000004)) {
+          storeFile_ = new com.google.protobuf.LazyStringArrayList(storeFile_);
+          bitField0_ |= 0x00000004;
+         }
+      }
+      /**
+       * <code>repeated string store_file = 3;</code>
+       *
+       * <pre>
+       * relative to store dir
+       * </pre>
+       */
+      public java.util.List<java.lang.String>
+          getStoreFileList() {
+        return java.util.Collections.unmodifiableList(storeFile_);
+      }
+      /**
+       * <code>repeated string store_file = 3;</code>
+       *
+       * <pre>
+       * relative to store dir
+       * </pre>
+       */
+      public int getStoreFileCount() {
+        return storeFile_.size();
+      }
+      /**
+       * <code>repeated string store_file = 3;</code>
+       *
+       * <pre>
+       * relative to store dir
+       * </pre>
+       */
+      public java.lang.String getStoreFile(int index) {
+        return storeFile_.get(index);
+      }
+      /**
+       * <code>repeated string store_file = 3;</code>
+       *
+       * <pre>
+       * relative to store dir
+       * </pre>
+       */
+      public com.google.protobuf.ByteString
+          getStoreFileBytes(int index) {
+        return storeFile_.getByteString(index);
+      }
+      /**
+       * <code>repeated string store_file = 3;</code>
+       *
+       * <pre>
+       * relative to store dir
+       * </pre>
+       */
+      public Builder setStoreFile(
+          int index, java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureStoreFileIsMutable();
+        storeFile_.set(index, value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string store_file = 3;</code>
+       *
+       * <pre>
+       * relative to store dir
+       * </pre>
+       */
+      public Builder addStoreFile(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureStoreFileIsMutable();
+        storeFile_.add(value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string store_file = 3;</code>
+       *
+       * <pre>
+       * relative to store dir
+       * </pre>
+       */
+      public Builder addAllStoreFile(
+          java.lang.Iterable<java.lang.String> values) {
+        ensureStoreFileIsMutable();
+        super.addAll(values, storeFile_);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string store_file = 3;</code>
+       *
+       * <pre>
+       * relative to store dir
+       * </pre>
+       */
+      public Builder clearStoreFile() {
+        storeFile_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000004);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string store_file = 3;</code>
+       *
+       * <pre>
+       * relative to store dir
+       * </pre>
+       */
+      public Builder addStoreFileBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureStoreFileIsMutable();
+        storeFile_.add(value);
+        onChanged();
+        return this;
+      }
 
-        public org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor getDefaultInstanceForType() {
-          return org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor.getDefaultInstance();
-        }
+      // @@protoc_insertion_point(builder_scope:StoreDescriptor)
+    }
 
-        public org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor build() {
-          org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor result = buildPartial();
-          if (!result.isInitialized()) {
-            throw newUninitializedMessageException(result);
-          }
-          return result;
-        }
+    static {
+      defaultInstance = new StoreDescriptor(true);
+      defaultInstance.initFields();
+    }
 
-        public org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor buildPartial() {
-          org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor result = new org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor(this);
-          int from_bitField0_ = bitField0_;
-          int to_bitField0_ = 0;
-          if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
-            to_bitField0_ |= 0x00000001;
-          }
-          result.familyName_ = familyName_;
-          if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
-            to_bitField0_ |= 0x00000002;
-          }
-          result.storeHomeDir_ = storeHomeDir_;
-          if (((bitField0_ & 0x00000004) == 0x00000004)) {
-            storeFile_ = new com.google.protobuf.UnmodifiableLazyStringList(
-                storeFile_);
-            bitField0_ = (bitField0_ & ~0x00000004);
-          }
-          result.storeFile_ = storeFile_;
-          result.bitField0_ = to_bitField0_;
-          onBuilt();
-          return result;
-        }
+    // @@protoc_insertion_point(class_scope:StoreDescriptor)
+  }
 
-        public Builder mergeFrom(com.google.protobuf.Message other) {
-          if (other instanceof org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor) {
-            return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor)other);
-          } else {
-            super.mergeFrom(other);
-            return this;
-          }
-        }
+  public interface BulkLoadDescriptorOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
 
-        public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor other) {
-          if (other == org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor.getDefaultInstance()) return this;
-          if (other.hasFamilyName()) {
-            setFamilyName(other.getFamilyName());
-          }
-          if (other.hasStoreHomeDir()) {
-            bitField0_ |= 0x00000002;
-            storeHomeDir_ = other.storeHomeDir_;
-            onChanged();
-          }
-          if (!other.storeFile_.isEmpty()) {
-            if (storeFile_.isEmpty()) {
-              storeFile_ = other.storeFile_;
-              bitField0_ = (bitField0_ & ~0x00000004);
-            } else {
-              ensureStoreFileIsMutable();
-              storeFile_.addAll(other.storeFile_);
-            }
-            onChanged();
-          }
-          this.mergeUnknownFields(other.getUnknownFields());
-          return this;
-        }
+    // required .TableName table_name = 1;
+    /**
+     * <code>required .TableName table_name = 1;</code>
+     */
+    boolean hasTableName();
+    /**
+     * <code>required .TableName table_name = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName();
+    /**
+     * <code>required .TableName table_name = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder();
 
-        public final boolean isInitialized() {
-          if (!hasFamilyName()) {
-            
-            return false;
-          }
-          if (!hasStoreHomeDir()) {
-            
-            return false;
-          }
-          return true;
-        }
+    // required bytes encoded_region_name = 2;
+    /**
+     * <code>required bytes encoded_region_name = 2;</code>
+     */
+    boolean hasEncodedRegionName();
+    /**
+     * <code>required bytes encoded_region_name = 2;</code>
+     */
+    com.google.protobuf.ByteString getEncodedRegionName();
 
-        public Builder mergeFrom(
-            com.google.protobuf.CodedInputStream input,
-            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-            throws java.io.IOException {
-          org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor parsedMessage = null;
-          try {
-            parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-          } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-            parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor) e.getUnfinishedMessage();
-            throw e;
-          } finally {
-            if (parsedMessage != null) {
-              mergeFrom(parsedMessage);
-            }
-          }
-          return this;
-        }
-        private int bitField0_;
+    // repeated .StoreDescriptor stores = 3;
+    /**
+     * <code>repeated .StoreDescriptor stores = 3;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor> 
+        getStoresList();
+    /**
+     * <code>repeated .StoreDescriptor stores = 3;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor getStores(int index);
+    /**
+     * <code>repeated .StoreDescriptor stores = 3;</code>
+     */
+    int getStoresCount();
+    /**
+     * <code>repeated .StoreDescriptor stores = 3;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptorOrBuilder> 
+        getStoresOrBuilderList();
+    /**
+     * <code>repeated .StoreDescriptor stores = 3;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptorOrBuilder getStoresOrBuilder(
+        int index);
 
-        // required bytes family_name = 1;
-        private com.google.protobuf.ByteString familyName_ = com.google.protobuf.ByteString.EMPTY;
-        /**
-         * <code>required bytes family_name = 1;</code>
-         */
-        public boolean hasFamilyName() {
-          return ((bitField0_ & 0x00000001) == 0x00000001);
-        }
-        /**
-         * <code>required bytes family_name = 1;</code>
-         */
-        public com.google.protobuf.ByteString getFamilyName() {
-          return familyName_;
-        }
-        /**
-         * <code>required bytes family_name = 1;</code>
-         */
-        public Builder setFamilyName(com.google.protobuf.ByteString value) {
-          if (value == null) {
-    throw new NullPointerException();
+    // required int64 bulkload_seq_num = 4;
+    /**
+     * <code>required int64 bulkload_seq_num = 4;</code>
+     */
+    boolean hasBulkloadSeqNum();
+    /**
+     * <code>required int64 bulkload_seq_num = 4;</code>
+     */
+    long getBulkloadSeqNum();
   }
-  bitField0_ |= 0x00000001;
-          familyName_ = value;
-          onChanged();
-          return this;
-        }
-        /**
-         * <code>required bytes family_name = 1;</code>
-         */
-        public Builder clearFamilyName() {
-          bitField0_ = (bitField0_ & ~0x00000001);
-          familyName_ = getDefaultInstance().getFamilyName();
-          onChanged();
-          return this;
-        }
+  /**
+   * Protobuf type {@code BulkLoadDescriptor}
+   *
+   * <pre>
+   **
+   * Special WAL entry used for writing bulk load events to WAL
+   * </pre>
+   */
+  public static final class BulkLoadDescriptor extends
+      com.google.protobuf.GeneratedMessage
+      implements BulkLoadDescriptorOrBuilder {
+    // Use BulkLoadDescriptor.newBuilder() to construct.
+    private BulkLoadDescriptor(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private BulkLoadDescriptor(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
 
-        // required string store_home_dir = 2;
-        private java.lang.Object storeHomeDir_ = "";
-        /**
-         * <code>required string store_home_dir = 2;</code>
-         *
-         * <pre>
-         *relative to region dir
-         * </pre>
-         */
-        public boolean hasStoreHomeDir() {
-          return ((bitField0_ & 0x00000002) == 0x00000002);
-        }
-        /**
-         * <code>required string store_home_dir = 2;</code>
-         *
-         * <pre>
-         *relative to region dir
-         * </pre>
-         */
-        public java.lang.String getStoreHomeDir() {
-          java.lang.Object ref = storeHomeDir_;
-          if (!(ref instanceof java.lang.String)) {
-            java.lang.String s = ((com.google.protobuf.ByteString) ref)
-                .toStringUtf8();
-            storeHomeDir_ = s;
-            return s;
-          } else {
-            return (java.lang.String) ref;
-          }
-        }
-        /**
-         * <code>required string store_home_dir = 2;</code>
-         *
-         * <pre>
-         *relative to region dir
-         * </pre>
-         */
-        public com.google.protobuf.ByteString
-            getStoreHomeDirBytes() {
-          java.lang.Object ref = storeHomeDir_;
-          if (ref instanceof String) {
-            com.google.protobuf.ByteString b = 
-                com.google.protobuf.ByteString.copyFromUtf8(
-                    (java.lang.String) ref);
-            storeHomeDir_ = b;
-            return b;
-          } else {
-            return (com.google.protobuf.ByteString) ref;
+    private static final BulkLoadDescriptor defaultInstance;
+    public static BulkLoadDescriptor getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public BulkLoadDescriptor getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private BulkLoadDescriptor(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000001) == 0x00000001)) {
+                subBuilder = tableName_.toBuilder();
+              }
+              tableName_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(tableName_);
+                tableName_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000001;
+              break;
+            }
+            case 18: {
+              bitField0_ |= 0x00000002;
+              encodedRegionName_ = input.readBytes();
+              break;
+            }
+            case 26: {
+              if (!((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
+                stores_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor>();
+                mutable_bitField0_ |= 0x00000004;
+              }
+              stores_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor.PARSER, extensionRegistry));
+              break;
+            }
+            case 32: {
+              bitField0_ |= 0x00000004;
+              bulkloadSeqNum_ = input.readInt64();
+              break;
+            }
           }
         }
-        /**
-         * <code>required string store_home_dir = 2;</code>
-         *
-         * <pre>
-         *relative to region dir
-         * </pre>
-         */
-        public Builder setStoreHomeDir(
-            java.lang.String value) {
-          if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000002;
-          storeHomeDir_ = value;
-          onChanged();
-          return this;
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
+          stores_ = java.util.Collections.unmodifiableList(stores_);
         }
-        /**
-         * <code>required string store_home_dir = 2;</code>
-         *
-         * <pre>
-         *relative to region dir
-         * </pre>
-         */
-        public Builder clearStoreHomeDir() {
-          bitField0_ = (bitField0_ & ~0x00000002);
-          storeHomeDir_ = getDefaultInstance().getStoreHomeDir();
-          onChanged();
-          return this;
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.WALProtos.internal_static_BulkLoadDescriptor_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.WALProtos.internal_static_BulkLoadDescriptor_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor.class, org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<BulkLoadDescriptor> PARSER =
+        new com.google.protobuf.AbstractParser<BulkLoadDescriptor>() {
+      public BulkLoadDescriptor parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new BulkLoadDescriptor(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<BulkLoadDescriptor> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // required .TableName table_name = 1;
+    public static final int TABLE_NAME_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName tableName_;
+    /**
+     * <code>required .TableName table_name = 1;</code>
+     */
+    public boolean hasTableName() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required .TableName table_name = 1;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName() {
+      return tableName_;
+    }
+    /**
+     * <code>required .TableName table_name = 1;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
+      return tableName_;
+    }
+
+    // required bytes encoded_region_name = 2;
+    public static final int ENCODED_REGION_NAME_FIELD_NUMBER = 2;
+    private com.google.protobuf.ByteString encodedRegionName_;
+    /**
+     * <code>required bytes encoded_region_name = 2;</code>
+     */
+    public boolean hasEncodedRegionName() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>required bytes encoded_region_name = 2;</code>
+     */
+    public com.google.protobuf.ByteString getEncodedRegionName() {
+      return encodedRegionName_;
+    }
+
+    // repeated .StoreDescriptor stores = 3;
+    public static final int STORES_FIELD_NUMBER = 3;
+    private java.util.List<org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor> stores_;
+    /**
+     * <code>repeated .StoreDescriptor stores = 3;</code>
+     */
+    public java.util.List<org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor> getStoresList() {
+      return stores_;
+    }
+    /**
+     * <code>repeated .StoreDescriptor stores = 3;</code>
+     */
+    public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptorOrBuilder> 
+        getStoresOrBuilderList() {
+      return stores_;
+    }
+    /**
+     * <code>repeated .StoreDescriptor stores = 3;</code>
+     */
+    public int getStoresCount() {
+      return stores_.size();
+    }
+    /**
+     * <code>repeated .StoreDescriptor stores = 3;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor getStores(int index) {
+      return stores_.get(index);
+    }
+    /**
+     * <code>repeated .StoreDescriptor stores = 3;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptorOrBuilder getStoresOrBuilder(
+        int index) {
+      return stores_.get(index);
+    }
+
+    // required int64 bulkload_seq_num = 4;
+    public static final int BULKLOAD_SEQ_NUM_FIELD_NUMBER = 4;
+    private long bulkloadSeqNum_;
+    /**
+     * <code>required int64 bulkload_seq_num = 4;</code>
+     */
+    public boolean hasBulkloadSeqNum() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>required int64 bulkload_seq_num = 4;</code>
+     */
+    public long getBulkloadSeqNum() {
+      return bulkloadSeqNum_;
+    }
+
+    private void initFields() {
+      tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+      encodedRegionName_ = com.google.protobuf.ByteString.EMPTY;
+      stores_ = java.util.Collections.emptyList();
+      bulkloadSeqNum_ = 0L;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (!hasTableName()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasEncodedRegionName()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasBulkloadSeqNum()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getTableName().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      for (int i = 0; i < getStoresCount(); i++) {
+        if (!getStores(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeMessage(1, tableName_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeBytes(2, encodedRegionName_);
+      }
+      for (int i = 0; i < stores_.size(); i++) {
+        output.writeMessage(3, stores_.get(i));
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeInt64(4, bulkloadSeqNum_);
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, tableName_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(2, encodedRegionName_);
+      }
+      for (int i = 0; i < stores_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(3, stores_.get(i));
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt64Size(4, bulkloadSeqNum_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor other = (org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor) obj;
+
+      boolean result = true;
+      result = result && (hasTableName() == other.hasTableName());
+      if (hasTableName()) {
+        result = result && getTableName()
+            .equals(other.getTableName());
+      }
+      result = result && (hasEncodedRegionName() == other.hasEncodedRegionName());
+      if (hasEncodedRegionName()) {
+        result = result && getEncodedRegionName()
+            .equals(other.getEncodedRegionName());
+      }
+      result = result && getStoresList()
+          .equals(other.getStoresList());
+      result = result && (hasBulkloadSeqNum() == other.hasBulkloadSeqNum());
+      if (hasBulkloadSeqNum()) {
+        result = result && (getBulkloadSeqNum()
+            == other.getBulkloadSeqNum());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasTableName()) {
+        hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER;
+        hash = (53 * hash) + getTableName().hashCode();
+      }
+      if (hasEncodedRegionName()) {
+        hash = (37 * hash) + ENCODED_REGION_NAME_FIELD_NUMBER;
+        hash = (53 * hash) + getEncodedRegionName().hashCode();
+      }
+      if (getStoresCount() > 0) {
+        hash = (37 * hash) + STORES_FIELD_NUMBER;
+        hash = (53 * hash) + getStoresList().hashCode();
+      }
+      if (hasBulkloadSeqNum()) {
+        hash = (37 * hash) + BULKLOAD_SEQ_NUM_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getBulkloadSeqNum());
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code BulkLoadDescriptor}
+     *
+     * <pre>
+     **
+     * Special WAL entry used for writing bulk load events to WAL
+     * </pre>
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptorOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.WALProtos.internal_static_BulkLoadDescriptor_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.WALProtos.internal_static_BulkLoadDescriptor_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor.class, org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getTableNameFieldBuilder();
+          getStoresFieldBuild

<TRUNCATED>

[47/50] [abbrv] hbase git commit: Merge branch 'master' (2/11/15) into hbase-11339

Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java
----------------------------------------------------------------------
diff --cc hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java
index 3c8fa87,0000000..aba81eb
mode 100644,000000..100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java
@@@ -1,546 -1,0 +1,548 @@@
 +/**
 + *
 + * 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.io.FileNotFoundException;
 +import java.io.IOException;
 +import java.util.ArrayList;
 +import java.util.Date;
 +import java.util.List;
 +import java.util.NavigableSet;
 +import java.util.UUID;
 +
 +import org.apache.hadoop.classification.InterfaceAudience;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.hbase.Cell;
 +import org.apache.hadoop.hbase.HColumnDescriptor;
 +import org.apache.hadoop.hbase.HConstants;
 +import org.apache.hadoop.hbase.KeyValue;
 +import org.apache.hadoop.hbase.KeyValue.KVComparator;
 +import org.apache.hadoop.hbase.KeyValue.Type;
 +import org.apache.hadoop.hbase.TableName;
 +import org.apache.hadoop.hbase.Tag;
 +import org.apache.hadoop.hbase.client.Scan;
 +import org.apache.hadoop.hbase.filter.Filter;
 +import org.apache.hadoop.hbase.filter.FilterList;
 +import org.apache.hadoop.hbase.io.compress.Compression;
 +import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 +import org.apache.hadoop.hbase.io.hfile.HFile;
 +import org.apache.hadoop.hbase.io.hfile.HFileContext;
 +import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
 +import org.apache.hadoop.hbase.master.TableLockManager;
 +import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
 +import org.apache.hadoop.hbase.mob.MobCacheConfig;
 +import org.apache.hadoop.hbase.mob.MobConstants;
 +import org.apache.hadoop.hbase.mob.MobFile;
 +import org.apache.hadoop.hbase.mob.MobFileName;
 +import org.apache.hadoop.hbase.mob.MobStoreEngine;
 +import org.apache.hadoop.hbase.mob.MobUtils;
 +import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
++import org.apache.hadoop.hbase.regionserver.compactions.CompactionThroughputController;
 +import org.apache.hadoop.hbase.util.Bytes;
 +import org.apache.hadoop.hbase.util.HFileArchiveUtil;
 +
 +/**
 + * The store implementation to save MOBs (medium objects), it extends the HStore.
 + * When a descriptor of a column family has the value "IS_MOB", it means this column family
 + * is a mob one. When a HRegion instantiate a store for this column family, the HMobStore is
 + * created.
 + * HMobStore is almost the same with the HStore except using different types of scanners.
 + * In the method of getScanner, the MobStoreScanner and MobReversedStoreScanner are returned.
 + * In these scanners, a additional seeks in the mob files should be performed after the seek
 + * to HBase is done.
 + * The store implements how we save MOBs by extending HStore. When a descriptor
 + * of a column family has the value "IS_MOB", it means this column family is a mob one. When a
 + * HRegion instantiate a store for this column family, the HMobStore is created. HMobStore is
 + * almost the same with the HStore except using different types of scanners. In the method of
 + * getScanner, the MobStoreScanner and MobReversedStoreScanner are returned. In these scanners, a
 + * additional seeks in the mob files should be performed after the seek in HBase is done.
 + */
 +@InterfaceAudience.Private
 +public class HMobStore extends HStore {
 +
 +  private MobCacheConfig mobCacheConfig;
 +  private Path homePath;
 +  private Path mobFamilyPath;
 +  private volatile long mobCompactedIntoMobCellsCount = 0;
 +  private volatile long mobCompactedFromMobCellsCount = 0;
 +  private volatile long mobCompactedIntoMobCellsSize = 0;
 +  private volatile long mobCompactedFromMobCellsSize = 0;
 +  private volatile long mobFlushCount = 0;
 +  private volatile long mobFlushedCellsCount = 0;
 +  private volatile long mobFlushedCellsSize = 0;
 +  private volatile long mobScanCellsCount = 0;
 +  private volatile long mobScanCellsSize = 0;
 +  private List<Path> mobDirLocations;
 +  private HColumnDescriptor family;
 +  private TableLockManager tableLockManager;
 +  private TableName tableLockName;
 +
 +  public HMobStore(final HRegion region, final HColumnDescriptor family,
 +      final Configuration confParam) throws IOException {
 +    super(region, family, confParam);
 +    this.family = family;
 +    this.mobCacheConfig = (MobCacheConfig) cacheConf;
 +    this.homePath = MobUtils.getMobHome(conf);
 +    this.mobFamilyPath = MobUtils.getMobFamilyPath(conf, this.getTableName(),
 +        family.getNameAsString());
 +    mobDirLocations = new ArrayList<Path>();
 +    mobDirLocations.add(mobFamilyPath);
 +    TableName tn = region.getTableDesc().getTableName();
 +    mobDirLocations.add(HFileArchiveUtil.getStoreArchivePath(conf, tn, MobUtils
 +        .getMobRegionInfo(tn).getEncodedName(), family.getNameAsString()));
 +    if (region.getRegionServerServices() != null) {
 +      tableLockManager = region.getRegionServerServices().getTableLockManager();
 +      tableLockName = MobUtils.getTableLockName(getTableName());
 +    }
 +  }
 +
 +  /**
 +   * Creates the mob cache config.
 +   */
 +  @Override
 +  protected void createCacheConf(HColumnDescriptor family) {
 +    cacheConf = new MobCacheConfig(conf, family);
 +  }
 +
 +  /**
 +   * Gets current config.
 +   */
 +  public Configuration getConfiguration() {
 +    return this.conf;
 +  }
 +
 +  /**
 +   * Gets the MobStoreScanner or MobReversedStoreScanner. In these scanners, a additional seeks in
 +   * the mob files should be performed after the seek in HBase is done.
 +   */
 +  @Override
 +  protected KeyValueScanner createScanner(Scan scan, final NavigableSet<byte[]> targetCols,
 +      long readPt, KeyValueScanner scanner) throws IOException {
 +    if (scanner == null) {
 +      if (MobUtils.isRefOnlyScan(scan)) {
 +        Filter refOnlyFilter = new MobReferenceOnlyFilter();
 +        Filter filter = scan.getFilter();
 +        if (filter != null) {
 +          scan.setFilter(new FilterList(filter, refOnlyFilter));
 +        } else {
 +          scan.setFilter(refOnlyFilter);
 +        }
 +      }
 +      scanner = scan.isReversed() ? new ReversedMobStoreScanner(this, getScanInfo(), scan,
 +          targetCols, readPt) : new MobStoreScanner(this, getScanInfo(), scan, targetCols, readPt);
 +    }
 +    return scanner;
 +  }
 +
 +  /**
 +   * Creates the mob store engine.
 +   */
 +  @Override
 +  protected StoreEngine<?, ?, ?, ?> createStoreEngine(Store store, Configuration conf,
 +      KVComparator kvComparator) throws IOException {
 +    MobStoreEngine engine = new MobStoreEngine();
 +    engine.createComponents(conf, store, kvComparator);
 +    return engine;
 +  }
 +
 +  /**
 +   * Gets the temp directory.
 +   * @return The temp directory.
 +   */
 +  private Path getTempDir() {
 +    return new Path(homePath, MobConstants.TEMP_DIR_NAME);
 +  }
 +
 +  /**
 +   * Creates the writer for the mob file in temp directory.
 +   * @param date The latest date of written cells.
 +   * @param maxKeyCount The key count.
 +   * @param compression The compression algorithm.
 +   * @param startKey The start key.
 +   * @return The writer for the mob file.
 +   * @throws IOException
 +   */
 +  public StoreFile.Writer createWriterInTmp(Date date, long maxKeyCount,
 +      Compression.Algorithm compression, byte[] startKey) throws IOException {
 +    if (startKey == null) {
 +      startKey = HConstants.EMPTY_START_ROW;
 +    }
 +    Path path = getTempDir();
 +    return createWriterInTmp(MobUtils.formatDate(date), path, maxKeyCount, compression, startKey);
 +  }
 +
 +  /**
 +   * Creates the writer for the del file in temp directory.
 +   * The del file keeps tracking the delete markers. Its name has a suffix _del,
 +   * the format is [0-9a-f]+(_del)?.
 +   * @param date The latest date of written cells.
 +   * @param maxKeyCount The key count.
 +   * @param compression The compression algorithm.
 +   * @param startKey The start key.
 +   * @return The writer for the del file.
 +   * @throws IOException
 +   */
 +  public StoreFile.Writer createDelFileWriterInTmp(Date date, long maxKeyCount,
 +      Compression.Algorithm compression, byte[] startKey) throws IOException {
 +    if (startKey == null) {
 +      startKey = HConstants.EMPTY_START_ROW;
 +    }
 +    Path path = getTempDir();
 +    String suffix = UUID
 +        .randomUUID().toString().replaceAll("-", "") + "_del";
 +    MobFileName mobFileName = MobFileName.create(startKey, MobUtils.formatDate(date), suffix);
 +    return createWriterInTmp(mobFileName, path, maxKeyCount, compression);
 +  }
 +
 +  /**
 +   * Creates the writer for the mob file in temp directory.
 +   * @param date The date string, its format is yyyymmmdd.
 +   * @param basePath The basic path for a temp directory.
 +   * @param maxKeyCount The key count.
 +   * @param compression The compression algorithm.
 +   * @param startKey The start key.
 +   * @return The writer for the mob file.
 +   * @throws IOException
 +   */
 +  public StoreFile.Writer createWriterInTmp(String date, Path basePath, long maxKeyCount,
 +      Compression.Algorithm compression, byte[] startKey) throws IOException {
 +    MobFileName mobFileName = MobFileName.create(startKey, date, UUID.randomUUID()
 +        .toString().replaceAll("-", ""));
 +    return createWriterInTmp(mobFileName, basePath, maxKeyCount, compression);
 +  }
 +
 +  /**
 +   * Creates the writer for the mob file in temp directory.
 +   * @param mobFileName The mob file name.
 +   * @param basePath The basic path for a temp directory.
 +   * @param maxKeyCount The key count.
 +   * @param compression The compression algorithm.
 +   * @return The writer for the mob file.
 +   * @throws IOException
 +   */
 +  public StoreFile.Writer createWriterInTmp(MobFileName mobFileName, Path basePath, long maxKeyCount,
 +      Compression.Algorithm compression) throws IOException {
 +    final CacheConfig writerCacheConf = mobCacheConfig;
 +    HFileContext hFileContext = new HFileContextBuilder().withCompression(compression)
 +        .withIncludesMvcc(false).withIncludesTags(true)
 +        .withChecksumType(HFile.DEFAULT_CHECKSUM_TYPE)
 +        .withBytesPerCheckSum(HFile.DEFAULT_BYTES_PER_CHECKSUM)
 +        .withBlockSize(getFamily().getBlocksize())
 +        .withHBaseCheckSum(true).withDataBlockEncoding(getFamily().getDataBlockEncoding()).build();
 +
 +    StoreFile.Writer w = new StoreFile.WriterBuilder(conf, writerCacheConf, region.getFilesystem())
 +        .withFilePath(new Path(basePath, mobFileName.getFileName()))
 +        .withComparator(KeyValue.COMPARATOR).withBloomType(BloomType.NONE)
 +        .withMaxKeyCount(maxKeyCount).withFileContext(hFileContext).build();
 +    return w;
 +  }
 +
 +  /**
 +   * Commits the mob file.
 +   * @param sourceFile The source file.
 +   * @param targetPath The directory path where the source file is renamed to.
 +   * @throws IOException
 +   */
 +  public void commitFile(final Path sourceFile, Path targetPath) throws IOException {
 +    if (sourceFile == null) {
 +      return;
 +    }
 +    Path dstPath = new Path(targetPath, sourceFile.getName());
 +    validateMobFile(sourceFile);
 +    String msg = "Renaming flushed file from " + sourceFile + " to " + dstPath;
 +    LOG.info(msg);
 +    Path parent = dstPath.getParent();
 +    if (!region.getFilesystem().exists(parent)) {
 +      region.getFilesystem().mkdirs(parent);
 +    }
 +    if (!region.getFilesystem().rename(sourceFile, dstPath)) {
 +      throw new IOException("Failed rename of " + sourceFile + " to " + dstPath);
 +    }
 +  }
 +
 +  /**
 +   * Validates a mob file by opening and closing it.
 +   *
 +   * @param path the path to the mob file
 +   */
 +  private void validateMobFile(Path path) throws IOException {
 +    StoreFile storeFile = null;
 +    try {
 +      storeFile =
 +          new StoreFile(region.getFilesystem(), path, conf, this.mobCacheConfig, BloomType.NONE);
 +      storeFile.createReader();
 +    } catch (IOException e) {
 +      LOG.error("Fail to open mob file[" + path + "], keep it in temp directory.", e);
 +      throw e;
 +    } finally {
 +      if (storeFile != null) {
 +        storeFile.closeReader(false);
 +      }
 +    }
 +  }
 +
 +  /**
 +   * Reads the cell from the mob file.
 +   * @param reference The cell found in the HBase, its value is a path to a mob file.
 +   * @param cacheBlocks Whether the scanner should cache blocks.
 +   * @return The cell found in the mob file.
 +   * @throws IOException
 +   */
 +  public Cell resolve(Cell reference, boolean cacheBlocks) throws IOException {
 +    Cell result = null;
 +    if (MobUtils.hasValidMobRefCellValue(reference)) {
 +      String fileName = MobUtils.getMobFileName(reference);
 +      result = readCell(mobDirLocations, fileName, reference, cacheBlocks);
 +      if (result == null) {
 +        result = readClonedCell(fileName, reference, cacheBlocks);
 +      }
 +    }
 +    if (result == null) {
 +      LOG.warn("The KeyValue result is null, assemble a new KeyValue with the same row,family,"
 +          + "qualifier,timestamp,type and tags but with an empty value to return.");
 +      result = new KeyValue(reference.getRowArray(), reference.getRowOffset(),
 +          reference.getRowLength(), reference.getFamilyArray(), reference.getFamilyOffset(),
 +          reference.getFamilyLength(), reference.getQualifierArray(),
 +          reference.getQualifierOffset(), reference.getQualifierLength(), reference.getTimestamp(),
 +          Type.codeToType(reference.getTypeByte()), HConstants.EMPTY_BYTE_ARRAY,
 +          0, 0, reference.getTagsArray(), reference.getTagsOffset(),
 +          reference.getTagsLength());
 +    }
 +    return result;
 +  }
 +
 +  /**
 +   * Reads the cell from a mob file.
 +   * The mob file might be located in different directories.
 +   * 1. The working directory.
 +   * 2. The archive directory.
 +   * Reads the cell from the files located in both of the above directories.
 +   * @param locations The possible locations where the mob files are saved.
 +   * @param fileName The file to be read.
 +   * @param search The cell to be searched.
 +   * @param cacheMobBlocks Whether the scanner should cache blocks.
 +   * @return The found cell. Null if there's no such a cell.
 +   * @throws IOException
 +   */
 +  private Cell readCell(List<Path> locations, String fileName, Cell search, boolean cacheMobBlocks)
 +      throws IOException {
 +    FileSystem fs = getFileSystem();
 +    for (Path location : locations) {
 +      MobFile file = null;
 +      Path path = new Path(location, fileName);
 +      try {
 +        file = mobCacheConfig.getMobFileCache().openFile(fs, path, mobCacheConfig);
 +        return file.readCell(search, cacheMobBlocks);
 +      } catch (IOException e) {
 +        mobCacheConfig.getMobFileCache().evictFile(fileName);
 +        if (e instanceof FileNotFoundException) {
 +          LOG.warn("Fail to read the cell, the mob file " + path + " doesn't exist", e);
 +        } else {
 +          throw e;
 +        }
 +      } finally {
 +        if (file != null) {
 +          mobCacheConfig.getMobFileCache().closeFile(file);
 +        }
 +      }
 +    }
 +    LOG.error("The mob file " + fileName + " could not be found in the locations "
 +        + mobDirLocations);
 +    return null;
 +  }
 +
 +  /**
 +   * Reads the cell from a mob file of source table.
 +   * The table might be cloned, in this case only hfile link is created in the new table,
 +   * and the mob file is located in the source table directories.
 +   * 1. The working directory of the source table.
 +   * 2. The archive directory of the source table.
 +   * Reads the cell from the files located in both of the above directories.
 +   * @param fileName The file to be read.
 +   * @param search The cell to be searched.
 +   * @param cacheMobBlocks Whether the scanner should cache blocks.
 +   * @return The found cell. Null if there's no such a cell.
 +   * @throws IOException
 +   */
 +  private Cell readClonedCell(String fileName, Cell search, boolean cacheMobBlocks)
 +      throws IOException {
 +    Tag tableNameTag = MobUtils.getTableNameTag(search);
 +    if (tableNameTag == null) {
 +      return null;
 +    }
 +    byte[] tableName = tableNameTag.getValue();
 +    if (Bytes.equals(this.getTableName().getName(), tableName)) {
 +      return null;
 +    }
 +    // the possible locations in the source table.
 +    List<Path> locations = new ArrayList<Path>();
 +    TableName tn = TableName.valueOf(tableName);
 +    locations.add(MobUtils.getMobFamilyPath(conf, tn, family.getNameAsString()));
 +    locations.add(HFileArchiveUtil.getStoreArchivePath(conf, tn, MobUtils.getMobRegionInfo(tn)
 +        .getEncodedName(), family.getNameAsString()));
 +    // read the cell from the source table.
 +    return readCell(locations, fileName, search, cacheMobBlocks);
 +  }
 +
 +  /**
 +   * Gets the mob file path.
 +   * @return The mob file path.
 +   */
 +  public Path getPath() {
 +    return mobFamilyPath;
 +  }
 +
 +  /**
 +   * The compaction in the store of mob.
 +   * The cells in this store contains the path of the mob files. There might be race
 +   * condition between the major compaction and the sweeping in mob files.
 +   * In order to avoid this, we need mutually exclude the running of the major compaction and
 +   * sweeping in mob files.
 +   * The minor compaction is not affected.
 +   * The major compaction is marked as retainDeleteMarkers when a sweeping is in progress.
 +   */
 +  @Override
-   public List<StoreFile> compact(CompactionContext compaction) throws IOException {
++  public List<StoreFile> compact(CompactionContext compaction,
++      CompactionThroughputController throughputController) throws IOException {
 +    // If it's major compaction, try to find whether there's a sweeper is running
 +    // If yes, mark the major compaction as retainDeleteMarkers
 +    if (compaction.getRequest().isAllFiles()) {
 +      // Use the Zookeeper to coordinate.
 +      // 1. Acquire a operation lock.
 +      //   1.1. If no, mark the major compaction as retainDeleteMarkers and continue the compaction.
 +      //   1.2. If the lock is obtained, search the node of sweeping.
 +      //      1.2.1. If the node is there, the sweeping is in progress, mark the major
 +      //             compaction as retainDeleteMarkers and continue the compaction.
 +      //      1.2.2. If the node is not there, add a child to the major compaction node, and
 +      //             run the compaction directly.
 +      TableLock lock = null;
 +      if (tableLockManager != null) {
 +        lock = tableLockManager.readLock(tableLockName, "Major compaction in HMobStore");
 +      }
 +      boolean tableLocked = false;
 +      String tableName = getTableName().getNameAsString();
 +      if (lock != null) {
 +        try {
 +          LOG.info("Start to acquire a read lock for the table[" + tableName
 +              + "], ready to perform the major compaction");
 +          lock.acquire();
 +          tableLocked = true;
 +        } catch (Exception e) {
 +          LOG.error("Fail to lock the table " + tableName, e);
 +        }
 +      } else {
 +        // If the tableLockManager is null, mark the tableLocked as true.
 +        tableLocked = true;
 +      }
 +      try {
 +        if (!tableLocked) {
 +          LOG.warn("Cannot obtain the table lock, maybe a sweep tool is running on this table["
 +              + tableName + "], forcing the delete markers to be retained");
 +          compaction.getRequest().forceRetainDeleteMarkers();
 +        }
-         return super.compact(compaction);
++        return super.compact(compaction, throughputController);
 +      } finally {
 +        if (tableLocked && lock != null) {
 +          try {
 +            lock.release();
 +          } catch (IOException e) {
 +            LOG.error("Fail to release the table lock " + tableName, e);
 +          }
 +        }
 +      }
 +    } else {
 +      // If it's not a major compaction, continue the compaction.
-       return super.compact(compaction);
++      return super.compact(compaction, throughputController);
 +    }
 +  }
 +
 +  public void updateMobCompactedIntoMobCellsCount(long count) {
 +    mobCompactedIntoMobCellsCount += count;
 +  }
 +
 +  public long getMobCompactedIntoMobCellsCount() {
 +    return mobCompactedIntoMobCellsCount;
 +  }
 +
 +  public void updateMobCompactedFromMobCellsCount(long count) {
 +    mobCompactedFromMobCellsCount += count;
 +  }
 +
 +  public long getMobCompactedFromMobCellsCount() {
 +    return mobCompactedFromMobCellsCount;
 +  }
 +
 +  public void updateMobCompactedIntoMobCellsSize(long size) {
 +    mobCompactedIntoMobCellsSize += size;
 +  }
 +
 +  public long getMobCompactedIntoMobCellsSize() {
 +    return mobCompactedIntoMobCellsSize;
 +  }
 +
 +  public void updateMobCompactedFromMobCellsSize(long size) {
 +    mobCompactedFromMobCellsSize += size;
 +  }
 +
 +  public long getMobCompactedFromMobCellsSize() {
 +    return mobCompactedFromMobCellsSize;
 +  }
 +
 +  public void updateMobFlushCount() {
 +    mobFlushCount++;
 +  }
 +
 +  public long getMobFlushCount() {
 +    return mobFlushCount;
 +  }
 +
 +  public void updateMobFlushedCellsCount(long count) {
 +    mobFlushedCellsCount += count;
 +  }
 +
 +  public long getMobFlushedCellsCount() {
 +    return mobFlushedCellsCount;
 +  }
 +
 +  public void updateMobFlushedCellsSize(long size) {
 +    mobFlushedCellsSize += size;
 +  }
 +
 +  public long getMobFlushedCellsSize() {
 +    return mobFlushedCellsSize;
 +  }
 +
 +  public void updateMobScanCellsCount(long count) {
 +    mobScanCellsCount += count;
 +  }
 +
 +  public long getMobScanCellsCount() {
 +    return mobScanCellsCount;
 +  }
 +
 +  public void updateMobScanCellsSize(long size) {
 +    mobScanCellsSize += size;
 +  }
 +
 +  public long getMobScanCellsSize() {
 +    return mobScanCellsSize;
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
diff --cc hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index df5c900,53e732a..ab0165d
--- 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
@@@ -33,9 -34,9 +34,10 @@@ import java.util.HashSet
  import java.util.Iterator;
  import java.util.List;
  import java.util.Map;
 +import java.util.Map.Entry;
  import java.util.NavigableMap;
  import java.util.NavigableSet;
+ import java.util.RandomAccess;
  import java.util.Set;
  import java.util.TreeMap;
  import java.util.concurrent.Callable;

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --cc hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index e11aac2,c170a65..787828b
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@@ -347,9 -372,7 +373,9 @@@ public class HRegionServer extends HasT
    private final RegionServerAccounting regionServerAccounting;
  
    // Cache configuration and block cache reference
-   final CacheConfig cacheConfig;
+   protected CacheConfig cacheConfig;
 +  // Cache configuration for mob
 +  final MobCacheConfig mobCacheConfig;
  
    /** The health check chore. */
    private HealthCheckChore healthCheckChore;
@@@ -831,10 -933,9 +938,10 @@@
        }
      }
      // Send cache a shutdown.
-     if (cacheConfig.isBlockCacheEnabled()) {
+     if (cacheConfig != null && cacheConfig.isBlockCacheEnabled()) {
        cacheConfig.getBlockCache().shutdown();
      }
 +    mobCacheConfig.getMobFileCache().shutdown();
  
      if (movedRegionsCleaner != null) {
        movedRegionsCleaner.stop("Region Server stopping");

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
----------------------------------------------------------------------
diff --cc hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
index b9f4038,252e5e1..f5bb67a
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
@@@ -55,10 -55,12 +55,13 @@@ import org.apache.hadoop.hbase.HColumnD
  import org.apache.hadoop.hbase.HConstants;
  import org.apache.hadoop.hbase.HRegionInfo;
  import org.apache.hadoop.hbase.KeyValue;
- import org.apache.hadoop.hbase.KeyValueUtil;
  import org.apache.hadoop.hbase.TableName;
 +import org.apache.hadoop.hbase.KeyValue.KVComparator;
+ import org.apache.hadoop.hbase.Tag;
+ import org.apache.hadoop.hbase.TagType;
+ import org.apache.hadoop.hbase.classification.InterfaceAudience;
  import org.apache.hadoop.hbase.client.Scan;
+ import org.apache.hadoop.hbase.conf.ConfigurationManager;
  import org.apache.hadoop.hbase.io.compress.Compression;
  import org.apache.hadoop.hbase.io.crypto.Cipher;
  import org.apache.hadoop.hbase.io.crypto.Encryption;
@@@ -133,11 -137,11 +138,11 @@@ public class HStore implements Store 
  
    protected final MemStore memstore;
    // This stores directory in the filesystem.
 -  private final HRegion region;
 +  protected final HRegion region;
    private final HColumnDescriptor family;
    private final HRegionFileSystem fs;
-   protected final Configuration conf;
 -  private Configuration conf;
 -  private final CacheConfig cacheConf;
++  protected Configuration conf;
 +  protected CacheConfig cacheConf;
    private long lastCompactSize = 0;
    volatile boolean forceMajor = false;
    /* how many bytes to write between status checks */

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
----------------------------------------------------------------------
diff --cc hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
index 4384d87,5e5590d..159ec55
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
@@@ -32,10 -33,11 +33,13 @@@ import org.apache.hadoop.hbase.ServerNa
  import org.apache.hadoop.hbase.io.hfile.BlockCache;
  import org.apache.hadoop.hbase.io.hfile.CacheConfig;
  import org.apache.hadoop.hbase.io.hfile.CacheStats;
 +import org.apache.hadoop.hbase.mob.MobCacheConfig;
 +import org.apache.hadoop.hbase.mob.MobFileCache;
+ import org.apache.hadoop.hbase.wal.DefaultWALProvider;
  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+ import org.apache.hadoop.hbase.util.FSUtils;
  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+ import org.apache.hadoop.hdfs.DFSHedgedReadMetrics;
  import org.apache.hadoop.metrics2.MetricsExecutor;
  
  /**
@@@ -50,11 -52,10 +54,11 @@@ class MetricsRegionServerWrapperImp
    private final HRegionServer regionServer;
  
    private BlockCache blockCache;
 +  private MobFileCache mobFileCache;
  
    private volatile long numStores = 0;
-   private volatile long numHLogFiles = 0;
-   private volatile long hlogFileSize = 0;
+   private volatile long numWALFiles = 0;
+   private volatile long walFileSize = 0;
    private volatile long numStoreFiles = 0;
    private volatile long memstoreSize = 0;
    private volatile long storeFileSize = 0;
@@@ -75,20 -76,7 +79,21 @@@
    private volatile long flushedCellsSize = 0;
    private volatile long compactedCellsSize = 0;
    private volatile long majorCompactedCellsSize = 0;
 +  private volatile long mobCompactedIntoMobCellsCount = 0;
 +  private volatile long mobCompactedFromMobCellsCount = 0;
 +  private volatile long mobCompactedIntoMobCellsSize = 0;
 +  private volatile long mobCompactedFromMobCellsSize = 0;
 +  private volatile long mobFlushCount = 0;
 +  private volatile long mobFlushedCellsCount = 0;
 +  private volatile long mobFlushedCellsSize = 0;
 +  private volatile long mobScanCellsCount = 0;
 +  private volatile long mobScanCellsSize = 0;
 +  private volatile long mobFileCacheAccessCount = 0;
 +  private volatile long mobFileCacheMissCount = 0;
 +  private volatile double mobFileCacheHitRatio = 0;
 +  private volatile long mobFileCacheEvictedCount = 0;
 +  private volatile long mobFileCacheCount = 0;
+   private volatile long blockedRequestsCount = 0L;
  
    private CacheStats cacheStats;
    private ScheduledExecutorService executor;
@@@ -526,15 -450,7 +549,16 @@@
        long tempFlushedCellsSize = 0;
        long tempCompactedCellsSize = 0;
        long tempMajorCompactedCellsSize = 0;
 +      long tempMobCompactedIntoMobCellsCount = 0;
 +      long tempMobCompactedFromMobCellsCount = 0;
 +      long tempMobCompactedIntoMobCellsSize = 0;
 +      long tempMobCompactedFromMobCellsSize = 0;
 +      long tempMobFlushCount = 0;
 +      long tempMobFlushedCellsCount = 0;
 +      long tempMobFlushedCellsSize = 0;
 +      long tempMobScanCellsCount = 0;
 +      long tempMobScanCellsSize = 0;
+       long tempBlockedRequestsCount = 0L;
  
        for (HRegion r : regionServer.getOnlineRegionsLocalContext()) {
          tempNumMutationsWithoutWAL += r.numMutationsWithoutWAL.get();
@@@ -631,20 -526,22 +646,36 @@@
        flushedCellsSize = tempFlushedCellsSize;
        compactedCellsSize = tempCompactedCellsSize;
        majorCompactedCellsSize = tempMajorCompactedCellsSize;
 +      mobCompactedIntoMobCellsCount = tempMobCompactedIntoMobCellsCount;
 +      mobCompactedFromMobCellsCount = tempMobCompactedFromMobCellsCount;
 +      mobCompactedIntoMobCellsSize = tempMobCompactedIntoMobCellsSize;
 +      mobCompactedFromMobCellsSize = tempMobCompactedFromMobCellsSize;
 +      mobFlushCount = tempMobFlushCount;
 +      mobFlushedCellsCount = tempMobFlushedCellsCount;
 +      mobFlushedCellsSize = tempMobFlushedCellsSize;
 +      mobScanCellsCount = tempMobScanCellsCount;
 +      mobScanCellsSize = tempMobScanCellsSize;
 +      mobFileCacheAccessCount = mobFileCache.getAccessCount();
 +      mobFileCacheMissCount = mobFileCache.getMissCount();
 +      mobFileCacheHitRatio = mobFileCache.getHitRatio();
 +      mobFileCacheEvictedCount = mobFileCache.getEvictedFileCount();
 +      mobFileCacheCount = mobFileCache.getCacheSize();
+       blockedRequestsCount = tempBlockedRequestsCount;
      }
    }
+ 
+   @Override
+   public long getHedgedReadOps() {
+     return this.dfsHedgedReadMetrics == null? 0: this.dfsHedgedReadMetrics.getHedgedReadOps();
+   }
+ 
+   @Override
+   public long getHedgedReadWins() {
+     return this.dfsHedgedReadMetrics == null? 0: this.dfsHedgedReadMetrics.getHedgedReadWins();
+   }
+ 
+   @Override
+   public long getBlockedRequestsCount() {
+     return blockedRequestsCount;
+   }
  }

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

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

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionRequest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
----------------------------------------------------------------------
diff --cc hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
index 13967c2,a92c17e..7870040
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
@@@ -43,10 -43,9 +43,12 @@@ import org.apache.hadoop.hbase.regionse
  import org.apache.hadoop.hbase.regionserver.StoreFile;
  import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
  import org.apache.hadoop.hbase.regionserver.StoreScanner;
 +import org.apache.hadoop.hbase.regionserver.TimeRangeTracker;
  import org.apache.hadoop.hbase.util.Bytes;
 +import org.apache.hadoop.hbase.util.Writables;
 +import org.apache.hadoop.util.StringUtils;
+ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+ import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
  
  /**
   * A compactor is a compaction algorithm associated a given policy. Base class also contains
@@@ -226,64 -220,87 +233,90 @@@ public abstract class Compactor 
      return store.getCoprocessorHost().preCompact(store, scanner, scanType, request);
    }
  
++  // TODO mob introduced the fd parameter; can we make this cleaner and easier to extend in future?
    /**
     * Performs the compaction.
-    * @param fd File details
++   * @param fd FileDetails of cell sink writer
     * @param scanner Where to read from.
     * @param writer Where to write to.
     * @param smallestReadPoint Smallest read point.
     * @param cleanSeqId When true, remove seqId(used to be mvcc) value which is <= smallestReadPoint
 +   * @param major Is a major compaction.
     * @return Whether compaction ended; false if it was interrupted for some reason.
     */
 -  protected boolean performCompaction(InternalScanner scanner, CellSink writer,
 +  protected boolean performCompaction(FileDetails fd, InternalScanner scanner, CellSink writer,
-       long smallestReadPoint, boolean cleanSeqId, boolean major) throws IOException {
-     int bytesWritten = 0;
+       long smallestReadPoint, boolean cleanSeqId,
 -      CompactionThroughputController throughputController) throws IOException {
++      CompactionThroughputController throughputController, boolean major) throws IOException {
+     long bytesWritten = 0;
+     long bytesWrittenProgress = 0;
      // Since scanner.next() can return 'false' but still be delivering data,
      // we have to use a do/while loop.
-     List<Cell> kvs = new ArrayList<Cell>();
-     int closeCheckInterval = HStore.getCloseCheckInterval();
-     long lastMillis;
+     List<Cell> cells = new ArrayList<Cell>();
+     long closeCheckInterval = HStore.getCloseCheckInterval();
+     long lastMillis = 0;
      if (LOG.isDebugEnabled()) {
-       lastMillis = System.currentTimeMillis();
-     } else {
-       lastMillis = 0;
+       lastMillis = EnvironmentEdgeManager.currentTime();
      }
+     String compactionName =
+         store.getRegionInfo().getRegionNameAsString() + "#" + store.getFamily().getNameAsString();
+     long now = 0;
      boolean hasMore;
-     do {
-       hasMore = scanner.next(kvs, compactionKVMax);
-       // output to writer:
-       for (Cell c : kvs) {
-         KeyValue kv = KeyValueUtil.ensureKeyValue(c);
-         resetSeqId(smallestReadPoint, cleanSeqId, kv);
-         writer.append(kv);
-         ++progress.currentCompactedKVs;
-         progress.totalCompactedSize += kv.getLength();
- 
-         // check periodically to see if a system stop is requested
-         if (closeCheckInterval > 0) {
-           bytesWritten += kv.getLength();
-           if (bytesWritten > closeCheckInterval) {
-             // Log the progress of long running compactions every minute if
-             // logging at DEBUG level
-             if (LOG.isDebugEnabled()) {
-               long now = System.currentTimeMillis();
-               if ((now - lastMillis) >= 60 * 1000) {
-                 LOG.debug("Compaction progress: " + progress + String.format(", rate=%.2f kB/sec",
-                   (bytesWritten / 1024.0) / ((now - lastMillis) / 1000.0)));
-                 lastMillis = now;
+     throughputController.start(compactionName);
+     try {
+       do {
+         hasMore = scanner.next(cells, compactionKVMax);
+         if (LOG.isDebugEnabled()) {
+           now = EnvironmentEdgeManager.currentTime();
+         }
+         // output to writer:
+         for (Cell c : cells) {
+           if (cleanSeqId && c.getSequenceId() <= smallestReadPoint) {
+             CellUtil.setSequenceId(c, 0);
+           }
+           writer.append(c);
+           int len = KeyValueUtil.length(c);
+           ++progress.currentCompactedKVs;
+           progress.totalCompactedSize += len;
+           if (LOG.isDebugEnabled()) {
+             bytesWrittenProgress += len;
+           }
+           throughputController.control(compactionName, len);
+           // check periodically to see if a system stop is requested
+           if (closeCheckInterval > 0) {
+             bytesWritten += len;
+             if (bytesWritten > closeCheckInterval) {
+               bytesWritten = 0;
+               if (!store.areWritesEnabled()) {
+                 progress.cancel();
+                 return false;
                }
              }
-             bytesWritten = 0;
-             if (!store.areWritesEnabled()) {
-               progress.cancel();
-               return false;
-             }
            }
          }
-       }
-       kvs.clear();
-     } while (hasMore);
+         // Log the progress of long running compactions every minute if
+         // logging at DEBUG level
+         if (LOG.isDebugEnabled()) {
+           if ((now - lastMillis) >= 60 * 1000) {
+             LOG.debug("Compaction progress: "
+                 + compactionName
+                 + " "
+                 + progress
+                 + String.format(", rate=%.2f kB/sec", (bytesWrittenProgress / 1024.0)
+                     / ((now - lastMillis) / 1000.0)) + ", throughputController is "
+                 + throughputController);
+             lastMillis = now;
+             bytesWrittenProgress = 0;
+           }
+         }
+         cells.clear();
+       } while (hasMore);
+     } catch (InterruptedException e) {
+       progress.cancel();
+       throw new InterruptedIOException("Interrupted while control throughput of compacting "
+           + compactionName);
+     } finally {
+       throughputController.finish(compactionName);
+     }
      progress.complete();
      return true;
    }
@@@ -321,29 -338,4 +354,17 @@@
      return new StoreScanner(store, store.getScanInfo(), scan, scanners, smallestReadPoint,
          earliestPutTs, dropDeletesFromRow, dropDeletesToRow);
    }
 +
 +  /**
-    * Resets the sequence id.
-    * @param smallestReadPoint The smallest mvcc readPoint across all the scanners in this region.
-    * @param cleanSeqId Should clean the sequence id.
-    * @param kv The current KeyValue.
-    */
-   protected void resetSeqId(long smallestReadPoint, boolean cleanSeqId, KeyValue kv) {
-     if (cleanSeqId && kv.getSequenceId() <= smallestReadPoint) {
-       kv.setSequenceId(0);
-     }
-   }
- 
-   /**
 +   * Appends the metadata and closes the writer.
 +   * @param writer The current store writer.
 +   * @param fd The file details.
 +   * @param isMajor Is a major compaction.
 +   * @throws IOException
 +   */
 +  protected void appendMetadataAndCloseWriter(StoreFile.Writer writer, FileDetails fd,
 +      boolean isMajor) throws IOException {
 +    writer.appendMetadata(fd.maxSeqId, isMajor);
 +    writer.close();
 +  }
  }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java
----------------------------------------------------------------------
diff --cc hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java
index be859c0,5d712c1..090be8c
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java
@@@ -76,9 -98,11 +98,10 @@@ public class DefaultCompactor extends C
            smallestReadPoint = Math.min(fd.minSeqIdToKeep, smallestReadPoint);
            cleanSeqId = true;
          }
 -        
 -        writer = store.createWriterInTmp(fd.maxKeyCount, this.compactionCompression, true,
 -            true, fd.maxTagsLength > 0);
 +        writer = createTmpWriter(fd, smallestReadPoint);
-         boolean finished = performCompaction(fd, scanner, writer, smallestReadPoint, cleanSeqId,
-             request.isAllFiles());
+         boolean finished =
 -            performCompaction(scanner, writer, smallestReadPoint, cleanSeqId, throughputController);
++            performCompaction(fd, scanner, writer, smallestReadPoint, cleanSeqId, throughputController,
++                    request.isAllFiles());
          if (!finished) {
            writer.close();
            store.getFileSystem().delete(writer.getPath(), false);
@@@ -102,22 -146,8 +145,22 @@@
    }
  
    /**
 +   * Creates a writer for a new file in a temporary directory.
 +   * @param fd The file details.
 +   * @param smallestReadPoint The smallest mvcc readPoint across all the scanners in this region.
 +   * @return Writer for a new StoreFile in the tmp dir.
 +   * @throws IOException
 +   */
 +  protected StoreFile.Writer createTmpWriter(FileDetails fd, long smallestReadPoint)
 +      throws IOException {
 +    StoreFile.Writer writer = store.createWriterInTmp(fd.maxKeyCount, this.compactionCompression,
 +        true, fd.maxMVCCReadpoint >= smallestReadPoint, fd.maxTagsLength > 0);
 +    return writer;
 +  }
 +
 +  /**
     * Compact a list of files for testing. Creates a fake {@link CompactionRequest} to pass to
-    * {@link #compact(CompactionRequest)};
+    * {@link #compact(CompactionRequest, CompactionThroughputController)};
     * @param filesToCompact the files to compact. These are used as the compactionSelection for
     *          the generated {@link CompactionRequest}.
     * @param isMajor true to major compact (prune all deletes, max versions, etc)

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/StripeCompactor.java
----------------------------------------------------------------------
diff --cc hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/StripeCompactor.java
index 3109015,b957e16..fab4c2f
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/StripeCompactor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/StripeCompactor.java
@@@ -127,8 -130,8 +130,9 @@@ public class StripeCompactor extends Co
        // It is ok here if storeScanner is null.
        StoreScanner storeScanner = (scanner instanceof StoreScanner) ? (StoreScanner)scanner : null;
        mw.init(storeScanner, factory, store.getComparator());
-       finished = performCompaction(fd, scanner, mw, smallestReadPoint, cleanSeqId,
-           request.isMajor());
+       finished =
 -          performCompaction(scanner, mw, smallestReadPoint, cleanSeqId, throughputController);
++          performCompaction(fd, scanner, mw, smallestReadPoint, cleanSeqId, throughputController,
++                  request.isMajor());
        if (!finished) {
          throw new InterruptedIOException( "Aborting compaction of store " + store +
              " in region " + store.getRegionInfo().getRegionNameAsString() +

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
----------------------------------------------------------------------
diff --cc hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
index 0d7efe7,2655e2b..841bc04
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
@@@ -52,21 -51,16 +51,20 @@@ import org.apache.hadoop.hbase.HConstan
  import org.apache.hadoop.hbase.HRegionInfo;
  import org.apache.hadoop.hbase.io.FileLink;
  import org.apache.hadoop.hbase.io.HFileLink;
- import org.apache.hadoop.hbase.io.HLogLink;
 +import org.apache.hadoop.hbase.io.hfile.HFile;
 +import org.apache.hadoop.hbase.mapreduce.JobUtil;
+ import org.apache.hadoop.hbase.io.WALLink;
  import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
 +import org.apache.hadoop.hbase.mob.MobUtils;
  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
  import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo;
  import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
- import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
  import org.apache.hadoop.hbase.util.FSUtils;
 +import org.apache.hadoop.hbase.util.HFileArchiveUtil;
  import org.apache.hadoop.hbase.util.Pair;
  import org.apache.hadoop.io.BytesWritable;
+ import org.apache.hadoop.io.IOUtils;
  import org.apache.hadoop.io.NullWritable;
- import org.apache.hadoop.io.SequenceFile;
  import org.apache.hadoop.io.Writable;
  import org.apache.hadoop.mapreduce.Job;
  import org.apache.hadoop.mapreduce.JobContext;
@@@ -423,10 -433,10 +437,10 @@@ public class ExportSnapshot extends Con
          switch (fileInfo.getType()) {
            case HFILE:
              Path inputPath = new Path(fileInfo.getHfile());
 -            link = HFileLink.buildFromHFileLinkPattern(conf, inputPath);
 +            link = getFileLink(inputPath, conf);
              break;
            case WAL:
-             link = new HLogLink(inputRoot, fileInfo.getWalServer(), fileInfo.getWalName());
+             link = new WALLink(inputRoot, fileInfo.getWalServer(), fileInfo.getWalName());
              break;
            default:
              throw new IOException("Invalid File Type: " + fileInfo.getType().toString());
@@@ -442,16 -452,6 +456,16 @@@
        }
      }
  
 +    private FileLink getFileLink(Path path, Configuration conf) throws IOException{
 +      String regionName = HFileLink.getReferencedRegionName(path.getName());
 +      TableName tableName = HFileLink.getReferencedTableName(path.getName());
 +      if(MobUtils.getMobRegionInfo(tableName).getEncodedName().equals(regionName)) {
-         return new HFileLink(MobUtils.getQualifiedMobRootDir(conf),
++        return HFileLink.buildFromHFileLinkPattern(MobUtils.getQualifiedMobRootDir(conf),
 +                HFileArchiveUtil.getArchivePath(conf), path);
 +      }
-       return new HFileLink(inputRoot, inputArchive, path);
++      return HFileLink.buildFromHFileLinkPattern(inputRoot, inputArchive, path);
 +    }
 +
      private FileChecksum getFileChecksum(final FileSystem fs, final Path path) {
        try {
          return fs.getFileChecksum(path);

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotInfo.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java
----------------------------------------------------------------------
diff --cc hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java
index 6f7d847,330ead4..9d3407a
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java
@@@ -34,17 -34,12 +34,18 @@@ import org.apache.hadoop.hbase.classifi
  import org.apache.hadoop.conf.Configuration;
  import org.apache.hadoop.fs.FSDataInputStream;
  import org.apache.hadoop.fs.FSDataOutputStream;
 +import org.apache.hadoop.fs.FileStatus;
  import org.apache.hadoop.fs.FileSystem;
  import org.apache.hadoop.fs.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.TableDescriptor;
  import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
 +import org.apache.hadoop.hbase.mob.MobConstants;
 +import org.apache.hadoop.hbase.mob.MobUtils;
 +import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
  import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDataManifest;
  import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotReferenceUtil.java
----------------------------------------------------------------------
diff --cc hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotReferenceUtil.java
index 2a599d3,d1f787a..50b5c9a
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotReferenceUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotReferenceUtil.java
@@@ -39,14 -39,11 +39,14 @@@ import org.apache.hadoop.fs.Path
  import org.apache.hadoop.hbase.HRegionInfo;
  import org.apache.hadoop.hbase.TableName;
  import org.apache.hadoop.hbase.io.HFileLink;
 +import org.apache.hadoop.hbase.mob.MobUtils;
  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
  import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
- import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
+ import org.apache.hadoop.hbase.wal.DefaultWALProvider;
  import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
 +import org.apache.hadoop.hbase.util.FSUtils;
  import org.apache.hadoop.hbase.util.FSVisitor;
 +import org.apache.hadoop.hbase.util.HFileArchiveUtil;
  
  /**
   * Utility methods for interacting with the snapshot referenced files.
@@@ -299,15 -296,7 +299,15 @@@ public final class SnapshotReferenceUti
      }
  
      // check if the linked file exists (in the archive, or in the table dir)
 -    HFileLink link = HFileLink.buildFromHFileLinkPattern(conf, linkPath);
 +    HFileLink link = null;
 +    if (MobUtils.isMobRegionInfo(regionInfo)) {
 +      // for mob region
-       link = new HFileLink(MobUtils.getQualifiedMobRootDir(conf),
++      link = HFileLink.buildFromHFileLinkPattern(MobUtils.getQualifiedMobRootDir(conf),
 +          HFileArchiveUtil.getArchivePath(conf), linkPath);
 +    } else {
 +      // not mob region
-       link = new HFileLink(conf, linkPath);
++      link = HFileLink.buildFromHFileLinkPattern(conf, linkPath);
 +    }
      try {
        FileStatus fstat = link.getFileStatus(fs);
        if (storeFile.hasFileSize() && storeFile.getFileSize() != fstat.getLen()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClient.java
----------------------------------------------------------------------
diff --cc hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClient.java
index a939422,0000000..27d53ba
mode 100644,000000..100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClient.java
@@@ -1,250 -1,0 +1,251 @@@
 +/**
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.hadoop.hbase.client;
 +
 +import java.io.IOException;
 +
 +import org.apache.commons.logging.Log;
 +import org.apache.commons.logging.LogFactory;
 +import org.apache.hadoop.hbase.HBaseTestingUtility;
 +import org.apache.hadoop.hbase.HConstants;
- import org.apache.hadoop.hbase.LargeTests;
 +import org.apache.hadoop.hbase.NamespaceDescriptor;
 +import org.apache.hadoop.hbase.NamespaceNotFoundException;
 +import org.apache.hadoop.hbase.TableName;
 +import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 +import org.apache.hadoop.hbase.mob.MobConstants;
 +import org.apache.hadoop.hbase.snapshot.MobSnapshotTestingUtils;
 +import org.apache.hadoop.hbase.snapshot.SnapshotDoesNotExistException;
 +import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
++import org.apache.hadoop.hbase.testclassification.ClientTests;
++import org.apache.hadoop.hbase.testclassification.LargeTests;
 +import org.apache.hadoop.hbase.util.Bytes;
 +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;
 +
 +/**
 + * Test clone snapshots from the client
 + */
- @Category(LargeTests.class)
++@Category({LargeTests.class, ClientTests.class})
 +public class TestMobCloneSnapshotFromClient {
 +  final Log LOG = LogFactory.getLog(getClass());
 +
 +  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 +
 +  private final byte[] FAMILY = Bytes.toBytes("cf");
 +
 +  private byte[] emptySnapshot;
 +  private byte[] snapshotName0;
 +  private byte[] snapshotName1;
 +  private byte[] snapshotName2;
 +  private int snapshot0Rows;
 +  private int snapshot1Rows;
 +  private TableName tableName;
 +  private Admin admin;
 +
 +  @BeforeClass
 +  public static void setUpBeforeClass() throws Exception {
 +    TEST_UTIL.getConfiguration().setBoolean(SnapshotManager.HBASE_SNAPSHOT_ENABLED, true);
 +    TEST_UTIL.getConfiguration().setBoolean("hbase.online.schema.update.enable", true);
 +    TEST_UTIL.getConfiguration().setInt("hbase.hstore.compactionThreshold", 10);
 +    TEST_UTIL.getConfiguration().setInt("hbase.regionserver.msginterval", 100);
 +    TEST_UTIL.getConfiguration().setInt("hbase.client.pause", 250);
 +    TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 6);
 +    TEST_UTIL.getConfiguration().setBoolean(
 +        "hbase.master.enabletable.roundrobin", true);
 +    TEST_UTIL.getConfiguration().setInt(MobConstants.MOB_FILE_CACHE_SIZE_KEY, 0);
 +    TEST_UTIL.startMiniCluster(3);
 +  }
 +
 +  @AfterClass
 +  public static void tearDownAfterClass() throws Exception {
 +    TEST_UTIL.shutdownMiniCluster();
 +  }
 +
 +  /**
 +   * Initialize the tests with a table filled with some data
 +   * and two snapshots (snapshotName0, snapshotName1) of different states.
 +   * The tableName, snapshotNames and the number of rows in the snapshot are initialized.
 +   */
 +  @Before
 +  public void setup() throws Exception {
 +    this.admin = TEST_UTIL.getHBaseAdmin();
 +
 +    long tid = System.currentTimeMillis();
 +    tableName = TableName.valueOf("testtb-" + tid);
 +    emptySnapshot = Bytes.toBytes("emptySnaptb-" + tid);
 +    snapshotName0 = Bytes.toBytes("snaptb0-" + tid);
 +    snapshotName1 = Bytes.toBytes("snaptb1-" + tid);
 +    snapshotName2 = Bytes.toBytes("snaptb2-" + tid);
 +
 +    // create Table and disable it
 +    MobSnapshotTestingUtils.createMobTable(TEST_UTIL, tableName, getNumReplicas(), FAMILY);
 +    admin.disableTable(tableName);
 +
 +    // take an empty snapshot
 +    admin.snapshot(emptySnapshot, tableName);
 +
 +    HTable table = new HTable(TEST_UTIL.getConfiguration(), tableName);
 +    try {
 +      // enable table and insert data
 +      admin.enableTable(tableName);
-       SnapshotTestingUtils.loadData(TEST_UTIL, table, 500, FAMILY);
++      SnapshotTestingUtils.loadData(TEST_UTIL, tableName, 500, FAMILY);
 +      snapshot0Rows = MobSnapshotTestingUtils.countMobRows(table);
 +      admin.disableTable(tableName);
 +
 +      // take a snapshot
 +      admin.snapshot(snapshotName0, tableName);
 +
 +      // enable table and insert more data
 +      admin.enableTable(tableName);
-       SnapshotTestingUtils.loadData(TEST_UTIL, table, 500, FAMILY);
++      SnapshotTestingUtils.loadData(TEST_UTIL, tableName, 500, FAMILY);
 +      snapshot1Rows = MobSnapshotTestingUtils.countMobRows(table);
 +      admin.disableTable(tableName);
 +
 +      // take a snapshot of the updated table
 +      admin.snapshot(snapshotName1, tableName);
 +
 +      // re-enable table
 +      admin.enableTable(tableName);
 +    } finally {
 +      table.close();
 +    }
 +  }
 +
 +  protected int getNumReplicas() {
 +    return 1;
 +  }
 +
 +  @After
 +  public void tearDown() throws Exception {
 +    if (admin.tableExists(tableName)) {
 +      TEST_UTIL.deleteTable(tableName);
 +    }
 +    SnapshotTestingUtils.deleteAllSnapshots(admin);
 +    SnapshotTestingUtils.deleteArchiveDirectory(TEST_UTIL);
 +  }
 +
 +  @Test(expected=SnapshotDoesNotExistException.class)
 +  public void testCloneNonExistentSnapshot() throws IOException, InterruptedException {
 +    String snapshotName = "random-snapshot-" + System.currentTimeMillis();
 +    TableName tableName = TableName.valueOf("random-table-" + System.currentTimeMillis());
 +    admin.cloneSnapshot(snapshotName, tableName);
 +  }
 +
 +  @Test(expected = NamespaceNotFoundException.class)
 +  public void testCloneOnMissingNamespace() throws IOException, InterruptedException {
 +    TableName clonedTableName = TableName.valueOf("unknownNS:clonetb");
 +    admin.cloneSnapshot(snapshotName1, clonedTableName);
 +  }
 +
 +  @Test
 +  public void testCloneSnapshot() throws IOException, InterruptedException {
 +    TableName clonedTableName = TableName.valueOf("clonedtb-" + System.currentTimeMillis());
 +    testCloneSnapshot(clonedTableName, snapshotName0, snapshot0Rows);
 +    testCloneSnapshot(clonedTableName, snapshotName1, snapshot1Rows);
 +    testCloneSnapshot(clonedTableName, emptySnapshot, 0);
 +  }
 +
 +  private void testCloneSnapshot(final TableName tableName, final byte[] snapshotName,
 +      int snapshotRows) throws IOException, InterruptedException {
 +    // create a new table from snapshot
 +    admin.cloneSnapshot(snapshotName, tableName);
 +    MobSnapshotTestingUtils.verifyMobRowCount(TEST_UTIL, tableName, snapshotRows);
 +
 +    verifyReplicasCameOnline(tableName);
 +    TEST_UTIL.deleteTable(tableName);
 +  }
 +
 +  protected void verifyReplicasCameOnline(TableName tableName) throws IOException {
 +    SnapshotTestingUtils.verifyReplicasCameOnline(tableName, admin, getNumReplicas());
 +  }
 +
 +  @Test
 +  public void testCloneSnapshotCrossNamespace() throws IOException, InterruptedException {
 +    String nsName = "testCloneSnapshotCrossNamespace";
 +    admin.createNamespace(NamespaceDescriptor.create(nsName).build());
 +    TableName clonedTableName =
 +        TableName.valueOf(nsName, "clonedtb-" + System.currentTimeMillis());
 +    testCloneSnapshot(clonedTableName, snapshotName0, snapshot0Rows);
 +    testCloneSnapshot(clonedTableName, snapshotName1, snapshot1Rows);
 +    testCloneSnapshot(clonedTableName, emptySnapshot, 0);
 +  }
 +
 +  /**
 +   * Verify that tables created from the snapshot are still alive after source table deletion.
 +   */
 +  @Test
 +  public void testCloneLinksAfterDelete() throws IOException, InterruptedException {
 +    // Clone a table from the first snapshot
 +    TableName clonedTableName = TableName.valueOf("clonedtb1-" + System.currentTimeMillis());
 +    admin.cloneSnapshot(snapshotName0, clonedTableName);
 +    MobSnapshotTestingUtils.verifyMobRowCount(TEST_UTIL, clonedTableName, snapshot0Rows);
 +
 +    // Take a snapshot of this cloned table.
 +    admin.disableTable(clonedTableName);
 +    admin.snapshot(snapshotName2, clonedTableName);
 +
 +    // Clone the snapshot of the cloned table
 +    TableName clonedTableName2 = TableName.valueOf("clonedtb2-" + System.currentTimeMillis());
 +    admin.cloneSnapshot(snapshotName2, clonedTableName2);
 +    MobSnapshotTestingUtils.verifyMobRowCount(TEST_UTIL, clonedTableName2, snapshot0Rows);
 +    admin.disableTable(clonedTableName2);
 +
 +    // Remove the original table
 +    TEST_UTIL.deleteTable(tableName);
 +    waitCleanerRun();
 +
 +    // Verify the first cloned table
 +    admin.enableTable(clonedTableName);
 +    MobSnapshotTestingUtils.verifyMobRowCount(TEST_UTIL, clonedTableName, snapshot0Rows);
 +
 +    // Verify the second cloned table
 +    admin.enableTable(clonedTableName2);
 +    MobSnapshotTestingUtils.verifyMobRowCount(TEST_UTIL, clonedTableName2, snapshot0Rows);
 +    admin.disableTable(clonedTableName2);
 +
 +    // Delete the first cloned table
 +    TEST_UTIL.deleteTable(clonedTableName);
 +    waitCleanerRun();
 +
 +    // Verify the second cloned table
 +    admin.enableTable(clonedTableName2);
 +    MobSnapshotTestingUtils.verifyMobRowCount(TEST_UTIL, clonedTableName2, snapshot0Rows);
 +
 +    // Clone a new table from cloned
 +    TableName clonedTableName3 = TableName.valueOf("clonedtb3-" + System.currentTimeMillis());
 +    admin.cloneSnapshot(snapshotName2, clonedTableName3);
 +    MobSnapshotTestingUtils.verifyMobRowCount(TEST_UTIL, clonedTableName3, snapshot0Rows);
 +
 +    // Delete the cloned tables
 +    TEST_UTIL.deleteTable(clonedTableName2);
 +    TEST_UTIL.deleteTable(clonedTableName3);
 +    admin.deleteSnapshot(snapshotName2);
 +  }
 +
 +  // ==========================================================================
 +  //  Helpers
 +  // ==========================================================================
 +
 +  private void waitCleanerRun() throws InterruptedException {
 +    TEST_UTIL.getMiniHBaseCluster().getMaster().getHFileCleaner().choreForTesting();
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobRestoreSnapshotFromClient.java
----------------------------------------------------------------------
diff --cc hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobRestoreSnapshotFromClient.java
index c75e006,0000000..0bb498d
mode 100644,000000..100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobRestoreSnapshotFromClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobRestoreSnapshotFromClient.java
@@@ -1,303 -1,0 +1,304 @@@
 +/**
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.hadoop.hbase.client;
 +
 +import static org.junit.Assert.assertEquals;
 +import static org.junit.Assert.assertFalse;
 +import static org.junit.Assert.fail;
 +
 +import java.io.IOException;
 +import java.util.HashSet;
 +import java.util.Set;
 +
 +import org.apache.commons.logging.Log;
 +import org.apache.commons.logging.LogFactory;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.hbase.HBaseTestingUtility;
 +import org.apache.hadoop.hbase.HColumnDescriptor;
 +import org.apache.hadoop.hbase.HConstants;
 +import org.apache.hadoop.hbase.HTableDescriptor;
- import org.apache.hadoop.hbase.LargeTests;
 +import org.apache.hadoop.hbase.TableName;
 +import org.apache.hadoop.hbase.master.MasterFileSystem;
 +import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 +import org.apache.hadoop.hbase.mob.MobConstants;
 +import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException;
 +import org.apache.hadoop.hbase.snapshot.CorruptedSnapshotException;
 +import org.apache.hadoop.hbase.snapshot.MobSnapshotTestingUtils;
 +import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
++import org.apache.hadoop.hbase.testclassification.ClientTests;
++import org.apache.hadoop.hbase.testclassification.LargeTests;
 +import org.apache.hadoop.hbase.util.Bytes;
 +import org.apache.hadoop.hbase.util.FSUtils;
 +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;
 +
 +/**
 + * Test restore snapshots from the client
 + */
- @Category(LargeTests.class)
++@Category({ClientTests.class, LargeTests.class})
 +public class TestMobRestoreSnapshotFromClient {
 +  final Log LOG = LogFactory.getLog(getClass());
 +
 +  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 +
 +  private final byte[] FAMILY = Bytes.toBytes("cf");
 +
 +  private byte[] emptySnapshot;
 +  private byte[] snapshotName0;
 +  private byte[] snapshotName1;
 +  private byte[] snapshotName2;
 +  private int snapshot0Rows;
 +  private int snapshot1Rows;
 +  private TableName tableName;
 +  private Admin admin;
 +
 +  @BeforeClass
 +  public static void setUpBeforeClass() throws Exception {
 +    TEST_UTIL.getConfiguration().setBoolean(SnapshotManager.HBASE_SNAPSHOT_ENABLED, true);
 +    TEST_UTIL.getConfiguration().setBoolean("hbase.online.schema.update.enable", true);
 +    TEST_UTIL.getConfiguration().setInt("hbase.hstore.compactionThreshold", 10);
 +    TEST_UTIL.getConfiguration().setInt("hbase.regionserver.msginterval", 100);
 +    TEST_UTIL.getConfiguration().setInt("hbase.client.pause", 250);
 +    TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 6);
 +    TEST_UTIL.getConfiguration().setBoolean(
 +        "hbase.master.enabletable.roundrobin", true);
 +    TEST_UTIL.getConfiguration().setInt(MobConstants.MOB_FILE_CACHE_SIZE_KEY, 0);
 +    TEST_UTIL.startMiniCluster(3);
 +  }
 +
 +  @AfterClass
 +  public static void tearDownAfterClass() throws Exception {
 +    TEST_UTIL.shutdownMiniCluster();
 +  }
 +
 +  /**
 +   * Initialize the tests with a table filled with some data
 +   * and two snapshots (snapshotName0, snapshotName1) of different states.
 +   * The tableName, snapshotNames and the number of rows in the snapshot are initialized.
 +   */
 +  @Before
 +  public void setup() throws Exception {
 +    this.admin = TEST_UTIL.getHBaseAdmin();
 +
 +    long tid = System.currentTimeMillis();
 +    tableName =
 +        TableName.valueOf("testtb-" + tid);
 +    emptySnapshot = Bytes.toBytes("emptySnaptb-" + tid);
 +    snapshotName0 = Bytes.toBytes("snaptb0-" + tid);
 +    snapshotName1 = Bytes.toBytes("snaptb1-" + tid);
 +    snapshotName2 = Bytes.toBytes("snaptb2-" + tid);
 +
 +    // create Table and disable it
 +    MobSnapshotTestingUtils.createMobTable(TEST_UTIL, tableName, getNumReplicas(), FAMILY);
 +
 +    admin.disableTable(tableName);
 +
 +    // take an empty snapshot
 +    admin.snapshot(emptySnapshot, tableName);
 +
 +    HTable table = new HTable(TEST_UTIL.getConfiguration(), tableName);
 +    // enable table and insert data
 +    admin.enableTable(tableName);
-     SnapshotTestingUtils.loadData(TEST_UTIL, table, 500, FAMILY);
++    SnapshotTestingUtils.loadData(TEST_UTIL, tableName, 500, FAMILY);
 +    snapshot0Rows = MobSnapshotTestingUtils.countMobRows(table);
 +    admin.disableTable(tableName);
 +
 +    // take a snapshot
 +    admin.snapshot(snapshotName0, tableName);
 +
 +    // enable table and insert more data
 +    admin.enableTable(tableName);
-     SnapshotTestingUtils.loadData(TEST_UTIL, table, 500, FAMILY);
++    SnapshotTestingUtils.loadData(TEST_UTIL, tableName, 500, FAMILY);
 +    snapshot1Rows = MobSnapshotTestingUtils.countMobRows(table);
 +    table.close();
 +  }
 +
 +  @After
 +  public void tearDown() throws Exception {
 +    TEST_UTIL.deleteTable(tableName);
 +    SnapshotTestingUtils.deleteAllSnapshots(TEST_UTIL.getHBaseAdmin());
 +    SnapshotTestingUtils.deleteArchiveDirectory(TEST_UTIL);
 +  }
 +
 +  @Test
 +  public void testRestoreSnapshot() throws IOException {
 +    MobSnapshotTestingUtils.verifyMobRowCount(TEST_UTIL, tableName, snapshot1Rows);
 +    admin.disableTable(tableName);
 +    admin.snapshot(snapshotName1, tableName);
 +    // Restore from snapshot-0
 +    admin.restoreSnapshot(snapshotName0);
 +    admin.enableTable(tableName);
 +    MobSnapshotTestingUtils.verifyMobRowCount(TEST_UTIL, tableName, snapshot0Rows);
 +    SnapshotTestingUtils.verifyReplicasCameOnline(tableName, admin, getNumReplicas());
 +
 +    // Restore from emptySnapshot
 +    admin.disableTable(tableName);
 +    admin.restoreSnapshot(emptySnapshot);
 +    admin.enableTable(tableName);
 +    MobSnapshotTestingUtils.verifyMobRowCount(TEST_UTIL, tableName, 0);
 +    SnapshotTestingUtils.verifyReplicasCameOnline(tableName, admin, getNumReplicas());
 +
 +    // Restore from snapshot-1
 +    admin.disableTable(tableName);
 +    admin.restoreSnapshot(snapshotName1);
 +    admin.enableTable(tableName);
 +    MobSnapshotTestingUtils.verifyMobRowCount(TEST_UTIL, tableName, snapshot1Rows);
 +    SnapshotTestingUtils.verifyReplicasCameOnline(tableName, admin, getNumReplicas());
 +
 +    // Restore from snapshot-1
 +    TEST_UTIL.deleteTable(tableName);
 +    admin.restoreSnapshot(snapshotName1);
 +    MobSnapshotTestingUtils.verifyMobRowCount(TEST_UTIL, tableName, snapshot1Rows);
 +    SnapshotTestingUtils.verifyReplicasCameOnline(tableName, admin, getNumReplicas());
 +  }
 +
 +  protected int getNumReplicas() {
 +    return 1;
 +  }
 +
 +  @Test
 +  public void testRestoreSchemaChange() throws Exception {
 +    byte[] TEST_FAMILY2 = Bytes.toBytes("cf2");
 +
 +    HTable table = new HTable(TEST_UTIL.getConfiguration(), tableName);
 +
 +    // Add one column family and put some data in it
 +    admin.disableTable(tableName);
 +    HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAMILY2);
 +    hcd.setMobEnabled(true);
 +    hcd.setMobThreshold(3L);
 +    admin.addColumn(tableName, hcd);
 +    admin.enableTable(tableName);
 +    assertEquals(2, table.getTableDescriptor().getFamilies().size());
 +    HTableDescriptor htd = admin.getTableDescriptor(tableName);
 +    assertEquals(2, htd.getFamilies().size());
-     SnapshotTestingUtils.loadData(TEST_UTIL, table, 500, TEST_FAMILY2);
++    SnapshotTestingUtils.loadData(TEST_UTIL, tableName, 500, TEST_FAMILY2);
 +    long snapshot2Rows = snapshot1Rows + 500;
 +    assertEquals(snapshot2Rows, MobSnapshotTestingUtils.countMobRows(table));
 +    assertEquals(500, MobSnapshotTestingUtils.countMobRows(table, TEST_FAMILY2));
 +    Set<String> fsFamilies = getFamiliesFromFS(tableName);
 +    assertEquals(2, fsFamilies.size());
 +
 +    // Take a snapshot
 +    admin.disableTable(tableName);
 +    admin.snapshot(snapshotName2, tableName);
 +
 +    // Restore the snapshot (without the cf)
 +    admin.restoreSnapshot(snapshotName0);
 +    admin.enableTable(tableName);
 +    assertEquals(1, table.getTableDescriptor().getFamilies().size());
 +    try {
 +      MobSnapshotTestingUtils.countMobRows(table, TEST_FAMILY2);
 +      fail("family '" + Bytes.toString(TEST_FAMILY2) + "' should not exists");
 +    } catch (NoSuchColumnFamilyException e) {
 +      // expected
 +    }
 +    assertEquals(snapshot0Rows, MobSnapshotTestingUtils.countMobRows(table));
 +    htd = admin.getTableDescriptor(tableName);
 +    assertEquals(1, htd.getFamilies().size());
 +    fsFamilies = getFamiliesFromFS(tableName);
 +    assertEquals(1, fsFamilies.size());
 +
 +    // Restore back the snapshot (with the cf)
 +    admin.disableTable(tableName);
 +    admin.restoreSnapshot(snapshotName2);
 +    admin.enableTable(tableName);
 +    htd = admin.getTableDescriptor(tableName);
 +    assertEquals(2, htd.getFamilies().size());
 +    assertEquals(2, table.getTableDescriptor().getFamilies().size());
 +    assertEquals(500, MobSnapshotTestingUtils.countMobRows(table, TEST_FAMILY2));
 +    assertEquals(snapshot2Rows, MobSnapshotTestingUtils.countMobRows(table));
 +    fsFamilies = getFamiliesFromFS(tableName);
 +    assertEquals(2, fsFamilies.size());
 +    table.close();
 +  }
 +
 +  @Test
 +  public void testCloneSnapshotOfCloned() throws IOException, InterruptedException {
 +    TableName clonedTableName =
 +        TableName.valueOf("clonedtb-" + System.currentTimeMillis());
 +    admin.cloneSnapshot(snapshotName0, clonedTableName);
 +    MobSnapshotTestingUtils.verifyMobRowCount(TEST_UTIL, clonedTableName, snapshot0Rows);
 +    SnapshotTestingUtils.verifyReplicasCameOnline(clonedTableName, admin, getNumReplicas());
 +    admin.disableTable(clonedTableName);
 +    admin.snapshot(snapshotName2, clonedTableName);
 +    TEST_UTIL.deleteTable(clonedTableName);
 +    waitCleanerRun();
 +
 +    admin.cloneSnapshot(snapshotName2, clonedTableName);
 +    MobSnapshotTestingUtils.verifyMobRowCount(TEST_UTIL, clonedTableName, snapshot0Rows);
 +    SnapshotTestingUtils.verifyReplicasCameOnline(clonedTableName, admin, getNumReplicas());
 +    TEST_UTIL.deleteTable(clonedTableName);
 +  }
 +
 +  @Test
 +  public void testCloneAndRestoreSnapshot() throws IOException, InterruptedException {
 +    TEST_UTIL.deleteTable(tableName);
 +    waitCleanerRun();
 +
 +    admin.cloneSnapshot(snapshotName0, tableName);
 +    MobSnapshotTestingUtils.verifyMobRowCount(TEST_UTIL, tableName, snapshot0Rows);
 +    SnapshotTestingUtils.verifyReplicasCameOnline(tableName, admin, getNumReplicas());
 +    waitCleanerRun();
 +
 +    admin.disableTable(tableName);
 +    admin.restoreSnapshot(snapshotName0);
 +    admin.enableTable(tableName);
 +    MobSnapshotTestingUtils.verifyMobRowCount(TEST_UTIL, tableName, snapshot0Rows);
 +    SnapshotTestingUtils.verifyReplicasCameOnline(tableName, admin, getNumReplicas());
 +  }
 +
 +  @Test
 +  public void testCorruptedSnapshot() throws IOException, InterruptedException {
 +    SnapshotTestingUtils.corruptSnapshot(TEST_UTIL, Bytes.toString(snapshotName0));
 +    TableName cloneName = TableName.valueOf("corruptedClone-" + System.currentTimeMillis());
 +    try {
 +      admin.cloneSnapshot(snapshotName0, cloneName);
 +      fail("Expected CorruptedSnapshotException, got succeeded cloneSnapshot()");
 +    } catch (CorruptedSnapshotException e) {
 +      // Got the expected corruption exception.
 +      // check for no references of the cloned table.
 +      assertFalse(admin.tableExists(cloneName));
 +    } catch (Exception e) {
 +      fail("Expected CorruptedSnapshotException got: " + e);
 +    }
 +  }
 +
 +  // ==========================================================================
 +  //  Helpers
 +  // ==========================================================================
 +  private void waitCleanerRun() throws InterruptedException {
 +    TEST_UTIL.getMiniHBaseCluster().getMaster().getHFileCleaner().choreForTesting();
 +  }
 +
 +  private Set<String> getFamiliesFromFS(final TableName tableName) throws IOException {
 +    MasterFileSystem mfs = TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterFileSystem();
 +    Set<String> families = new HashSet<String>();
 +    Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), tableName);
 +    for (Path regionDir: FSUtils.getRegionDirs(mfs.getFileSystem(), tableDir)) {
 +      for (Path familyDir: FSUtils.getFamilyDirs(mfs.getFileSystem(), regionDir)) {
 +        families.add(familyDir.getName());
 +      }
 +    }
 +    return families;
 +  }
 +}


[26/50] [abbrv] hbase git commit: HBASE-12701 Document how to set the split policy on a given table

Posted by jm...@apache.org.
HBASE-12701 Document how to set the split policy on a given table


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

Branch: refs/heads/hbase-11339
Commit: 9283b93e225edfaddbb8b24dd1b8214bcd328e97
Parents: 200ec5b
Author: Misty Stanley-Jones <ms...@cloudera.com>
Authored: Tue Feb 10 14:05:00 2015 +1000
Committer: Misty Stanley-Jones <ms...@cloudera.com>
Committed: Tue Feb 10 14:05:41 2015 +1000

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/9283b93e/src/main/asciidoc/_chapters/architecture.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/architecture.adoc b/src/main/asciidoc/_chapters/architecture.adoc
index cd9a4a9..1833cfc 100644
--- a/src/main/asciidoc/_chapters/architecture.adoc
+++ b/src/main/asciidoc/_chapters/architecture.adoc
@@ -1331,6 +1331,35 @@ The RegionServer splits a region, offlines the split region and then adds the da
 See <<disable.splitting>> for how to manually manage splits (and for why you might do this).
 
 ==== Custom Split Policies
+ou can override the default split policy using a custom link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.html[RegionSplitPolicy](HBase 0.94+). Typically a custom split policy should extend
+HBase's default split policy: link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/regionserver/IncreasingToUpperBoundRegionSplitPolicy.html[IncreasingToUpperBoundRegionSplitPolicy].
+
+The policy can set globally through the HBase configuration or on a per-table
+basis.
+
+.Configuring the Split Policy Globally in _hbase-site.xml_
+[source,xml]
+----
+<property>
+  <name>hbase.regionserver.region.split.policy</name>
+  <value>org.apache.hadoop.hbase.regionserver.IncreasingToUpperBoundRegionSplitPolicy</value>
+</property>
+----
+
+.Configuring a Split Policy On a Table Using the Java API
+[source,java]
+HTableDescriptor tableDesc = new HTableDescriptor("test");
+tableDesc.setValue(HTableDescriptor.SPLIT_POLICY, ConstantSizeRegionSplitPolicy.class.getName());
+tableDesc.addFamily(new HColumnDescriptor(Bytes.toBytes("cf1")));
+admin.createTable(tableDesc);              
+----
+
+[source]
+.Configuring the Split Policy On a Table Using HBase Shell
+----
+hbase> create 'test', {METHOD => 'table_att', CONFIG => {'SPLIT_POLICY' => 'org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy'}},
+{NAME => 'cf1'}
+----
 
 The default split policy can be overwritten using a custom link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.html[RegionSplitPolicy(HBase 0.94+)]. Typically a custom split policy should extend HBase's default split policy: link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/regionserver/ConstantSizeRegionSplitPolicy.html[ConstantSizeRegionSplitPolicy].
 


[04/50] [abbrv] hbase git commit: HBASE-12962 - TestHFileBlockIndex.testBlockIndex() commented out during HBASE-10531 (Ram)

Posted by jm...@apache.org.
HBASE-12962 - TestHFileBlockIndex.testBlockIndex() commented out during
HBASE-10531 (Ram)


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

Branch: refs/heads/hbase-11339
Commit: 37ef793fabc633557c994ab0ff8e6024d799d48a
Parents: fd0bb89
Author: Ramkrishna <ra...@intel.com>
Authored: Wed Feb 4 11:42:55 2015 +0530
Committer: Ramkrishna <ra...@intel.com>
Committed: Wed Feb 4 11:42:55 2015 +0530

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/37ef793f/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java
index c0f2fed..4e79381 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java
@@ -119,7 +119,7 @@ public class TestHFileBlockIndex {
     fs = HFileSystem.get(conf);
   }
 
-  //@Test
+  @Test
   public void testBlockIndex() throws IOException {
     testBlockIndexInternals(false);
     clear();


[20/50] [abbrv] hbase git commit: HBASE-12985 Javadoc warning and findbugs fixes to get us green again

Posted by jm...@apache.org.
HBASE-12985 Javadoc warning and findbugs fixes to get us green again


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

Branch: refs/heads/hbase-11339
Commit: eea9873ceff60381d50799994e260e8319ee68a7
Parents: ac175b1
Author: stack <st...@apache.org>
Authored: Sat Feb 7 17:05:11 2015 -0800
Committer: stack <st...@apache.org>
Committed: Sat Feb 7 17:05:11 2015 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/thrift/HttpDoAsClient.java    |  6 ------
 .../java/org/apache/hadoop/hbase/SplitLogCounters.java    |  2 +-
 .../hadoop/hbase/client/ClientSideRegionScanner.java      |  5 +----
 .../main/java/org/apache/hadoop/hbase/ipc/RpcServer.java  |  5 +++--
 .../hadoop/hbase/master/balancer/BaseLoadBalancer.java    |  6 ++----
 .../hbase/procedure/flush/FlushTableSubprocedure.java     |  2 +-
 .../apache/hadoop/hbase/regionserver/HRegionServer.java   |  2 +-
 .../hadoop/hbase/regionserver/wal/WALCellCodec.java       | 10 +++++-----
 .../hadoop/hbase/security/token/ZKSecretWatcher.java      |  2 +-
 9 files changed, 15 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/eea9873c/hbase-examples/src/main/java/org/apache/hadoop/hbase/thrift/HttpDoAsClient.java
----------------------------------------------------------------------
diff --git a/hbase-examples/src/main/java/org/apache/hadoop/hbase/thrift/HttpDoAsClient.java b/hbase-examples/src/main/java/org/apache/hadoop/hbase/thrift/HttpDoAsClient.java
index 9da79ac..9ef1bd2 100644
--- a/hbase-examples/src/main/java/org/apache/hadoop/hbase/thrift/HttpDoAsClient.java
+++ b/hbase-examples/src/main/java/org/apache/hadoop/hbase/thrift/HttpDoAsClient.java
@@ -248,12 +248,6 @@ public class HttpDoAsClient {
     System.out.println("row: " + utf8(rowResult.row.array()) + ", cols: " + rowStr);
   }
 
-  private void printRow(List<TRowResult> rows) {
-    for (TRowResult rowResult : rows) {
-      printRow(rowResult);
-    }
-  }
-
   static Subject getSubject() throws Exception {
     if (!secure) return new Subject();
     /*

http://git-wip-us.apache.org/repos/asf/hbase/blob/eea9873c/hbase-server/src/main/java/org/apache/hadoop/hbase/SplitLogCounters.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/SplitLogCounters.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/SplitLogCounters.java
index f1a8c59..6af5045 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/SplitLogCounters.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/SplitLogCounters.java
@@ -86,7 +86,7 @@ public class SplitLogCounters {
   public final static AtomicLong tot_wkr_task_grabing = new AtomicLong(0);
 
   public static void resetCounters() throws Exception {
-    Class<?> cl = (new SplitLogCounters()).getClass();
+    Class<?> cl = SplitLogCounters.class;
     for (Field fld : cl.getDeclaredFields()) {
       if (!fld.isSynthetic()) ((AtomicLong)fld.get(null)).set(0);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/eea9873c/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClientSideRegionScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClientSideRegionScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClientSideRegionScanner.java
index ff34460..2bab21b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClientSideRegionScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClientSideRegionScanner.java
@@ -43,16 +43,13 @@ import org.mortbay.log.Log;
 public class ClientSideRegionScanner extends AbstractClientScanner {
 
   private HRegion region;
-  private Scan scan;
   RegionScanner scanner;
   List<Cell> values;
 
   public ClientSideRegionScanner(Configuration conf, FileSystem fs,
-      Path rootDir, HTableDescriptor htd, HRegionInfo hri, Scan scan, ScanMetrics scanMetrics) 
+      Path rootDir, HTableDescriptor htd, HRegionInfo hri, Scan scan, ScanMetrics scanMetrics)
           throws IOException {
 
-    this.scan = scan;
-
     // region is immutable, set isolation level
     scan.setIsolationLevel(IsolationLevel.READ_UNCOMMITTED);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/eea9873c/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
index b469fac..fac1ac9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
@@ -1874,8 +1874,9 @@ public class RpcServer implements RpcServerInterface {
    * instance else pass null for no authentication check.
    * @param name Used keying this rpc servers' metrics and for naming the Listener thread.
    * @param services A list of services.
-   * @param bindAddres Where to listen
-   * @throws IOException
+   * @param bindAddress Where to listen
+   * @param conf
+   * @param scheduler
    */
   public RpcServer(final Server server, final String name,
       final List<BlockingServiceAndInterface> services,

http://git-wip-us.apache.org/repos/asf/hbase/blob/eea9873c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
index dea0828..f527931 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
@@ -61,8 +61,8 @@ import com.google.common.collect.Sets;
 
 /**
  * The base class for load balancers. It provides the the functions used to by
- * {@link org.apache.hadoop.hbase.master.AssignmentManager} to assign regions 
- * in the edge cases. It doesn't provide an implementation of the 
+ * {@link org.apache.hadoop.hbase.master.AssignmentManager} to assign regions
+ * in the edge cases. It doesn't provide an implementation of the
  * actual balancing algorithm.
  *
  */
@@ -138,8 +138,6 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
     int numRegions;
 
     int numMovedRegions = 0; //num moved regions from the initial configuration
-    // num of moved regions away from master that should be on the master
-    int numMovedMetaRegions = 0;       //num of moved regions that are META
     Map<ServerName, List<HRegionInfo>> clusterState;
 
     protected final RackManager rackManager;

http://git-wip-us.apache.org/repos/asf/hbase/blob/eea9873c/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/FlushTableSubprocedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/FlushTableSubprocedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/FlushTableSubprocedure.java
index cfc6807..d032eba 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/FlushTableSubprocedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/FlushTableSubprocedure.java
@@ -53,7 +53,7 @@ public class FlushTableSubprocedure extends Subprocedure {
     this.taskManager = taskManager;
   }
 
-  private class RegionFlushTask implements Callable<Void> {
+  private static class RegionFlushTask implements Callable<Void> {
     HRegion region;
     RegionFlushTask(HRegion region) {
       this.region = region;

http://git-wip-us.apache.org/repos/asf/hbase/blob/eea9873c/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index fcb8c6f..90b29ef 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -1475,7 +1475,7 @@ public class HRegionServer extends HasThread implements
     }
   }
 
-  class PeriodicMemstoreFlusher extends ScheduledChore {
+  static class PeriodicMemstoreFlusher extends ScheduledChore {
     final HRegionServer server;
     final static int RANGE_OF_DELAY = 20000; //millisec
     final static int MIN_DELAY_TIME = 3000; //millisec

http://git-wip-us.apache.org/repos/asf/hbase/blob/eea9873c/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
index 56137e8..a80c508 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
@@ -82,7 +82,7 @@ public class WALCellCodec implements Codec {
   static String getWALCellCodecClass(Configuration conf) {
     return conf.get(WAL_CELL_CODEC_CLASS_KEY, WALCellCodec.class.getName());
   }
-  
+
   /**
    * Create and setup a {@link WALCellCodec} from the {@code cellCodecClsName} and
    * CompressionContext, if {@code cellCodecClsName} is specified.
@@ -106,7 +106,7 @@ public class WALCellCodec implements Codec {
   }
 
   /**
-   * Create and setup a {@link WALCellCodec} from the 
+   * Create and setup a {@link WALCellCodec} from the
    * CompressionContext.
    * Cell Codec classname is read from {@link Configuration}.
    * Fully prepares the codec for use.
@@ -122,7 +122,7 @@ public class WALCellCodec implements Codec {
     return ReflectionUtils.instantiateWithCustomCtor(cellCodecClsName, new Class[]
         { Configuration.class, CompressionContext.class }, new Object[] { conf, compression });
   }
-  
+
   public interface ByteStringCompressor {
     ByteString compress(byte[] data, Dictionary dict) throws IOException;
   }
@@ -249,7 +249,7 @@ public class WALCellCodec implements Codec {
     protected Cell parseCell() throws IOException {
       int keylength = StreamUtils.readRawVarint32(in);
       int vlength = StreamUtils.readRawVarint32(in);
-      
+
       int tagsLength = StreamUtils.readRawVarint32(in);
       int length = 0;
       if(tagsLength == 0) {
@@ -328,7 +328,7 @@ public class WALCellCodec implements Codec {
     }
   }
 
-  public class EnsureKvEncoder extends BaseEncoder {
+  public static class EnsureKvEncoder extends BaseEncoder {
     public EnsureKvEncoder(OutputStream out) {
       super(out);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/eea9873c/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/ZKSecretWatcher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/ZKSecretWatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/ZKSecretWatcher.java
index c9196aa..5fb3d40 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/ZKSecretWatcher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/ZKSecretWatcher.java
@@ -85,7 +85,7 @@ public class ZKSecretWatcher extends ZooKeeperListener {
     if (keysParentZNode.equals(ZKUtil.getParent(path))) {
       String keyId = ZKUtil.getNodeName(path);
       try {
-        Integer id = new Integer(keyId);
+        Integer id = Integer.valueOf(keyId);
         secretManager.removeKey(id);
       } catch (NumberFormatException nfe) {
         LOG.error("Invalid znode name for key ID '"+keyId+"'", nfe);


[21/50] [abbrv] hbase git commit: HBASE-12992 TestChoreService doesn't close services, that can break test on slow virtual hosts (Andrey Stepachev)

Posted by jm...@apache.org.
HBASE-12992 TestChoreService doesn't close services, that can break test on slow virtual hosts (Andrey Stepachev)


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

Branch: refs/heads/hbase-11339
Commit: 9d6b237ae8676750c97dad2b9d2655dbd43f67fa
Parents: eea9873
Author: tedyu <yu...@gmail.com>
Authored: Mon Feb 9 12:15:23 2015 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Mon Feb 9 12:15:23 2015 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/TestChoreService.java   | 53 +++++++++++++++++---
 1 file changed, 46 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/9d6b237a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestChoreService.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestChoreService.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestChoreService.java
index ed5fa60..b1bad3d 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestChoreService.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestChoreService.java
@@ -90,7 +90,7 @@ public class TestChoreService {
         try {
           Thread.sleep(getPeriod() * 2);
         } catch (InterruptedException e) {
-          e.printStackTrace();
+          //e.printStackTrace();
         }
       }
     }
@@ -258,6 +258,7 @@ public class TestChoreService {
     }
 
     assertFalse(brokeOutOfLoop);
+    shutdownService(service);
   }
 
   @Test (timeout=20000)
@@ -272,6 +273,7 @@ public class TestChoreService {
     chore1.cancel(true);
     assertFalse(chore1.isScheduled());
     assertTrue(service.getNumberOfScheduledChores() == 0);
+    shutdownService(service);
   }
 
   @Test (timeout=20000)
@@ -308,7 +310,7 @@ public class TestChoreService {
   }
 
   @Test (timeout=20000)
-  public void testChoreServiceConstruction() {
+  public void testChoreServiceConstruction() throws InterruptedException {
     final int corePoolSize = 10;
     final int defaultCorePoolSize = ChoreService.MIN_CORE_POOL_SIZE;
 
@@ -320,6 +322,11 @@ public class TestChoreService {
 
     ChoreService invalidInit = new ChoreService(TEST_SERVER_NAME, -10);
     assertEquals(defaultCorePoolSize, invalidInit.getCorePoolSize());
+
+    shutdownService(customInit);
+    shutdownService(defaultInit);
+    shutdownService(invalidInit);
+
   }
 
   @Test (timeout=20000)
@@ -336,6 +343,15 @@ public class TestChoreService {
 
     Thread.sleep(10 * period);
     assertTrue(chore.getCountOfChoreCalls() == 21);
+
+    shutdownService(service);
+  }
+
+  public void shutdownService(ChoreService service) throws InterruptedException {
+    service.shutdown();
+    while (!service.isTerminated()) {
+      Thread.sleep(100);
+    }
   }
 
   @Test (timeout=20000)
@@ -367,6 +383,8 @@ public class TestChoreService {
     Thread.sleep(10 * period + delta);
 
     assertTrue(chore.getCountOfChoreCalls() == 26);
+
+    shutdownService(service);
   }
 
   @Test (timeout=20000)
@@ -402,13 +420,15 @@ public class TestChoreService {
     Thread.sleep(slowChorePeriod * 10);
     assertEquals("Chores are missing their start time. Should expand core pool size", 5,
       service.getCorePoolSize());
+
+    shutdownService(service);
   }
 
   @Test (timeout=20000)
   public void testCorePoolDecrease() throws InterruptedException {
     final int initialCorePoolSize = 3;
     ChoreService service = new ChoreService(TEST_SERVER_NAME, initialCorePoolSize);
-    final int chorePeriod = 10;
+    final int chorePeriod = 100;
 
     // Slow chores always miss their start time and thus the core pool size should be at least as
     // large as the number of running slow chores
@@ -504,6 +524,8 @@ public class TestChoreService {
     Thread.sleep(chorePeriod * 10);
     assertEquals(service.getNumberOfChoresMissingStartTime(), 2);
     assertEquals("Should not change", 3, service.getCorePoolSize());
+
+    shutdownService(service);
   }
 
   @Test (timeout=20000)
@@ -541,6 +563,8 @@ public class TestChoreService {
     dn5.cancel();
     Thread.sleep(sleepTime);
     assertEquals("Scheduled chore mismatch", 0, service.getNumberOfScheduledChores());
+
+    shutdownService(service);
   }
 
   @Test (timeout=20000)
@@ -580,6 +604,8 @@ public class TestChoreService {
     sc5.cancel();
     Thread.sleep(sleepTime);
     assertEquals(0, service.getNumberOfChoresMissingStartTime());
+
+    shutdownService(service);
   }
 
   /**
@@ -591,7 +617,7 @@ public class TestChoreService {
   public void testMaximumChoreServiceThreads() throws InterruptedException {
     ChoreService service = new ChoreService(TEST_SERVER_NAME);
 
-    final int period = 10;
+    final int period = 100;
     final int sleepTime = 5 * period;
 
     // Slow chores sleep for a length of time LONGER than their period. Thus, SlowChores
@@ -628,6 +654,8 @@ public class TestChoreService {
 
     Thread.sleep(sleepTime);
     assertTrue(service.getCorePoolSize() <= service.getNumberOfScheduledChores());
+
+    shutdownService(service);
   }
 
   @Test (timeout=20000)
@@ -655,6 +683,8 @@ public class TestChoreService {
     assertTrue(!chore.isInitialChoreComplete());
     assertTrue(chore.getTimeOfLastRun() == -1);
     assertTrue(chore.getTimeOfThisRun() == -1);
+
+    shutdownService(service);
   }
 
   @Test (timeout=20000)
@@ -689,6 +719,9 @@ public class TestChoreService {
     assertFalse(service1.isChoreScheduled(chore));
     assertFalse(service2.isChoreScheduled(chore));
     assertTrue(chore.getChoreServicer() == null);
+
+    shutdownService(service1);
+    shutdownService(service2);
   }
 
   @Test (timeout=20000)
@@ -714,6 +747,8 @@ public class TestChoreService {
     assertTrue(chore.triggerNow());
     Thread.sleep(sleep);
     assertEquals(5, chore.getCountOfChoreCalls());
+
+    shutdownService(service);
   }
 
   @Test (timeout=20000)
@@ -765,6 +800,8 @@ public class TestChoreService {
     assertFalse(chore1_group2.isScheduled());
     assertFalse(chore2_group2.isScheduled());
     assertFalse(chore3_group2.isScheduled());
+
+    shutdownService(service);
   }
 
   @Test (timeout=20000)
@@ -782,7 +819,7 @@ public class TestChoreService {
     assertTrue(service.scheduleChore(successChore3));
     assertTrue(successChore3.isScheduled());
 
-    service.shutdown();
+    shutdownService(service);
 
     assertFalse(successChore1.isScheduled());
     assertFalse(successChore2.isScheduled());
@@ -803,7 +840,7 @@ public class TestChoreService {
     assertTrue(service.scheduleChore(slowChore3));
 
     Thread.sleep(sleep / 2);
-    service.shutdown();
+    shutdownService(service);
 
     assertFalse(slowChore1.isScheduled());
     assertFalse(slowChore2.isScheduled());
@@ -812,6 +849,8 @@ public class TestChoreService {
 
     Thread.sleep(5);
     assertTrue(service.isTerminated());
+
+    shutdownService(service);
   }
 
   @Test (timeout=20000)
@@ -832,7 +871,7 @@ public class TestChoreService {
     assertTrue(service.scheduleChore(successChore3));
     assertTrue(successChore3.isScheduled());
 
-    service.shutdown();
+    shutdownService(service);
 
     assertFalse(service.scheduleChore(failChore1));
     assertFalse(failChore1.isScheduled());


[44/50] [abbrv] hbase git commit: Merge branch 'master' (2/11/15) into hbase-11339

Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDeleteMobTable.java
----------------------------------------------------------------------
diff --cc hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDeleteMobTable.java
index a64b7d6,0000000..028e602
mode 100644,000000..100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDeleteMobTable.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDeleteMobTable.java
@@@ -1,225 -1,0 +1,225 @@@
 +/**
 + * 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.io.IOException;
 +import java.util.Random;
 +
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.hbase.HBaseTestingUtility;
 +import org.apache.hadoop.hbase.HColumnDescriptor;
 +import org.apache.hadoop.hbase.HTableDescriptor;
- import org.apache.hadoop.hbase.MediumTests;
 +import org.apache.hadoop.hbase.TableName;
 +import org.apache.hadoop.hbase.client.HBaseAdmin;
 +import org.apache.hadoop.hbase.client.HTable;
 +import org.apache.hadoop.hbase.client.Put;
 +import org.apache.hadoop.hbase.client.Result;
 +import org.apache.hadoop.hbase.client.ResultScanner;
 +import org.apache.hadoop.hbase.client.Scan;
 +import org.apache.hadoop.hbase.mob.MobConstants;
 +import org.apache.hadoop.hbase.mob.MobUtils;
++import org.apache.hadoop.hbase.testclassification.MediumTests;
 +import org.apache.hadoop.hbase.util.Bytes;
 +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 +import org.apache.hadoop.hbase.util.FSUtils;
 +import org.apache.hadoop.hbase.util.HFileArchiveUtil;
 +import org.junit.AfterClass;
 +import org.junit.Assert;
 +import org.junit.BeforeClass;
 +import org.junit.Test;
 +import org.junit.experimental.categories.Category;
 +
 +@Category(MediumTests.class)
 +public class TestDeleteMobTable {
 +
 +  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 +  private final static byte[] FAMILY = Bytes.toBytes("family");
 +  private final static byte[] QF = Bytes.toBytes("qualifier");
 +  private static Random random = new Random();
 +
 +  @BeforeClass
 +  public static void setUpBeforeClass() throws Exception {
 +    TEST_UTIL.getConfiguration().setInt("hbase.master.info.port", 0);
 +    TEST_UTIL.getConfiguration().setBoolean("hbase.regionserver.info.port.auto", true);
 +    TEST_UTIL.startMiniCluster(1);
 +  }
 +
 +  @AfterClass
 +  public static void tearDownAfterClass() throws Exception {
 +    TEST_UTIL.shutdownMiniCluster();
 +  }
 +
 +  /**
 +   * Generate the mob value.
 +   *
 +   * @param size
 +   *          the size of the value
 +   * @return the mob value generated
 +   */
 +  private static byte[] generateMobValue(int size) {
 +    byte[] mobVal = new byte[size];
 +    random.nextBytes(mobVal);
 +    return mobVal;
 +  }
 +
 +  @Test
 +  public void testDeleteMobTable() throws Exception {
 +    byte[] tableName = Bytes.toBytes("testDeleteMobTable");
 +    TableName tn = TableName.valueOf(tableName);
 +    HTableDescriptor htd = new HTableDescriptor(tn);
 +    HColumnDescriptor hcd = new HColumnDescriptor(FAMILY);
 +    hcd.setMobEnabled(true);
 +    hcd.setMobThreshold(0);
 +    htd.addFamily(hcd);
 +    HBaseAdmin admin = null;
 +    HTable table = null;
 +    try {
 +      admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
 +      admin.createTable(htd);
 +      table = new HTable(TEST_UTIL.getConfiguration(), tableName);
 +      byte[] value = generateMobValue(10);
 +
 +      byte[] row = Bytes.toBytes("row");
 +      Put put = new Put(row);
 +      put.add(FAMILY, QF, EnvironmentEdgeManager.currentTime(), value);
 +      table.put(put);
 +
 +      table.flushCommits();
 +      admin.flush(tableName);
 +
 +      // the mob file exists
 +      Assert.assertEquals(1, countMobFiles(tn, hcd.getNameAsString()));
 +      Assert.assertEquals(0, countArchiveMobFiles(tn, hcd.getNameAsString()));
 +      String fileName = assertHasOneMobRow(table, tn, hcd.getNameAsString());
 +      Assert.assertFalse(mobArchiveExist(tn, hcd.getNameAsString(), fileName));
 +      Assert.assertTrue(mobTableDirExist(tn));
 +      table.close();
 +
 +      admin.disableTable(tn);
 +      admin.deleteTable(tn);
 +
 +      Assert.assertFalse(admin.tableExists(tn));
 +      Assert.assertEquals(0, countMobFiles(tn, hcd.getNameAsString()));
 +      Assert.assertEquals(1, countArchiveMobFiles(tn, hcd.getNameAsString()));
 +      Assert.assertTrue(mobArchiveExist(tn, hcd.getNameAsString(), fileName));
 +      Assert.assertFalse(mobTableDirExist(tn));
 +    } finally {
 +      if (admin != null) {
 +        admin.close();
 +      }
 +    }
 +  }
 +
 +  @Test
 +  public void testDeleteNonMobTable() throws Exception {
 +    byte[] tableName = Bytes.toBytes("testDeleteNonMobTable");
 +    TableName tn = TableName.valueOf(tableName);
 +    HTableDescriptor htd = new HTableDescriptor(tn);
 +    HColumnDescriptor hcd = new HColumnDescriptor(FAMILY);
 +    htd.addFamily(hcd);
 +    HBaseAdmin admin = null;
 +    HTable table = null;
 +    try {
 +      admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
 +      admin.createTable(htd);
 +      table = new HTable(TEST_UTIL.getConfiguration(), tableName);
 +      byte[] value = generateMobValue(10);
 +
 +      byte[] row = Bytes.toBytes("row");
 +      Put put = new Put(row);
 +      put.add(FAMILY, QF, EnvironmentEdgeManager.currentTime(), value);
 +      table.put(put);
 +
 +      table.flushCommits();
 +      admin.flush(tableName);
 +      table.close();
 +
 +      // the mob file doesn't exist
 +      Assert.assertEquals(0, countMobFiles(tn, hcd.getNameAsString()));
 +      Assert.assertEquals(0, countArchiveMobFiles(tn, hcd.getNameAsString()));
 +      Assert.assertFalse(mobTableDirExist(tn));
 +
 +      admin.disableTable(tn);
 +      admin.deleteTable(tn);
 +
 +      Assert.assertFalse(admin.tableExists(tn));
 +      Assert.assertEquals(0, countMobFiles(tn, hcd.getNameAsString()));
 +      Assert.assertEquals(0, countArchiveMobFiles(tn, hcd.getNameAsString()));
 +      Assert.assertFalse(mobTableDirExist(tn));
 +    } finally {
 +      if (admin != null) {
 +        admin.close();
 +      }
 +    }
 +  }
 +
 +  private int countMobFiles(TableName tn, String familyName) throws IOException {
 +    FileSystem fs = TEST_UTIL.getTestFileSystem();
 +    Path mobFileDir = MobUtils.getMobFamilyPath(TEST_UTIL.getConfiguration(), tn, familyName);
 +    if (fs.exists(mobFileDir)) {
 +      return fs.listStatus(mobFileDir).length;
 +    } else {
 +      return 0;
 +    }
 +  }
 +
 +  private int countArchiveMobFiles(TableName tn, String familyName)
 +      throws IOException {
 +    FileSystem fs = TEST_UTIL.getTestFileSystem();
 +    Path storePath = HFileArchiveUtil.getStoreArchivePath(TEST_UTIL.getConfiguration(), tn,
 +        MobUtils.getMobRegionInfo(tn).getEncodedName(), familyName);
 +    if (fs.exists(storePath)) {
 +      return fs.listStatus(storePath).length;
 +    } else {
 +      return 0;
 +    }
 +  }
 +
 +  private boolean mobTableDirExist(TableName tn) throws IOException {
 +    FileSystem fs = TEST_UTIL.getTestFileSystem();
 +    Path tableDir = FSUtils.getTableDir(MobUtils.getMobHome(TEST_UTIL.getConfiguration()), tn);
 +    return fs.exists(tableDir);
 +  }
 +
 +  private boolean mobArchiveExist(TableName tn, String familyName, String fileName)
 +      throws IOException {
 +    FileSystem fs = TEST_UTIL.getTestFileSystem();
 +    Path storePath = HFileArchiveUtil.getStoreArchivePath(TEST_UTIL.getConfiguration(), tn,
 +        MobUtils.getMobRegionInfo(tn).getEncodedName(), familyName);
 +    return fs.exists(new Path(storePath, fileName));
 +  }
 +
 +  private String assertHasOneMobRow(HTable table, TableName tn, String familyName)
 +      throws IOException {
 +    Scan scan = new Scan();
 +    scan.setAttribute(MobConstants.MOB_SCAN_RAW, Bytes.toBytes(Boolean.TRUE));
 +    ResultScanner rs = table.getScanner(scan);
 +    Result r = rs.next();
 +    Assert.assertNotNull(r);
 +    byte[] value = r.getValue(FAMILY, QF);
 +    String fileName = Bytes.toString(value, Bytes.SIZEOF_INT, value.length - Bytes.SIZEOF_INT);
 +    Path filePath = new Path(
 +        MobUtils.getMobFamilyPath(TEST_UTIL.getConfiguration(), tn, familyName), fileName);
 +    FileSystem fs = TEST_UTIL.getTestFileSystem();
 +    Assert.assertTrue(fs.exists(filePath));
 +    r = rs.next();
 +    Assert.assertNull(r);
 +    return fileName;
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
----------------------------------------------------------------------
diff --cc hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
index 30b74d4,0000000..5f76655
mode 100644,000000..100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
@@@ -1,471 -1,0 +1,475 @@@
 +/**
 + *
 + * 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.io.IOException;
 +import java.util.ArrayList;
 +import java.util.Collections;
 +import java.util.Date;
 +import java.util.Iterator;
 +import java.util.List;
 +import java.util.NavigableSet;
 +import java.util.concurrent.ConcurrentSkipListSet;
 +
 +import org.apache.commons.logging.Log;
 +import org.apache.commons.logging.LogFactory;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.hbase.Cell;
 +import org.apache.hadoop.hbase.CellUtil;
 +import org.apache.hadoop.hbase.HBaseConfiguration;
 +import org.apache.hadoop.hbase.HBaseTestingUtility;
 +import org.apache.hadoop.hbase.HColumnDescriptor;
 +import org.apache.hadoop.hbase.HRegionInfo;
 +import org.apache.hadoop.hbase.HTableDescriptor;
 +import org.apache.hadoop.hbase.KeyValue;
- import org.apache.hadoop.hbase.MediumTests;
++import org.apache.hadoop.hbase.testclassification.MediumTests;
 +import org.apache.hadoop.hbase.TableName;
 +import org.apache.hadoop.hbase.client.Get;
 +import org.apache.hadoop.hbase.client.Scan;
 +import org.apache.hadoop.hbase.mob.MobConstants;
 +import org.apache.hadoop.hbase.mob.MobUtils;
 +import org.apache.hadoop.hbase.monitoring.MonitoredTask;
- import org.apache.hadoop.hbase.regionserver.wal.HLog;
- import org.apache.hadoop.hbase.regionserver.wal.HLogFactory;
 +import org.apache.hadoop.hbase.util.Bytes;
 +import org.apache.hadoop.hbase.util.FSUtils;
++import org.apache.hadoop.hbase.wal.WAL;
++import org.apache.hadoop.hbase.wal.WALFactory;
 +import org.junit.Assert;
 +import org.junit.Before;
 +import org.junit.Rule;
 +import org.junit.Test;
 +import org.junit.experimental.categories.Category;
 +import org.junit.rules.TestName;
 +import org.mockito.Mockito;
 +
 +@Category(MediumTests.class)
 +public class TestHMobStore {
 +  public static final Log LOG = LogFactory.getLog(TestHMobStore.class);
 +  @Rule public TestName name = new TestName();
 +
 +  private HMobStore store;
 +  private HRegion region;
 +  private HColumnDescriptor hcd;
 +  private FileSystem fs;
 +  private byte [] table = Bytes.toBytes("table");
 +  private byte [] family = Bytes.toBytes("family");
 +  private byte [] row = Bytes.toBytes("row");
 +  private byte [] row2 = Bytes.toBytes("row2");
 +  private byte [] qf1 = Bytes.toBytes("qf1");
 +  private byte [] qf2 = Bytes.toBytes("qf2");
 +  private byte [] qf3 = Bytes.toBytes("qf3");
 +  private byte [] qf4 = Bytes.toBytes("qf4");
 +  private byte [] qf5 = Bytes.toBytes("qf5");
 +  private byte [] qf6 = Bytes.toBytes("qf6");
 +  private byte[] value = Bytes.toBytes("value");
 +  private byte[] value2 = Bytes.toBytes("value2");
 +  private Path mobFilePath;
 +  private Date currentDate = new Date();
 +  private KeyValue seekKey1;
 +  private KeyValue seekKey2;
 +  private KeyValue seekKey3;
 +  private NavigableSet<byte[]> qualifiers =
 +    new ConcurrentSkipListSet<byte[]>(Bytes.BYTES_COMPARATOR);
 +  private List<Cell> expected = new ArrayList<Cell>();
 +  private long id = System.currentTimeMillis();
 +  private Get get = new Get(row);
 +  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 +  private final String DIR = TEST_UTIL.getDataTestDir("TestHMobStore").toString();
 +
 +  /**
 +   * Setup
 +   * @throws Exception
 +   */
 +  @Before
 +  public void setUp() throws Exception {
 +    qualifiers.add(qf1);
 +    qualifiers.add(qf3);
 +    qualifiers.add(qf5);
 +
 +    Iterator<byte[]> iter = qualifiers.iterator();
 +    while(iter.hasNext()){
 +      byte [] next = iter.next();
 +      expected.add(new KeyValue(row, family, next, 1, value));
 +      get.addColumn(family, next);
 +      get.setMaxVersions(); // all versions.
 +    }
 +  }
 +
 +  private void init(String methodName, Configuration conf, boolean testStore)
 +  throws IOException {
 +    hcd = new HColumnDescriptor(family);
 +    hcd.setMobEnabled(true);
 +    hcd.setMobThreshold(3L);
 +    hcd.setMaxVersions(4);
 +    init(methodName, conf, hcd, testStore);
 +  }
 +
 +  private void init(String methodName, Configuration conf,
 +      HColumnDescriptor hcd, boolean testStore) throws IOException {
 +    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(table));
 +    init(methodName, conf, htd, hcd, testStore);
 +  }
 +
 +  private void init(String methodName, Configuration conf, HTableDescriptor htd,
 +      HColumnDescriptor hcd, boolean testStore) throws IOException {
 +    //Setting up tje Region and Store
 +    Path basedir = new Path(DIR+methodName);
 +    Path tableDir = FSUtils.getTableDir(basedir, htd.getTableName());
 +    String logName = "logs";
 +    Path logdir = new Path(basedir, logName);
 +    FileSystem fs = FileSystem.get(conf);
 +    fs.delete(logdir, true);
 +
 +    htd.addFamily(hcd);
 +    HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
-     HLog hlog = HLogFactory.createHLog(fs, basedir, logName, conf);
-     region = new HRegion(tableDir, hlog, fs, conf, info, htd, null);
++
++    final Configuration walConf = new Configuration(conf);
++    FSUtils.setRootDir(walConf, basedir);
++    final WALFactory wals = new WALFactory(walConf, null, methodName);
++    region = new HRegion(tableDir, wals.getWAL(info.getEncodedNameAsBytes()), fs, conf,
++            info, htd, null);
 +    store = new HMobStore(region, hcd, conf);
 +    if(testStore) {
 +      init(conf, hcd);
 +    }
 +  }
 +
 +  private void init(Configuration conf, HColumnDescriptor hcd)
 +      throws IOException {
 +    Path basedir = FSUtils.getRootDir(conf);
 +    fs = FileSystem.get(conf);
 +    Path homePath = new Path(basedir, Bytes.toString(family) + Path.SEPARATOR
 +        + Bytes.toString(family));
 +    fs.mkdirs(homePath);
 +
 +    KeyValue key1 = new KeyValue(row, family, qf1, 1, value);
 +    KeyValue key2 = new KeyValue(row, family, qf2, 1, value);
 +    KeyValue key3 = new KeyValue(row2, family, qf3, 1, value2);
 +    KeyValue[] keys = new KeyValue[] { key1, key2, key3 };
 +    int maxKeyCount = keys.length;
 +    StoreFile.Writer mobWriter = store.createWriterInTmp(currentDate,
 +        maxKeyCount, hcd.getCompactionCompression(), region.getStartKey());
 +    mobFilePath = mobWriter.getPath();
 +
 +    mobWriter.append(key1);
 +    mobWriter.append(key2);
 +    mobWriter.append(key3);
 +    mobWriter.close();
 +
 +    int valueLength1 = key1.getValueLength();
 +    int valueLength2 = key2.getValueLength();
 +    int valueLength3 = key3.getValueLength();
 +
 +    String targetPathName = MobUtils.formatDate(currentDate);
 +    byte[] referenceValue =
 +            Bytes.toBytes(targetPathName + Path.SEPARATOR
 +                + mobFilePath.getName());
 +    byte[] newReferenceValue1 = Bytes.add(Bytes.toBytes(valueLength1), referenceValue);
 +    byte[] newReferenceValue2 = Bytes.add(Bytes.toBytes(valueLength2), referenceValue);
 +    byte[] newReferenceValue3 = Bytes.add(Bytes.toBytes(valueLength3), referenceValue);
 +    seekKey1 = new KeyValue(row, family, qf1, Long.MAX_VALUE, newReferenceValue1);
 +    seekKey2 = new KeyValue(row, family, qf2, Long.MAX_VALUE, newReferenceValue2);
 +    seekKey3 = new KeyValue(row2, family, qf3, Long.MAX_VALUE, newReferenceValue3);
 +  }
 +
 +  /**
 +   * Getting data from memstore
 +   * @throws IOException
 +   */
 +  @Test
 +  public void testGetFromMemStore() throws IOException {
 +    final Configuration conf = HBaseConfiguration.create();
 +    init(name.getMethodName(), conf, false);
 +
 +    //Put data in memstore
 +    this.store.add(new KeyValue(row, family, qf1, 1, value));
 +    this.store.add(new KeyValue(row, family, qf2, 1, value));
 +    this.store.add(new KeyValue(row, family, qf3, 1, value));
 +    this.store.add(new KeyValue(row, family, qf4, 1, value));
 +    this.store.add(new KeyValue(row, family, qf5, 1, value));
 +    this.store.add(new KeyValue(row, family, qf6, 1, value));
 +
 +    Scan scan = new Scan(get);
 +    InternalScanner scanner = (InternalScanner) store.getScanner(scan,
 +        scan.getFamilyMap().get(store.getFamily().getName()),
 +        0);
 +
 +    List<Cell> results = new ArrayList<Cell>();
 +    scanner.next(results);
 +    Collections.sort(results, KeyValue.COMPARATOR);
 +    scanner.close();
 +
 +    //Compare
 +    Assert.assertEquals(expected.size(), results.size());
 +    for(int i=0; i<results.size(); i++) {
 +      // Verify the values
 +      Assert.assertEquals(expected.get(i), results.get(i));
 +    }
 +  }
 +
 +  /**
 +   * Getting MOB data from files
 +   * @throws IOException
 +   */
 +  @Test
 +  public void testGetFromFiles() throws IOException {
 +    final Configuration conf = TEST_UTIL.getConfiguration();
 +    init(name.getMethodName(), conf, false);
 +
 +    //Put data in memstore
 +    this.store.add(new KeyValue(row, family, qf1, 1, value));
 +    this.store.add(new KeyValue(row, family, qf2, 1, value));
 +    //flush
 +    flush(1);
 +
 +    //Add more data
 +    this.store.add(new KeyValue(row, family, qf3, 1, value));
 +    this.store.add(new KeyValue(row, family, qf4, 1, value));
 +    //flush
 +    flush(2);
 +
 +    //Add more data
 +    this.store.add(new KeyValue(row, family, qf5, 1, value));
 +    this.store.add(new KeyValue(row, family, qf6, 1, value));
 +    //flush
 +    flush(3);
 +
 +    Scan scan = new Scan(get);
 +    InternalScanner scanner = (InternalScanner) store.getScanner(scan,
 +        scan.getFamilyMap().get(store.getFamily().getName()),
 +        0);
 +
 +    List<Cell> results = new ArrayList<Cell>();
 +    scanner.next(results);
 +    Collections.sort(results, KeyValue.COMPARATOR);
 +    scanner.close();
 +
 +    //Compare
 +    Assert.assertEquals(expected.size(), results.size());
 +    for(int i=0; i<results.size(); i++) {
 +      Assert.assertEquals(expected.get(i), results.get(i));
 +    }
 +  }
 +
 +  /**
 +   * Getting the reference data from files
 +   * @throws IOException
 +   */
 +  @Test
 +  public void testGetReferencesFromFiles() throws IOException {
 +    final Configuration conf = HBaseConfiguration.create();
 +    init(name.getMethodName(), conf, false);
 +
 +    //Put data in memstore
 +    this.store.add(new KeyValue(row, family, qf1, 1, value));
 +    this.store.add(new KeyValue(row, family, qf2, 1, value));
 +    //flush
 +    flush(1);
 +
 +    //Add more data
 +    this.store.add(new KeyValue(row, family, qf3, 1, value));
 +    this.store.add(new KeyValue(row, family, qf4, 1, value));
 +    //flush
 +    flush(2);
 +
 +    //Add more data
 +    this.store.add(new KeyValue(row, family, qf5, 1, value));
 +    this.store.add(new KeyValue(row, family, qf6, 1, value));
 +    //flush
 +    flush(3);
 +
 +    Scan scan = new Scan(get);
 +    scan.setAttribute(MobConstants.MOB_SCAN_RAW, Bytes.toBytes(Boolean.TRUE));
 +    InternalScanner scanner = (InternalScanner) store.getScanner(scan,
 +      scan.getFamilyMap().get(store.getFamily().getName()),
 +      0);
 +
 +    List<Cell> results = new ArrayList<Cell>();
 +    scanner.next(results);
 +    Collections.sort(results, KeyValue.COMPARATOR);
 +    scanner.close();
 +
 +    //Compare
 +    Assert.assertEquals(expected.size(), results.size());
 +    for(int i=0; i<results.size(); i++) {
 +      Cell cell = results.get(i);
 +      Assert.assertTrue(MobUtils.isMobReferenceCell(cell));
 +    }
 +  }
 +
 +  /**
 +   * Getting data from memstore and files
 +   * @throws IOException
 +   */
 +  @Test
 +  public void testGetFromMemStoreAndFiles() throws IOException {
 +
 +    final Configuration conf = HBaseConfiguration.create();
 +
 +    init(name.getMethodName(), conf, false);
 +
 +    //Put data in memstore
 +    this.store.add(new KeyValue(row, family, qf1, 1, value));
 +    this.store.add(new KeyValue(row, family, qf2, 1, value));
 +    //flush
 +    flush(1);
 +
 +    //Add more data
 +    this.store.add(new KeyValue(row, family, qf3, 1, value));
 +    this.store.add(new KeyValue(row, family, qf4, 1, value));
 +    //flush
 +    flush(2);
 +
 +    //Add more data
 +    this.store.add(new KeyValue(row, family, qf5, 1, value));
 +    this.store.add(new KeyValue(row, family, qf6, 1, value));
 +
 +    Scan scan = new Scan(get);
 +    InternalScanner scanner = (InternalScanner) store.getScanner(scan,
 +        scan.getFamilyMap().get(store.getFamily().getName()),
 +        0);
 +
 +    List<Cell> results = new ArrayList<Cell>();
 +    scanner.next(results);
 +    Collections.sort(results, KeyValue.COMPARATOR);
 +    scanner.close();
 +
 +    //Compare
 +    Assert.assertEquals(expected.size(), results.size());
 +    for(int i=0; i<results.size(); i++) {
 +      Assert.assertEquals(expected.get(i), results.get(i));
 +    }
 +  }
 +
 +  /**
 +   * Getting data from memstore and files
 +   * @throws IOException
 +   */
 +  @Test
 +  public void testMobCellSizeThreshold() throws IOException {
 +
 +    final Configuration conf = HBaseConfiguration.create();
 +
 +    HColumnDescriptor hcd;
 +    hcd = new HColumnDescriptor(family);
 +    hcd.setMobEnabled(true);
 +    hcd.setMobThreshold(100);
 +    hcd.setMaxVersions(4);
 +    init(name.getMethodName(), conf, hcd, false);
 +
 +    //Put data in memstore
 +    this.store.add(new KeyValue(row, family, qf1, 1, value));
 +    this.store.add(new KeyValue(row, family, qf2, 1, value));
 +    //flush
 +    flush(1);
 +
 +    //Add more data
 +    this.store.add(new KeyValue(row, family, qf3, 1, value));
 +    this.store.add(new KeyValue(row, family, qf4, 1, value));
 +    //flush
 +    flush(2);
 +
 +    //Add more data
 +    this.store.add(new KeyValue(row, family, qf5, 1, value));
 +    this.store.add(new KeyValue(row, family, qf6, 1, value));
 +    //flush
 +    flush(3);
 +
 +    Scan scan = new Scan(get);
 +    scan.setAttribute(MobConstants.MOB_SCAN_RAW, Bytes.toBytes(Boolean.TRUE));
 +    InternalScanner scanner = (InternalScanner) store.getScanner(scan,
 +      scan.getFamilyMap().get(store.getFamily().getName()),
 +      0);
 +
 +    List<Cell> results = new ArrayList<Cell>();
 +    scanner.next(results);
 +    Collections.sort(results, KeyValue.COMPARATOR);
 +    scanner.close();
 +
 +    //Compare
 +    Assert.assertEquals(expected.size(), results.size());
 +    for(int i=0; i<results.size(); i++) {
 +      Cell cell = results.get(i);
 +      //this is not mob reference cell.
 +      Assert.assertFalse(MobUtils.isMobReferenceCell(cell));
 +      Assert.assertEquals(expected.get(i), results.get(i));
 +      Assert.assertEquals(100, store.getFamily().getMobThreshold());
 +    }
 +  }
 +
 +  @Test
 +  public void testCommitFile() throws Exception {
 +    final Configuration conf = HBaseConfiguration.create();
 +    init(name.getMethodName(), conf, true);
 +    String targetPathName = MobUtils.formatDate(new Date());
 +    Path targetPath = new Path(store.getPath(), (targetPathName
 +        + Path.SEPARATOR + mobFilePath.getName()));
 +    fs.delete(targetPath, true);
 +    Assert.assertFalse(fs.exists(targetPath));
 +    //commit file
 +    store.commitFile(mobFilePath, targetPath);
 +    Assert.assertTrue(fs.exists(targetPath));
 +  }
 +
 +  @Test
 +  public void testResolve() throws Exception {
 +    final Configuration conf = HBaseConfiguration.create();
 +    init(name.getMethodName(), conf, true);
 +    String targetPathName = MobUtils.formatDate(currentDate);
 +    Path targetPath = new Path(store.getPath(), targetPathName);
 +    store.commitFile(mobFilePath, targetPath);
 +    //resolve
 +    Cell resultCell1 = store.resolve(seekKey1, false);
 +    Cell resultCell2 = store.resolve(seekKey2, false);
 +    Cell resultCell3 = store.resolve(seekKey3, false);
 +    //compare
 +    Assert.assertEquals(Bytes.toString(value),
 +        Bytes.toString(CellUtil.cloneValue(resultCell1)));
 +    Assert.assertEquals(Bytes.toString(value),
 +        Bytes.toString(CellUtil.cloneValue(resultCell2)));
 +    Assert.assertEquals(Bytes.toString(value2),
 +        Bytes.toString(CellUtil.cloneValue(resultCell3)));
 +  }
 +
 +  /**
 +   * Flush the memstore
 +   * @param storeFilesSize
 +   * @throws IOException
 +   */
 +  private void flush(int storeFilesSize) throws IOException{
 +    this.store.snapshot();
 +    flushStore(store, id++);
 +    Assert.assertEquals(storeFilesSize, this.store.getStorefiles().size());
-     Assert.assertEquals(0, ((DefaultMemStore)this.store.memstore).kvset.size());
++    Assert.assertEquals(0, ((DefaultMemStore)this.store.memstore).cellSet.size());
 +  }
 +
 +  /**
 +   * Flush the memstore
 +   * @param store
 +   * @param id
 +   * @throws IOException
 +   */
 +  private static void flushStore(HMobStore store, long id) throws IOException {
 +    StoreFlushContext storeFlushCtx = store.createFlushContext(id);
 +    storeFlushCtx.prepare();
 +    storeFlushCtx.flushCache(Mockito.mock(MonitoredTask.class));
 +    storeFlushCtx.commit(Mockito.mock(MonitoredTask.class));
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobCompaction.java
----------------------------------------------------------------------
diff --cc hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobCompaction.java
index 2d68cd1,0000000..d429de5
mode 100644,000000..100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobCompaction.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobCompaction.java
@@@ -1,467 -1,0 +1,467 @@@
 +/**
 + *
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.hadoop.hbase.regionserver;
 +
 +import static org.apache.hadoop.hbase.HBaseTestingUtility.START_KEY;
 +import static org.apache.hadoop.hbase.HBaseTestingUtility.fam1;
 +import static org.junit.Assert.assertEquals;
 +import static org.junit.Assert.assertTrue;
 +
 +import java.io.IOException;
 +import java.util.ArrayList;
 +import java.util.HashSet;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Random;
 +import java.util.Set;
 +
 +import org.apache.commons.logging.Log;
 +import org.apache.commons.logging.LogFactory;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.FileStatus;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.hbase.Cell;
 +import org.apache.hadoop.hbase.CellUtil;
 +import org.apache.hadoop.hbase.HBaseTestCase.HRegionIncommon;
 +import org.apache.hadoop.hbase.HBaseTestingUtility;
 +import org.apache.hadoop.hbase.HColumnDescriptor;
 +import org.apache.hadoop.hbase.HConstants;
 +import org.apache.hadoop.hbase.HTableDescriptor;
 +import org.apache.hadoop.hbase.KeyValue;
 +import org.apache.hadoop.hbase.KeyValueUtil;
- import org.apache.hadoop.hbase.MediumTests;
 +import org.apache.hadoop.hbase.client.Delete;
 +import org.apache.hadoop.hbase.client.Durability;
 +import org.apache.hadoop.hbase.client.Put;
 +import org.apache.hadoop.hbase.client.Scan;
 +import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 +import org.apache.hadoop.hbase.io.hfile.HFile;
 +import org.apache.hadoop.hbase.io.hfile.HFileContext;
 +import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
 +import org.apache.hadoop.hbase.mob.MobConstants;
 +import org.apache.hadoop.hbase.mob.MobUtils;
++import org.apache.hadoop.hbase.testclassification.MediumTests;
 +import org.apache.hadoop.hbase.util.Bytes;
 +import org.apache.hadoop.hbase.util.FSUtils;
 +import org.apache.hadoop.hbase.util.Pair;
 +import org.junit.After;
 +import org.junit.AfterClass;
 +import org.junit.BeforeClass;
 +import org.junit.Rule;
 +import org.junit.Test;
 +import org.junit.experimental.categories.Category;
 +import org.junit.rules.TestName;
 +
 +/**
 + * Test mob compaction
 + */
 +@Category(MediumTests.class)
 +public class TestMobCompaction {
 +  @Rule
 +  public TestName name = new TestName();
 +  static final Log LOG = LogFactory.getLog(TestMobCompaction.class.getName());
 +  private final static HBaseTestingUtility UTIL = new HBaseTestingUtility();
 +  private Configuration conf = null;
 +
 +  private HRegion region = null;
 +  private HTableDescriptor htd = null;
 +  private HColumnDescriptor hcd = null;
 +  private long mobCellThreshold = 1000;
 +
 +  private FileSystem fs;
 +
 +  private static final byte[] COLUMN_FAMILY = fam1;
 +  private final byte[] STARTROW = Bytes.toBytes(START_KEY);
 +  private int compactionThreshold;
 +
 +  @BeforeClass
 +  public static void setUpBeforeClass() throws Exception {
 +    UTIL.getConfiguration().setInt("hbase.master.info.port", 0);
 +    UTIL.getConfiguration().setBoolean("hbase.regionserver.info.port.auto", true);
 +    UTIL.startMiniCluster(1);
 +  }
 +
 +  @AfterClass
 +  public static void tearDownAfterClass() throws Exception {
 +    UTIL.shutdownMiniCluster();
 +  }
 +
 +  private void init(Configuration conf, long mobThreshold) throws Exception {
 +    this.conf = conf;
 +    this.mobCellThreshold = mobThreshold;
 +    HBaseTestingUtility UTIL = new HBaseTestingUtility(conf);
 +
 +    compactionThreshold = conf.getInt("hbase.hstore.compactionThreshold", 3);
 +    htd = UTIL.createTableDescriptor(name.getMethodName());
 +    hcd = new HColumnDescriptor(COLUMN_FAMILY);
 +    hcd.setMobEnabled(true);
 +    hcd.setMobThreshold(mobThreshold);
 +    hcd.setMaxVersions(1);
-     htd.addFamily(hcd);
++    htd.modifyFamily(hcd);
 +
 +    region = UTIL.createLocalHRegion(htd, null, null);
 +    fs = FileSystem.get(conf);
 +  }
 +
 +  @After
 +  public void tearDown() throws Exception {
 +    region.close();
 +    fs.delete(UTIL.getDataTestDir(), true);
 +  }
 +
 +  /**
 +   * During compaction, cells smaller than the threshold won't be affected.
 +   */
 +  @Test
 +  public void testSmallerValue() throws Exception {
 +    init(UTIL.getConfiguration(), 500);
 +    byte[] dummyData = makeDummyData(300); // smaller than mob threshold
 +    HRegionIncommon loader = new HRegionIncommon(region);
 +    // one hfile per row
 +    for (int i = 0; i < compactionThreshold; i++) {
 +      Put p = createPut(i, dummyData);
 +      loader.put(p);
 +      loader.flushcache();
 +    }
 +    assertEquals("Before compaction: store files", compactionThreshold, countStoreFiles());
 +    assertEquals("Before compaction: mob file count", 0, countMobFiles());
 +    assertEquals("Before compaction: rows", compactionThreshold, countRows());
 +    assertEquals("Before compaction: mob rows", 0, countMobRows());
 +
 +    region.compactStores();
 +
 +    assertEquals("After compaction: store files", 1, countStoreFiles());
 +    assertEquals("After compaction: mob file count", 0, countMobFiles());
 +    assertEquals("After compaction: referenced mob file count", 0, countReferencedMobFiles());
 +    assertEquals("After compaction: rows", compactionThreshold, countRows());
 +    assertEquals("After compaction: mob rows", 0, countMobRows());
 +  }
 +
 +  /**
 +   * During compaction, the mob threshold size is changed.
 +   */
 +  @Test
 +  public void testLargerValue() throws Exception {
 +    init(UTIL.getConfiguration(), 200);
 +    byte[] dummyData = makeDummyData(300); // larger than mob threshold
 +    HRegionIncommon loader = new HRegionIncommon(region);
 +    for (int i = 0; i < compactionThreshold; i++) {
 +      Put p = createPut(i, dummyData);
 +      loader.put(p);
 +      loader.flushcache();
 +    }
 +    assertEquals("Before compaction: store files", compactionThreshold, countStoreFiles());
 +    assertEquals("Before compaction: mob file count", compactionThreshold, countMobFiles());
 +    assertEquals("Before compaction: rows", compactionThreshold, countRows());
 +    assertEquals("Before compaction: mob rows", compactionThreshold, countMobRows());
 +    assertEquals("Before compaction: number of mob cells", compactionThreshold,
 +        countMobCellsInMetadata());
 +    // Change the threshold larger than the data size
 +    region.getTableDesc().getFamily(COLUMN_FAMILY).setMobThreshold(500);
 +    region.initialize();
 +    region.compactStores();
 +
 +    assertEquals("After compaction: store files", 1, countStoreFiles());
 +    assertEquals("After compaction: mob file count", compactionThreshold, countMobFiles());
 +    assertEquals("After compaction: referenced mob file count", 0, countReferencedMobFiles());
 +    assertEquals("After compaction: rows", compactionThreshold, countRows());
 +    assertEquals("After compaction: mob rows", 0, countMobRows());
 +  }
 +
 +  /**
 +   * This test will first generate store files, then bulk load them and trigger the compaction. When
 +   * compaction, the cell value will be larger than the threshold.
 +   */
 +  @Test
 +  public void testMobCompactionWithBulkload() throws Exception {
 +    // The following will produce store files of 600.
 +    init(UTIL.getConfiguration(), 300);
 +    byte[] dummyData = makeDummyData(600);
 +
 +    Path hbaseRootDir = FSUtils.getRootDir(conf);
 +    Path basedir = new Path(hbaseRootDir, htd.getNameAsString());
 +    List<Pair<byte[], String>> hfiles = new ArrayList<Pair<byte[], String>>(1);
 +    for (int i = 0; i < compactionThreshold; i++) {
 +      Path hpath = new Path(basedir, "hfile" + i);
 +      hfiles.add(Pair.newPair(COLUMN_FAMILY, hpath.toString()));
 +      createHFile(hpath, i, dummyData);
 +    }
 +
 +    // The following will bulk load the above generated store files and compact, with 600(fileSize)
 +    // > 300(threshold)
 +    boolean result = region.bulkLoadHFiles(hfiles, true);
 +    assertTrue("Bulkload result:", result);
 +    assertEquals("Before compaction: store files", compactionThreshold, countStoreFiles());
 +    assertEquals("Before compaction: mob file count", 0, countMobFiles());
 +    assertEquals("Before compaction: rows", compactionThreshold, countRows());
 +    assertEquals("Before compaction: mob rows", 0, countMobRows());
 +    assertEquals("Before compaction: referenced mob file count", 0, countReferencedMobFiles());
 +
 +    region.compactStores();
 +
 +    assertEquals("After compaction: store files", 1, countStoreFiles());
 +    assertEquals("After compaction: mob file count:", 1, countMobFiles());
 +    assertEquals("After compaction: rows", compactionThreshold, countRows());
 +    assertEquals("After compaction: mob rows", compactionThreshold, countMobRows());
 +    assertEquals("After compaction: referenced mob file count", 1, countReferencedMobFiles());
 +    assertEquals("After compaction: number of mob cells", compactionThreshold,
 +        countMobCellsInMetadata());
 +  }
 +
 +  @Test
 +  public void testMajorCompactionAfterDelete() throws Exception {
 +    init(UTIL.getConfiguration(), 100);
 +    byte[] dummyData = makeDummyData(200); // larger than mob threshold
 +    HRegionIncommon loader = new HRegionIncommon(region);
 +    // create hfiles and mob hfiles but don't trigger compaction
 +    int numHfiles = compactionThreshold - 1;
 +    byte[] deleteRow = Bytes.add(STARTROW, Bytes.toBytes(0));
 +    for (int i = 0; i < numHfiles; i++) {
 +      Put p = createPut(i, dummyData);
 +      loader.put(p);
 +      loader.flushcache();
 +    }
 +    assertEquals("Before compaction: store files", numHfiles, countStoreFiles());
 +    assertEquals("Before compaction: mob file count", numHfiles, countMobFiles());
 +    assertEquals("Before compaction: rows", numHfiles, countRows());
 +    assertEquals("Before compaction: mob rows", numHfiles, countMobRows());
 +    assertEquals("Before compaction: number of mob cells", numHfiles, countMobCellsInMetadata());
 +    // now let's delete some cells that contain mobs
 +    Delete delete = new Delete(deleteRow);
 +    delete.deleteFamily(COLUMN_FAMILY);
 +    region.delete(delete);
 +    loader.flushcache();
 +
 +    assertEquals("Before compaction: store files", numHfiles + 1, countStoreFiles());
 +    assertEquals("Before compaction: mob files", numHfiles, countMobFiles());
 +    region.compactStores(true);
 +    assertEquals("After compaction: store files", 1, countStoreFiles());
 +    // still have original mob hfiles and now added a mob del file
 +    assertEquals("After compaction: mob files", numHfiles + 1, countMobFiles());
 +
 +    Scan scan = new Scan();
 +    scan.setRaw(true);
 +    InternalScanner scanner = region.getScanner(scan);
 +    List<Cell> results = new ArrayList<Cell>();
 +    scanner.next(results);
 +    int deleteCount = 0;
 +    while (!results.isEmpty()) {
 +      for (Cell c : results) {
 +        if (c.getTypeByte() == KeyValue.Type.DeleteFamily.getCode()) {
 +          deleteCount++;
 +          assertTrue(Bytes.equals(CellUtil.cloneRow(c), deleteRow));
 +        }
 +      }
 +      results.clear();
 +      scanner.next(results);
 +    }
 +    // assert the delete mark is not retained after the major compaction
 +    assertEquals(0, deleteCount);
 +    scanner.close();
 +    // assert the deleted cell is not counted
 +    assertEquals("The cells in mob files", numHfiles - 1, countMobCellsInMobFiles(1));
 +  }
 +
 +  private int countStoreFiles() throws IOException {
 +    Store store = region.getStore(COLUMN_FAMILY);
 +    return store.getStorefilesCount();
 +  }
 +
 +  private int countMobFiles() throws IOException {
 +    Path mobDirPath = new Path(MobUtils.getMobRegionPath(conf, htd.getTableName()),
 +        hcd.getNameAsString());
 +    if (fs.exists(mobDirPath)) {
 +      FileStatus[] files = UTIL.getTestFileSystem().listStatus(mobDirPath);
 +      return files.length;
 +    }
 +    return 0;
 +  }
 +
 +  private long countMobCellsInMetadata() throws IOException {
 +    long mobCellsCount = 0;
 +    Path mobDirPath = new Path(MobUtils.getMobRegionPath(conf, htd.getTableName()),
 +        hcd.getNameAsString());
 +    Configuration copyOfConf = new Configuration(conf);
 +    copyOfConf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0f);
 +    CacheConfig cacheConfig = new CacheConfig(copyOfConf);
 +    if (fs.exists(mobDirPath)) {
 +      FileStatus[] files = UTIL.getTestFileSystem().listStatus(mobDirPath);
 +      for (FileStatus file : files) {
 +        StoreFile sf = new StoreFile(fs, file.getPath(), conf, cacheConfig, BloomType.NONE);
 +        Map<byte[], byte[]> fileInfo = sf.createReader().loadFileInfo();
 +        byte[] count = fileInfo.get(StoreFile.MOB_CELLS_COUNT);
 +        assertTrue(count != null);
 +        mobCellsCount += Bytes.toLong(count);
 +      }
 +    }
 +    return mobCellsCount;
 +  }
 +
 +  private Put createPut(int rowIdx, byte[] dummyData) throws IOException {
 +    Put p = new Put(Bytes.add(STARTROW, Bytes.toBytes(rowIdx)));
 +    p.setDurability(Durability.SKIP_WAL);
 +    p.add(COLUMN_FAMILY, Bytes.toBytes("colX"), dummyData);
 +    return p;
 +  }
 +
 +  /**
 +   * Create an HFile with the given number of bytes
 +   */
 +  private void createHFile(Path path, int rowIdx, byte[] dummyData) throws IOException {
 +    HFileContext meta = new HFileContextBuilder().build();
 +    HFile.Writer writer = HFile.getWriterFactory(conf, new CacheConfig(conf)).withPath(fs, path)
 +        .withFileContext(meta).create();
 +    long now = System.currentTimeMillis();
 +    try {
 +      KeyValue kv = new KeyValue(Bytes.add(STARTROW, Bytes.toBytes(rowIdx)), COLUMN_FAMILY,
 +          Bytes.toBytes("colX"), now, dummyData);
 +      writer.append(kv);
 +    } finally {
 +      writer.appendFileInfo(StoreFile.BULKLOAD_TIME_KEY, Bytes.toBytes(System.currentTimeMillis()));
 +      writer.close();
 +    }
 +  }
 +
 +  private int countMobRows() throws IOException {
 +    Scan scan = new Scan();
 +    // Do not retrieve the mob data when scanning
 +    scan.setAttribute(MobConstants.MOB_SCAN_RAW, Bytes.toBytes(Boolean.TRUE));
 +    InternalScanner scanner = region.getScanner(scan);
 +
 +    int scannedCount = 0;
 +    List<Cell> results = new ArrayList<Cell>();
 +    boolean hasMore = true;
 +    while (hasMore) {
 +      hasMore = scanner.next(results);
 +      for (Cell c : results) {
 +        if (MobUtils.isMobReferenceCell(c)) {
 +          scannedCount++;
 +        }
 +      }
 +      results.clear();
 +    }
 +    scanner.close();
 +
 +    return scannedCount;
 +  }
 +
 +  private int countRows() throws IOException {
 +    Scan scan = new Scan();
 +    // Do not retrieve the mob data when scanning
 +    InternalScanner scanner = region.getScanner(scan);
 +
 +    int scannedCount = 0;
 +    List<Cell> results = new ArrayList<Cell>();
 +    boolean hasMore = true;
 +    while (hasMore) {
 +      hasMore = scanner.next(results);
 +      scannedCount += results.size();
 +      results.clear();
 +    }
 +    scanner.close();
 +
 +    return scannedCount;
 +  }
 +
 +  private byte[] makeDummyData(int size) {
 +    byte[] dummyData = new byte[size];
 +    new Random().nextBytes(dummyData);
 +    return dummyData;
 +  }
 +
 +  private int countReferencedMobFiles() throws IOException {
 +    Scan scan = new Scan();
 +    // Do not retrieve the mob data when scanning
 +    scan.setAttribute(MobConstants.MOB_SCAN_RAW, Bytes.toBytes(Boolean.TRUE));
 +    InternalScanner scanner = region.getScanner(scan);
 +
 +    List<Cell> kvs = new ArrayList<Cell>();
 +    boolean hasMore = true;
 +    String fileName;
 +    Set<String> files = new HashSet<String>();
 +    do {
 +      kvs.clear();
 +      hasMore = scanner.next(kvs);
 +      for (Cell c : kvs) {
 +        KeyValue kv = KeyValueUtil.ensureKeyValue(c);
 +        if (!MobUtils.isMobReferenceCell(kv)) {
 +          continue;
 +        }
 +        if (!MobUtils.hasValidMobRefCellValue(kv)) {
 +          continue;
 +        }
 +        int size = MobUtils.getMobValueLength(kv);
 +        if (size <= mobCellThreshold) {
 +          continue;
 +        }
 +        fileName = MobUtils.getMobFileName(kv);
 +        if (fileName.isEmpty()) {
 +          continue;
 +        }
 +        files.add(fileName);
 +        Path familyPath = MobUtils.getMobFamilyPath(conf, htd.getTableName(),
 +            hcd.getNameAsString());
 +        assertTrue(fs.exists(new Path(familyPath, fileName)));
 +      }
 +    } while (hasMore);
 +
 +    scanner.close();
 +
 +    return files.size();
 +  }
 +
 +  private int countMobCellsInMobFiles(int expectedNumDelfiles) throws IOException {
 +    Configuration copyOfConf = new Configuration(conf);
 +    copyOfConf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0f);
 +    CacheConfig cacheConfig = new CacheConfig(copyOfConf);
 +    Path mobDirPath = new Path(MobUtils.getMobRegionPath(conf, htd.getTableName()),
 +        hcd.getNameAsString());
 +    List<StoreFile> sfs = new ArrayList<StoreFile>();
 +    int numDelfiles = 0;
 +    int size = 0;
 +    if (fs.exists(mobDirPath)) {
 +      for (FileStatus f : fs.listStatus(mobDirPath)) {
 +        StoreFile sf = new StoreFile(fs, f.getPath(), conf, cacheConfig, BloomType.NONE);
 +        sfs.add(sf);
 +        if (StoreFileInfo.isDelFile(sf.getPath())) {
 +          numDelfiles++;
 +        }
 +      }
 +      List scanners = StoreFileScanner.getScannersForStoreFiles(sfs, false, true, false, null,
 +          HConstants.LATEST_TIMESTAMP);
 +      Scan scan = new Scan();
 +      scan.setMaxVersions(hcd.getMaxVersions());
 +      long timeToPurgeDeletes = Math.max(conf.getLong("hbase.hstore.time.to.purge.deletes", 0), 0);
 +      long ttl = HStore.determineTTLFromFamily(hcd);
 +      ScanInfo scanInfo = new ScanInfo(hcd, ttl, timeToPurgeDeletes, KeyValue.COMPARATOR);
 +      StoreScanner scanner = new StoreScanner(scan, scanInfo, ScanType.COMPACT_DROP_DELETES, null,
 +          scanners, 0L, HConstants.LATEST_TIMESTAMP);
 +      List<Cell> results = new ArrayList<>();
 +      boolean hasMore = true;
 +      while (hasMore) {
 +        hasMore = scanner.next(results);
 +        size += results.size();
 +        results.clear();
 +      }
 +    }
 +    // assert the number of the existing del files
 +    assertEquals(expectedNumDelfiles, numDelfiles);
 +    return size;
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreScanner.java
----------------------------------------------------------------------
diff --cc hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreScanner.java
index bb73dba,0000000..1112b12
mode 100644,000000..100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreScanner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreScanner.java
@@@ -1,393 -1,0 +1,393 @@@
 +/**
 + *
 + * 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.io.IOException;
 +import java.util.List;
 +import java.util.Random;
 +
 +import org.apache.hadoop.fs.FileStatus;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.hbase.Cell;
 +import org.apache.hadoop.hbase.CellUtil;
 +import org.apache.hadoop.hbase.HBaseTestingUtility;
 +import org.apache.hadoop.hbase.HColumnDescriptor;
 +import org.apache.hadoop.hbase.HRegionInfo;
 +import org.apache.hadoop.hbase.HTableDescriptor;
- import org.apache.hadoop.hbase.MediumTests;
 +import org.apache.hadoop.hbase.TableName;
 +import org.apache.hadoop.hbase.client.HBaseAdmin;
 +import org.apache.hadoop.hbase.client.HTable;
 +import org.apache.hadoop.hbase.client.Put;
 +import org.apache.hadoop.hbase.client.Result;
 +import org.apache.hadoop.hbase.client.ResultScanner;
 +import org.apache.hadoop.hbase.client.Scan;
 +import org.apache.hadoop.hbase.mob.MobConstants;
 +import org.apache.hadoop.hbase.mob.MobUtils;
++import org.apache.hadoop.hbase.testclassification.MediumTests;
 +import org.apache.hadoop.hbase.util.Bytes;
 +import org.apache.hadoop.hbase.util.FSUtils;
 +import org.apache.hadoop.hbase.util.HFileArchiveUtil;
 +import org.junit.AfterClass;
 +import org.junit.Assert;
 +import org.junit.BeforeClass;
 +import org.junit.Test;
 +import org.junit.experimental.categories.Category;
 +
 +@Category(MediumTests.class)
 +public class TestMobStoreScanner {
 +
 +  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 +  private final static byte [] row1 = Bytes.toBytes("row1");
 +  private final static byte [] family = Bytes.toBytes("family");
 +  private final static byte [] qf1 = Bytes.toBytes("qualifier1");
 +  private final static byte [] qf2 = Bytes.toBytes("qualifier2");
 +  protected final byte[] qf3 = Bytes.toBytes("qualifier3");
 +  private static HTable table;
 +  private static HBaseAdmin admin;
 +  private static HColumnDescriptor hcd;
 +  private static HTableDescriptor desc;
 +  private static Random random = new Random();
 +  private static long defaultThreshold = 10;
 +
 +  @BeforeClass
 +  public static void setUpBeforeClass() throws Exception {
 +    TEST_UTIL.getConfiguration().setInt("hbase.master.info.port", 0);
 +    TEST_UTIL.getConfiguration().setBoolean("hbase.regionserver.info.port.auto", true);
 +
 +    TEST_UTIL.startMiniCluster(1);
 +  }
 +
 +  @AfterClass
 +  public static void tearDownAfterClass() throws Exception {
 +    TEST_UTIL.shutdownMiniCluster();
 +  }
 +
 +  public void setUp(long threshold, String TN) throws Exception {
 +    desc = new HTableDescriptor(TableName.valueOf(TN));
 +    hcd = new HColumnDescriptor(family);
 +    hcd.setMobEnabled(true);
 +    hcd.setMobThreshold(threshold);
 +    hcd.setMaxVersions(4);
 +    desc.addFamily(hcd);
 +    admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
 +    admin.createTable(desc);
 +    table = new HTable(TEST_UTIL.getConfiguration(), TN);
 +  }
 +
 +  /**
 +   * Generate the mob value.
 +   *
 +   * @param size the size of the value
 +   * @return the mob value generated
 +   */
 +  private static byte[] generateMobValue(int size) {
 +    byte[] mobVal = new byte[size];
 +    random.nextBytes(mobVal);
 +    return mobVal;
 +  }
 +
 +  /**
 +   * Set the scan attribute
 +   *
 +   * @param reversed if true, scan will be backward order
 +   * @param mobScanRaw if true, scan will get the mob reference
 +   * @return this
 +   */
 +  public void setScan(Scan scan, boolean reversed, boolean mobScanRaw) {
 +    scan.setReversed(reversed);
 +    scan.setMaxVersions(4);
 +    if(mobScanRaw) {
 +      scan.setAttribute(MobConstants.MOB_SCAN_RAW, Bytes.toBytes(Boolean.TRUE));
 +    }
 +  }
 +
 +  @Test
 +  public void testMobStoreScanner() throws Exception {
 +	  testGetFromFiles(false);
 +	  testGetFromMemStore(false);
 +    testGetReferences(false);
 +    testMobThreshold(false);
 +    testGetFromArchive(false);
 +  }
 +
 +  @Test
 +  public void testReversedMobStoreScanner() throws Exception {
 +	  testGetFromFiles(true);
 +	  testGetFromMemStore(true);
 +    testGetReferences(true);
 +    testMobThreshold(true);
 +    testGetFromArchive(true);
 +  }
 +
 +  public void testGetFromFiles(boolean reversed) throws Exception {
 +    String TN = "testGetFromFiles" + reversed;
 +    setUp(defaultThreshold, TN);
 +    long ts1 = System.currentTimeMillis();
 +    long ts2 = ts1 + 1;
 +    long ts3 = ts1 + 2;
 +    byte [] value = generateMobValue((int)defaultThreshold+1);
 +
 +    Put put1 = new Put(row1);
 +    put1.add(family, qf1, ts3, value);
 +    put1.add(family, qf2, ts2, value);
 +    put1.add(family, qf3, ts1, value);
 +    table.put(put1);
 +
 +    table.flushCommits();
 +    admin.flush(TN);
 +
 +    Scan scan = new Scan();
 +    setScan(scan, reversed, false);
 +
 +    ResultScanner results = table.getScanner(scan);
 +    int count = 0;
 +    for (Result res : results) {
 +      List<Cell> cells = res.listCells();
 +      for(Cell cell : cells) {
 +        // Verify the value
 +        Assert.assertEquals(Bytes.toString(value),
 +            Bytes.toString(CellUtil.cloneValue(cell)));
 +        count++;
 +      }
 +    }
 +    results.close();
 +    Assert.assertEquals(3, count);
 +  }
 +
 +  public void testGetFromMemStore(boolean reversed) throws Exception {
 +    String TN = "testGetFromMemStore" + reversed;
 +    setUp(defaultThreshold, TN);
 +    long ts1 = System.currentTimeMillis();
 +    long ts2 = ts1 + 1;
 +    long ts3 = ts1 + 2;
 +    byte [] value = generateMobValue((int)defaultThreshold+1);;
 +
 +    Put put1 = new Put(row1);
 +    put1.add(family, qf1, ts3, value);
 +    put1.add(family, qf2, ts2, value);
 +    put1.add(family, qf3, ts1, value);
 +    table.put(put1);
 +
 +    Scan scan = new Scan();
 +    setScan(scan, reversed, false);
 +
 +    ResultScanner results = table.getScanner(scan);
 +    int count = 0;
 +    for (Result res : results) {
 +      List<Cell> cells = res.listCells();
 +      for(Cell cell : cells) {
 +        // Verify the value
 +        Assert.assertEquals(Bytes.toString(value),
 +            Bytes.toString(CellUtil.cloneValue(cell)));
 +        count++;
 +      }
 +    }
 +    results.close();
 +    Assert.assertEquals(3, count);
 +  }
 +
 +  public void testGetReferences(boolean reversed) throws Exception {
 +    String TN = "testGetReferences" + reversed;
 +    setUp(defaultThreshold, TN);
 +    long ts1 = System.currentTimeMillis();
 +    long ts2 = ts1 + 1;
 +    long ts3 = ts1 + 2;
 +    byte [] value = generateMobValue((int)defaultThreshold+1);;
 +
 +    Put put1 = new Put(row1);
 +    put1.add(family, qf1, ts3, value);
 +    put1.add(family, qf2, ts2, value);
 +    put1.add(family, qf3, ts1, value);
 +    table.put(put1);
 +
 +    table.flushCommits();
 +    admin.flush(TN);
 +
 +    Scan scan = new Scan();
 +    setScan(scan, reversed, true);
 +
 +    ResultScanner results = table.getScanner(scan);
 +    int count = 0;
 +    for (Result res : results) {
 +      List<Cell> cells = res.listCells();
 +      for(Cell cell : cells) {
 +        // Verify the value
 +        assertIsMobReference(cell, row1, family, value, TN);
 +        count++;
 +      }
 +    }
 +    results.close();
 +    Assert.assertEquals(3, count);
 +  }
 +
 +  public void testMobThreshold(boolean reversed) throws Exception {
 +    String TN = "testMobThreshold" + reversed;
 +    setUp(defaultThreshold, TN);
 +    byte [] valueLess = generateMobValue((int)defaultThreshold-1);
 +    byte [] valueEqual = generateMobValue((int)defaultThreshold);
 +    byte [] valueGreater = generateMobValue((int)defaultThreshold+1);
 +    long ts1 = System.currentTimeMillis();
 +    long ts2 = ts1 + 1;
 +    long ts3 = ts1 + 2;
 +
 +    Put put1 = new Put(row1);
 +    put1.add(family, qf1, ts3, valueLess);
 +    put1.add(family, qf2, ts2, valueEqual);
 +    put1.add(family, qf3, ts1, valueGreater);
 +    table.put(put1);
 +
 +    table.flushCommits();
 +    admin.flush(TN);
 +
 +    Scan scan = new Scan();
 +    setScan(scan, reversed, true);
 +
 +    Cell cellLess= null;
 +    Cell cellEqual = null;
 +    Cell cellGreater = null;
 +    ResultScanner results = table.getScanner(scan);
 +    int count = 0;
 +    for (Result res : results) {
 +      List<Cell> cells = res.listCells();
 +      for(Cell cell : cells) {
 +        // Verify the value
 +        String qf = Bytes.toString(CellUtil.cloneQualifier(cell));
 +        if(qf.equals(Bytes.toString(qf1))) {
 +          cellLess = cell;
 +        }
 +        if(qf.equals(Bytes.toString(qf2))) {
 +          cellEqual = cell;
 +        }
 +        if(qf.equals(Bytes.toString(qf3))) {
 +          cellGreater = cell;
 +        }
 +        count++;
 +      }
 +    }
 +    Assert.assertEquals(3, count);
 +    assertNotMobReference(cellLess, row1, family, valueLess);
 +    assertNotMobReference(cellEqual, row1, family, valueEqual);
 +    assertIsMobReference(cellGreater, row1, family, valueGreater, TN);
 +    results.close();
 +  }
 +
 +  public void testGetFromArchive(boolean reversed) throws Exception {
 +    String TN = "testGetFromArchive" + reversed;
 +    setUp(defaultThreshold, TN);
 +    long ts1 = System.currentTimeMillis();
 +    long ts2 = ts1 + 1;
 +    long ts3 = ts1 + 2;
 +    byte [] value = generateMobValue((int)defaultThreshold+1);;
 +    // Put some data
 +    Put put1 = new Put(row1);
 +    put1.add(family, qf1, ts3, value);
 +    put1.add(family, qf2, ts2, value);
 +    put1.add(family, qf3, ts1, value);
 +    table.put(put1);
 +
 +    table.flushCommits();
 +    admin.flush(TN);
 +
 +    // Get the files in the mob path
 +    Path mobFamilyPath;
 +    mobFamilyPath = new Path(MobUtils.getMobRegionPath(TEST_UTIL.getConfiguration(),
 +        TableName.valueOf(TN)), hcd.getNameAsString());
 +    FileSystem fs = FileSystem.get(TEST_UTIL.getConfiguration());
 +    FileStatus[] files = fs.listStatus(mobFamilyPath);
 +
 +    // Get the archive path
 +    Path rootDir = FSUtils.getRootDir(TEST_UTIL.getConfiguration());
 +    Path tableDir = FSUtils.getTableDir(rootDir, TableName.valueOf(TN));
 +    HRegionInfo regionInfo = MobUtils.getMobRegionInfo(TableName.valueOf(TN));
 +    Path storeArchiveDir = HFileArchiveUtil.getStoreArchivePath(TEST_UTIL.getConfiguration(),
 +        regionInfo, tableDir, family);
 +
 +    // Move the files from mob path to archive path
 +    fs.mkdirs(storeArchiveDir);
 +    int fileCount = 0;
 +    for(FileStatus file : files) {
 +      fileCount++;
 +      Path filePath = file.getPath();
 +      Path src = new Path(mobFamilyPath, filePath.getName());
 +      Path dst = new Path(storeArchiveDir, filePath.getName());
 +      fs.rename(src, dst);
 +    }
 +
 +    // Verify the moving success
 +    FileStatus[] files1 = fs.listStatus(mobFamilyPath);
 +    Assert.assertEquals(0, files1.length);
 +    FileStatus[] files2 = fs.listStatus(storeArchiveDir);
 +    Assert.assertEquals(fileCount, files2.length);
 +
 +    // Scan from archive
 +    Scan scan = new Scan();
 +    setScan(scan, reversed, false);
 +    ResultScanner results = table.getScanner(scan);
 +    int count = 0;
 +    for (Result res : results) {
 +      List<Cell> cells = res.listCells();
 +      for(Cell cell : cells) {
 +        // Verify the value
 +        Assert.assertEquals(Bytes.toString(value),
 +            Bytes.toString(CellUtil.cloneValue(cell)));
 +        count++;
 +      }
 +    }
 +    results.close();
 +    Assert.assertEquals(3, count);
 +  }
 +
 +  /**
 +   * Assert the value is not store in mob.
 +   */
 +  private static void assertNotMobReference(Cell cell, byte[] row, byte[] family,
 +      byte[] value) throws IOException {
 +    Assert.assertEquals(Bytes.toString(row),
 +        Bytes.toString(CellUtil.cloneRow(cell)));
 +    Assert.assertEquals(Bytes.toString(family),
 +        Bytes.toString(CellUtil.cloneFamily(cell)));
 +    Assert.assertTrue(Bytes.toString(value).equals(
 +        Bytes.toString(CellUtil.cloneValue(cell))));
 +  }
 +
 +  /**
 +   * Assert the value is store in mob.
 +   */
 +  private static void assertIsMobReference(Cell cell, byte[] row, byte[] family,
 +      byte[] value, String TN) throws IOException {
 +    Assert.assertEquals(Bytes.toString(row),
 +        Bytes.toString(CellUtil.cloneRow(cell)));
 +    Assert.assertEquals(Bytes.toString(family),
 +        Bytes.toString(CellUtil.cloneFamily(cell)));
 +    Assert.assertFalse(Bytes.toString(value).equals(
 +        Bytes.toString(CellUtil.cloneValue(cell))));
 +    byte[] referenceValue = CellUtil.cloneValue(cell);
 +    String fileName = Bytes.toString(referenceValue, Bytes.SIZEOF_INT,
 +        referenceValue.length - Bytes.SIZEOF_INT);
 +    int valLen = Bytes.toInt(referenceValue, 0, Bytes.SIZEOF_INT);
 +    Assert.assertEquals(value.length, valLen);
 +    Path mobFamilyPath;
 +    mobFamilyPath = new Path(MobUtils.getMobRegionPath(TEST_UTIL.getConfiguration(),
 +        TableName.valueOf(TN)), hcd.getNameAsString());
 +    Path targetPath = new Path(mobFamilyPath, fileName);
 +    FileSystem fs = FileSystem.get(TEST_UTIL.getConfiguration());
 +    Assert.assertTrue(fs.exists(targetPath));
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
----------------------------------------------------------------------
diff --cc hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
index 48eba8e,aa071ef..5714351
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
@@@ -350,92 -333,35 +333,97 @@@ public class TestRegionServerMetrics 
      for (int insertCount =0; insertCount < 100; insertCount++) {
        Put p = new Put(Bytes.toBytes("" + insertCount + "row"));
        p.add(cf, qualifier, val);
-       t.put(p);
+       puts.add(p);
      }
-     t.flushCommits();
- 
-     Scan s = new Scan();
-     s.setBatch(1);
-     s.setCaching(1);
-     ResultScanner resultScanners = t.getScanner(s);
- 
-     for (int nextCount = 0; nextCount < 30; nextCount++) {
-       Result result = resultScanners.next();
-       assertNotNull(result);
-       assertEquals(1, result.size());
+     try (HTable t = TEST_UTIL.createTable(tableName, cf)) {
+       t.put(puts);
+ 
+       Scan s = new Scan();
+       s.setBatch(1);
+       s.setCaching(1);
+       ResultScanner resultScanners = t.getScanner(s);
+ 
+       for (int nextCount = 0; nextCount < 30; nextCount++) {
+         Result result = resultScanners.next();
+         assertNotNull(result);
+         assertEquals(1, result.size());
+       }
      }
-     for ( HRegionInfo i:t.getRegionLocations().keySet()) {
-       MetricsRegionAggregateSource agg = rs.getRegion(i.getRegionName())
-           .getMetrics()
-           .getSource()
-           .getAggregateSource();
-       String prefix = "namespace_"+NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR+
-           "_table_"+tableNameString +
-           "_region_" + i.getEncodedName()+
-           "_metric";
-       metricsHelper.assertCounter(prefix + "_scanNextNumOps", 30, agg);
+     try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
+       for ( HRegionLocation location: locator.getAllRegionLocations()) {
+         HRegionInfo i = location.getRegionInfo();
+         MetricsRegionAggregateSource agg = rs.getRegion(i.getRegionName())
+             .getMetrics()
+             .getSource()
+             .getAggregateSource();
+         String prefix = "namespace_"+NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR+
+             "_table_"+tableNameString +
+             "_region_" + i.getEncodedName()+
+             "_metric";
+         metricsHelper.assertCounter(prefix + "_scanNextNumOps", 30, agg);
+       }
      }
    }
 +
 +  @Test
 +  public void testMobMetrics() throws IOException, InterruptedException {
 +    String tableNameString = "testMobMetrics";
 +    TableName tableName = TableName.valueOf(tableNameString);
 +    byte[] cf = Bytes.toBytes("d");
 +    byte[] qualifier = Bytes.toBytes("qual");
 +    byte[] val = Bytes.toBytes("mobdata");
 +    int numHfiles = conf.getInt("hbase.hstore.compactionThreshold", 3) - 1;
 +    HTableDescriptor htd = new HTableDescriptor(tableName);
 +    HColumnDescriptor hcd = new HColumnDescriptor(cf);
 +    hcd.setMobEnabled(true);
 +    hcd.setMobThreshold(0);
 +    htd.addFamily(hcd);
 +    HBaseAdmin admin = new HBaseAdmin(conf);
 +    HTable t = TEST_UTIL.createTable(htd, new byte[0][0], conf);
 +    HRegion region = rs.getOnlineRegions(tableName).get(0);
 +    t.setAutoFlush(true, true);
 +    for (int insertCount = 0; insertCount < numHfiles; insertCount++) {
 +      Put p = new Put(Bytes.toBytes(insertCount));
 +      p.add(cf, qualifier, val);
 +      t.put(p);
 +      admin.flush(tableName);
 +    }
 +    metricsRegionServer.getRegionServerWrapper().forceRecompute();
 +    metricsHelper.assertCounter("mobFlushCount", numHfiles, serverSource);
 +    Scan scan = new Scan(Bytes.toBytes(0), Bytes.toBytes(2));
 +    ResultScanner scanner = t.getScanner(scan);
 +    scanner.next(100);
 +    scanner.close();
 +    metricsRegionServer.getRegionServerWrapper().forceRecompute();
 +    metricsHelper.assertCounter("mobScanCellsCount", 2, serverSource);
 +    region.getTableDesc().getFamily(cf).setMobThreshold(100);
 +    region.initialize();
 +    region.compactStores(true);
 +    metricsRegionServer.getRegionServerWrapper().forceRecompute();
 +    metricsHelper.assertCounter("mobCompactedFromMobCellsCount", numHfiles,
 +        serverSource);
 +    metricsHelper.assertCounter("mobCompactedIntoMobCellsCount", 0, serverSource);
 +    scanner = t.getScanner(scan);
 +    scanner.next(100);
 +    metricsRegionServer.getRegionServerWrapper().forceRecompute();
 +    // metrics are reset by the region initialization
 +    metricsHelper.assertCounter("mobScanCellsCount", 0, serverSource);
 +    for (int insertCount = numHfiles;
 +        insertCount < 2 * numHfiles - 1; insertCount++) {
 +      Put p = new Put(Bytes.toBytes(insertCount));
 +      p.add(cf, qualifier, val);
 +      t.put(p);
 +      admin.flush(tableName);
 +    }
 +    region.getTableDesc().getFamily(cf).setMobThreshold(0);
 +    region.initialize();
 +    region.compactStores(true);
 +    metricsRegionServer.getRegionServerWrapper().forceRecompute();
 +    // metrics are reset by the region initialization
 +    metricsHelper.assertCounter("mobCompactedFromMobCellsCount", 0, serverSource);
 +    metricsHelper.assertCounter("mobCompactedIntoMobCellsCount", 2 * numHfiles - 1,
 +        serverSource);
 +    t.close();
 +    admin.close();
 +  }
  }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/MobSnapshotTestingUtils.java
----------------------------------------------------------------------
diff --cc hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/MobSnapshotTestingUtils.java
index 2dcf83a,0000000..b7af75e
mode 100644,000000..100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/MobSnapshotTestingUtils.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/MobSnapshotTestingUtils.java
@@@ -1,355 -1,0 +1,349 @@@
 +/**
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.hadoop.hbase.snapshot;
 +
 +import static org.junit.Assert.assertEquals;
 +import static org.junit.Assert.assertTrue;
 +
 +import java.io.IOException;
 +import java.util.List;
 +
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.FSDataOutputStream;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.fs.Path;
- import org.apache.hadoop.hbase.Cell;
- import org.apache.hadoop.hbase.CellUtil;
- import org.apache.hadoop.hbase.HBaseTestingUtility;
- import org.apache.hadoop.hbase.HColumnDescriptor;
- import org.apache.hadoop.hbase.HRegionInfo;
- import org.apache.hadoop.hbase.HTableDescriptor;
- import org.apache.hadoop.hbase.TableName;
++import org.apache.hadoop.hbase.*;
 +import org.apache.hadoop.hbase.client.HTable;
 +import org.apache.hadoop.hbase.client.Result;
 +import org.apache.hadoop.hbase.client.ResultScanner;
 +import org.apache.hadoop.hbase.client.Scan;
 +import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
 +import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 +import org.apache.hadoop.hbase.regionserver.BloomType;
 +import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 +import org.apache.hadoop.hbase.util.Bytes;
 +import org.apache.hadoop.hbase.util.FSTableDescriptors;
 +import org.apache.hadoop.hbase.util.FSUtils;
 +import org.junit.Assert;
 +
 +public class MobSnapshotTestingUtils {
 +
 +  /**
 +   * Create the Mob Table.
 +   */
 +  public static void createMobTable(final HBaseTestingUtility util,
 +      final TableName tableName, int regionReplication,
 +      final byte[]... families) throws IOException, InterruptedException {
 +    HTableDescriptor htd = new HTableDescriptor(tableName);
 +    htd.setRegionReplication(regionReplication);
 +    for (byte[] family : families) {
 +      HColumnDescriptor hcd = new HColumnDescriptor(family);
 +      hcd.setMobEnabled(true);
 +      hcd.setMobThreshold(0L);
 +      htd.addFamily(hcd);
 +    }
 +    byte[][] splitKeys = SnapshotTestingUtils.getSplitKeys();
 +    util.getHBaseAdmin().createTable(htd, splitKeys);
 +    SnapshotTestingUtils.waitForTableToBeOnline(util, tableName);
 +    assertEquals((splitKeys.length + 1) * regionReplication, util
 +        .getHBaseAdmin().getTableRegions(tableName).size());
 +  }
 +
 +  /**
 +   * Create a Mob table.
 +   *
 +   * @param util
 +   * @param tableName
 +   * @param families
 +   * @return An HTable instance for the created table.
 +   * @throws IOException
 +   */
 +  public static HTable createMobTable(final HBaseTestingUtility util,
 +      final TableName tableName, final byte[]... families) throws IOException {
 +    HTableDescriptor htd = new HTableDescriptor(tableName);
 +    for (byte[] family : families) {
 +      HColumnDescriptor hcd = new HColumnDescriptor(family);
 +      // Disable blooms (they are on by default as of 0.95) but we disable them
 +      // here because
 +      // tests have hard coded counts of what to expect in block cache, etc.,
 +      // and blooms being
 +      // on is interfering.
 +      hcd.setBloomFilterType(BloomType.NONE);
 +      hcd.setMobEnabled(true);
 +      hcd.setMobThreshold(0L);
 +      htd.addFamily(hcd);
 +    }
 +    util.getHBaseAdmin().createTable(htd);
 +    // HBaseAdmin only waits for regions to appear in hbase:meta we should wait
 +    // until they are assigned
 +    util.waitUntilAllRegionsAssigned(htd.getTableName());
 +    return new HTable(util.getConfiguration(), htd.getTableName());
 +  }
 +
 +  /**
 +   * Return the number of rows in the given table.
 +   */
 +  public static int countMobRows(final HTable table) throws IOException {
 +    Scan scan = new Scan();
 +    ResultScanner results = table.getScanner(scan);
 +    int count = 0;
 +    for (Result res : results) {
 +      count++;
 +      List<Cell> cells = res.listCells();
 +      for (Cell cell : cells) {
 +        // Verify the value
 +        Assert.assertTrue(CellUtil.cloneValue(cell).length > 0);
 +      }
 +    }
 +    results.close();
 +    return count;
 +  }
 +
 +  /**
 +   * Return the number of rows in the given table.
 +   */
 +  public static int countMobRows(final HTable table, final byte[]... families)
 +      throws IOException {
 +    Scan scan = new Scan();
 +    for (byte[] family : families) {
 +      scan.addFamily(family);
 +    }
 +    ResultScanner results = table.getScanner(scan);
 +    int count = 0;
 +    for (Result res : results) {
 +      count++;
 +      List<Cell> cells = res.listCells();
 +      for (Cell cell : cells) {
 +        // Verify the value
 +        Assert.assertTrue(CellUtil.cloneValue(cell).length > 0);
 +      }
 +    }
 +    results.close();
 +    return count;
 +  }
 +
 +  public static void verifyMobRowCount(final HBaseTestingUtility util,
 +      final TableName tableName, long expectedRows) throws IOException {
 +    HTable table = new HTable(util.getConfiguration(), tableName);
 +    try {
 +      assertEquals(expectedRows, countMobRows(table));
 +    } finally {
 +      table.close();
 +    }
 +  }
 +
 +  // ==========================================================================
 +  // Snapshot Mock
 +  // ==========================================================================
 +  public static class SnapshotMock {
 +    private final static String TEST_FAMILY = "cf";
 +    public final static int TEST_NUM_REGIONS = 4;
 +
 +    private final Configuration conf;
 +    private final FileSystem fs;
 +    private final Path rootDir;
 +
 +    static class RegionData {
 +      public HRegionInfo hri;
 +      public Path tableDir;
 +      public Path[] files;
 +
 +      public RegionData(final Path tableDir, final HRegionInfo hri,
 +          final int nfiles) {
 +        this.tableDir = tableDir;
 +        this.hri = hri;
 +        this.files = new Path[nfiles];
 +      }
 +    }
 +
 +    public static class SnapshotBuilder {
 +      private final RegionData[] tableRegions;
 +      private final SnapshotDescription desc;
 +      private final HTableDescriptor htd;
 +      private final Configuration conf;
 +      private final FileSystem fs;
 +      private final Path rootDir;
 +      private Path snapshotDir;
 +      private int snapshotted = 0;
 +
 +      public SnapshotBuilder(final Configuration conf, final FileSystem fs,
 +          final Path rootDir, final HTableDescriptor htd,
 +          final SnapshotDescription desc, final RegionData[] tableRegions)
 +          throws IOException {
 +        this.fs = fs;
 +        this.conf = conf;
 +        this.rootDir = rootDir;
 +        this.htd = htd;
 +        this.desc = desc;
 +        this.tableRegions = tableRegions;
 +        this.snapshotDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(desc,
 +            rootDir);
 +        new FSTableDescriptors(conf).createTableDescriptorForTableDirectory(
-             snapshotDir, htd, false);
++            snapshotDir, new TableDescriptor(htd), false);
 +      }
 +
 +      public HTableDescriptor getTableDescriptor() {
 +        return this.htd;
 +      }
 +
 +      public SnapshotDescription getSnapshotDescription() {
 +        return this.desc;
 +      }
 +
 +      public Path getSnapshotsDir() {
 +        return this.snapshotDir;
 +      }
 +
 +      public Path[] addRegion() throws IOException {
 +        return addRegion(desc);
 +      }
 +
 +      public Path[] addRegionV1() throws IOException {
 +        return addRegion(desc.toBuilder()
 +            .setVersion(SnapshotManifestV1.DESCRIPTOR_VERSION).build());
 +      }
 +
 +      public Path[] addRegionV2() throws IOException {
 +        return addRegion(desc.toBuilder()
 +            .setVersion(SnapshotManifestV2.DESCRIPTOR_VERSION).build());
 +      }
 +
 +      private Path[] addRegion(final SnapshotDescription desc)
 +          throws IOException {
 +        if (this.snapshotted == tableRegions.length) {
 +          throw new UnsupportedOperationException(
 +              "No more regions in the table");
 +        }
 +
 +        RegionData regionData = tableRegions[this.snapshotted++];
 +        ForeignExceptionDispatcher monitor = new ForeignExceptionDispatcher(
 +            desc.getName());
 +        SnapshotManifest manifest = SnapshotManifest.create(conf, fs,
 +            snapshotDir, desc, monitor);
 +        manifest.addRegion(regionData.tableDir, regionData.hri);
 +        return regionData.files;
 +      }
 +
 +      public Path commit() throws IOException {
 +        ForeignExceptionDispatcher monitor = new ForeignExceptionDispatcher(
 +            desc.getName());
 +        SnapshotManifest manifest = SnapshotManifest.create(conf, fs,
 +            snapshotDir, desc, monitor);
 +        manifest.addTableDescriptor(htd);
 +        manifest.consolidate();
 +        SnapshotDescriptionUtils.completeSnapshot(desc, rootDir, snapshotDir,
 +            fs);
 +        snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(desc,
 +            rootDir);
 +        return snapshotDir;
 +      }
 +    }
 +
 +    public SnapshotMock(final Configuration conf, final FileSystem fs,
 +        final Path rootDir) {
 +      this.fs = fs;
 +      this.conf = conf;
 +      this.rootDir = rootDir;
 +    }
 +
 +    public SnapshotBuilder createSnapshotV1(final String snapshotName)
 +        throws IOException {
 +      return createSnapshot(snapshotName, SnapshotManifestV1.DESCRIPTOR_VERSION);
 +    }
 +
 +    public SnapshotBuilder createSnapshotV2(final String snapshotName)
 +        throws IOException {
 +      return createSnapshot(snapshotName, SnapshotManifestV2.DESCRIPTOR_VERSION);
 +    }
 +
 +    private SnapshotBuilder createSnapshot(final String snapshotName,
 +        final int version) throws IOException {
 +      HTableDescriptor htd = createHtd(snapshotName);
 +
 +      RegionData[] regions = createTable(htd, TEST_NUM_REGIONS);
 +
 +      SnapshotDescription desc = SnapshotDescription.newBuilder()
 +          .setTable(htd.getNameAsString()).setName(snapshotName)
 +          .setVersion(version).build();
 +
 +      Path workingDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(desc,
 +          rootDir);
 +      SnapshotDescriptionUtils.writeSnapshotInfo(desc, workingDir, fs);
 +      return new SnapshotBuilder(conf, fs, rootDir, htd, desc, regions);
 +    }
 +
 +    public HTableDescriptor createHtd(final String tableName) {
 +      HTableDescriptor htd = new HTableDescriptor(tableName);
 +      HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAMILY);
 +      hcd.setMobEnabled(true);
 +      hcd.setMobThreshold(0L);
 +      htd.addFamily(hcd);
 +      return htd;
 +    }
 +
 +    private RegionData[] createTable(final HTableDescriptor htd,
 +        final int nregions) throws IOException {
 +      Path tableDir = FSUtils.getTableDir(rootDir, htd.getTableName());
 +      new FSTableDescriptors(conf).createTableDescriptorForTableDirectory(
-           tableDir, htd, false);
++          tableDir, new TableDescriptor(htd), false);
 +
 +      assertTrue(nregions % 2 == 0);
 +      RegionData[] regions = new RegionData[nregions];
 +      for (int i = 0; i < regions.length; i += 2) {
 +        byte[] startKey = Bytes.toBytes(0 + i * 2);
 +        byte[] endKey = Bytes.toBytes(1 + i * 2);
 +
 +        // First region, simple with one plain hfile.
 +        HRegionInfo hri = new HRegionInfo(htd.getTableName(), startKey, endKey);
 +        HRegionFileSystem rfs = HRegionFileSystem.createRegionOnFileSystem(
 +            conf, fs, tableDir, hri);
 +        regions[i] = new RegionData(tableDir, hri, 3);
 +        for (int j = 0; j < regions[i].files.length; ++j) {
 +          Path storeFile = createStoreFile(rfs.createTempName());
 +          regions[i].files[j] = rfs.commitStoreFile(TEST_FAMILY, storeFile);
 +        }
 +
 +        // Second region, used to test the split case.
 +        // This region contains a reference to the hfile in the first region.
 +        startKey = Bytes.toBytes(2 + i * 2);
 +        endKey = Bytes.toBytes(3 + i * 2);
 +        hri = new HRegionInfo(htd.getTableName());
 +        rfs = HRegionFileSystem.createRegionOnFileSystem(conf, fs, tableDir,
 +            hri);
 +        regions[i + 1] = new RegionData(tableDir, hri, regions[i].files.length);
 +        for (int j = 0; j < regions[i].files.length; ++j) {
 +          String refName = regions[i].files[j].getName() + '.'
 +              + regions[i].hri.getEncodedName();
 +          Path refFile = createStoreFile(new Path(rootDir, refName));
 +          regions[i + 1].files[j] = rfs.commitStoreFile(TEST_FAMILY, refFile);
 +        }
 +      }
 +      return regions;
 +    }
 +
 +    private Path createStoreFile(final Path storeFile) throws IOException {
 +      FSDataOutputStream out = fs.create(storeFile);
 +      try {
 +        out.write(Bytes.toBytes(storeFile.toString()));
 +      } finally {
 +        out.close();
 +      }
 +      return storeFile;
 +    }
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java
----------------------------------------------------------------------


[12/50] [abbrv] hbase git commit: HBASE-12891 Parallel execution for Hbck checkRegionConsistency

Posted by jm...@apache.org.
HBASE-12891 Parallel execution for Hbck checkRegionConsistency

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


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

Branch: refs/heads/hbase-11339
Commit: eddd5739a14ceb5cfc9b9c7d2e357eea96bd9703
Parents: 3b56d2a
Author: rahulgidwani <ra...@flurry.com>
Authored: Fri Feb 6 15:14:18 2015 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Fri Feb 6 15:14:18 2015 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/util/HBaseFsck.java | 28 +++++++++++++++++++-
 1 file changed, 27 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/eddd5739/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
index 8e1d848..96bd0f7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
@@ -1703,9 +1703,19 @@ public class HBaseFsck extends Configured implements Closeable {
   throws IOException, KeeperException, InterruptedException {
     // Divide the checks in two phases. One for default/primary replicas and another
     // for the non-primary ones. Keeps code cleaner this way.
+    List<WorkItemRegionConsistency> workItems =
+        new ArrayList<WorkItemRegionConsistency>(regionInfoMap.size());
     for (java.util.Map.Entry<String, HbckInfo> e: regionInfoMap.entrySet()) {
       if (e.getValue().getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID) {
-        checkRegionConsistency(e.getKey(), e.getValue());
+        workItems.add(new WorkItemRegionConsistency(e.getKey(), e.getValue()));
+      }
+    }
+    List<Future<Void>> workFutures = executor.invokeAll(workItems);
+    for(Future<Void> f: workFutures) {
+      try {
+        f.get();
+      } catch(ExecutionException e1) {
+        LOG.warn("Could not check region consistency " , e1.getCause());
       }
     }
     boolean prevHdfsCheck = shouldCheckHdfs();
@@ -2355,6 +2365,22 @@ public class HBaseFsck extends Configured implements Closeable {
     }
   };
 
+  class WorkItemRegionConsistency implements Callable<Void> {
+    private final String key;
+    private final HbckInfo hbi;
+
+    WorkItemRegionConsistency(String key, HbckInfo hbi) {
+      this.key = key;
+      this.hbi = hbi;
+    }
+
+    @Override
+    public synchronized Void call() throws Exception {
+      checkRegionConsistency(key, hbi);
+      return null;
+    }
+  }
+
 
   /**
    * Maintain information about a particular table.


[18/50] [abbrv] hbase git commit: HBASE-11567 Write bulk load COMMIT events to WAL (Alex Newman, Jeffrey Zhong)

Posted by jm...@apache.org.
HBASE-11567 Write bulk load COMMIT events to WAL (Alex Newman, Jeffrey Zhong)


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

Branch: refs/heads/hbase-11339
Commit: 3f4427739d9ff698d39f2687f11f65967c67340d
Parents: 7f4146b
Author: Jeffrey Zhong <je...@apache.org>
Authored: Wed Feb 4 15:52:01 2015 -0800
Committer: Jeffrey Zhong <je...@apache.org>
Committed: Fri Feb 6 18:07:30 2015 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/protobuf/ProtobufUtil.java     |   35 +-
 .../hbase/protobuf/generated/FilterProtos.java  |    6 +-
 .../generated/VisibilityLabelsProtos.java       |    6 +-
 .../hbase/protobuf/generated/WALProtos.java     | 3441 ++++++++++++------
 hbase-protocol/src/main/protobuf/WAL.proto      |   23 +-
 .../hadoop/hbase/regionserver/HRegion.java      |   95 +-
 .../hadoop/hbase/regionserver/wal/WALEdit.java  |   35 +-
 .../hadoop/hbase/regionserver/wal/WALUtil.java  |   39 +
 .../hadoop/hbase/regionserver/TestBulkLoad.java |  312 ++
 .../hadoop/hbase/regionserver/TestHRegion.java  |    2 +-
 .../regionserver/TestHRegionServerBulkLoad.java |   46 +-
 .../wal/TestWALActionsListener.java             |    2 +-
 pom.xml                                         |   17 +
 13 files changed, 2869 insertions(+), 1190 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/3f442773/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
index 9c383f8..caae1bb 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
@@ -125,6 +125,8 @@ import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor.FlushAction;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.EventType;
+import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor;
+import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor;
 import org.apache.hadoop.hbase.quotas.QuotaScope;
 import org.apache.hadoop.hbase.quotas.QuotaType;
 import org.apache.hadoop.hbase.quotas.ThrottleType;
@@ -2608,8 +2610,7 @@ public final class ProtobufUtil {
         .setServer(toServerName(server));
 
     for (Map.Entry<byte[], List<Path>> entry : storeFiles.entrySet()) {
-      RegionEventDescriptor.StoreDescriptor.Builder builder
-        = RegionEventDescriptor.StoreDescriptor.newBuilder()
+      StoreDescriptor.Builder builder = StoreDescriptor.newBuilder()
           .setFamilyName(ByteStringer.wrap(entry.getKey()))
           .setStoreHomeDir(Bytes.toString(entry.getKey()));
       for (Path path : entry.getValue()) {
@@ -2963,4 +2964,34 @@ public final class ProtobufUtil {
             .setScope(toProtoQuotaScope(scope))
             .build();
   }
+
+  /**
+   * Generates a marker for the WAL so that we propagate the notion of a bulk region load
+   * throughout the WAL.
+   *
+   * @param tableName         The tableName into which the bulk load is being imported into.
+   * @param encodedRegionName Encoded region name of the region which is being bulk loaded.
+   * @param storeFiles        A set of store files of a column family are bulk loaded.
+   * @param bulkloadSeqId     sequence ID (by a force flush) used to create bulk load hfile
+   *                          name
+   * @return The WAL log marker for bulk loads.
+   */
+  public static WALProtos.BulkLoadDescriptor toBulkLoadDescriptor(TableName tableName,
+      ByteString encodedRegionName, Map<byte[], List<Path>> storeFiles, long bulkloadSeqId) {
+    BulkLoadDescriptor.Builder desc = BulkLoadDescriptor.newBuilder()
+        .setTableName(ProtobufUtil.toProtoTableName(tableName))
+        .setEncodedRegionName(encodedRegionName).setBulkloadSeqNum(bulkloadSeqId);
+
+    for (Map.Entry<byte[], List<Path>> entry : storeFiles.entrySet()) {
+      WALProtos.StoreDescriptor.Builder builder = StoreDescriptor.newBuilder()
+          .setFamilyName(ByteStringer.wrap(entry.getKey()))
+          .setStoreHomeDir(Bytes.toString(entry.getKey())); // relative to region
+      for (Path path : entry.getValue()) {
+        builder.addStoreFile(path.getName());
+      }
+      desc.addStores(builder);
+    }
+
+    return desc.build();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/3f442773/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/FilterProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/FilterProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/FilterProtos.java
index 3fb466d..af1f33d 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/FilterProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/FilterProtos.java
@@ -16611,7 +16611,7 @@ public final class FilterProtos {
     /**
      * <code>repeated .RowRange row_range_list = 1;</code>
      */
-    java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRangeOrBuilder>
+    java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRangeOrBuilder> 
         getRowRangeListOrBuilderList();
     /**
      * <code>repeated .RowRange row_range_list = 1;</code>
@@ -17270,12 +17270,12 @@ public final class FilterProtos {
       /**
        * <code>repeated .RowRange row_range_list = 1;</code>
        */
-      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange.Builder>
+      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange.Builder> 
            getRowRangeListBuilderList() {
         return getRowRangeListFieldBuilder().getBuilderList();
       }
       private com.google.protobuf.RepeatedFieldBuilder<
-          org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange.Builder, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRangeOrBuilder>
+          org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange.Builder, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRangeOrBuilder> 
           getRowRangeListFieldBuilder() {
         if (rowRangeListBuilder_ == null) {
           rowRangeListBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<

http://git-wip-us.apache.org/repos/asf/hbase/blob/3f442773/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/VisibilityLabelsProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/VisibilityLabelsProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/VisibilityLabelsProtos.java
index 294772e..70593b0 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/VisibilityLabelsProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/VisibilityLabelsProtos.java
@@ -5092,7 +5092,7 @@ public final class VisibilityLabelsProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -5108,7 +5108,7 @@ public final class VisibilityLabelsProtos {
         getRegexBytes() {
       java.lang.Object ref = regex_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         regex_ = b;
@@ -5414,7 +5414,7 @@ public final class VisibilityLabelsProtos {
           getRegexBytes() {
         java.lang.Object ref = regex_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           regex_ = b;


[45/50] [abbrv] hbase git commit: Merge branch 'master' (2/11/15) into hbase-11339

Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/filecompactions/TestMobFileCompactor.java
----------------------------------------------------------------------
diff --cc hbase-server/src/test/java/org/apache/hadoop/hbase/mob/filecompactions/TestMobFileCompactor.java
index 9a8b7d9,0000000..4bf1623
mode 100644,000000..100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/filecompactions/TestMobFileCompactor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/filecompactions/TestMobFileCompactor.java
@@@ -1,652 -1,0 +1,652 @@@
 +/**
 + *
 + * 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.mob.filecompactions;
 +
 +import static org.junit.Assert.assertEquals;
 +
 +import java.io.IOException;
 +import java.util.ArrayList;
 +import java.util.Collections;
 +import java.util.List;
 +import java.util.Random;
 +import java.util.concurrent.ExecutorService;
 +import java.util.concurrent.RejectedExecutionException;
 +import java.util.concurrent.RejectedExecutionHandler;
 +import java.util.concurrent.SynchronousQueue;
 +import java.util.concurrent.ThreadPoolExecutor;
 +import java.util.concurrent.TimeUnit;
 +
 +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.Cell;
 +import org.apache.hadoop.hbase.CellUtil;
 +import org.apache.hadoop.hbase.HBaseTestingUtility;
 +import org.apache.hadoop.hbase.HColumnDescriptor;
 +import org.apache.hadoop.hbase.HTableDescriptor;
- import org.apache.hadoop.hbase.LargeTests;
++import org.apache.hadoop.hbase.testclassification.LargeTests;
 +import org.apache.hadoop.hbase.TableName;
 +import org.apache.hadoop.hbase.client.Admin;
 +import org.apache.hadoop.hbase.client.Delete;
 +import org.apache.hadoop.hbase.client.Durability;
 +import org.apache.hadoop.hbase.client.HTable;
 +import org.apache.hadoop.hbase.client.Put;
 +import org.apache.hadoop.hbase.client.Result;
 +import org.apache.hadoop.hbase.client.ResultScanner;
 +import org.apache.hadoop.hbase.client.Scan;
 +import org.apache.hadoop.hbase.io.HFileLink;
 +import org.apache.hadoop.hbase.mob.MobConstants;
 +import org.apache.hadoop.hbase.mob.MobUtils;
 +import org.apache.hadoop.hbase.regionserver.HRegion;
 +import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
 +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(LargeTests.class)
 +public class TestMobFileCompactor {
 +  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 +  private Configuration conf = null;
 +  private String tableNameAsString;
 +  private TableName tableName;
 +  private static HTable hTable;
 +  private static Admin admin;
 +  private static HTableDescriptor desc;
 +  private static HColumnDescriptor hcd1;
 +  private static HColumnDescriptor hcd2;
 +  private static FileSystem fs;
 +  private final static String family1 = "family1";
 +  private final static String family2 = "family2";
 +  private final static String qf1 = "qualifier1";
 +  private final static String qf2 = "qualifier2";
 +  private static byte[] KEYS = Bytes.toBytes("012");
 +  private static int regionNum = KEYS.length;
 +  private static int delRowNum = 1;
 +  private static int delCellNum = 6;
 +  private static int cellNumPerRow = 3;
 +  private static int rowNumPerFile = 2;
 +  private static ExecutorService pool;
 +
 +  @BeforeClass
 +  public static void setUpBeforeClass() throws Exception {
 +    TEST_UTIL.getConfiguration().setInt("hbase.master.info.port", 0);
 +    TEST_UTIL.getConfiguration().setBoolean("hbase.regionserver.info.port.auto", true);
 +    TEST_UTIL.startMiniCluster(1);
 +    pool = createThreadPool(TEST_UTIL.getConfiguration());
 +  }
 +
 +  @AfterClass
 +  public static void tearDownAfterClass() throws Exception {
 +    pool.shutdown();
 +    TEST_UTIL.shutdownMiniCluster();
 +  }
 +
 +  @Before
 +  public void setUp() throws Exception {
 +    fs = TEST_UTIL.getTestFileSystem();
 +    conf = TEST_UTIL.getConfiguration();
 +    long tid = System.currentTimeMillis();
 +    tableNameAsString = "testMob" + tid;
 +    tableName = TableName.valueOf(tableNameAsString);
 +    hcd1 = new HColumnDescriptor(family1);
 +    hcd1.setMobEnabled(true);
 +    hcd1.setMobThreshold(0L);
 +    hcd1.setMaxVersions(4);
 +    hcd2 = new HColumnDescriptor(family2);
 +    hcd2.setMobEnabled(true);
 +    hcd2.setMobThreshold(0L);
 +    hcd2.setMaxVersions(4);
 +    desc = new HTableDescriptor(tableName);
 +    desc.addFamily(hcd1);
 +    desc.addFamily(hcd2);
 +    admin = TEST_UTIL.getHBaseAdmin();
 +    admin.createTable(desc, getSplitKeys());
 +    hTable = new HTable(conf, tableNameAsString);
 +    hTable.setAutoFlush(false, false);
 +  }
 +
 +  @After
 +  public void tearDown() throws Exception {
 +    admin.disableTable(tableName);
 +    admin.deleteTable(tableName);
 +    admin.close();
 +    hTable.close();
 +    fs.delete(TEST_UTIL.getDataTestDir(), true);
 +  }
 +
 +  @Test
 +  public void testCompactionWithoutDelFiles() throws Exception {
 +    resetConf();
 +    int count = 4;
 +    // generate mob files
 +    loadData(count, rowNumPerFile);
 +    int rowNumPerRegion = count*rowNumPerFile;
 +
 +    assertEquals("Before compaction: mob rows count", regionNum*rowNumPerRegion,
 +        countMobRows(hTable));
 +    assertEquals("Before compaction: mob file count", regionNum*count, countFiles(true, family1));
 +    assertEquals("Before compaction: del file count", 0, countFiles(false, family1));
 +
 +    MobFileCompactor compactor = new PartitionedMobFileCompactor(conf, fs, tableName, hcd1, pool);
 +    compactor.compact();
 +
 +    assertEquals("After compaction: mob rows count", regionNum*rowNumPerRegion,
 +        countMobRows(hTable));
 +    assertEquals("After compaction: mob file count", regionNum, countFiles(true, family1));
 +    assertEquals("After compaction: del file count", 0, countFiles(false, family1));
 +  }
 +
 +  @Test
 +  public void testCompactionWithDelFiles() throws Exception {
 +    resetConf();
 +    int count = 4;
 +    // generate mob files
 +    loadData(count, rowNumPerFile);
 +    int rowNumPerRegion = count*rowNumPerFile;
 +
 +    assertEquals("Before deleting: mob rows count", regionNum*rowNumPerRegion,
 +        countMobRows(hTable));
 +    assertEquals("Before deleting: mob cells count", regionNum*cellNumPerRow*rowNumPerRegion,
 +        countMobCells(hTable));
 +    assertEquals("Before deleting: family1 mob file count", regionNum*count,
 +        countFiles(true, family1));
 +    assertEquals("Before deleting: family2 mob file count", regionNum*count,
 +        countFiles(true, family2));
 +
 +    createDelFile();
 +
 +    assertEquals("Before compaction: mob rows count", regionNum*(rowNumPerRegion-delRowNum),
 +        countMobRows(hTable));
 +    assertEquals("Before compaction: mob cells count",
 +        regionNum*(cellNumPerRow*rowNumPerRegion-delCellNum), countMobCells(hTable));
 +    assertEquals("Before compaction: family1 mob file count", regionNum*count,
 +        countFiles(true, family1));
 +    assertEquals("Before compaction: family2 file count", regionNum*count,
 +        countFiles(true, family2));
 +    assertEquals("Before compaction: family1 del file count", regionNum,
 +        countFiles(false, family1));
 +    assertEquals("Before compaction: family2 del file count", regionNum,
 +        countFiles(false, family2));
 +
 +    // do the mob file compaction
 +    MobFileCompactor compactor = new PartitionedMobFileCompactor(conf, fs, tableName, hcd1, pool);
 +    compactor.compact();
 +
 +    assertEquals("After compaction: mob rows count", regionNum*(rowNumPerRegion-delRowNum),
 +        countMobRows(hTable));
 +    assertEquals("After compaction: mob cells count",
 +        regionNum*(cellNumPerRow*rowNumPerRegion-delCellNum), countMobCells(hTable));
 +    assertEquals("After compaction: family1 mob file count", regionNum,
 +        countFiles(true, family1));
 +    assertEquals("After compaction: family2 mob file count", regionNum*count,
 +        countFiles(true, family2));
 +    assertEquals("After compaction: family1 del file count", 0, countFiles(false, family1));
 +    assertEquals("After compaction: family2 del file count", regionNum,
 +        countFiles(false, family2));
 +    assertRefFileNameEqual(family1);
 +  }
 +
 +  private void assertRefFileNameEqual(String familyName) throws IOException {
 +    Scan scan = new Scan();
 +    scan.addFamily(Bytes.toBytes(familyName));
 +    // Do not retrieve the mob data when scanning
 +    scan.setAttribute(MobConstants.MOB_SCAN_RAW, Bytes.toBytes(Boolean.TRUE));
 +    ResultScanner results = hTable.getScanner(scan);
 +    Path mobFamilyPath = new Path(MobUtils.getMobRegionPath(TEST_UTIL.getConfiguration(),
 +        tableName), familyName);
 +    List<Path> actualFilePaths = new ArrayList<>();
 +    List<Path> expectFilePaths = new ArrayList<>();
 +    for (Result res : results) {
 +      for (Cell cell : res.listCells()) {
 +        byte[] referenceValue = CellUtil.cloneValue(cell);
 +        String fileName = Bytes.toString(referenceValue, Bytes.SIZEOF_INT,
 +            referenceValue.length - Bytes.SIZEOF_INT);
 +        Path targetPath = new Path(mobFamilyPath, fileName);
 +        if(!actualFilePaths.contains(targetPath)) {
 +          actualFilePaths.add(targetPath);
 +        }
 +      }
 +    }
 +    results.close();
 +    if (fs.exists(mobFamilyPath)) {
 +      FileStatus[] files = fs.listStatus(mobFamilyPath);
 +      for (FileStatus file : files) {
 +        if (!StoreFileInfo.isDelFile(file.getPath())) {
 +          expectFilePaths.add(file.getPath());
 +        }
 +      }
 +    }
 +    Collections.sort(actualFilePaths);
 +    Collections.sort(expectFilePaths);
 +    assertEquals(expectFilePaths, actualFilePaths);
 +  }
 +
 +  @Test
 +  public void testCompactionWithDelFilesAndNotMergeAllFiles() throws Exception {
 +    resetConf();
 +    int mergeSize = 5000;
 +    // change the mob compaction merge size
 +    conf.setLong(MobConstants.MOB_FILE_COMPACTION_MERGEABLE_THRESHOLD, mergeSize);
 +
 +    int count = 4;
 +    // generate mob files
 +    loadData(count, rowNumPerFile);
 +    int rowNumPerRegion = count*rowNumPerFile;
 +
 +    assertEquals("Before deleting: mob rows count", regionNum*rowNumPerRegion,
 +        countMobRows(hTable));
 +    assertEquals("Before deleting: mob cells count", regionNum*cellNumPerRow*rowNumPerRegion,
 +        countMobCells(hTable));
 +    assertEquals("Before deleting: mob file count", regionNum*count, countFiles(true, family1));
 +
 +    int largeFilesCount = countLargeFiles(mergeSize, family1);
 +    createDelFile();
 +
 +    assertEquals("Before compaction: mob rows count", regionNum*(rowNumPerRegion-delRowNum),
 +        countMobRows(hTable));
 +    assertEquals("Before compaction: mob cells count",
 +        regionNum*(cellNumPerRow*rowNumPerRegion-delCellNum), countMobCells(hTable));
 +    assertEquals("Before compaction: family1 mob file count", regionNum*count,
 +        countFiles(true, family1));
 +    assertEquals("Before compaction: family2 mob file count", regionNum*count,
 +        countFiles(true, family2));
 +    assertEquals("Before compaction: family1 del file count", regionNum,
 +        countFiles(false, family1));
 +    assertEquals("Before compaction: family2 del file count", regionNum,
 +        countFiles(false, family2));
 +
 +    // do the mob file compaction
 +    MobFileCompactor compactor = new PartitionedMobFileCompactor(conf, fs, tableName, hcd1, pool);
 +    compactor.compact();
 +
 +    assertEquals("After compaction: mob rows count", regionNum*(rowNumPerRegion-delRowNum),
 +        countMobRows(hTable));
 +    assertEquals("After compaction: mob cells count",
 +        regionNum*(cellNumPerRow*rowNumPerRegion-delCellNum), countMobCells(hTable));
 +    // After the compaction, the files smaller than the mob compaction merge size
 +    // is merge to one file
 +    assertEquals("After compaction: family1 mob file count", largeFilesCount + regionNum,
 +        countFiles(true, family1));
 +    assertEquals("After compaction: family2 mob file count", regionNum*count,
 +        countFiles(true, family2));
 +    assertEquals("After compaction: family1 del file count", regionNum,
 +        countFiles(false, family1));
 +    assertEquals("After compaction: family2 del file count", regionNum,
 +        countFiles(false, family2));
 +  }
 +
 +  @Test
 +  public void testCompactionWithDelFilesAndWithSmallCompactionBatchSize() throws Exception {
 +    resetConf();
 +    int batchSize = 2;
 +    conf.setInt(MobConstants.MOB_FILE_COMPACTION_BATCH_SIZE, batchSize);
 +    int count = 4;
 +    // generate mob files
 +    loadData(count, rowNumPerFile);
 +    int rowNumPerRegion = count*rowNumPerFile;
 +
 +    assertEquals("Before deleting: mob row count", regionNum*rowNumPerRegion,
 +        countMobRows(hTable));
 +    assertEquals("Before deleting: family1 mob file count", regionNum*count,
 +        countFiles(true, family1));
 +    assertEquals("Before deleting: family2 mob file count", regionNum*count,
 +        countFiles(true, family2));
 +
 +    createDelFile();
 +
 +    assertEquals("Before compaction: mob rows count", regionNum*(rowNumPerRegion-delRowNum),
 +        countMobRows(hTable));
 +    assertEquals("Before compaction: mob cells count",
 +        regionNum*(cellNumPerRow*rowNumPerRegion-delCellNum), countMobCells(hTable));
 +    assertEquals("Before compaction: family1 mob file count", regionNum*count,
 +        countFiles(true, family1));
 +    assertEquals("Before compaction: family2 mob file count", regionNum*count,
 +        countFiles(true, family2));
 +    assertEquals("Before compaction: family1 del file count", regionNum,
 +        countFiles(false, family1));
 +    assertEquals("Before compaction: family2 del file count", regionNum,
 +        countFiles(false, family2));
 +
 +    // do the mob file compaction
 +    MobFileCompactor compactor = new PartitionedMobFileCompactor(conf, fs, tableName, hcd1, pool);
 +    compactor.compact();
 +
 +    assertEquals("After compaction: mob rows count", regionNum*(rowNumPerRegion-delRowNum),
 +        countMobRows(hTable));
 +    assertEquals("After compaction: mob cells count",
 +        regionNum*(cellNumPerRow*rowNumPerRegion-delCellNum), countMobCells(hTable));
 +    assertEquals("After compaction: family1 mob file count", regionNum*(count/batchSize),
 +        countFiles(true, family1));
 +    assertEquals("After compaction: family2 mob file count", regionNum*count,
 +        countFiles(true, family2));
 +    assertEquals("After compaction: family1 del file count", 0, countFiles(false, family1));
 +    assertEquals("After compaction: family2 del file count", regionNum,
 +        countFiles(false, family2));
 +  }
 +
 +  @Test
 +  public void testCompactionWithHFileLink() throws IOException, InterruptedException {
 +    resetConf();
 +    int count = 4;
 +    // generate mob files
 +    loadData(count, rowNumPerFile);
 +    int rowNumPerRegion = count*rowNumPerFile;
 +
 +    long tid = System.currentTimeMillis();
 +    byte[] snapshotName1 = Bytes.toBytes("snaptb-" + tid);
 +    // take a snapshot
 +    admin.snapshot(snapshotName1, tableName);
 +
 +    createDelFile();
 +
 +    assertEquals("Before compaction: mob rows count", regionNum*(rowNumPerRegion-delRowNum),
 +        countMobRows(hTable));
 +    assertEquals("Before compaction: mob cells count",
 +        regionNum*(cellNumPerRow*rowNumPerRegion-delCellNum), countMobCells(hTable));
 +    assertEquals("Before compaction: family1 mob file count", regionNum*count,
 +        countFiles(true, family1));
 +    assertEquals("Before compaction: family2 mob file count", regionNum*count,
 +        countFiles(true, family2));
 +    assertEquals("Before compaction: family1 del file count", regionNum,
 +        countFiles(false, family1));
 +    assertEquals("Before compaction: family2 del file count", regionNum,
 +        countFiles(false, family2));
 +
 +    // do the mob file compaction
 +    MobFileCompactor compactor = new PartitionedMobFileCompactor(conf, fs, tableName, hcd1, pool);
 +    compactor.compact();
 +
 +    assertEquals("After first compaction: mob rows count", regionNum*(rowNumPerRegion-delRowNum),
 +        countMobRows(hTable));
 +    assertEquals("After first compaction: mob cells count",
 +        regionNum*(cellNumPerRow*rowNumPerRegion-delCellNum), countMobCells(hTable));
 +    assertEquals("After first compaction: family1 mob file count", regionNum,
 +        countFiles(true, family1));
 +    assertEquals("After first compaction: family2 mob file count", regionNum*count,
 +        countFiles(true, family2));
 +    assertEquals("After first compaction: family1 del file count", 0, countFiles(false, family1));
 +    assertEquals("After first compaction: family2 del file count", regionNum,
 +        countFiles(false, family2));
 +    assertEquals("After first compaction: family1 hfilelink count", 0, countHFileLinks(family1));
 +    assertEquals("After first compaction: family2 hfilelink count", 0, countHFileLinks(family2));
 +
 +    admin.disableTable(tableName);
 +    // Restore from snapshot, the hfilelink will exist in mob dir
 +    admin.restoreSnapshot(snapshotName1);
 +    admin.enableTable(tableName);
 +
 +    assertEquals("After restoring snapshot: mob rows count", regionNum*rowNumPerRegion,
 +        countMobRows(hTable));
 +    assertEquals("After restoring snapshot: mob cells count",
 +        regionNum*cellNumPerRow*rowNumPerRegion, countMobCells(hTable));
 +    assertEquals("After restoring snapshot: family1 mob file count", regionNum*count,
 +        countFiles(true, family1));
 +    assertEquals("After restoring snapshot: family2 mob file count", regionNum*count,
 +        countFiles(true, family2));
 +    assertEquals("After restoring snapshot: family1 del file count", 0,
 +        countFiles(false, family1));
 +    assertEquals("After restoring snapshot: family2 del file count", 0,
 +        countFiles(false, family2));
 +    assertEquals("After restoring snapshot: family1 hfilelink count", regionNum*count,
 +        countHFileLinks(family1));
 +    assertEquals("After restoring snapshot: family2 hfilelink count", 0,
 +        countHFileLinks(family2));
 +
 +    compactor.compact();
 +
 +    assertEquals("After second compaction: mob rows count", regionNum*rowNumPerRegion,
 +        countMobRows(hTable));
 +    assertEquals("After second compaction: mob cells count",
 +        regionNum*cellNumPerRow*rowNumPerRegion, countMobCells(hTable));
 +    assertEquals("After second compaction: family1 mob file count", regionNum,
 +        countFiles(true, family1));
 +    assertEquals("After second compaction: family2 mob file count", regionNum*count,
 +        countFiles(true, family2));
 +    assertEquals("After second compaction: family1 del file count", 0, countFiles(false, family1));
 +    assertEquals("After second compaction: family2 del file count", 0, countFiles(false, family2));
 +    assertEquals("After second compaction: family1 hfilelink count", 0, countHFileLinks(family1));
 +    assertEquals("After second compaction: family2 hfilelink count", 0, countHFileLinks(family2));
 +  }
 +
 +  /**
 +   * Gets the number of rows in the given table.
 +   * @param table to get the  scanner
 +   * @return the number of rows
 +   */
 +  private int countMobRows(final HTable table) throws IOException {
 +    Scan scan = new Scan();
 +    // Do not retrieve the mob data when scanning
 +    scan.setAttribute(MobConstants.MOB_SCAN_RAW, Bytes.toBytes(Boolean.TRUE));
 +    ResultScanner results = table.getScanner(scan);
 +    int count = 0;
 +    for (Result res : results) {
 +      count++;
 +    }
 +    results.close();
 +    return count;
 +  }
 +
 +  /**
 +   * Gets the number of cells in the given table.
 +   * @param table to get the  scanner
 +   * @return the number of cells
 +   */
 +  private int countMobCells(final HTable table) throws IOException {
 +    Scan scan = new Scan();
 +    // Do not retrieve the mob data when scanning
 +    scan.setAttribute(MobConstants.MOB_SCAN_RAW, Bytes.toBytes(Boolean.TRUE));
 +    ResultScanner results = table.getScanner(scan);
 +    int count = 0;
 +    for (Result res : results) {
 +      for (Cell cell : res.listCells()) {
 +        count++;
 +      }
 +    }
 +    results.close();
 +    return count;
 +  }
 +
 +  /**
 +   * Gets the number of files in the mob path.
 +   * @param isMobFile gets number of the mob files or del files
 +   * @param familyName the family name
 +   * @return the number of the files
 +   */
 +  private int countFiles(boolean isMobFile, String familyName) throws IOException {
 +    Path mobDirPath = MobUtils.getMobFamilyPath(
 +        MobUtils.getMobRegionPath(conf, tableName), familyName);
 +    int count = 0;
 +    if (fs.exists(mobDirPath)) {
 +      FileStatus[] files = fs.listStatus(mobDirPath);
 +      for (FileStatus file : files) {
 +        if (isMobFile == true) {
 +          if (!StoreFileInfo.isDelFile(file.getPath())) {
 +            count++;
 +          }
 +        } else {
 +          if (StoreFileInfo.isDelFile(file.getPath())) {
 +            count++;
 +          }
 +        }
 +      }
 +    }
 +    return count;
 +  }
 +
 +  /**
 +   * Gets the number of HFileLink in the mob path.
 +   * @param familyName the family name
 +   * @return the number of the HFileLink
 +   */
 +  private int countHFileLinks(String familyName) throws IOException {
 +    Path mobDirPath = MobUtils.getMobFamilyPath(
 +        MobUtils.getMobRegionPath(conf, tableName), familyName);
 +    int count = 0;
 +    if (fs.exists(mobDirPath)) {
 +      FileStatus[] files = fs.listStatus(mobDirPath);
 +      for (FileStatus file : files) {
 +        if (HFileLink.isHFileLink(file.getPath())) {
 +          count++;
 +        }
 +      }
 +    }
 +    return count;
 +  }
 +
 +  /**
 +   * Gets the number of files.
 +   * @param size the size of the file
 +   * @param familyName the family name
 +   * @return the number of files large than the size
 +   */
 +  private int countLargeFiles(int size, String familyName) throws IOException {
 +    Path mobDirPath = MobUtils.getMobFamilyPath(
 +        MobUtils.getMobRegionPath(conf, tableName), familyName);
 +    int count = 0;
 +    if (fs.exists(mobDirPath)) {
 +      FileStatus[] files = fs.listStatus(mobDirPath);
 +      for (FileStatus file : files) {
 +        // ignore the del files in the mob path
 +        if ((!StoreFileInfo.isDelFile(file.getPath()))
 +            && (file.getLen() > size)) {
 +          count++;
 +        }
 +      }
 +    }
 +    return count;
 +  }
 +
 +  /**
 +   * loads some data to the table.
 +   * @param count the mob file number
 +   */
 +  private void loadData(int fileNum, int rowNumPerFile) throws IOException,
 +      InterruptedException {
 +    if (fileNum <= 0) {
 +      throw new IllegalArgumentException();
 +    }
 +    for (byte k0 : KEYS) {
 +      byte[] k = new byte[] { k0 };
 +      for (int i = 0; i < fileNum * rowNumPerFile; i++) {
 +        byte[] key = Bytes.add(k, Bytes.toBytes(i));
 +        byte[] mobVal = makeDummyData(10 * (i + 1));
 +        Put put = new Put(key);
 +        put.setDurability(Durability.SKIP_WAL);
 +        put.add(Bytes.toBytes(family1), Bytes.toBytes(qf1), mobVal);
 +        put.add(Bytes.toBytes(family1), Bytes.toBytes(qf2), mobVal);
 +        put.add(Bytes.toBytes(family2), Bytes.toBytes(qf1), mobVal);
 +        hTable.put(put);
 +        if ((i + 1) % rowNumPerFile == 0) {
 +          hTable.flushCommits();
 +          admin.flush(tableName);
 +        }
 +      }
 +    }
 +  }
 +
 +  /**
 +   * delete the row, family and cell to create the del file
 +   */
 +  private void createDelFile() throws IOException, InterruptedException {
 +    for (byte k0 : KEYS) {
 +      byte[] k = new byte[] { k0 };
 +      // delete a family
 +      byte[] key1 = Bytes.add(k, Bytes.toBytes(0));
 +      Delete delete1 = new Delete(key1);
 +      delete1.deleteFamily(Bytes.toBytes(family1));
 +      hTable.delete(delete1);
 +      // delete one row
 +      byte[] key2 = Bytes.add(k, Bytes.toBytes(2));
 +      Delete delete2 = new Delete(key2);
 +      hTable.delete(delete2);
 +      // delete one cell
 +      byte[] key3 = Bytes.add(k, Bytes.toBytes(4));
 +      Delete delete3 = new Delete(key3);
 +      delete3.deleteColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1));
 +      hTable.delete(delete3);
 +      hTable.flushCommits();
 +      admin.flush(tableName);
 +      List<HRegion> regions = TEST_UTIL.getHBaseCluster().getRegions(
 +          Bytes.toBytes(tableNameAsString));
 +      for (HRegion region : regions) {
 +        region.waitForFlushesAndCompactions();
 +        region.compactStores(true);
 +      }
 +    }
 +  }
 +  /**
 +   * Creates the dummy data with a specific size.
 +   * @param the size of data
 +   * @return the dummy data
 +   */
 +  private byte[] makeDummyData(int size) {
 +    byte[] dummyData = new byte[size];
 +    new Random().nextBytes(dummyData);
 +    return dummyData;
 +  }
 +
 +  /**
 +   * Gets the split keys
 +   */
 +  public static byte[][] getSplitKeys() {
 +    byte[][] splitKeys = new byte[KEYS.length - 1][];
 +    for (int i = 0; i < splitKeys.length; ++i) {
 +      splitKeys[i] = new byte[] { KEYS[i + 1] };
 +    }
 +    return splitKeys;
 +  }
 +
 +  private static ExecutorService createThreadPool(Configuration conf) {
 +    int maxThreads = 10;
 +    long keepAliveTime = 60;
 +    final SynchronousQueue<Runnable> queue = new SynchronousQueue<Runnable>();
 +    ThreadPoolExecutor pool = new ThreadPoolExecutor(1, maxThreads,
 +        keepAliveTime, TimeUnit.SECONDS, queue,
 +        Threads.newDaemonThreadFactory("MobFileCompactionChore"),
 +        new RejectedExecutionHandler() {
 +          @Override
 +          public void rejectedExecution(Runnable r, ThreadPoolExecutor executor) {
 +            try {
 +              // waiting for a thread to pick up instead of throwing exceptions.
 +              queue.put(r);
 +            } catch (InterruptedException e) {
 +              throw new RejectedExecutionException(e);
 +            }
 +          }
 +        });
 +    ((ThreadPoolExecutor) pool).allowCoreThreadTimeOut(true);
 +    return pool;
 +  }
 +
 +  /**
 +   * Resets the configuration.
 +   */
 +  private void resetConf() {
 +    conf.setLong(MobConstants.MOB_FILE_COMPACTION_MERGEABLE_THRESHOLD,
 +      MobConstants.DEFAULT_MOB_FILE_COMPACTION_MERGEABLE_THRESHOLD);
 +    conf.setInt(MobConstants.MOB_FILE_COMPACTION_BATCH_SIZE,
 +      MobConstants.DEFAULT_MOB_FILE_COMPACTION_BATCH_SIZE);
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/filecompactions/TestPartitionedMobFileCompactionRequest.java
----------------------------------------------------------------------
diff --cc hbase-server/src/test/java/org/apache/hadoop/hbase/mob/filecompactions/TestPartitionedMobFileCompactionRequest.java
index ac66d95,0000000..f9159aa
mode 100644,000000..100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/filecompactions/TestPartitionedMobFileCompactionRequest.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/filecompactions/TestPartitionedMobFileCompactionRequest.java
@@@ -1,60 -1,0 +1,60 @@@
 +/**
 + *
 + * 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.mob.filecompactions;
 +
 +import org.apache.hadoop.fs.FileStatus;
 +import org.apache.hadoop.fs.Path;
- import org.apache.hadoop.hbase.SmallTests;
++import org.apache.hadoop.hbase.testclassification.SmallTests;
 +import org.apache.hadoop.hbase.mob.filecompactions.PartitionedMobFileCompactionRequest.CompactionPartition;
 +import org.apache.hadoop.hbase.mob.filecompactions.PartitionedMobFileCompactionRequest.CompactionPartitionId;
 +import org.junit.Assert;
 +import org.junit.Test;
 +import org.junit.experimental.categories.Category;
 +
 +@Category(SmallTests.class)
 +public class TestPartitionedMobFileCompactionRequest {
 +
 +  @Test
 +  public void testCompactedPartitionId() {
 +    String startKey1 = "startKey1";
 +    String startKey2 = "startKey2";
 +    String date1 = "date1";
 +    String date2 = "date2";
 +    CompactionPartitionId partitionId1 = new CompactionPartitionId(startKey1, date1);
 +    CompactionPartitionId partitionId2 = new CompactionPartitionId(startKey2, date2);
 +    CompactionPartitionId partitionId3 = new CompactionPartitionId(startKey1, date2);
 +
 +    Assert.assertTrue(partitionId1.equals(partitionId1));
 +    Assert.assertFalse(partitionId1.equals(partitionId2));
 +    Assert.assertFalse(partitionId1.equals(partitionId3));
 +    Assert.assertFalse(partitionId2.equals(partitionId3));
 +
 +    Assert.assertEquals(startKey1, partitionId1.getStartKey());
 +    Assert.assertEquals(date1, partitionId1.getDate());
 +  }
 +
 +  @Test
 +  public void testCompactedPartition() {
 +    CompactionPartitionId partitionId = new CompactionPartitionId("startKey1", "date1");
 +    CompactionPartition partition = new CompactionPartition(partitionId);
 +    FileStatus file = new FileStatus(1, false, 1, 1024, 1, new Path("/test"));
 +    partition.addFile(file);
 +    Assert.assertEquals(file, partition.listFiles().get(0));
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/filecompactions/TestPartitionedMobFileCompactor.java
----------------------------------------------------------------------
diff --cc hbase-server/src/test/java/org/apache/hadoop/hbase/mob/filecompactions/TestPartitionedMobFileCompactor.java
index 1d64c0c,0000000..12c88b2
mode 100644,000000..100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/filecompactions/TestPartitionedMobFileCompactor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/filecompactions/TestPartitionedMobFileCompactor.java
@@@ -1,423 -1,0 +1,423 @@@
 +/**
 + *
 + * 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.mob.filecompactions;
 +
 +import java.io.IOException;
 +import java.util.ArrayList;
 +import java.util.Collection;
 +import java.util.Collections;
 +import java.util.Date;
 +import java.util.List;
 +import java.util.Random;
 +import java.util.UUID;
 +import java.util.concurrent.ExecutorService;
 +import java.util.concurrent.RejectedExecutionException;
 +import java.util.concurrent.RejectedExecutionHandler;
 +import java.util.concurrent.SynchronousQueue;
 +import java.util.concurrent.ThreadPoolExecutor;
 +import java.util.concurrent.TimeUnit;
 +
 +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.Cell;
 +import org.apache.hadoop.hbase.HBaseTestingUtility;
 +import org.apache.hadoop.hbase.HColumnDescriptor;
 +import org.apache.hadoop.hbase.HConstants;
 +import org.apache.hadoop.hbase.KeyValue;
 +import org.apache.hadoop.hbase.KeyValue.Type;
- import org.apache.hadoop.hbase.LargeTests;
++import org.apache.hadoop.hbase.testclassification.LargeTests;
 +import org.apache.hadoop.hbase.TableName;
 +import org.apache.hadoop.hbase.client.Scan;
 +import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 +import org.apache.hadoop.hbase.io.hfile.HFileContext;
 +import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
 +import org.apache.hadoop.hbase.mob.MobConstants;
 +import org.apache.hadoop.hbase.mob.MobFileName;
 +import org.apache.hadoop.hbase.mob.MobUtils;
 +import org.apache.hadoop.hbase.mob.filecompactions.MobFileCompactionRequest.CompactionType;
 +import org.apache.hadoop.hbase.mob.filecompactions.PartitionedMobFileCompactionRequest.CompactionPartition;
 +import org.apache.hadoop.hbase.regionserver.BloomType;
 +import org.apache.hadoop.hbase.regionserver.HStore;
 +import org.apache.hadoop.hbase.regionserver.ScanInfo;
 +import org.apache.hadoop.hbase.regionserver.ScanType;
 +import org.apache.hadoop.hbase.regionserver.StoreFile;
 +import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
 +import org.apache.hadoop.hbase.regionserver.StoreScanner;
 +import org.apache.hadoop.hbase.util.Bytes;
 +import org.apache.hadoop.hbase.util.FSUtils;
 +import org.apache.hadoop.hbase.util.Threads;
 +import org.junit.AfterClass;
 +import org.junit.Assert;
 +import org.junit.BeforeClass;
 +import org.junit.Test;
 +import org.junit.experimental.categories.Category;
 +
 +@Category(LargeTests.class)
 +public class TestPartitionedMobFileCompactor {
 +  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 +  private final static String family = "family";
 +  private final static String qf = "qf";
 +  private HColumnDescriptor hcd = new HColumnDescriptor(family);
 +  private Configuration conf = TEST_UTIL.getConfiguration();
 +  private CacheConfig cacheConf = new CacheConfig(conf);
 +  private FileSystem fs;
 +  private List<FileStatus> mobFiles = new ArrayList<>();
 +  private List<FileStatus> delFiles = new ArrayList<>();
 +  private List<FileStatus> allFiles = new ArrayList<>();
 +  private Path basePath;
 +  private String mobSuffix;
 +  private String delSuffix;
 +  private static ExecutorService pool;
 +
 +  @BeforeClass
 +  public static void setUpBeforeClass() throws Exception {
 +    TEST_UTIL.getConfiguration().setInt("hbase.master.info.port", 0);
 +    TEST_UTIL.getConfiguration().setBoolean("hbase.regionserver.info.port.auto", true);
 +    TEST_UTIL.getConfiguration().setInt("hfile.format.version", 3);
 +    TEST_UTIL.startMiniCluster(1);
 +    pool = createThreadPool(TEST_UTIL.getConfiguration());
 +  }
 +
 +  @AfterClass
 +  public static void tearDownAfterClass() throws Exception {
 +    pool.shutdown();
 +    TEST_UTIL.shutdownMiniCluster();
 +  }
 +
 +  private void init(String tableName) throws Exception {
 +    fs = FileSystem.get(conf);
 +    Path testDir = FSUtils.getRootDir(conf);
 +    Path mobTestDir = new Path(testDir, MobConstants.MOB_DIR_NAME);
 +    basePath = new Path(new Path(mobTestDir, tableName), family);
 +    mobSuffix = UUID.randomUUID().toString().replaceAll("-", "");
 +    delSuffix = UUID.randomUUID().toString().replaceAll("-", "") + "_del";
 +  }
 +
 +  @Test
 +  public void testCompactionSelectWithAllFiles() throws Exception {
 +    resetConf();
 +    String tableName = "testCompactionSelectWithAllFiles";
 +    init(tableName);
 +    int count = 10;
 +    // create 10 mob files.
 +    createStoreFiles(basePath, family, qf, count, Type.Put);
 +    // create 10 del files
 +    createStoreFiles(basePath, family, qf, count, Type.Delete);
 +    listFiles();
 +    long mergeSize = MobConstants.DEFAULT_MOB_FILE_COMPACTION_MERGEABLE_THRESHOLD;
 +    List<String> expectedStartKeys = new ArrayList<>();
 +    for(FileStatus file : mobFiles) {
 +      if(file.getLen() < mergeSize) {
 +        String fileName = file.getPath().getName();
 +        String startKey = fileName.substring(0, 32);
 +        expectedStartKeys.add(startKey);
 +      }
 +    }
 +    testSelectFiles(tableName, CompactionType.ALL_FILES, expectedStartKeys);
 +  }
 +
 +  @Test
 +  public void testCompactionSelectWithPartFiles() throws Exception {
 +    resetConf();
 +    String tableName = "testCompactionSelectWithPartFiles";
 +    init(tableName);
 +    int count = 10;
 +    // create 10 mob files.
 +    createStoreFiles(basePath, family, qf, count, Type.Put);
 +    // create 10 del files
 +    createStoreFiles(basePath, family, qf, count, Type.Delete);
 +    listFiles();
 +    long mergeSize = 4000;
 +    List<String> expectedStartKeys = new ArrayList<>();
 +    for(FileStatus file : mobFiles) {
 +      if(file.getLen() < 4000) {
 +        String fileName = file.getPath().getName();
 +        String startKey = fileName.substring(0, 32);
 +        expectedStartKeys.add(startKey);
 +      }
 +    }
 +    // set the mob file compaction mergeable threshold
 +    conf.setLong(MobConstants.MOB_FILE_COMPACTION_MERGEABLE_THRESHOLD, mergeSize);
 +    testSelectFiles(tableName, CompactionType.PART_FILES, expectedStartKeys);
 +  }
 +
 +  @Test
 +  public void testCompactDelFilesWithDefaultBatchSize() throws Exception {
 +    resetConf();
 +    String tableName = "testCompactDelFilesWithDefaultBatchSize";
 +    init(tableName);
 +    // create 20 mob files.
 +    createStoreFiles(basePath, family, qf, 20, Type.Put);
 +    // create 13 del files
 +    createStoreFiles(basePath, family, qf, 13, Type.Delete);
 +    listFiles();
 +    testCompactDelFiles(tableName, 1, 13);
 +  }
 +
 +  @Test
 +  public void testCompactDelFilesWithSmallBatchSize() throws Exception {
 +    resetConf();
 +    String tableName = "testCompactDelFilesWithSmallBatchSize";
 +    init(tableName);
 +    // create 20 mob files.
 +    createStoreFiles(basePath, family, qf, 20, Type.Put);
 +    // create 13 del files
 +    createStoreFiles(basePath, family, qf, 13, Type.Delete);
 +    listFiles();
 +
 +    // set the mob file compaction batch size
 +    conf.setInt(MobConstants.MOB_FILE_COMPACTION_BATCH_SIZE, 4);
 +    testCompactDelFiles(tableName, 1, 13);
 +  }
 +
 +  @Test
 +  public void testCompactDelFilesChangeMaxDelFileCount() throws Exception {
 +    resetConf();
 +    String tableName = "testCompactDelFilesWithSmallBatchSize";
 +    init(tableName);
 +    // create 20 mob files.
 +    createStoreFiles(basePath, family, qf, 20, Type.Put);
 +    // create 13 del files
 +    createStoreFiles(basePath, family, qf, 13, Type.Delete);
 +    listFiles();
 +
 +    // set the max del file count
 +    conf.setInt(MobConstants.MOB_DELFILE_MAX_COUNT, 5);
 +    // set the mob file compaction batch size
 +    conf.setInt(MobConstants.MOB_FILE_COMPACTION_BATCH_SIZE, 2);
 +    testCompactDelFiles(tableName, 4, 13);
 +  }
 +
 +  /**
 +   * Tests the selectFiles
 +   * @param tableName the table name
 +   * @param type the expected compaction type
 +   * @param expected the expected start keys
 +   */
 +  private void testSelectFiles(String tableName, final CompactionType type,
 +      final List<String> expected) throws IOException {
 +    PartitionedMobFileCompactor compactor = new PartitionedMobFileCompactor(conf, fs,
 +      TableName.valueOf(tableName), hcd, pool) {
 +      @Override
 +      public List<Path> compact(List<FileStatus> files) throws IOException {
 +        if (files == null || files.isEmpty()) {
 +          return null;
 +        }
 +        PartitionedMobFileCompactionRequest request = select(files);
 +        // assert the compaction type is ALL_FILES
 +        Assert.assertEquals(type, request.type);
 +        // assert get the right partitions
 +        compareCompactedPartitions(expected, request.compactionPartitions);
 +        // assert get the right del files
 +        compareDelFiles(request.delFiles);
 +        return null;
 +      }
 +    };
 +    compactor.compact(allFiles);
 +  }
 +
 +  /**
 +   * Tests the compacteDelFile
 +   * @param tableName the table name
 +   * @param expectedFileCount the expected file count
 +   * @param expectedCellCount the expected cell count
 +   */
 +  private void testCompactDelFiles(String tableName, final int expectedFileCount,
 +      final int expectedCellCount) throws IOException {
 +    PartitionedMobFileCompactor compactor = new PartitionedMobFileCompactor(conf, fs,
 +      TableName.valueOf(tableName), hcd, pool) {
 +      @Override
 +      protected List<Path> performCompaction(PartitionedMobFileCompactionRequest request)
 +          throws IOException {
 +        List<Path> delFilePaths = new ArrayList<Path>();
 +        for (FileStatus delFile : request.delFiles) {
 +          delFilePaths.add(delFile.getPath());
 +        }
 +        List<Path> newDelPaths = compactDelFiles(request, delFilePaths);
 +        // assert the del files are merged.
 +        Assert.assertEquals(expectedFileCount, newDelPaths.size());
 +        Assert.assertEquals(expectedCellCount, countDelCellsInDelFiles(newDelPaths));
 +        return null;
 +      }
 +    };
 +
 +    compactor.compact(allFiles);
 +  }
 +
 +  /**
 +   * Lists the files in the path
 +   */
 +  private void listFiles() throws IOException {
 +    for (FileStatus file : fs.listStatus(basePath)) {
 +      allFiles.add(file);
 +      if (file.getPath().getName().endsWith("_del")) {
 +        delFiles.add(file);
 +      } else {
 +        mobFiles.add(file);
 +      }
 +    }
 +  }
 +
 +  /**
 +   * Compares the compacted partitions.
 +   * @param partitions the collection of CompactedPartitions
 +   */
 +  private void compareCompactedPartitions(List<String> expected,
 +      Collection<CompactionPartition> partitions) {
 +    List<String> actualKeys = new ArrayList<>();
 +    for (CompactionPartition partition : partitions) {
 +      actualKeys.add(partition.getPartitionId().getStartKey());
 +    }
 +    Collections.sort(expected);
 +    Collections.sort(actualKeys);
 +    Assert.assertEquals(expected.size(), actualKeys.size());
 +    for (int i = 0; i < expected.size(); i++) {
 +      Assert.assertEquals(expected.get(i), actualKeys.get(i));
 +    }
 +  }
 +
 +  /**
 +   * Compares the del files.
 +   * @param allDelFiles all the del files
 +   */
 +  private void compareDelFiles(Collection<FileStatus> allDelFiles) {
 +    int i = 0;
 +    for (FileStatus file : allDelFiles) {
 +      Assert.assertEquals(delFiles.get(i), file);
 +      i++;
 +    }
 +  }
 +
 +  /**
 +   * Creates store files.
 +   * @param basePath the path to create file
 +   * @family the family name
 +   * @qualifier the column qualifier
 +   * @count the store file number
 +   * @type the key type
 +   */
 +  private void createStoreFiles(Path basePath, String family, String qualifier, int count,
 +      Type type) throws IOException {
 +    HFileContext meta = new HFileContextBuilder().withBlockSize(8 * 1024).build();
 +    String startKey = "row_";
 +    MobFileName mobFileName = null;
 +    for (int i = 0; i < count; i++) {
 +      byte[] startRow = Bytes.toBytes(startKey + i) ;
 +      if(type.equals(Type.Delete)) {
 +        mobFileName = MobFileName.create(startRow, MobUtils.formatDate(
 +            new Date()), delSuffix);
 +      }
 +      if(type.equals(Type.Put)){
 +        mobFileName = MobFileName.create(Bytes.toBytes(startKey + i), MobUtils.formatDate(
 +            new Date()), mobSuffix);
 +      }
 +      StoreFile.Writer mobFileWriter = new StoreFile.WriterBuilder(conf, cacheConf, fs)
 +      .withFileContext(meta).withFilePath(new Path(basePath, mobFileName.getFileName())).build();
 +      writeStoreFile(mobFileWriter, startRow, Bytes.toBytes(family), Bytes.toBytes(qualifier),
 +          type, (i+1)*1000);
 +    }
 +  }
 +
 +  /**
 +   * Writes data to store file.
 +   * @param writer the store file writer
 +   * @param row the row key
 +   * @param family the family name
 +   * @param qualifier the column qualifier
 +   * @param type the key type
 +   * @param size the size of value
 +   */
 +  private static void writeStoreFile(final StoreFile.Writer writer, byte[]row, byte[] family,
 +      byte[] qualifier, Type type, int size) throws IOException {
 +    long now = System.currentTimeMillis();
 +    try {
 +      byte[] dummyData = new byte[size];
 +      new Random().nextBytes(dummyData);
 +      writer.append(new KeyValue(row, family, qualifier, now, type, dummyData));
 +    } finally {
 +      writer.close();
 +    }
 +  }
 +
 +  /**
 +   * Gets the number of del cell in the del files
 +   * @param paths the del file paths
 +   * @return the cell size
 +   */
 +  private int countDelCellsInDelFiles(List<Path> paths) throws IOException {
 +    List<StoreFile> sfs = new ArrayList<StoreFile>();
 +    int size = 0;
 +    for(Path path : paths) {
 +      StoreFile sf = new StoreFile(fs, path, conf, cacheConf, BloomType.NONE);
 +      sfs.add(sf);
 +    }
 +    List scanners = StoreFileScanner.getScannersForStoreFiles(sfs, false, true,
 +        false, null, HConstants.LATEST_TIMESTAMP);
 +    Scan scan = new Scan();
 +    scan.setMaxVersions(hcd.getMaxVersions());
 +    long timeToPurgeDeletes = Math.max(conf.getLong("hbase.hstore.time.to.purge.deletes", 0), 0);
 +    long ttl = HStore.determineTTLFromFamily(hcd);
 +    ScanInfo scanInfo = new ScanInfo(hcd, ttl, timeToPurgeDeletes, KeyValue.COMPARATOR);
 +    StoreScanner scanner = new StoreScanner(scan, scanInfo, ScanType.COMPACT_RETAIN_DELETES, null,
 +        scanners, 0L, HConstants.LATEST_TIMESTAMP);
 +    List<Cell> results = new ArrayList<>();
 +    boolean hasMore = true;
 +    while (hasMore) {
 +      hasMore = scanner.next(results);
 +      size += results.size();
 +      results.clear();
 +    }
 +    scanner.close();
 +    return size;
 +  }
 +
 +  private static ExecutorService createThreadPool(Configuration conf) {
 +    int maxThreads = 10;
 +    long keepAliveTime = 60;
 +    final SynchronousQueue<Runnable> queue = new SynchronousQueue<Runnable>();
 +    ThreadPoolExecutor pool = new ThreadPoolExecutor(1, maxThreads, keepAliveTime,
 +      TimeUnit.SECONDS, queue, Threads.newDaemonThreadFactory("MobFileCompactionChore"),
 +      new RejectedExecutionHandler() {
 +        @Override
 +        public void rejectedExecution(Runnable r, ThreadPoolExecutor executor) {
 +          try {
 +            // waiting for a thread to pick up instead of throwing exceptions.
 +            queue.put(r);
 +          } catch (InterruptedException e) {
 +            throw new RejectedExecutionException(e);
 +          }
 +        }
 +      });
 +    ((ThreadPoolExecutor) pool).allowCoreThreadTimeOut(true);
 +    return pool;
 +  }
 +
 +  /**
 +   * Resets the configuration.
 +   */
 +  private void resetConf() {
 +    conf.setLong(MobConstants.MOB_FILE_COMPACTION_MERGEABLE_THRESHOLD,
 +      MobConstants.DEFAULT_MOB_FILE_COMPACTION_MERGEABLE_THRESHOLD);
 +    conf.setInt(MobConstants.MOB_DELFILE_MAX_COUNT, MobConstants.DEFAULT_MOB_DELFILE_MAX_COUNT);
 +    conf.setInt(MobConstants.MOB_FILE_COMPACTION_BATCH_SIZE,
 +      MobConstants.DEFAULT_MOB_FILE_COMPACTION_BATCH_SIZE);
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/mapreduce/TestMobSweepJob.java
----------------------------------------------------------------------
diff --cc hbase-server/src/test/java/org/apache/hadoop/hbase/mob/mapreduce/TestMobSweepJob.java
index e0b9a83,0000000..49345e4
mode 100644,000000..100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/mapreduce/TestMobSweepJob.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/mapreduce/TestMobSweepJob.java
@@@ -1,168 -1,0 +1,168 @@@
 +/**
 + * 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.mob.mapreduce;
 +
 +import static org.junit.Assert.assertEquals;
 +
 +import java.io.IOException;
 +import java.util.List;
 +
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.CommonConfigurationKeys;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.hbase.HBaseTestingUtility;
- import org.apache.hadoop.hbase.MediumTests;
 +import org.apache.hadoop.hbase.mob.MobConstants;
 +import org.apache.hadoop.hbase.mob.MobUtils;
++import org.apache.hadoop.hbase.testclassification.MediumTests;
 +import org.apache.hadoop.io.IOUtils;
 +import org.apache.hadoop.io.SequenceFile;
 +import org.apache.hadoop.io.serializer.JavaSerialization;
 +import org.apache.hadoop.io.serializer.WritableSerialization;
 +import org.junit.AfterClass;
 +import org.junit.BeforeClass;
 +import org.junit.Test;
 +import org.junit.experimental.categories.Category;
 +
 +@Category(MediumTests.class)
 +public class TestMobSweepJob {
 +
 +  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 +
 +  @BeforeClass
 +  public static void setUpBeforeClass() throws Exception {
 +    TEST_UTIL.getConfiguration().setInt("hbase.master.info.port", 0);
 +    TEST_UTIL.getConfiguration().setBoolean("hbase.regionserver.info.port.auto", true);
 +    TEST_UTIL.getConfiguration().set(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY,
 +        JavaSerialization.class.getName() + "," + WritableSerialization.class.getName());
 +    TEST_UTIL.startMiniCluster();
 +  }
 +
 +  @AfterClass
 +  public static void tearDownAfterClass() throws Exception {
 +    TEST_UTIL.shutdownMiniCluster();
 +  }
 +
 +  private void writeFileNames(FileSystem fs, Configuration conf, Path path,
 +      String[] filesNames) throws IOException {
 +    // write the names to a sequence file
 +    SequenceFile.Writer writer = SequenceFile.createWriter(fs, conf, path,
 +        String.class, String.class);
 +    try {
 +      for (String fileName : filesNames) {
 +        writer.append(fileName, MobConstants.EMPTY_STRING);
 +      }
 +    } finally {
 +      IOUtils.closeStream(writer);
 +    }
 +  }
 +
 +  @Test
 +  public void testSweeperJobWithOutUnusedFile() throws Exception {
 +    FileSystem fs = TEST_UTIL.getTestFileSystem();
 +    Configuration configuration = new Configuration(
 +        TEST_UTIL.getConfiguration());
 +    Path vistiedFileNamesPath = new Path(MobUtils.getMobHome(configuration),
 +        "/hbase/mobcompaction/SweepJob/working/names/0/visited");
 +    Path allFileNamesPath = new Path(MobUtils.getMobHome(configuration),
 +        "/hbase/mobcompaction/SweepJob/working/names/0/all");
 +    configuration.set(SweepJob.WORKING_VISITED_DIR_KEY,
 +        vistiedFileNamesPath.toString());
 +    configuration.set(SweepJob.WORKING_ALLNAMES_FILE_KEY,
 +        allFileNamesPath.toString());
 +
 +    writeFileNames(fs, configuration, allFileNamesPath, new String[] { "1",
 +        "2", "3", "4", "5", "6"});
 +
 +    Path r0 = new Path(vistiedFileNamesPath, "r0");
 +    writeFileNames(fs, configuration, r0, new String[] { "1",
 +        "2", "3"});
 +    Path r1 = new Path(vistiedFileNamesPath, "r1");
 +    writeFileNames(fs, configuration, r1, new String[] { "1", "4", "5"});
 +    Path r2 = new Path(vistiedFileNamesPath, "r2");
 +    writeFileNames(fs, configuration, r2, new String[] { "2", "3", "6"});
 +
 +    SweepJob sweepJob = new SweepJob(configuration, fs);
 +    List<String> toBeArchived = sweepJob.getUnusedFiles(configuration);
 +
 +    assertEquals(0, toBeArchived.size());
 +  }
 +
 +  @Test
 +  public void testSweeperJobWithUnusedFile() throws Exception {
 +    FileSystem fs = TEST_UTIL.getTestFileSystem();
 +    Configuration configuration = new Configuration(
 +        TEST_UTIL.getConfiguration());
 +    Path vistiedFileNamesPath = new Path(MobUtils.getMobHome(configuration),
 +        "/hbase/mobcompaction/SweepJob/working/names/1/visited");
 +    Path allFileNamesPath = new Path(MobUtils.getMobHome(configuration),
 +        "/hbase/mobcompaction/SweepJob/working/names/1/all");
 +    configuration.set(SweepJob.WORKING_VISITED_DIR_KEY,
 +        vistiedFileNamesPath.toString());
 +    configuration.set(SweepJob.WORKING_ALLNAMES_FILE_KEY,
 +        allFileNamesPath.toString());
 +
 +    writeFileNames(fs, configuration, allFileNamesPath, new String[] { "1",
 +        "2", "3", "4", "5", "6"});
 +
 +    Path r0 = new Path(vistiedFileNamesPath, "r0");
 +    writeFileNames(fs, configuration, r0, new String[] { "1",
 +        "2", "3"});
 +    Path r1 = new Path(vistiedFileNamesPath, "r1");
 +    writeFileNames(fs, configuration, r1, new String[] { "1", "5"});
 +    Path r2 = new Path(vistiedFileNamesPath, "r2");
 +    writeFileNames(fs, configuration, r2, new String[] { "2", "3"});
 +
 +    SweepJob sweepJob = new SweepJob(configuration, fs);
 +    List<String> toBeArchived = sweepJob.getUnusedFiles(configuration);
 +
 +    assertEquals(2, toBeArchived.size());
 +    assertEquals(new String[] { "4", "6" }, toBeArchived.toArray(new String[0]));
 +  }
 +
 +  @Test
 +  public void testSweeperJobWithRedundantFile() throws Exception {
 +    FileSystem fs = TEST_UTIL.getTestFileSystem();
 +    Configuration configuration = new Configuration(
 +        TEST_UTIL.getConfiguration());
 +    Path vistiedFileNamesPath = new Path(MobUtils.getMobHome(configuration),
 +        "/hbase/mobcompaction/SweepJob/working/names/2/visited");
 +    Path allFileNamesPath = new Path(MobUtils.getMobHome(configuration),
 +        "/hbase/mobcompaction/SweepJob/working/names/2/all");
 +    configuration.set(SweepJob.WORKING_VISITED_DIR_KEY,
 +        vistiedFileNamesPath.toString());
 +    configuration.set(SweepJob.WORKING_ALLNAMES_FILE_KEY,
 +        allFileNamesPath.toString());
 +
 +    writeFileNames(fs, configuration, allFileNamesPath, new String[] { "1",
 +        "2", "3", "4", "5", "6"});
 +
 +    Path r0 = new Path(vistiedFileNamesPath, "r0");
 +    writeFileNames(fs, configuration, r0, new String[] { "1",
 +        "2", "3"});
 +    Path r1 = new Path(vistiedFileNamesPath, "r1");
 +    writeFileNames(fs, configuration, r1, new String[] { "1", "5", "6", "7"});
 +    Path r2 = new Path(vistiedFileNamesPath, "r2");
 +    writeFileNames(fs, configuration, r2, new String[] { "2", "3", "4"});
 +
 +    SweepJob sweepJob = new SweepJob(configuration, fs);
 +    List<String> toBeArchived = sweepJob.getUnusedFiles(configuration);
 +
 +    assertEquals(0, toBeArchived.size());
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/mapreduce/TestMobSweepMapper.java
----------------------------------------------------------------------
diff --cc hbase-server/src/test/java/org/apache/hadoop/hbase/mob/mapreduce/TestMobSweepMapper.java
index 2aa3a4a,0000000..9e95a39
mode 100644,000000..100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/mapreduce/TestMobSweepMapper.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/mapreduce/TestMobSweepMapper.java
@@@ -1,120 -1,0 +1,120 @@@
 +/**
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.hadoop.hbase.mob.mapreduce;
 +
 +import static org.junit.Assert.assertEquals;
 +import static org.mockito.Matchers.any;
 +import static org.mockito.Mockito.doAnswer;
 +import static org.mockito.Mockito.mock;
 +import static org.mockito.Mockito.when;
 +
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.hbase.HBaseTestingUtility;
 +import org.apache.hadoop.hbase.KeyValue;
 +import org.apache.hadoop.hbase.ServerName;
- import org.apache.hadoop.hbase.SmallTests;
 +import org.apache.hadoop.hbase.TableName;
++import org.apache.hadoop.hbase.testclassification.SmallTests;
 +import org.apache.hadoop.hbase.client.Result;
 +import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 +import org.apache.hadoop.hbase.master.TableLockManager;
 +import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
 +import org.apache.hadoop.hbase.mob.MobUtils;
 +import org.apache.hadoop.hbase.mob.mapreduce.SweepJob.DummyMobAbortable;
 +import org.apache.hadoop.hbase.util.Bytes;
 +import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 +import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 +import org.apache.hadoop.io.Text;
 +import org.apache.hadoop.mapreduce.Mapper;
 +import org.junit.AfterClass;
 +import org.junit.BeforeClass;
 +import org.junit.Test;
 +import org.junit.experimental.categories.Category;
 +import org.mockito.invocation.InvocationOnMock;
 +import org.mockito.stubbing.Answer;
 +
 +@Category(SmallTests.class)
 +public class TestMobSweepMapper {
 +
 +  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 +
 +  @BeforeClass
 +  public static void setUpBeforeClass() throws Exception {
 +    TEST_UTIL.getConfiguration().setInt("hbase.master.info.port", 0);
 +    TEST_UTIL.getConfiguration().setBoolean("hbase.regionserver.info.port.auto", true);
 +    TEST_UTIL.getConfiguration().setInt("hfile.format.version", 3);
 +    TEST_UTIL.startMiniCluster(1);
 +  }
 +
 +  @AfterClass
 +  public static void tearDownAfterClass() throws Exception {
 +    TEST_UTIL.shutdownMiniCluster();
 +  }
 +
 +  @Test
 +  public void TestMap() throws Exception {
 +    String prefix = "0000";
 +    final String fileName = "19691231f2cd014ea28f42788214560a21a44cef";
 +    final String mobFilePath = prefix + fileName;
 +
 +    ImmutableBytesWritable r = new ImmutableBytesWritable(Bytes.toBytes("r"));
 +    final KeyValue[] kvList = new KeyValue[1];
 +    kvList[0] = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("family"),
 +            Bytes.toBytes("column"), Bytes.toBytes(mobFilePath));
 +
 +    Result columns = mock(Result.class);
-     when(columns.raw()).thenReturn(kvList);
++    when(columns.rawCells()).thenReturn(kvList);
 +
 +    Configuration configuration = new Configuration(TEST_UTIL.getConfiguration());
 +    ZooKeeperWatcher zkw = new ZooKeeperWatcher(configuration, "1", new DummyMobAbortable());
 +    TableName tn = TableName.valueOf("testSweepMapper");
 +    TableName lockName = MobUtils.getTableLockName(tn);
 +    String znode = ZKUtil.joinZNode(zkw.tableLockZNode, lockName.getNameAsString());
 +    configuration.set(SweepJob.SWEEP_JOB_ID, "1");
 +    configuration.set(SweepJob.SWEEP_JOB_TABLE_NODE, znode);
 +    ServerName serverName = SweepJob.getCurrentServerName(configuration);
 +    configuration.set(SweepJob.SWEEP_JOB_SERVERNAME, serverName.toString());
 +
 +    TableLockManager tableLockManager = TableLockManager.createTableLockManager(configuration, zkw,
 +        serverName);
 +    TableLock lock = tableLockManager.writeLock(lockName, "Run sweep tool");
 +    lock.acquire();
 +    try {
 +      Mapper<ImmutableBytesWritable, Result, Text, KeyValue>.Context ctx =
 +        mock(Mapper.Context.class);
 +      when(ctx.getConfiguration()).thenReturn(configuration);
 +      SweepMapper map = new SweepMapper();
 +      doAnswer(new Answer<Void>() {
 +
 +        @Override
 +        public Void answer(InvocationOnMock invocation) throws Throwable {
 +          Text text = (Text) invocation.getArguments()[0];
 +          KeyValue kv = (KeyValue) invocation.getArguments()[1];
 +
 +          assertEquals(Bytes.toString(text.getBytes(), 0, text.getLength()), fileName);
 +          assertEquals(0, Bytes.compareTo(kv.getKey(), kvList[0].getKey()));
 +
 +          return null;
 +        }
 +      }).when(ctx).write(any(Text.class), any(KeyValue.class));
 +
 +      map.map(r, columns, ctx);
 +    } finally {
 +      lock.release();
 +    }
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/mapreduce/TestMobSweepReducer.java
----------------------------------------------------------------------
diff --cc hbase-server/src/test/java/org/apache/hadoop/hbase/mob/mapreduce/TestMobSweepReducer.java
index 1a69d06,0000000..308b50e
mode 100644,000000..100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/mapreduce/TestMobSweepReducer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/mapreduce/TestMobSweepReducer.java
@@@ -1,220 -1,0 +1,220 @@@
 +/**
 + * 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.mob.mapreduce;
 +
 +import static org.junit.Assert.assertEquals;
 +import static org.mockito.Mockito.mock;
 +import static org.mockito.Mockito.when;
 +
 +import java.util.ArrayList;
 +import java.util.List;
 +import java.util.Set;
 +import java.util.TreeSet;
 +
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.CommonConfigurationKeys;
 +import org.apache.hadoop.fs.FileStatus;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.hbase.HBaseTestingUtility;
 +import org.apache.hadoop.hbase.HColumnDescriptor;
 +import org.apache.hadoop.hbase.HTableDescriptor;
 +import org.apache.hadoop.hbase.KeyValue;
- import org.apache.hadoop.hbase.MediumTests;
 +import org.apache.hadoop.hbase.ServerName;
 +import org.apache.hadoop.hbase.TableName;
 +import org.apache.hadoop.hbase.client.Admin;
 +import org.apache.hadoop.hbase.client.HTable;
 +import org.apache.hadoop.hbase.client.Put;
 +import org.apache.hadoop.hbase.mapreduce.TableInputFormat;
 +import org.apache.hadoop.hbase.master.TableLockManager;
 +import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
 +import org.apache.hadoop.hbase.mob.MobConstants;
 +import org.apache.hadoop.hbase.mob.MobUtils;
 +import org.apache.hadoop.hbase.mob.mapreduce.SweepJob.DummyMobAbortable;
 +import org.apache.hadoop.hbase.mob.mapreduce.SweepJob.SweepCounter;
++import org.apache.hadoop.hbase.testclassification.MediumTests;
 +import org.apache.hadoop.hbase.util.Bytes;
 +import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 +import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 +import org.apache.hadoop.io.SequenceFile;
 +import org.apache.hadoop.io.Text;
 +import org.apache.hadoop.io.Writable;
 +import org.apache.hadoop.io.serializer.JavaSerialization;
 +import org.apache.hadoop.mapreduce.Counter;
 +import org.apache.hadoop.mapreduce.Reducer;
 +import org.apache.hadoop.mapreduce.counters.GenericCounter;
 +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;
 +import org.mockito.Matchers;
 +
 +@Category(MediumTests.class)
 +public class TestMobSweepReducer {
 +
 +  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 +  private final static String tableName = "testSweepReducer";
 +  private final static String row = "row";
 +  private final static String family = "family";
 +  private final static String qf = "qf";
 +  private static HTable table;
 +  private static Admin admin;
 +
 +  @BeforeClass
 +  public static void setUpBeforeClass() throws Exception {
 +    TEST_UTIL.getConfiguration().setInt("hbase.master.info.port", 0);
 +    TEST_UTIL.getConfiguration().setBoolean("hbase.regionserver.info.port.auto", true);
 +
 +    TEST_UTIL.getConfiguration().setInt("hfile.format.version", 3);
 +
 +    TEST_UTIL.startMiniCluster(1);
 +  }
 +
 +  @AfterClass
 +  public static void tearDownAfterClass() throws Exception {
 +    TEST_UTIL.shutdownMiniCluster();
 +  }
 +
 +  @SuppressWarnings("deprecation")
 +  @Before
 +  public void setUp() throws Exception {
 +    HTableDescriptor desc = new HTableDescriptor(tableName);
 +    HColumnDescriptor hcd = new HColumnDescriptor(family);
 +    hcd.setMobEnabled(true);
 +    hcd.setMobThreshold(3L);
 +    hcd.setMaxVersions(4);
 +    desc.addFamily(hcd);
 +
 +    admin = TEST_UTIL.getHBaseAdmin();
 +    admin.createTable(desc);
 +    table = new HTable(TEST_UTIL.getConfiguration(), tableName);
 +  }
 +
 +  @After
 +  public void tearDown() throws Exception {
 +    admin.disableTable(TableName.valueOf(tableName));
 +    admin.deleteTable(TableName.valueOf(tableName));
 +    admin.close();
 +  }
 +
 +  private List<String> getKeyFromSequenceFile(FileSystem fs, Path path,
 +                                              Configuration conf) throws Exception {
 +    List<String> list = new ArrayList<String>();
 +    SequenceFile.Reader reader = new SequenceFile.Reader(conf, SequenceFile.Reader.file(path));
 +
 +    String next = (String) reader.next((String) null);
 +    while (next != null) {
 +      list.add(next);
 +      next = (String) reader.next((String) null);
 +    }
 +    reader.close();
 +    return list;
 +  }
 +
 +  @Test
 +  public void testRun() throws Exception {
 +
 +    TableName tn = TableName.valueOf(tableName);
 +    byte[] mobValueBytes = new byte[100];
 +
 +    //get the path where mob files lie in
 +    Path mobFamilyPath = MobUtils.getMobFamilyPath(TEST_UTIL.getConfiguration(), tn, family);
 +
 +    Put put = new Put(Bytes.toBytes(row));
 +    put.add(Bytes.toBytes(family), Bytes.toBytes(qf), 1, mobValueBytes);
 +    Put put2 = new Put(Bytes.toBytes(row + "ignore"));
 +    put2.add(Bytes.toBytes(family), Bytes.toBytes(qf), 1, mobValueBytes);
 +    table.put(put);
 +    table.put(put2);
 +    table.flushCommits();
 +    admin.flush(tn);
 +
 +    FileStatus[] fileStatuses = TEST_UTIL.getTestFileSystem().listStatus(mobFamilyPath);
 +    //check the generation of a mob file
 +    assertEquals(1, fileStatuses.length);
 +
 +    String mobFile1 = fileStatuses[0].getPath().getName();
 +
 +    Configuration configuration = new Configuration(TEST_UTIL.getConfiguration());
 +    configuration.setFloat(MobConstants.MOB_SWEEP_TOOL_COMPACTION_RATIO, 0.6f);
 +    configuration.setStrings(TableInputFormat.INPUT_TABLE, tableName);
 +    configuration.setStrings(TableInputFormat.SCAN_COLUMN_FAMILY, family);
 +    configuration.setStrings(SweepJob.WORKING_VISITED_DIR_KEY, "jobWorkingNamesDir");
 +    configuration.setStrings(SweepJob.WORKING_FILES_DIR_KEY, "compactionFileDir");
 +    configuration.setStrings(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY,
 +            JavaSerialization.class.getName());
 +    configuration.set(SweepJob.WORKING_VISITED_DIR_KEY, "compactionVisitedDir");
 +    configuration.setLong(MobConstants.MOB_SWEEP_TOOL_COMPACTION_START_DATE,
 +        System.currentTimeMillis() + 24 * 3600 * 1000);
 +
 +    ZooKeeperWatcher zkw = new ZooKeeperWatcher(configuration, "1", new DummyMobAbortable());
 +    TableName lockName = MobUtils.getTableLockName(tn);
 +    String znode = ZKUtil.joinZNode(zkw.tableLockZNode, lockName.getNameAsString());
 +    configuration.set(SweepJob.SWEEP_JOB_ID, "1");
 +    configuration.set(SweepJob.SWEEP_JOB_TABLE_NODE, znode);
 +    ServerName serverName = SweepJob.getCurrentServerName(configuration);
 +    configuration.set(SweepJob.SWEEP_JOB_SERVERNAME, serverName.toString());
 +
 +    TableLockManager tableLockManager = TableLockManager.createTableLockManager(configuration, zkw,
 +        serverName);
 +    TableLock lock = tableLockManager.writeLock(lockName, "Run sweep tool");
 +    lock.acquire();
 +    try {
 +      // use the same counter when mocking
 +      Counter counter = new GenericCounter();
 +      Reducer<Text, KeyValue, Writable, Writable>.Context ctx = mock(Reducer.Context.class);
 +      when(ctx.getConfiguration()).thenReturn(configuration);
 +      when(ctx.getCounter(Matchers.any(SweepCounter.class))).thenReturn(counter);
 +      when(ctx.nextKey()).thenReturn(true).thenReturn(false);
 +      when(ctx.getCurrentKey()).thenReturn(new Text(mobFile1));
 +
 +      byte[] refBytes = Bytes.toBytes(mobFile1);
 +      long valueLength = refBytes.length;
 +      byte[] newValue = Bytes.add(Bytes.toBytes(valueLength), refBytes);
 +      KeyValue kv2 = new KeyValue(Bytes.toBytes(row), Bytes.toBytes(family), Bytes.toBytes(qf), 1,
 +        KeyValue.Type.Put, newValue);
 +      List<KeyValue> list = new ArrayList<KeyValue>();
 +      list.add(kv2);
 +
 +      when(ctx.getValues()).thenReturn(list);
 +
 +      SweepReducer reducer = new SweepReducer();
 +      reducer.run(ctx);
 +    } finally {
 +      lock.release();
 +    }
 +    FileStatus[] filsStatuses2 = TEST_UTIL.getTestFileSystem().listStatus(mobFamilyPath);
 +    String mobFile2 = filsStatuses2[0].getPath().getName();
 +    //new mob file is generated, old one has been archived
 +    assertEquals(1, filsStatuses2.length);
 +    assertEquals(false, mobFile2.equalsIgnoreCase(mobFile1));
 +
 +    //test sequence file
 +    String workingPath = configuration.get(SweepJob.WORKING_VISITED_DIR_KEY);
 +    FileStatus[] statuses = TEST_UTIL.getTestFileSystem().listStatus(new Path(workingPath));
 +    Set<String> files = new TreeSet<String>();
 +    for (FileStatus st : statuses) {
 +      files.addAll(getKeyFromSequenceFile(TEST_UTIL.getTestFileSystem(),
 +              st.getPath(), configuration));
 +    }
 +    assertEquals(1, files.size());
 +    assertEquals(true, files.contains(mobFile1));
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/mapreduce/TestMobSweeper.java
----------------------------------------------------------------------
diff --cc hbase-server/src/test/java/org/apache/hadoop/hbase/mob/mapreduce/TestMobSweeper.java
index c4817aa,0000000..1689c2a
mode 100644,000000..100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/mapreduce/TestMobSweeper.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/mapreduce/TestMobSweeper.java
@@@ -1,307 -1,0 +1,307 @@@
 +/**
 + * 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.mob.mapreduce;
 +
 +import static org.junit.Assert.assertEquals;
 +
 +import java.io.IOException;
 +import java.util.Random;
 +import java.util.Set;
 +import java.util.TreeSet;
 +
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.FileStatus;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.hbase.HBaseTestingUtility;
 +import org.apache.hadoop.hbase.HColumnDescriptor;
 +import org.apache.hadoop.hbase.HTableDescriptor;
- import org.apache.hadoop.hbase.MediumTests;
 +import org.apache.hadoop.hbase.TableName;
 +import org.apache.hadoop.hbase.client.Admin;
 +import org.apache.hadoop.hbase.client.HTable;
 +import org.apache.hadoop.hbase.client.Put;
 +import org.apache.hadoop.hbase.client.Result;
 +import org.apache.hadoop.hbase.client.ResultScanner;
 +import org.apache.hadoop.hbase.client.Scan;
 +import org.apache.hadoop.hbase.mob.MobConstants;
 +import org.apache.hadoop.hbase.mob.MobUtils;
++import org.apache.hadoop.hbase.testclassification.MediumTests;
 +import org.apache.hadoop.hbase.util.Bytes;
 +import org.apache.hadoop.util.ToolRunner;
 +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(MediumTests.class)
 +public class TestMobSweeper {
 +  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 +  private String tableName;
 +  private final static String row = "row_";
 +  private final static String family = "family";
 +  private final static String column = "column";
 +  private static HTable table;
 +  private static Admin admin;
 +
 +  private Random random = new Random();
 +  @BeforeClass
 +  public static void setUpBeforeClass() throws Exception {
 +    TEST_UTIL.getConfiguration().setInt("hbase.master.info.port", 0);
 +    TEST_UTIL.getConfiguration().setBoolean("hbase.regionserver.info.port.auto", true);
 +    TEST_UTIL.getConfiguration().setInt("hbase.hstore.compaction.min", 15); // avoid major compactions
 +    TEST_UTIL.getConfiguration().setInt("hbase.hstore.compaction.max", 30); // avoid major compactions
 +    TEST_UTIL.getConfiguration().setInt("hfile.format.version", 3);
 +
 +    TEST_UTIL.startMiniCluster();
 +
 +    TEST_UTIL.startMiniMapReduceCluster();
 +  }
 +
 +  @AfterClass
 +  public static void tearDownAfterClass() throws Exception {
 +    TEST_UTIL.shutdownMiniCluster();
 +    TEST_UTIL.shutdownMiniMapReduceCluster();
 +  }
 +
 +  @SuppressWarnings("deprecation")
 +  @Before
 +  public void setUp() throws Exception {
 +    long tid = System.currentTimeMillis();
 +    tableName = "testSweeper" + tid;
 +    HTableDescriptor desc = new HTableDescriptor(tableName);
 +    HColumnDescriptor hcd = new HColumnDescriptor(family);
 +    hcd.setMobEnabled(true);
 +    hcd.setMobThreshold(3L);
 +    hcd.setMaxVersions(4);
 +    desc.addFamily(hcd);
 +
 +    admin = TEST_UTIL.getHBaseAdmin();
 +    admin.createTable(desc);
 +    table = new HTable(TEST_UTIL.getConfiguration(), tableName);
 +    table.setAutoFlush(false);
 +
 +  }
 +
 +  @After
 +  public void tearDown() throws Exception {
 +    admin.disableTable(TableName.valueOf(tableName));
 +    admin.deleteTable(TableName.valueOf(tableName));
 +    admin.close();
 +  }
 +
 +  private Path getMobFamilyPath(Configuration conf, String tableNameStr,
 +                                String familyName) {
 +    Path p = new Path(MobUtils.getMobRegionPath(conf, TableName.valueOf(tableNameStr)),
 +            familyName);
 +    return p;
 +  }
 +
 +
 +  private String mergeString(Set<String> set) {
 +    StringBuilder sb = new StringBuilder();
 +    for (String s : set)
 +      sb.append(s);
 +    return sb.toString();
 +  }
 +
 +
 +  private void generateMobTable(int count, int flushStep)
 +          throws IOException, InterruptedException {
 +    if (count <= 0 || flushStep <= 0)
 +      return;
 +    int index = 0;
 +    for (int i = 0; i < count; i++) {
 +      byte[] mobVal = new byte[101*1024];
 +      random.nextBytes(mobVal);
 +
 +      Put put = new Put(Bytes.toBytes(row + i));
 +      put.add(Bytes.toBytes(family), Bytes.toBytes(column), mobVal);
 +      table.put(put);
 +      if (index++ % flushStep == 0) {
 +        table.flushCommits();
 +        admin.flush(TableName.valueOf(tableName));
 +      }
 +
 +
 +    }
 +    table.flushCommits();
 +    admin.flush(TableName.valueOf(tableName));
 +  }
 +
 +  @Test
 +  public void testSweeper() throws Exception {
 +
 +    int count = 10;
 +    //create table and generate 10 mob files
 +    generateMobTable(count, 1);
 +
 +    //get mob files
 +    Path mobFamilyPath = getMobFamilyPath(TEST_UTIL.getConfiguration(), tableName, family);
 +    FileStatus[] fileStatuses = TEST_UTIL.getTestFileSystem().listStatus(mobFamilyPath);
 +    // mobFileSet0 stores the orignal mob files
 +    TreeSet<String> mobFilesSet = new TreeSet<String>();
 +    for (FileStatus status : fileStatuses) {
 +      mobFilesSet.add(status.getPath().getName());
 +    }
 +
 +    //scan the table, retreive the references
 +    Scan scan = new Scan();
 +    scan.setAttribute(MobConstants.MOB_SCAN_RAW, Bytes.toBytes(Boolean.TRUE));
 +    scan.setAttribute(MobConstants.MOB_SCAN_REF_ONLY, Bytes.toBytes(Boolean.TRUE));
 +    ResultScanner rs = table.getScanner(scan);
 +    TreeSet<String> mobFilesScanned = new TreeSet<String>();
 +    for (Result res : rs) {
 +      byte[] valueBytes = res.getValue(Bytes.toBytes(family),
 +          Bytes.toBytes(column));
 +      mobFilesScanned.add(Bytes.toString(valueBytes, Bytes.SIZEOF_INT,
 +          valueBytes.length - Bytes.SIZEOF_INT));
 +    }
 +
 +    //there should be 10 mob files
 +    assertEquals(10, mobFilesScanned.size());
 +    //check if we store the correct reference of mob files
 +    assertEquals(mergeString(mobFilesSet), mergeString(mobFilesScanned));
 +
 +
 +    Configuration conf = TEST_UTIL.getConfiguration();
 +    conf.setLong(SweepJob.MOB_SWEEP_JOB_DELAY, 24 * 60 * 60 * 1000);
 +
 +    String[] args = new String[2];
 +    args[0] = tableName;
 +    args[1] = family;
 +    ToolRunner.run(conf, new Sweeper(), args);
 +
 +
 +    mobFamilyPath = getMobFamilyPath(TEST_UTIL.getConfiguration(), tableName, family);
 +    fileStatuses = TEST_UTIL.getTestFileSystem().listStatus(mobFamilyPath);
 +    mobFilesSet = new TreeSet<String>();
 +    for (FileStatus status : fileStatuses) {
 +      mobFilesSet.add(status.getPath().getName());
 +    }
 +
 +    assertEquals(10, mobFilesSet.size());
 +
 +
 +    scan = new Scan();
 +    scan.setAttribute(MobConstants.MOB_SCAN_RAW, Bytes.toBytes(Boolean.TRUE));
 +    scan.setAttribute(MobConstants.MOB_SCAN_REF_ONLY, Bytes.toBytes(Boolean.TRUE));
 +    rs = table.getScanner(scan);
 +    TreeSet<String> mobFilesScannedAfterJob = new TreeSet<String>();
 +    for (Result res : rs) {
 +      byte[] valueBytes = res.getValue(Bytes.toBytes(family), Bytes.toBytes(
 +          column));
 +      mobFilesScannedAfterJob.add(Bytes.toString(valueBytes, Bytes.SIZEOF_INT,
 +          valueBytes.length - Bytes.SIZEOF_INT));
 +    }
 +
 +    assertEquals(10, mobFilesScannedAfterJob.size());
 +
 +    fileStatuses = TEST_UTIL.getTestFileSystem().listStatus(mobFamilyPath);
 +    mobFilesSet = new TreeSet<String>();
 +    for (FileStatus status : fileStatuses) {
 +      mobFilesSet.add(status.getPath().getName());
 +    }
 +
 +    assertEquals(10, mobFilesSet.size());
 +    assertEquals(true, mobFilesScannedAfterJob.iterator().next()
 +            .equalsIgnoreCase(mobFilesSet.iterator().next()));
 +
 +  }
 +
 +  @Test
 +  public void testCompactionDelaySweeper() throws Exception {
 +
 +    int count = 10;
 +    //create table and generate 10 mob files
 +    generateMobTable(count, 1);
 +
 +    //get mob files
 +    Path mobFamilyPath = getMobFamilyPath(TEST_UTIL.getConfiguration(), tableName, family);
 +    FileStatus[] fileStatuses = TEST_UTIL.getTestFileSystem().listStatus(mobFamilyPath);
 +    // mobFileSet0 stores the orignal mob files
 +    TreeSet<String> mobFilesSet = new TreeSet<String>();
 +    for (FileStatus status : fileStatuses) {
 +      mobFilesSet.add(status.getPath().getName());
 +    }
 +
 +    //scan the table, retreive the references
 +    Scan scan = new Scan();
 +    scan.setAttribute(MobConstants.MOB_SCAN_RAW, Bytes.toBytes(Boolean.TRUE));
 +    scan.setAttribute(MobConstants.MOB_SCAN_REF_ONLY, Bytes.toBytes(Boolean.TRUE));
 +    ResultScanner rs = table.getScanner(scan);
 +    TreeSet<String> mobFilesScanned = new TreeSet<String>();
 +    for (Result res : rs) {
 +      byte[] valueBytes = res.getValue(Bytes.toBytes(family),
 +              Bytes.toBytes(column));
 +      mobFilesScanned.add(Bytes.toString(valueBytes, Bytes.SIZEOF_INT,
 +          valueBytes.length - Bytes.SIZEOF_INT));
 +    }
 +
 +    //there should be 10 mob files
 +    assertEquals(10, mobFilesScanned.size());
 +    //check if we store the correct reference of mob files
 +    assertEquals(mergeString(mobFilesSet), mergeString(mobFilesScanned));
 +
 +
 +    Configuration conf = TEST_UTIL.getConfiguration();
 +    conf.setLong(SweepJob.MOB_SWEEP_JOB_DELAY, 0);
 +
 +    String[] args = new String[2];
 +    args[0] = tableName;
 +    args[1] = family;
 +    ToolRunner.run(conf, new Sweeper(), args);
 +
 +
 +    mobFamilyPath = getMobFamilyPath(TEST_UTIL.getConfiguration(), tableName, family);
 +    fileStatuses = TEST_UTIL.getTestFileSystem().listStatus(mobFamilyPath);
 +    mobFilesSet = new TreeSet<String>();
 +    for (FileStatus status : fileStatuses) {
 +      mobFilesSet.add(status.getPath().getName());
 +    }
 +
 +    assertEquals(1, mobFilesSet.size());
 +
 +
 +    scan = new Scan();
 +    scan.setAttribute(MobConstants.MOB_SCAN_RAW, Bytes.toBytes(Boolean.TRUE));
 +    scan.setAttribute(MobConstants.MOB_SCAN_REF_ONLY, Bytes.toBytes(Boolean.TRUE));
 +    rs = table.getScanner(scan);
 +    TreeSet<String> mobFilesScannedAfterJob = new TreeSet<String>();
 +    for (Result res : rs) {
 +      byte[] valueBytes = res.getValue(Bytes.toBytes(family), Bytes.toBytes(
 +              column));
 +      mobFilesScannedAfterJob.add(Bytes.toString(valueBytes, Bytes.SIZEOF_INT,
 +          valueBytes.length - Bytes.SIZEOF_INT));
 +    }
 +
 +    assertEquals(1, mobFilesScannedAfterJob.size());
 +
 +    fileStatuses = TEST_UTIL.getTestFileSystem().listStatus(mobFamilyPath);
 +    mobFilesSet = new TreeSet<String>();
 +    for (FileStatus status : fileStatuses) {
 +      mobFilesSet.add(status.getPath().getName());
 +    }
 +
 +    assertEquals(1, mobFilesSet.size());
 +    assertEquals(true, mobFilesScannedAfterJob.iterator().next()
 +            .equalsIgnoreCase(mobFilesSet.iterator().next()));
 +
 +  }
 +
 +}

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe335b68/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java
----------------------------------------------------------------------
diff --cc hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java
index c7d146b,852d319..0d28e54
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java
@@@ -242,73 -242,22 +242,92 @@@ public class MetricsRegionServerWrapper
    }
  
    @Override
+   public long getHedgedReadOps() {
+     return 100;
+   }
+ 
+   @Override
+   public long getHedgedReadWins() {
+     return 10;
+   }
+ 
+   @Override
+   public long getBlockedRequestsCount() {
+     return 0;
+   }
+ 
+   @Override
+   public int getSplitQueueSize() {
+     return 0;
+   }
++
++  @Override
 +  public long getMobCompactedIntoMobCellsCount() {
 +    return 20;
 +  }
 +
 +  @Override
 +  public long getMobCompactedFromMobCellsCount() {
 +    return 10;
 +  }
 +
 +  @Override
 +  public long getMobCompactedIntoMobCellsSize() {
 +    return 200;
 +  }
 +
 +  @Override
 +  public long getMobCompactedFromMobCellsSize() {
 +    return 100;
 +  }
 +
 +  @Override
 +  public long getMobFlushCount() {
 +    return 1;
 +  }
 +
 +  @Override
 +  public long getMobFlushedCellsCount() {
 +    return 10;
 +  }
 +
 +  @Override
 +  public long getMobFlushedCellsSize() {
 +    return 1000;
 +  }
 +
 +  @Override
 +  public long getMobScanCellsCount() {
 +    return 10;
 +  }
 +
 +  @Override
 +  public long getMobScanCellsSize() {
 +    return 1000;
 +  }
 +
 +  @Override
 +  public long getMobFileCacheAccessCount() {
 +    return 100;
 +  }
 +
 +  @Override
 +  public long getMobFileCacheMissCount() {
 +    return 50;
 +  }
 +
 +  @Override
 +  public long getMobFileCacheEvictedCount() {
 +    return 0;
 +  }
 +
 +  @Override
 +  public long getMobFileCacheCount() {
 +    return 100;
 +  }
 +
 +  @Override
 +  public int getMobFileCacheHitPercent() {
 +    return 50;
 +  }
- 
- }
+ }


[41/50] [abbrv] hbase git commit: HBASE-12996 Reversed field on Filter should be transient (Ian Friedman)

Posted by jm...@apache.org.
HBASE-12996 Reversed field on Filter should be transient (Ian Friedman)


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

Branch: refs/heads/hbase-11339
Commit: dc522d010cd2078d4ce6888df564610ed29bbdc7
Parents: 3f7b18c
Author: Andrew Purtell <ap...@apache.org>
Authored: Wed Feb 11 12:17:51 2015 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Feb 11 12:17:51 2015 -0800

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/dc522d01/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java
index d66ad50..88bf842 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java
@@ -55,7 +55,7 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException;
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public abstract class Filter {
-  protected boolean reversed;
+  protected transient boolean reversed;
   /**
    * Reset the state of the filter between rows.
    * 


[02/50] [abbrv] hbase git commit: HBASE-12957 region_mover#isSuccessfulScan may be extremely slow on region with lots of expired data (Hongyu Bi)

Posted by jm...@apache.org.
HBASE-12957 region_mover#isSuccessfulScan may be extremely slow on region with lots of expired data (Hongyu Bi)


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

Branch: refs/heads/hbase-11339
Commit: 4388fed83028325cfe75fc0a8787183db2a58855
Parents: 5c1b08c
Author: tedyu <yu...@gmail.com>
Authored: Tue Feb 3 20:06:23 2015 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Tue Feb 3 20:06:23 2015 -0800

----------------------------------------------------------------------
 bin/region_mover.rb                             |  6 +-
 .../regionserver/TestRegionServerHostname.java  | 98 ++++++++++++++++++++
 2 files changed, 102 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/4388fed8/bin/region_mover.rb
----------------------------------------------------------------------
diff --git a/bin/region_mover.rb b/bin/region_mover.rb
index 7a09854..78979d9 100644
--- a/bin/region_mover.rb
+++ b/bin/region_mover.rb
@@ -31,6 +31,8 @@ import org.apache.hadoop.hbase.client.Scan
 import org.apache.hadoop.hbase.client.HTable
 import org.apache.hadoop.hbase.client.HConnectionManager
 import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
+import org.apache.hadoop.hbase.filter.InclusiveStopFilter;
+import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.util.Bytes
 import org.apache.hadoop.hbase.util.Writables
 import org.apache.hadoop.conf.Configuration
@@ -95,10 +97,10 @@ end
 # Trys to scan a row from passed region
 # Throws exception if can't
 def isSuccessfulScan(admin, r)
-  scan = Scan.new(r.getStartKey()) 
+  scan = Scan.new(r.getStartKey(), r.getStartKey())
   scan.setBatch(1)
   scan.setCaching(1)
-  scan.setFilter(FirstKeyOnlyFilter.new()) 
+  scan.setFilter(FilterList.new(FirstKeyOnlyFilter.new(),InclusiveStopFilter().new(r.getStartKey())))
   begin
     table = HTable.new(admin.getConfiguration(), r.getTableName())
     scanner = table.getScanner(scan)

http://git-wip-us.apache.org/repos/asf/hbase/blob/4388fed8/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerHostname.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerHostname.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerHostname.java
new file mode 100644
index 0000000..523911b
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerHostname.java
@@ -0,0 +1,98 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver;
+
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.NetworkInterface;
+import java.util.Enumeration;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Tests for the hostname specification by region server
+ */
+@Category({RegionServerTests.class, MediumTests.class})
+public class TestRegionServerHostname {
+  private static final Log LOG = LogFactory.getLog(TestRegionServerHostname.class);
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+  @Test (timeout=30000)
+  public void testInvalidRegionServerHostname() throws Exception {
+    final int NUM_MASTERS = 1;
+    final int NUM_RS = 1;
+    String invalidHostname = "hostAddr";
+    TEST_UTIL.getConfiguration().set(HRegionServer.HOSTNAME_KEY, invalidHostname);
+    try {
+      TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS);
+    } catch (IOException ioe) {
+      Throwable t1 = ioe.getCause();
+      Throwable t2 = t1.getCause();
+      assertTrue(t2.getMessage().contains("Failed resolve of " + invalidHostname));
+      return;
+    } finally {
+      TEST_UTIL.shutdownMiniCluster();
+    }
+    assertTrue("Failed to validate against invalid hostname", false);
+  }
+
+  @Test(timeout=120000)
+  public void testRegionServerHostname() throws Exception {
+    final int NUM_MASTERS = 1;
+    final int NUM_RS = 1;
+    Enumeration<NetworkInterface> netInterfaceList = NetworkInterface.getNetworkInterfaces();
+
+    while (netInterfaceList.hasMoreElements()) {
+      NetworkInterface ni = netInterfaceList.nextElement();
+      Enumeration<InetAddress> addrList = ni.getInetAddresses();
+      // iterate through host addresses and use each as hostname
+      while (addrList.hasMoreElements()) {
+        InetAddress addr = addrList.nextElement();
+        if (addr.isLoopbackAddress() || addr.isLinkLocalAddress()) continue;
+        if (addr.isMulticastAddress()) continue;
+        String hostAddr = addr.getHostAddress();
+        LOG.info("Found " + hostAddr + " on " + ni);
+        
+        TEST_UTIL.getConfiguration().set(HRegionServer.HOSTNAME_KEY, hostAddr);
+        TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS);
+        try {
+          ZooKeeperWatcher zkw = TEST_UTIL.getZooKeeperWatcher();
+          List<String> servers = ZKUtil.listChildrenNoWatch(zkw, zkw.rsZNode);
+          assertTrue(servers.size() > 0);
+          for (String server : servers) {
+            assertTrue(server.startsWith(hostAddr+","));
+          }
+          zkw.close();
+        } finally {
+          TEST_UTIL.shutdownMiniCluster();
+        }
+      }
+    }
+  }
+}


[31/50] [abbrv] hbase git commit: HBASE-12035 Keep table state in META (Andrey Stepachev)

Posted by jm...@apache.org.
HBASE-12035 Keep table state in META (Andrey Stepachev)


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

Branch: refs/heads/hbase-11339
Commit: fa852c4ce68999cbe30e11820cbfd099d1c5cfaa
Parents: ede20aa
Author: stack <st...@apache.org>
Authored: Tue Feb 10 14:20:47 2015 -0800
Committer: stack <st...@apache.org>
Committed: Tue Feb 10 14:20:47 2015 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/HTableDescriptor.java   |  11 +
 .../apache/hadoop/hbase/MetaTableAccessor.java  | 385 ++++++++++++++-----
 .../hadoop/hbase/client/ConnectionCallable.java |  56 +++
 .../hadoop/hbase/client/ConnectionManager.java  | 137 +++----
 .../apache/hadoop/hbase/client/HBaseAdmin.java  |  88 +++--
 .../apache/hadoop/hbase/client/TableState.java  |  52 ++-
 .../org/apache/hadoop/hbase/HConstants.java     |  14 +-
 .../hbase/protobuf/generated/HBaseProtos.java   | 130 +++----
 hbase-protocol/src/main/protobuf/HBase.proto    |   4 +-
 .../apache/hadoop/hbase/TableDescriptor.java    |  51 ++-
 .../org/apache/hadoop/hbase/ipc/RpcServer.java  |   4 +-
 .../hadoop/hbase/master/AssignmentManager.java  |  11 +-
 .../org/apache/hadoop/hbase/master/HMaster.java |  39 +-
 .../hadoop/hbase/master/LoadBalancer.java       |   2 +
 .../hadoop/hbase/master/MasterFileSystem.java   |   2 +-
 .../hadoop/hbase/master/MasterRpcServices.java  |   2 -
 .../hadoop/hbase/master/RegionStateStore.java   |   3 +-
 .../hadoop/hbase/master/RegionStates.java       |   7 +-
 .../SnapshotOfRegionAssignmentFromMeta.java     |   2 +-
 .../hadoop/hbase/master/TableStateManager.java  | 206 +++++-----
 .../master/handler/CreateTableHandler.java      |  15 +-
 .../master/handler/TruncateTableHandler.java    |  11 +-
 .../hbase/regionserver/HRegionServer.java       |   8 +-
 .../hadoop/hbase/snapshot/SnapshotManifest.java |   3 +-
 .../hadoop/hbase/util/FSTableDescriptors.java   |   9 +-
 .../org/apache/hadoop/hbase/util/HBaseFsck.java | 141 ++++---
 .../hadoop/hbase/HBaseTestingUtility.java       |  42 +-
 .../hadoop/hbase/TestMetaTableAccessor.java     |   7 +-
 .../apache/hadoop/hbase/client/TestAdmin1.java  |  28 +-
 .../hbase/client/TestMetaWithReplicas.java      |  69 +++-
 .../hbase/client/TestReplicaWithCluster.java    |   6 +-
 .../master/TestAssignmentManagerOnCluster.java  |   5 +-
 .../hadoop/hbase/master/TestCatalogJanitor.java |   3 +-
 .../TestMasterOperationsForRegionReplicas.java  |   6 +-
 .../hadoop/hbase/master/TestRegionStates.java   |   4 +-
 .../security/access/TestAccessController.java   |   1 +
 .../hbase/util/TestCoprocessorScanPolicy.java   |   1 +
 .../hbase/util/TestFSTableDescriptors.java      |  14 +-
 .../apache/hadoop/hbase/util/TestHBaseFsck.java |  31 +-
 .../util/hbck/OfflineMetaRebuildTestCore.java   |  13 +-
 .../util/hbck/TestOfflineMetaRebuildBase.java   |  13 +-
 41 files changed, 1095 insertions(+), 541 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
index cc91aed..51352bb 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
@@ -1332,6 +1332,17 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
               .setBloomFilterType(BloomType.NONE)
               // Enable cache of data blocks in L1 if more than one caching tier deployed:
               // e.g. if using CombinedBlockCache (BucketCache).
+              .setCacheDataInL1(true),
+          new HColumnDescriptor(HConstants.TABLE_FAMILY)
+              // Ten is arbitrary number.  Keep versions to help debugging.
+              .setMaxVersions(10)
+              .setInMemory(true)
+              .setBlocksize(8 * 1024)
+              .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+                  // Disable blooms for meta.  Needs work.  Seems to mess w/ getClosestOrBefore.
+              .setBloomFilterType(BloomType.NONE)
+                  // Enable cache of data blocks in L1 if more than one caching tier deployed:
+                  // e.g. if using CombinedBlockCache (BucketCache).
               .setCacheDataInL1(true)
       });
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
index 8f3a20e..2e6723a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
@@ -17,9 +17,23 @@
  */
 package org.apache.hadoop.hbase;
 
+import javax.annotation.Nullable;
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
 import com.google.common.annotations.VisibleForTesting;
 import com.google.protobuf.ServiceException;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -38,6 +52,8 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
@@ -48,18 +64,6 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.PairOfSameType;
 import org.apache.hadoop.hbase.util.Threads;
 
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.NavigableMap;
-import java.util.Set;
-import java.util.SortedMap;
-import java.util.TreeMap;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
 /**
  * Read/write operations on region and assignment information store in
  * <code>hbase:meta</code>.
@@ -78,6 +82,11 @@ public class MetaTableAccessor {
    * HRI defined which is called default replica.
    *
    * Meta layout (as of 0.98 + HBASE-10070) is like:
+   *
+   * For each table there is single row in column family 'table' formatted:
+   * <tableName> including namespace and columns are:
+   * table: state             => contains table state
+   *
    * For each table range, there is a single row, formatted like:
    * <tableName>,<startKey>,<regionId>,<encodedRegionName>. This row corresponds to the regionName
    * of the default region replica.
@@ -120,6 +129,24 @@ public class MetaTableAccessor {
       META_REGION_PREFIX, 0, len);
   }
 
+
+  @InterfaceAudience.Private
+  public enum QueryType {
+    ALL(HConstants.TABLE_FAMILY, HConstants.CATALOG_FAMILY),
+    REGION(HConstants.CATALOG_FAMILY),
+    TABLE(HConstants.TABLE_FAMILY);
+
+    private final byte[][] families;
+
+    QueryType(byte[]... families) {
+      this.families = families;
+    }
+
+    byte[][] getFamilies() {
+      return this.families;
+    }
+  }
+
   /** The delimiter for meta columns for replicaIds > 0 */
   protected static final char META_REPLICA_ID_DELIMITER = '_';
 
@@ -131,40 +158,64 @@ public class MetaTableAccessor {
   // Reading operations //
   ////////////////////////
 
- /**
-   * Performs a full scan of a <code>hbase:meta</code> table.
-   * @return List of {@link org.apache.hadoop.hbase.client.Result}
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
    * @throws IOException
    */
-  public static List<Result> fullScanOfMeta(Connection connection)
-  throws IOException {
-    CollectAllVisitor v = new CollectAllVisitor();
-    fullScan(connection, v, null);
-    return v.getResults();
+  public static void fullScanRegions(Connection connection,
+      final Visitor visitor)
+      throws IOException {
+    fullScan(connection, visitor, null, QueryType.REGION);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @throws IOException
+   */
+  public static List<Result> fullScanRegions(Connection connection)
+      throws IOException {
+    return fullScan(connection, QueryType.REGION);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for tables.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in tables family.
+   * @throws IOException
+   */
+  public static void fullScanTables(Connection connection,
+      final Visitor visitor)
+      throws IOException {
+    fullScan(connection, visitor, null, QueryType.TABLE);
   }
 
   /**
    * Performs a full scan of <code>hbase:meta</code>.
    * @param connection connection we're using
    * @param visitor Visitor invoked against each row.
+   * @param type scanned part of meta
    * @throws IOException
    */
   public static void fullScan(Connection connection,
-      final Visitor visitor)
+      final Visitor visitor, QueryType type)
   throws IOException {
-    fullScan(connection, visitor, null);
+    fullScan(connection, visitor, null, type);
   }
 
   /**
    * Performs a full scan of <code>hbase:meta</code>.
    * @param connection connection we're using
+   * @param type scanned part of meta
    * @return List of {@link Result}
    * @throws IOException
    */
-  public static List<Result> fullScan(Connection connection)
+  public static List<Result> fullScan(Connection connection, QueryType type)
     throws IOException {
     CollectAllVisitor v = new CollectAllVisitor();
-    fullScan(connection, v, null);
+    fullScan(connection, v, null, type);
     return v.getResults();
   }
 
@@ -306,6 +357,7 @@ public class MetaTableAccessor {
    * @return null if it doesn't contain merge qualifier, else two merge regions
    * @throws IOException
    */
+  @Nullable
   public static Pair<HRegionInfo, HRegionInfo> getRegionsFromMergeQualifier(
       Connection connection, byte[] regionName) throws IOException {
     Result result = getRegionResult(connection, regionName);
@@ -328,42 +380,9 @@ public class MetaTableAccessor {
   public static boolean tableExists(Connection connection,
       final TableName tableName)
   throws IOException {
-    if (tableName.equals(TableName.META_TABLE_NAME)) {
-      // Catalog tables always exist.
-      return true;
-    }
-    // Make a version of ResultCollectingVisitor that only collects the first
-    CollectingVisitor<HRegionInfo> visitor = new CollectingVisitor<HRegionInfo>() {
-      private HRegionInfo current = null;
-
-      @Override
-      public boolean visit(Result r) throws IOException {
-        RegionLocations locations = getRegionLocations(r);
-        if (locations == null || locations.getRegionLocation().getRegionInfo() == null) {
-          LOG.warn("No serialized HRegionInfo in " + r);
-          return true;
-        }
-        this.current = locations.getRegionLocation().getRegionInfo();
-        if (this.current == null) {
-          LOG.warn("No serialized HRegionInfo in " + r);
-          return true;
-        }
-        if (!isInsideTable(this.current, tableName)) return false;
-        // Else call super and add this Result to the collection.
-        super.visit(r);
-        // Stop collecting regions from table after we get one.
-        return false;
-      }
-
-      @Override
-      void add(Result r) {
-        // Add the current HRI.
-        this.results.add(this.current);
-      }
-    };
-    fullScan(connection, visitor, getTableStartRowForMeta(tableName));
-    // If visitor has results >= 1 then table exists.
-    return visitor.getResults().size() >= 1;
+    // Catalog tables always exist.
+    return tableName.equals(TableName.META_TABLE_NAME)
+        || getTableState(connection, tableName) != null;
   }
 
   /**
@@ -400,6 +419,7 @@ public class MetaTableAccessor {
     return getListOfHRegionInfos(result);
   }
 
+  @Nullable
   static List<HRegionInfo> getListOfHRegionInfos(final List<Pair<HRegionInfo, ServerName>> pairs) {
     if (pairs == null || pairs.isEmpty()) return null;
     List<HRegionInfo> result = new ArrayList<HRegionInfo>(pairs.size());
@@ -470,6 +490,7 @@ public class MetaTableAccessor {
    * Do not use this method to get meta table regions, use methods in MetaTableLocator instead.
    * @param connection connection we're using
    * @param tableName table to work with
+   * @param excludeOfflinedSplitParents don't return split parents
    * @return Return list of regioninfos and server addresses.
    * @throws IOException
    */
@@ -512,7 +533,7 @@ public class MetaTableAccessor {
           }
         }
       };
-    fullScan(connection, visitor, getTableStartRowForMeta(tableName));
+    fullScan(connection, visitor, getTableStartRowForMeta(tableName), QueryType.REGION);
     return visitor.getResults();
   }
 
@@ -544,7 +565,7 @@ public class MetaTableAccessor {
         }
       }
     };
-    fullScan(connection, v);
+    fullScan(connection, v, QueryType.REGION);
     return hris;
   }
 
@@ -555,17 +576,22 @@ public class MetaTableAccessor {
       public boolean visit(Result r) throws IOException {
         if (r ==  null || r.isEmpty()) return true;
         LOG.info("fullScanMetaAndPrint.Current Meta Row: " + r);
-        RegionLocations locations = getRegionLocations(r);
-        if (locations == null) return true;
-        for (HRegionLocation loc : locations.getRegionLocations()) {
-          if (loc != null) {
-            LOG.info("fullScanMetaAndPrint.HRI Print= " + loc.getRegionInfo());
+        TableState state = getTableState(r);
+        if (state != null) {
+          LOG.info("Table State: " + state);
+        } else {
+          RegionLocations locations = getRegionLocations(r);
+          if (locations == null) return true;
+          for (HRegionLocation loc : locations.getRegionLocations()) {
+            if (loc != null) {
+              LOG.info("fullScanMetaAndPrint.HRI Print= " + loc.getRegionInfo());
+            }
           }
         }
         return true;
       }
     };
-    fullScan(connection, v);
+    fullScan(connection, v, QueryType.ALL);
   }
 
   /**
@@ -574,20 +600,40 @@ public class MetaTableAccessor {
    * @param visitor Visitor invoked against each row.
    * @param startrow Where to start the scan. Pass null if want to begin scan
    * at first row.
+   * @param type scanned part of meta
    * <code>hbase:meta</code>, the default (pass false to scan hbase:meta)
    * @throws IOException
    */
   public static void fullScan(Connection connection,
-    final Visitor visitor, final byte [] startrow)
+      final Visitor visitor, @Nullable final byte[] startrow, QueryType type) throws IOException {
+    fullScan(connection, visitor, startrow, type, false);
+  }
+
+  /**
+   * Performs a full scan of a catalog table.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row.
+   * @param startrow Where to start the scan. Pass null if want to begin scan
+   * at first row.
+   * @param type scanned part of meta
+   * @param raw read raw data including Delete tumbstones
+   * <code>hbase:meta</code>, the default (pass false to scan hbase:meta)
+   * @throws IOException
+   */
+  public static void fullScan(Connection connection,
+      final Visitor visitor, @Nullable final byte[] startrow, QueryType type, boolean raw)
   throws IOException {
     Scan scan = new Scan();
+    scan.setRaw(raw);
     if (startrow != null) scan.setStartRow(startrow);
     if (startrow == null) {
       int caching = connection.getConfiguration()
           .getInt(HConstants.HBASE_META_SCANNER_CACHING, 100);
       scan.setCaching(caching);
     }
-    scan.addFamily(HConstants.CATALOG_FAMILY);
+    for (byte[] family : type.getFamilies()) {
+      scan.addFamily(family);
+    }
     Table metaTable = getMetaHTable(connection);
     ResultScanner scanner = null;
     try {
@@ -608,11 +654,19 @@ public class MetaTableAccessor {
    * Returns the column family used for meta columns.
    * @return HConstants.CATALOG_FAMILY.
    */
-  protected static byte[] getFamily() {
+  protected static byte[] getCatalogFamily() {
     return HConstants.CATALOG_FAMILY;
   }
 
   /**
+   * Returns the column family used for table columns.
+   * @return HConstants.TABLE_FAMILY.
+   */
+  protected static byte[] getTableFamily() {
+    return HConstants.TABLE_FAMILY;
+  }
+
+  /**
    * Returns the column qualifier for serialized region info
    * @return HConstants.REGIONINFO_QUALIFIER
    */
@@ -621,6 +675,15 @@ public class MetaTableAccessor {
   }
 
   /**
+   * Returns the column qualifier for serialized table state
+   *
+   * @return HConstants.TABLE_STATE_QUALIFIER
+   */
+  protected static byte[] getStateColumn() {
+    return HConstants.TABLE_STATE_QUALIFIER;
+  }
+
+  /**
    * Returns the column qualifier for server column for replicaId
    * @param replicaId the replicaId of the region
    * @return a byte[] for server column qualifier
@@ -686,14 +749,15 @@ public class MetaTableAccessor {
    * @param r Result to pull from
    * @return A ServerName instance or null if necessary fields not found or empty.
    */
+  @Nullable
   private static ServerName getServerName(final Result r, final int replicaId) {
     byte[] serverColumn = getServerColumn(replicaId);
-    Cell cell = r.getColumnLatestCell(getFamily(), serverColumn);
+    Cell cell = r.getColumnLatestCell(getCatalogFamily(), serverColumn);
     if (cell == null || cell.getValueLength() == 0) return null;
     String hostAndPort = Bytes.toString(
       cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
     byte[] startcodeColumn = getStartCodeColumn(replicaId);
-    cell = r.getColumnLatestCell(getFamily(), startcodeColumn);
+    cell = r.getColumnLatestCell(getCatalogFamily(), startcodeColumn);
     if (cell == null || cell.getValueLength() == 0) return null;
     return ServerName.valueOf(hostAndPort,
       Bytes.toLong(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()));
@@ -706,7 +770,7 @@ public class MetaTableAccessor {
    * @return SeqNum, or HConstants.NO_SEQNUM if there's no value written.
    */
   private static long getSeqNumDuringOpen(final Result r, final int replicaId) {
-    Cell cell = r.getColumnLatestCell(getFamily(), getSeqNumColumn(replicaId));
+    Cell cell = r.getColumnLatestCell(getCatalogFamily(), getSeqNumColumn(replicaId));
     if (cell == null || cell.getValueLength() == 0) return HConstants.NO_SEQNUM;
     return Bytes.toLong(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
   }
@@ -716,6 +780,7 @@ public class MetaTableAccessor {
    * @return an HRegionLocationList containing all locations for the region range or null if
    *  we can't deserialize the result.
    */
+  @Nullable
   public static RegionLocations getRegionLocations(final Result r) {
     if (r == null) return null;
     HRegionInfo regionInfo = getHRegionInfo(r, getRegionInfoColumn());
@@ -726,7 +791,7 @@ public class MetaTableAccessor {
 
     locations.add(getRegionLocation(r, regionInfo, 0));
 
-    NavigableMap<byte[], byte[]> infoMap = familyMap.get(getFamily());
+    NavigableMap<byte[], byte[]> infoMap = familyMap.get(getCatalogFamily());
     if (infoMap == null) return new RegionLocations(locations);
 
     // iterate until all serverName columns are seen
@@ -788,8 +853,9 @@ public class MetaTableAccessor {
    * @param qualifier Column family qualifier
    * @return An HRegionInfo instance or null.
    */
+  @Nullable
   private static HRegionInfo getHRegionInfo(final Result r, byte [] qualifier) {
-    Cell cell = r.getColumnLatestCell(getFamily(), qualifier);
+    Cell cell = r.getColumnLatestCell(getCatalogFamily(), qualifier);
     if (cell == null) return null;
     return HRegionInfo.parseFromOrNull(cell.getValueArray(),
       cell.getValueOffset(), cell.getValueLength());
@@ -824,6 +890,80 @@ public class MetaTableAccessor {
   }
 
   /**
+   * Fetch table state for given table from META table
+   * @param conn connection to use
+   * @param tableName table to fetch state for
+   * @return state
+   * @throws IOException
+   */
+  @Nullable
+  public static TableState getTableState(Connection conn, TableName tableName)
+      throws IOException {
+    Table metaHTable = getMetaHTable(conn);
+    Get get = new Get(tableName.getName()).addColumn(getTableFamily(), getStateColumn());
+    long time = EnvironmentEdgeManager.currentTime();
+    get.setTimeRange(0, time);
+    Result result =
+        metaHTable.get(get);
+    return getTableState(result);
+  }
+
+  /**
+   * Fetch table states from META table
+   * @param conn connection to use
+   * @return map {tableName -> state}
+   * @throws IOException
+   */
+  public static Map<TableName, TableState> getTableStates(Connection conn)
+      throws IOException {
+    final Map<TableName, TableState> states = new LinkedHashMap<>();
+    Visitor collector = new Visitor() {
+      @Override
+      public boolean visit(Result r) throws IOException {
+        TableState state = getTableState(r);
+        if (state != null)
+          states.put(state.getTableName(), state);
+        return true;
+      }
+    };
+    fullScanTables(conn, collector);
+    return states;
+  }
+
+  /**
+   * Updates state in META
+   * @param conn connection to use
+   * @param tableName table to look for
+   * @throws IOException
+   */
+  public static void updateTableState(Connection conn, TableName tableName,
+      TableState.State actual) throws IOException {
+    updateTableState(conn, new TableState(tableName, actual));
+  }
+
+  /**
+   * Decode table state from META Result.
+   * Should contain cell from HConstants.TABLE_FAMILY
+   * @param r result
+   * @return null if not found
+   * @throws IOException
+   */
+  @Nullable
+  public static TableState getTableState(Result r)
+      throws IOException {
+    Cell cell = r.getColumnLatestCell(getTableFamily(), getStateColumn());
+    if (cell == null) return null;
+    try {
+      return TableState.parseFrom(TableName.valueOf(r.getRow()),
+          Arrays.copyOfRange(cell.getValueArray(),
+          cell.getValueOffset(), cell.getValueOffset() + cell.getValueLength()));
+    } catch (DeserializationException e) {
+      throw new IOException(e);
+    }
+
+  }
+
+  /**
    * Implementations 'visit' a catalog table row.
    */
   public interface Visitor {
@@ -920,7 +1060,8 @@ public class MetaTableAccessor {
    */
   public static Put makePutFromRegionInfo(HRegionInfo regionInfo)
     throws IOException {
-    Put put = new Put(regionInfo.getRegionName());
+    long now = EnvironmentEdgeManager.currentTime();
+    Put put = new Put(regionInfo.getRegionName(), now);
     addRegionInfo(put, regionInfo);
     return put;
   }
@@ -933,7 +1074,9 @@ public class MetaTableAccessor {
     if (regionInfo == null) {
       throw new IllegalArgumentException("Can't make a delete for null region");
     }
+    long now = EnvironmentEdgeManager.currentTime();
     Delete delete = new Delete(regionInfo.getRegionName());
+    delete.addFamily(getCatalogFamily(), now);
     return delete;
   }
 
@@ -1034,14 +1177,15 @@ public class MetaTableAccessor {
       throws IOException {
     int absoluteIndex = replicaIndexToDeleteFrom + numReplicasToRemove;
     for (byte[] row : metaRows) {
+      long now = EnvironmentEdgeManager.currentTime();
       Delete deleteReplicaLocations = new Delete(row);
       for (int i = replicaIndexToDeleteFrom; i < absoluteIndex; i++) {
-        deleteReplicaLocations.deleteColumns(HConstants.CATALOG_FAMILY,
-          getServerColumn(i));
-        deleteReplicaLocations.deleteColumns(HConstants.CATALOG_FAMILY,
-          getSeqNumColumn(i));
-        deleteReplicaLocations.deleteColumns(HConstants.CATALOG_FAMILY,
-          getStartCodeColumn(i));
+        deleteReplicaLocations.addColumns(getCatalogFamily(),
+          getServerColumn(i), now);
+        deleteReplicaLocations.addColumns(getCatalogFamily(),
+          getSeqNumColumn(i), now);
+        deleteReplicaLocations.addColumns(getCatalogFamily(),
+          getStartCodeColumn(i), now);
       }
       deleteFromMetaTable(connection, deleteReplicaLocations);
     }
@@ -1171,7 +1315,8 @@ public class MetaTableAccessor {
   public static void addDaughter(final Connection connection,
       final HRegionInfo regionInfo, final ServerName sn, final long openSeqNum)
       throws NotAllMetaRegionsOnlineException, IOException {
-    Put put = new Put(regionInfo.getRegionName());
+    long now = EnvironmentEdgeManager.currentTime();
+    Put put = new Put(regionInfo.getRegionName(), now);
     addRegionInfo(put, regionInfo);
     if (sn != null) {
       addLocation(put, sn, openSeqNum, regionInfo.getReplicaId());
@@ -1273,6 +1418,45 @@ public class MetaTableAccessor {
   }
 
   /**
+   * Update state of the table in meta.
+   * @param connection what we use for update
+   * @param state new state
+   * @throws IOException
+   */
+  public static void updateTableState(Connection connection, TableState state)
+      throws IOException {
+    Put put = makePutFromTableState(state);
+    putToMetaTable(connection, put);
+    LOG.info(
+        "Updated table " + state.getTableName() + " state to " + state.getState() + " in META");
+  }
+
+  /**
+   * Construct PUT for given state
+   * @param state new state
+   */
+  public static Put makePutFromTableState(TableState state) {
+    long time = EnvironmentEdgeManager.currentTime();
+    Put put = new Put(state.getTableName().getName(), time);
+    put.add(getTableFamily(), getStateColumn(), state.convert().toByteArray());
+    return put;
+  }
+
+  /**
+   * Remove state for table from meta
+   * @param connection to use for deletion
+   * @param table to delete state for
+   */
+  public static void deleteTableState(Connection connection, TableName table)
+      throws IOException {
+    long time = EnvironmentEdgeManager.currentTime();
+    Delete delete = new Delete(table.getName());
+    delete.addColumns(getTableFamily(), getStateColumn(), time);
+    deleteFromMetaTable(connection, delete);
+    LOG.info("Deleted table " + table + " state from META");
+  }
+
+  /**
    * Performs an atomic multi-Mutate operation against the given table.
    */
   private static void multiMutate(Table table, byte[] row, Mutation... mutations)
@@ -1337,7 +1521,8 @@ public class MetaTableAccessor {
                                      HRegionInfo regionInfo, ServerName sn, long openSeqNum)
     throws IOException {
     // region replicas are kept in the primary region's row
-    Put put = new Put(getMetaKeyForRegion(regionInfo));
+    long time = EnvironmentEdgeManager.currentTime();
+    Put put = new Put(getMetaKeyForRegion(regionInfo), time);
     addLocation(put, sn, openSeqNum, regionInfo.getReplicaId());
     putToMetaTable(connection, put);
     LOG.info("Updated row " + regionInfo.getRegionNameAsString() +
@@ -1353,7 +1538,9 @@ public class MetaTableAccessor {
   public static void deleteRegion(Connection connection,
                                   HRegionInfo regionInfo)
     throws IOException {
+    long time = EnvironmentEdgeManager.currentTime();
     Delete delete = new Delete(regionInfo.getRegionName());
+    delete.addFamily(getCatalogFamily(), time);
     deleteFromMetaTable(connection, delete);
     LOG.info("Deleted " + regionInfo.getRegionNameAsString());
   }
@@ -1367,8 +1554,11 @@ public class MetaTableAccessor {
   public static void deleteRegions(Connection connection,
                                    List<HRegionInfo> regionsInfo) throws IOException {
     List<Delete> deletes = new ArrayList<Delete>(regionsInfo.size());
+    long time = EnvironmentEdgeManager.currentTime();
     for (HRegionInfo hri: regionsInfo) {
-      deletes.add(new Delete(hri.getRegionName()));
+      Delete e = new Delete(hri.getRegionName());
+      e.addFamily(getCatalogFamily(), time);
+      deletes.add(e);
     }
     deleteFromMetaTable(connection, deletes);
     LOG.info("Deleted " + regionsInfo);
@@ -1388,7 +1578,7 @@ public class MetaTableAccessor {
     List<Mutation> mutation = new ArrayList<Mutation>();
     if (regionsToRemove != null) {
       for (HRegionInfo hri: regionsToRemove) {
-        mutation.add(new Delete(hri.getRegionName()));
+        mutation.add(makeDeleteFromRegionInfo(hri));
       }
     }
     if (regionsToAdd != null) {
@@ -1431,9 +1621,10 @@ public class MetaTableAccessor {
    */
   public static void deleteMergeQualifiers(Connection connection,
                                            final HRegionInfo mergedRegion) throws IOException {
+    long time = EnvironmentEdgeManager.currentTime();
     Delete delete = new Delete(mergedRegion.getRegionName());
-    delete.deleteColumns(HConstants.CATALOG_FAMILY, HConstants.MERGEA_QUALIFIER);
-    delete.deleteColumns(HConstants.CATALOG_FAMILY, HConstants.MERGEB_QUALIFIER);
+    delete.addColumns(getCatalogFamily(), HConstants.MERGEA_QUALIFIER, time);
+    delete.addColumns(getCatalogFamily(), HConstants.MERGEB_QUALIFIER, time);
     deleteFromMetaTable(connection, delete);
     LOG.info("Deleted references in merged region "
       + mergedRegion.getRegionNameAsString() + ", qualifier="
@@ -1443,7 +1634,7 @@ public class MetaTableAccessor {
 
   private static Put addRegionInfo(final Put p, final HRegionInfo hri)
     throws IOException {
-    p.addImmutable(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
+    p.addImmutable(getCatalogFamily(), HConstants.REGIONINFO_QUALIFIER,
       hri.toByteArray());
     return p;
   }
@@ -1452,20 +1643,20 @@ public class MetaTableAccessor {
     // using regionserver's local time as the timestamp of Put.
     // See: HBASE-11536
     long now = EnvironmentEdgeManager.currentTime();
-    p.addImmutable(HConstants.CATALOG_FAMILY, getServerColumn(replicaId), now,
+    p.addImmutable(getCatalogFamily(), getServerColumn(replicaId), now,
       Bytes.toBytes(sn.getHostAndPort()));
-    p.addImmutable(HConstants.CATALOG_FAMILY, getStartCodeColumn(replicaId), now,
+    p.addImmutable(getCatalogFamily(), getStartCodeColumn(replicaId), now,
       Bytes.toBytes(sn.getStartcode()));
-    p.addImmutable(HConstants.CATALOG_FAMILY, getSeqNumColumn(replicaId), now,
+    p.addImmutable(getCatalogFamily(), getSeqNumColumn(replicaId), now,
       Bytes.toBytes(openSeqNum));
     return p;
   }
 
   public static Put addEmptyLocation(final Put p, int replicaId) {
     long now = EnvironmentEdgeManager.currentTime();
-    p.addImmutable(HConstants.CATALOG_FAMILY, getServerColumn(replicaId), now, null);
-    p.addImmutable(HConstants.CATALOG_FAMILY, getStartCodeColumn(replicaId), now, null);
-    p.addImmutable(HConstants.CATALOG_FAMILY, getSeqNumColumn(replicaId), now, null);
+    p.addImmutable(getCatalogFamily(), getServerColumn(replicaId), now, null);
+    p.addImmutable(getCatalogFamily(), getStartCodeColumn(replicaId), now, null);
+    p.addImmutable(getCatalogFamily(), getSeqNumColumn(replicaId), now, null);
     return p;
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionCallable.java
new file mode 100644
index 0000000..3f44927
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionCallable.java
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.client;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+ * A RetryingCallable for generic connection operations.
+ * @param <V> return type
+ */
+abstract class ConnectionCallable<V> implements RetryingCallable<V>, Closeable {
+  protected Connection connection;
+
+  public ConnectionCallable(final Connection connection) {
+    this.connection = connection;
+  }
+
+  @Override
+  public void prepare(boolean reload) throws IOException {
+  }
+
+  @Override
+  public void close() throws IOException {
+  }
+
+  @Override
+  public void throwable(Throwable t, boolean retrying) {
+  }
+
+  @Override
+  public String getExceptionMessageAdditionalDetail() {
+    return "";
+  }
+
+  @Override
+  public long sleep(long pause, int tries) {
+    return ConnectionUtils.getPauseTime(pause, tries);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java
index dbd555c..e986156 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java
@@ -18,6 +18,7 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import javax.annotation.Nullable;
 import java.io.Closeable;
 import java.io.IOException;
 import java.io.InterruptedIOException;
@@ -37,9 +38,12 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.BlockingRpcChannel;
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -60,8 +64,6 @@ import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.AsyncProcess.AsyncRequestFuture;
-import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
-import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase;
 import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicy;
 import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicyFactory;
 import org.apache.hadoop.hbase.client.coprocessor.Batch;
@@ -172,6 +174,7 @@ import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.ExceptionUtil;
+import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
@@ -179,11 +182,6 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.zookeeper.KeeperException;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.protobuf.BlockingRpcChannel;
-import com.google.protobuf.RpcController;
-import com.google.protobuf.ServiceException;
-
 /**
  * An internal, non-instantiable class that manages creation of {@link HConnection}s.
  */
@@ -929,30 +927,7 @@ final class ConnectionManager {
 
     @Override
     public boolean isTableAvailable(final TableName tableName) throws IOException {
-      final AtomicBoolean available = new AtomicBoolean(true);
-      final AtomicInteger regionCount = new AtomicInteger(0);
-      MetaScannerVisitor visitor = new MetaScannerVisitorBase() {
-        @Override
-        public boolean processRow(Result row) throws IOException {
-          HRegionInfo info = MetaScanner.getHRegionInfo(row);
-          if (info != null && !info.isSplitParent()) {
-            if (tableName.equals(info.getTable())) {
-              ServerName server = HRegionInfo.getServerName(row);
-              if (server == null) {
-                available.set(false);
-                return false;
-              }
-              regionCount.incrementAndGet();
-            } else if (tableName.compareTo(info.getTable()) < 0) {
-              // Return if we are done with the current table
-              return false;
-            }
-          }
-          return true;
-        }
-      };
-      MetaScanner.metaScan(this, visitor, tableName);
-      return available.get() && (regionCount.get() > 0);
+      return isTableAvailable(tableName, null);
     }
 
     @Override
@@ -961,44 +936,61 @@ final class ConnectionManager {
     }
 
     @Override
-    public boolean isTableAvailable(final TableName tableName, final byte[][] splitKeys)
+    public boolean isTableAvailable(final TableName tableName, @Nullable final byte[][] splitKeys)
         throws IOException {
-      final AtomicBoolean available = new AtomicBoolean(true);
-      final AtomicInteger regionCount = new AtomicInteger(0);
-      MetaScannerVisitor visitor = new MetaScannerVisitorBase() {
-        @Override
-        public boolean processRow(Result row) throws IOException {
-          HRegionInfo info = MetaScanner.getHRegionInfo(row);
-          if (info != null && !info.isSplitParent()) {
-            if (tableName.equals(info.getTable())) {
-              ServerName server = HRegionInfo.getServerName(row);
-              if (server == null) {
-                available.set(false);
-                return false;
-              }
-              if (!Bytes.equals(info.getStartKey(), HConstants.EMPTY_BYTE_ARRAY)) {
-                for (byte[] splitKey : splitKeys) {
-                  // Just check if the splitkey is available
-                  if (Bytes.equals(info.getStartKey(), splitKey)) {
-                    regionCount.incrementAndGet();
-                    break;
-                  }
-                }
-              } else {
-                // Always empty start row should be counted
-                regionCount.incrementAndGet();
+      try {
+        if (!isTableEnabled(tableName)) {
+          LOG.debug("Table " + tableName + " not enabled");
+          return false;
+        }
+        ClusterConnection connection = getConnectionInternal(getConfiguration());
+        List<Pair<HRegionInfo, ServerName>> locations = MetaTableAccessor
+            .getTableRegionsAndLocations(connection, tableName, true);
+        int notDeployed = 0;
+        int regionCount = 0;
+        for (Pair<HRegionInfo, ServerName> pair : locations) {
+          HRegionInfo info = pair.getFirst();
+          if (pair.getSecond() == null) {
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("Table " + tableName + " has not deployed region " + pair.getFirst()
+                  .getEncodedName());
+            }
+            notDeployed++;
+          } else if (splitKeys != null
+              && !Bytes.equals(info.getStartKey(), HConstants.EMPTY_BYTE_ARRAY)) {
+            for (byte[] splitKey : splitKeys) {
+              // Just check if the splitkey is available
+              if (Bytes.equals(info.getStartKey(), splitKey)) {
+                regionCount++;
+                break;
               }
-            } else if (tableName.compareTo(info.getTable()) < 0) {
-              // Return if we are done with the current table
-              return false;
             }
+          } else {
+            // Always empty start row should be counted
+            regionCount++;
+          }
+        }
+        if (notDeployed > 0) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Table " + tableName + " has " + notDeployed + " regions");
+          }
+          return false;
+        } else if (splitKeys != null && regionCount != splitKeys.length + 1) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Table " + tableName + " expected to have " + (splitKeys.length + 1)
+                + " regions, but only " + regionCount + " available");
+          }
+          return false;
+        } else {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Table " + tableName + " should be available");
           }
           return true;
         }
-      };
-      MetaScanner.metaScan(this, visitor, tableName);
-      // +1 needs to be added so that the empty start row is also taken into account
-      return available.get() && (regionCount.get() == splitKeys.length + 1);
+      } catch (TableNotFoundException tnfe) {
+        LOG.warn("Table " + tableName + " not enabled, it is not exists");
+        return false;
+      }
     }
 
     @Override
@@ -2485,7 +2477,7 @@ final class ConnectionManager {
       GetTableDescriptorsResponse htds;
       try {
         GetTableDescriptorsRequest req =
-          RequestConverter.buildGetTableDescriptorsRequest(tableName);
+            RequestConverter.buildGetTableDescriptorsRequest(tableName);
         htds = master.getTableDescriptors(null, req);
       } catch (ServiceException se) {
         throw ProtobufUtil.getRemoteException(se);
@@ -2510,16 +2502,11 @@ final class ConnectionManager {
 
     @Override
     public TableState getTableState(TableName tableName) throws IOException {
-      MasterKeepAliveConnection master = getKeepAliveMasterService();
-      try {
-        GetTableStateResponse resp = master.getTableState(null,
-                RequestConverter.buildGetTableStateRequest(tableName));
-        return TableState.convert(resp.getTableState());
-      } catch (ServiceException se) {
-        throw ProtobufUtil.getRemoteException(se);
-      } finally {
-        master.close();
-      }
+      ClusterConnection conn = getConnectionInternal(getConfiguration());
+      TableState tableState = MetaTableAccessor.getTableState(conn, tableName);
+      if (tableState == null)
+        throw new TableNotFoundException(tableName);
+      return tableState;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index d14e369..3acaaf9 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -18,6 +18,7 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import java.io.Closeable;
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.net.SocketTimeoutException;
@@ -286,7 +287,12 @@ public class HBaseAdmin implements Admin {
    */
   @Override
   public boolean tableExists(final TableName tableName) throws IOException {
-    return MetaTableAccessor.tableExists(connection, tableName);
+    return executeCallable(new ConnectionCallable<Boolean>(getConnection()) {
+      @Override
+      public Boolean call(int callTimeout) throws ServiceException, IOException {
+        return MetaTableAccessor.tableExists(connection, tableName);
+      }
+    });
   }
 
   public boolean tableExists(final byte[] tableName)
@@ -547,11 +553,11 @@ public class HBaseAdmin implements Admin {
     }
     int numRegs = (splitKeys == null ? 1 : splitKeys.length + 1) * desc.getRegionReplication();
     int prevRegCount = 0;
-    boolean doneWithMetaScan = false;
+    boolean tableWasEnabled = false;
     for (int tries = 0; tries < this.numRetries * this.retryLongerMultiplier;
       ++tries) {
-      if (!doneWithMetaScan) {
-        // Wait for new table to come on-line
+      if (tableWasEnabled) {
+        // Wait all table regions comes online
         final AtomicInteger actualRegCount = new AtomicInteger(0);
         MetaScannerVisitor visitor = new MetaScannerVisitorBase() {
           @Override
@@ -599,17 +605,26 @@ public class HBaseAdmin implements Admin {
             tries = -1;
           }
         } else {
-          doneWithMetaScan = true;
-          tries = -1;
+          return;
         }
-      } else if (isTableEnabled(desc.getTableName())) {
-        return;
       } else {
-        try { // Sleep
-          Thread.sleep(getPauseTime(tries));
-        } catch (InterruptedException e) {
-          throw new InterruptedIOException("Interrupted when waiting" +
-            " for table to be enabled; meta scan was done");
+        try {
+          tableWasEnabled = isTableAvailable(desc.getTableName());
+        } catch (TableNotFoundException tnfe) {
+          LOG.debug(
+              "Table " + desc.getTableName() + " was not enabled, sleeping, still " + numRetries
+                  + " retries left");
+        }
+        if (tableWasEnabled) {
+          // no we will scan meta to ensure all regions are online
+          tries = -1;
+        } else {
+          try { // Sleep
+            Thread.sleep(getPauseTime(tries));
+          } catch (InterruptedException e) {
+            throw new InterruptedIOException("Interrupted when waiting" +
+                " for table to be enabled; meta scan was done");
+          }
         }
       }
     }
@@ -698,24 +713,11 @@ public class HBaseAdmin implements Admin {
     });
 
     int failures = 0;
-    // Wait until all regions deleted
     for (int tries = 0; tries < (this.numRetries * this.retryLongerMultiplier); tries++) {
       try {
-        // Find whether all regions are deleted.
-        List<RegionLocations> regionLations =
-            MetaScanner.listTableRegionLocations(conf, connection, tableName);
-
-        // let us wait until hbase:meta table is updated and
-        // HMaster removes the table from its HTableDescriptors
-        if (regionLations == null || regionLations.size() == 0) {
-          HTableDescriptor htd = getTableDescriptorByTableName(tableName);
-
-          if (htd == null) {
-            // table could not be found in master - we are done.
-            tableExists = false;
-            break;
-          }
-        }
+        tableExists = tableExists(tableName);
+        if (!tableExists)
+          break;
       } catch (IOException ex) {
         failures++;
         if(failures >= numRetries - 1) {           // no more tries left
@@ -1109,9 +1111,17 @@ public class HBaseAdmin implements Admin {
    * @throws IOException if a remote or network exception occurs
    */
   @Override
-  public boolean isTableEnabled(TableName tableName) throws IOException {
+  public boolean isTableEnabled(final TableName tableName) throws IOException {
     checkTableExistence(tableName);
-    return connection.isTableEnabled(tableName);
+    return executeCallable(new ConnectionCallable<Boolean>(getConnection()) {
+      @Override
+      public Boolean call(int callTimeout) throws ServiceException, IOException {
+        TableState tableState = MetaTableAccessor.getTableState(connection, tableName);
+        if (tableState == null)
+          throw new TableNotFoundException(tableName);
+        return tableState.inStates(TableState.State.ENABLED);
+      }
+    });
   }
 
   public boolean isTableEnabled(byte[] tableName) throws IOException {
@@ -2296,10 +2306,15 @@ public class HBaseAdmin implements Admin {
    */
   private TableName checkTableExists(final TableName tableName)
       throws IOException {
-    if (!MetaTableAccessor.tableExists(connection, tableName)) {
-      throw new TableNotFoundException(tableName);
-    }
-    return tableName;
+    return executeCallable(new ConnectionCallable<TableName>(getConnection()) {
+      @Override
+      public TableName call(int callTimeout) throws ServiceException, IOException {
+        if (!MetaTableAccessor.tableExists(connection, tableName)) {
+          throw new TableNotFoundException(tableName);
+        }
+        return tableName;
+      }
+    });
   }
 
   /**
@@ -3667,7 +3682,8 @@ public class HBaseAdmin implements Admin {
     return QuotaRetriever.open(conf, filter);
   }
 
-  private <V> V executeCallable(MasterCallable<V> callable) throws IOException {
+  private <C extends RetryingCallable<V> & Closeable, V> V executeCallable(C callable)
+      throws IOException {
     RpcRetryingCaller<V> caller = rpcCallerFactory.newCaller();
     try {
       return caller.callWithRetries(callable, operationTimeout);

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableState.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableState.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableState.java
index be9b80c..77c90f5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableState.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableState.java
@@ -17,9 +17,11 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import com.google.protobuf.InvalidProtocolBufferException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 
@@ -171,20 +173,30 @@ public class TableState {
   public HBaseProtos.TableState convert() {
     return HBaseProtos.TableState.newBuilder()
         .setState(this.state.convert())
-        .setTable(ProtobufUtil.toProtoTableName(this.tableName))
+        .setTable(ProtobufUtil.toProtoTableName(this.tableName)) // set for backward compatibility
         .setTimestamp(this.timestamp)
             .build();
   }
 
   /**
    * Covert from PB version of TableState
+   *
+   * @param tableName table this state of
    * @param tableState convert from
    * @return POJO
    */
-  public static TableState convert(HBaseProtos.TableState tableState) {
+  public static TableState convert(TableName tableName, HBaseProtos.TableState tableState) {
     TableState.State state = State.convert(tableState.getState());
-    return new TableState(ProtobufUtil.toTableName(tableState.getTable()),
-        state, tableState.getTimestamp());
+    return new TableState(tableName, state, tableState.getTimestamp());
+  }
+
+  public static TableState parseFrom(TableName tableName, byte[] bytes)
+      throws DeserializationException {
+    try {
+      return convert(tableName, HBaseProtos.TableState.parseFrom(bytes));
+    } catch (InvalidProtocolBufferException e) {
+      throw new DeserializationException(e);
+    }
   }
 
   /**
@@ -200,4 +212,36 @@ public class TableState {
     }
     return false;
   }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    TableState that = (TableState) o;
+
+    if (timestamp != that.timestamp) return false;
+    if (state != that.state) return false;
+    if (tableName != null ? !tableName.equals(that.tableName) : that.tableName != null)
+      return false;
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    int result = (int) (timestamp ^ (timestamp >>> 32));
+    result = 31 * result + (tableName != null ? tableName.hashCode() : 0);
+    result = 31 * result + (state != null ? state.hashCode() : 0);
+    return result;
+  }
+
+  @Override
+  public String toString() {
+    return "TableState{" +
+        "timestamp=" + timestamp +
+        ", tableName=" + tableName +
+        ", state=" + state +
+        '}';
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
index 2ee55f7..8a07397 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.hbase;
 
 import static org.apache.hadoop.hbase.io.hfile.BlockType.MAGIC_LENGTH;
 
-import java.nio.ByteBuffer;
 import java.nio.charset.Charset;
 import java.util.Arrays;
 import java.util.Collections;
@@ -451,6 +450,16 @@ public final class HConstants {
   /** The upper-half merge region column qualifier */
   public static final byte[] MERGEB_QUALIFIER = Bytes.toBytes("mergeB");
 
+  /** The catalog family as a string*/
+  public static final String TABLE_FAMILY_STR = "table";
+
+  /** The catalog family */
+  public static final byte [] TABLE_FAMILY = Bytes.toBytes(TABLE_FAMILY_STR);
+
+  /** The serialized table state qualifier */
+  public static final byte[] TABLE_STATE_QUALIFIER = Bytes.toBytes("state");
+
+
   /**
    * The meta table version column qualifier.
    * We keep current version of the meta table in this column in <code>-ROOT-</code>
@@ -738,7 +747,8 @@ public final class HConstants {
   /**
    * The client scanner timeout period in milliseconds.
    */
-  public static final String HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD = "hbase.client.scanner.timeout.period";
+  public static final String HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD =
+      "hbase.client.scanner.timeout.period";
 
   /**
    * Use {@link #HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD} instead.

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java
index 2947f40..a96ef17 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java
@@ -3450,15 +3450,15 @@ public final class HBaseProtos {
      */
     org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getSchemaOrBuilder();
 
-    // optional .TableState.State state = 2 [default = ENABLED];
+    // optional .TableState.State state = 2 [default = ENABLED, deprecated = true];
     /**
-     * <code>optional .TableState.State state = 2 [default = ENABLED];</code>
+     * <code>optional .TableState.State state = 2 [default = ENABLED, deprecated = true];</code>
      */
-    boolean hasState();
+    @java.lang.Deprecated boolean hasState();
     /**
-     * <code>optional .TableState.State state = 2 [default = ENABLED];</code>
+     * <code>optional .TableState.State state = 2 [default = ENABLED, deprecated = true];</code>
      */
-    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State getState();
+    @java.lang.Deprecated org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State getState();
   }
   /**
    * Protobuf type {@code TableDescriptor}
@@ -3601,19 +3601,19 @@ public final class HBaseProtos {
       return schema_;
     }
 
-    // optional .TableState.State state = 2 [default = ENABLED];
+    // optional .TableState.State state = 2 [default = ENABLED, deprecated = true];
     public static final int STATE_FIELD_NUMBER = 2;
     private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State state_;
     /**
-     * <code>optional .TableState.State state = 2 [default = ENABLED];</code>
+     * <code>optional .TableState.State state = 2 [default = ENABLED, deprecated = true];</code>
      */
-    public boolean hasState() {
+    @java.lang.Deprecated public boolean hasState() {
       return ((bitField0_ & 0x00000002) == 0x00000002);
     }
     /**
-     * <code>optional .TableState.State state = 2 [default = ENABLED];</code>
+     * <code>optional .TableState.State state = 2 [default = ENABLED, deprecated = true];</code>
      */
-    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State getState() {
+    @java.lang.Deprecated public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State getState() {
       return state_;
     }
 
@@ -4054,24 +4054,24 @@ public final class HBaseProtos {
         return schemaBuilder_;
       }
 
-      // optional .TableState.State state = 2 [default = ENABLED];
+      // optional .TableState.State state = 2 [default = ENABLED, deprecated = true];
       private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State state_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State.ENABLED;
       /**
-       * <code>optional .TableState.State state = 2 [default = ENABLED];</code>
+       * <code>optional .TableState.State state = 2 [default = ENABLED, deprecated = true];</code>
        */
-      public boolean hasState() {
+      @java.lang.Deprecated public boolean hasState() {
         return ((bitField0_ & 0x00000002) == 0x00000002);
       }
       /**
-       * <code>optional .TableState.State state = 2 [default = ENABLED];</code>
+       * <code>optional .TableState.State state = 2 [default = ENABLED, deprecated = true];</code>
        */
-      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State getState() {
+      @java.lang.Deprecated public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State getState() {
         return state_;
       }
       /**
-       * <code>optional .TableState.State state = 2 [default = ENABLED];</code>
+       * <code>optional .TableState.State state = 2 [default = ENABLED, deprecated = true];</code>
        */
-      public Builder setState(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State value) {
+      @java.lang.Deprecated public Builder setState(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State value) {
         if (value == null) {
           throw new NullPointerException();
         }
@@ -4081,9 +4081,9 @@ public final class HBaseProtos {
         return this;
       }
       /**
-       * <code>optional .TableState.State state = 2 [default = ENABLED];</code>
+       * <code>optional .TableState.State state = 2 [default = ENABLED, deprecated = true];</code>
        */
-      public Builder clearState() {
+      @java.lang.Deprecated public Builder clearState() {
         bitField0_ = (bitField0_ & ~0x00000002);
         state_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State.ENABLED;
         onChanged();
@@ -18197,52 +18197,52 @@ public final class HBaseProtos {
       "TableState.State\022\031\n\005table\030\002 \002(\0132\n.TableN" +
       "ame\022\021\n\ttimestamp\030\003 \001(\004\"?\n\005State\022\013\n\007ENABL" +
       "ED\020\000\022\014\n\010DISABLED\020\001\022\r\n\tDISABLING\020\002\022\014\n\010ENA",
-      "BLING\020\003\"Z\n\017TableDescriptor\022\034\n\006schema\030\001 \002" +
-      "(\0132\014.TableSchema\022)\n\005state\030\002 \001(\0162\021.TableS" +
-      "tate.State:\007ENABLED\"o\n\022ColumnFamilySchem" +
-      "a\022\014\n\004name\030\001 \002(\014\022#\n\nattributes\030\002 \003(\0132\017.By" +
-      "tesBytesPair\022&\n\rconfiguration\030\003 \003(\0132\017.Na" +
-      "meStringPair\"\232\001\n\nRegionInfo\022\021\n\tregion_id" +
-      "\030\001 \002(\004\022\036\n\ntable_name\030\002 \002(\0132\n.TableName\022\021" +
-      "\n\tstart_key\030\003 \001(\014\022\017\n\007end_key\030\004 \001(\014\022\017\n\007of" +
-      "fline\030\005 \001(\010\022\r\n\005split\030\006 \001(\010\022\025\n\nreplica_id" +
-      "\030\007 \001(\005:\0010\"1\n\014FavoredNodes\022!\n\014favored_nod",
-      "e\030\001 \003(\0132\013.ServerName\"\225\001\n\017RegionSpecifier" +
-      "\0222\n\004type\030\001 \002(\0162$.RegionSpecifier.RegionS" +
-      "pecifierType\022\r\n\005value\030\002 \002(\014\"?\n\023RegionSpe" +
-      "cifierType\022\017\n\013REGION_NAME\020\001\022\027\n\023ENCODED_R" +
-      "EGION_NAME\020\002\"%\n\tTimeRange\022\014\n\004from\030\001 \001(\004\022" +
-      "\n\n\002to\030\002 \001(\004\"A\n\nServerName\022\021\n\thost_name\030\001" +
-      " \002(\t\022\014\n\004port\030\002 \001(\r\022\022\n\nstart_code\030\003 \001(\004\"\033" +
-      "\n\013Coprocessor\022\014\n\004name\030\001 \002(\t\"-\n\016NameStrin" +
-      "gPair\022\014\n\004name\030\001 \002(\t\022\r\n\005value\030\002 \002(\t\",\n\rNa" +
-      "meBytesPair\022\014\n\004name\030\001 \002(\t\022\r\n\005value\030\002 \001(\014",
-      "\"/\n\016BytesBytesPair\022\r\n\005first\030\001 \002(\014\022\016\n\006sec" +
-      "ond\030\002 \002(\014\",\n\rNameInt64Pair\022\014\n\004name\030\001 \001(\t" +
-      "\022\r\n\005value\030\002 \001(\003\"\314\001\n\023SnapshotDescription\022" +
-      "\014\n\004name\030\001 \002(\t\022\r\n\005table\030\002 \001(\t\022\030\n\rcreation" +
-      "_time\030\003 \001(\003:\0010\022.\n\004type\030\004 \001(\0162\031.SnapshotD" +
-      "escription.Type:\005FLUSH\022\017\n\007version\030\005 \001(\005\022" +
-      "\r\n\005owner\030\006 \001(\t\".\n\004Type\022\014\n\010DISABLED\020\000\022\t\n\005" +
-      "FLUSH\020\001\022\r\n\tSKIPFLUSH\020\002\"}\n\024ProcedureDescr" +
-      "iption\022\021\n\tsignature\030\001 \002(\t\022\020\n\010instance\030\002 " +
-      "\001(\t\022\030\n\rcreation_time\030\003 \001(\003:\0010\022&\n\rconfigu",
-      "ration\030\004 \003(\0132\017.NameStringPair\"\n\n\010EmptyMs" +
-      "g\"\033\n\007LongMsg\022\020\n\010long_msg\030\001 \002(\003\"\037\n\tDouble" +
-      "Msg\022\022\n\ndouble_msg\030\001 \002(\001\"\'\n\rBigDecimalMsg" +
-      "\022\026\n\016bigdecimal_msg\030\001 \002(\014\"5\n\004UUID\022\026\n\016leas" +
-      "t_sig_bits\030\001 \002(\004\022\025\n\rmost_sig_bits\030\002 \002(\004\"" +
-      "K\n\023NamespaceDescriptor\022\014\n\004name\030\001 \002(\014\022&\n\r" +
-      "configuration\030\002 \003(\0132\017.NameStringPair\"$\n\020" +
-      "RegionServerInfo\022\020\n\010infoPort\030\001 \001(\005*r\n\013Co" +
-      "mpareType\022\010\n\004LESS\020\000\022\021\n\rLESS_OR_EQUAL\020\001\022\t" +
-      "\n\005EQUAL\020\002\022\r\n\tNOT_EQUAL\020\003\022\024\n\020GREATER_OR_E",
-      "QUAL\020\004\022\013\n\007GREATER\020\005\022\t\n\005NO_OP\020\006*n\n\010TimeUn" +
-      "it\022\017\n\013NANOSECONDS\020\001\022\020\n\014MICROSECONDS\020\002\022\020\n" +
-      "\014MILLISECONDS\020\003\022\013\n\007SECONDS\020\004\022\013\n\007MINUTES\020" +
-      "\005\022\t\n\005HOURS\020\006\022\010\n\004DAYS\020\007B>\n*org.apache.had" +
-      "oop.hbase.protobuf.generatedB\013HBaseProto" +
-      "sH\001\240\001\001"
+      "BLING\020\003\"^\n\017TableDescriptor\022\034\n\006schema\030\001 \002" +
+      "(\0132\014.TableSchema\022-\n\005state\030\002 \001(\0162\021.TableS" +
+      "tate.State:\007ENABLEDB\002\030\001\"o\n\022ColumnFamilyS" +
+      "chema\022\014\n\004name\030\001 \002(\014\022#\n\nattributes\030\002 \003(\0132" +
+      "\017.BytesBytesPair\022&\n\rconfiguration\030\003 \003(\0132" +
+      "\017.NameStringPair\"\232\001\n\nRegionInfo\022\021\n\tregio" +
+      "n_id\030\001 \002(\004\022\036\n\ntable_name\030\002 \002(\0132\n.TableNa" +
+      "me\022\021\n\tstart_key\030\003 \001(\014\022\017\n\007end_key\030\004 \001(\014\022\017" +
+      "\n\007offline\030\005 \001(\010\022\r\n\005split\030\006 \001(\010\022\025\n\nreplic" +
+      "a_id\030\007 \001(\005:\0010\"1\n\014FavoredNodes\022!\n\014favored",
+      "_node\030\001 \003(\0132\013.ServerName\"\225\001\n\017RegionSpeci" +
+      "fier\0222\n\004type\030\001 \002(\0162$.RegionSpecifier.Reg" +
+      "ionSpecifierType\022\r\n\005value\030\002 \002(\014\"?\n\023Regio" +
+      "nSpecifierType\022\017\n\013REGION_NAME\020\001\022\027\n\023ENCOD" +
+      "ED_REGION_NAME\020\002\"%\n\tTimeRange\022\014\n\004from\030\001 " +
+      "\001(\004\022\n\n\002to\030\002 \001(\004\"A\n\nServerName\022\021\n\thost_na" +
+      "me\030\001 \002(\t\022\014\n\004port\030\002 \001(\r\022\022\n\nstart_code\030\003 \001" +
+      "(\004\"\033\n\013Coprocessor\022\014\n\004name\030\001 \002(\t\"-\n\016NameS" +
+      "tringPair\022\014\n\004name\030\001 \002(\t\022\r\n\005value\030\002 \002(\t\"," +
+      "\n\rNameBytesPair\022\014\n\004name\030\001 \002(\t\022\r\n\005value\030\002",
+      " \001(\014\"/\n\016BytesBytesPair\022\r\n\005first\030\001 \002(\014\022\016\n" +
+      "\006second\030\002 \002(\014\",\n\rNameInt64Pair\022\014\n\004name\030\001" +
+      " \001(\t\022\r\n\005value\030\002 \001(\003\"\314\001\n\023SnapshotDescript" +
+      "ion\022\014\n\004name\030\001 \002(\t\022\r\n\005table\030\002 \001(\t\022\030\n\rcrea" +
+      "tion_time\030\003 \001(\003:\0010\022.\n\004type\030\004 \001(\0162\031.Snaps" +
+      "hotDescription.Type:\005FLUSH\022\017\n\007version\030\005 " +
+      "\001(\005\022\r\n\005owner\030\006 \001(\t\".\n\004Type\022\014\n\010DISABLED\020\000" +
+      "\022\t\n\005FLUSH\020\001\022\r\n\tSKIPFLUSH\020\002\"}\n\024ProcedureD" +
+      "escription\022\021\n\tsignature\030\001 \002(\t\022\020\n\010instanc" +
+      "e\030\002 \001(\t\022\030\n\rcreation_time\030\003 \001(\003:\0010\022&\n\rcon",
+      "figuration\030\004 \003(\0132\017.NameStringPair\"\n\n\010Emp" +
+      "tyMsg\"\033\n\007LongMsg\022\020\n\010long_msg\030\001 \002(\003\"\037\n\tDo" +
+      "ubleMsg\022\022\n\ndouble_msg\030\001 \002(\001\"\'\n\rBigDecima" +
+      "lMsg\022\026\n\016bigdecimal_msg\030\001 \002(\014\"5\n\004UUID\022\026\n\016" +
+      "least_sig_bits\030\001 \002(\004\022\025\n\rmost_sig_bits\030\002 " +
+      "\002(\004\"K\n\023NamespaceDescriptor\022\014\n\004name\030\001 \002(\014" +
+      "\022&\n\rconfiguration\030\002 \003(\0132\017.NameStringPair" +
+      "\"$\n\020RegionServerInfo\022\020\n\010infoPort\030\001 \001(\005*r" +
+      "\n\013CompareType\022\010\n\004LESS\020\000\022\021\n\rLESS_OR_EQUAL" +
+      "\020\001\022\t\n\005EQUAL\020\002\022\r\n\tNOT_EQUAL\020\003\022\024\n\020GREATER_",
+      "OR_EQUAL\020\004\022\013\n\007GREATER\020\005\022\t\n\005NO_OP\020\006*n\n\010Ti" +
+      "meUnit\022\017\n\013NANOSECONDS\020\001\022\020\n\014MICROSECONDS\020" +
+      "\002\022\020\n\014MILLISECONDS\020\003\022\013\n\007SECONDS\020\004\022\013\n\007MINU" +
+      "TES\020\005\022\t\n\005HOURS\020\006\022\010\n\004DAYS\020\007B>\n*org.apache" +
+      ".hadoop.hbase.protobuf.generatedB\013HBaseP" +
+      "rotosH\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-protocol/src/main/protobuf/HBase.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/HBase.proto b/hbase-protocol/src/main/protobuf/HBase.proto
index c3c8c6a..1566846 100644
--- a/hbase-protocol/src/main/protobuf/HBase.proto
+++ b/hbase-protocol/src/main/protobuf/HBase.proto
@@ -55,14 +55,14 @@ message TableState {
   }
   // This is the table's state.
   required State state = 1;
-  required TableName table = 2;
+  required TableName table = 2 [deprecated = true];
   optional uint64 timestamp = 3;
 }
 
 /** On HDFS representation of table state. */
 message TableDescriptor {
   required TableSchema schema = 1;
-  optional TableState.State state = 2 [ default = ENABLED ];
+  optional TableState.State state = 2 [ default = ENABLED, deprecated = true ];
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-server/src/main/java/org/apache/hadoop/hbase/TableDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/TableDescriptor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/TableDescriptor.java
index d27bfb7..d1935db 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/TableDescriptor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/TableDescriptor.java
@@ -17,12 +17,13 @@
  */
 package org.apache.hadoop.hbase;
 
+import javax.annotation.Nullable;
 import java.io.IOException;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.protobuf.InvalidProtocolBufferException;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@@ -35,15 +36,23 @@ import org.apache.hadoop.hbase.regionserver.BloomType;
 @InterfaceAudience.Private
 public class TableDescriptor {
   private HTableDescriptor hTableDescriptor;
+  /**
+   * Don't use, state was moved to meta, use MetaTableAccessor instead
+   * @deprecated state was moved to meta
+   */
+  @Deprecated
+  @Nullable
   private TableState.State tableState;
 
   /**
    * Creates TableDescriptor with all fields.
    * @param hTableDescriptor HTableDescriptor to use
    * @param tableState table state
+   * @deprecated state was moved to meta
    */
+  @Deprecated
   public TableDescriptor(HTableDescriptor hTableDescriptor,
-      TableState.State tableState) {
+      @Nullable TableState.State tableState) {
     this.hTableDescriptor = hTableDescriptor;
     this.tableState = tableState;
   }
@@ -69,22 +78,35 @@ public class TableDescriptor {
     this.hTableDescriptor = hTableDescriptor;
   }
 
+  /**
+   * @return table state
+   * @deprecated state was moved to meta
+   */
+  @Deprecated
+  @Nullable
   public TableState.State getTableState() {
     return tableState;
   }
 
-  public void setTableState(TableState.State tableState) {
+  /**
+   * @param tableState state to set for table
+   * @deprecated state was moved to meta
+   */
+  @Deprecated
+  public void setTableState(@Nullable TableState.State tableState) {
     this.tableState = tableState;
   }
 
   /**
    * Convert to PB.
    */
+  @SuppressWarnings("deprecation")
   public HBaseProtos.TableDescriptor convert() {
-    return HBaseProtos.TableDescriptor.newBuilder()
-        .setSchema(hTableDescriptor.convert())
-        .setState(tableState.convert())
-        .build();
+    HBaseProtos.TableDescriptor.Builder builder = HBaseProtos.TableDescriptor.newBuilder()
+        .setSchema(hTableDescriptor.convert());
+    if (tableState!= null)
+      builder.setState(tableState.convert());
+    return builder.build();
   }
 
   /**
@@ -92,7 +114,9 @@ public class TableDescriptor {
    */
   public static TableDescriptor convert(HBaseProtos.TableDescriptor proto) {
     HTableDescriptor hTableDescriptor = HTableDescriptor.convert(proto.getSchema());
-    TableState.State state = TableState.State.convert(proto.getState());
+    TableState.State state = proto.hasState()?
+        TableState.State.convert(proto.getState())
+        :null;
     return new TableDescriptor(hTableDescriptor, state);
   }
 
@@ -170,6 +194,17 @@ public class TableDescriptor {
                 .setBloomFilterType(BloomType.NONE)
                     // Enable cache of data blocks in L1 if more than one caching tier deployed:
                     // e.g. if using CombinedBlockCache (BucketCache).
+                .setCacheDataInL1(true),
+            new HColumnDescriptor(HConstants.TABLE_FAMILY)
+                // Ten is arbitrary number.  Keep versions to help debugging.
+                .setMaxVersions(10)
+                .setInMemory(true)
+                .setBlocksize(8 * 1024)
+                .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+                    // Disable blooms for meta.  Needs work.  Seems to mess w/ getClosestOrBefore.
+                .setBloomFilterType(BloomType.NONE)
+                    // Enable cache of data blocks in L1 if more than one caching tier deployed:
+                    // e.g. if using CombinedBlockCache (BucketCache).
                 .setCacheDataInL1(true)
         }) {
     };

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
index fac1ac9..064771c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
@@ -198,6 +198,7 @@ public class RpcServer implements RpcServerInterface {
 
   protected final InetSocketAddress bindAddress;
   protected int port;                             // port we listen on
+  protected InetSocketAddress address;            // inet address we listen on
   private int readThreads;                        // number of read threads
   protected int maxIdleTime;                      // the maximum idle time after
                                                   // which a client may be
@@ -528,6 +529,7 @@ public class RpcServer implements RpcServerInterface {
       // Bind the server socket to the binding addrees (can be different from the default interface)
       bind(acceptChannel.socket(), bindAddress, backlogLength);
       port = acceptChannel.socket().getLocalPort(); //Could be an ephemeral port
+      address = (InetSocketAddress)acceptChannel.socket().getLocalSocketAddress();
       // create a selector;
       selector= Selector.open();
 
@@ -754,7 +756,7 @@ public class RpcServer implements RpcServerInterface {
     }
 
     InetSocketAddress getAddress() {
-      return (InetSocketAddress)acceptChannel.socket().getLocalSocketAddress();
+      return address;
     }
 
     void doAccept(SelectionKey key) throws IOException, OutOfMemoryError {

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/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
index 4d9ff13..f861529 100644
--- 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
@@ -54,7 +54,6 @@ 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.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
@@ -99,7 +98,7 @@ import com.google.common.annotations.VisibleForTesting;
 public class AssignmentManager {
   private static final Log LOG = LogFactory.getLog(AssignmentManager.class);
 
-  protected final Server server;
+  protected final MasterServices server;
 
   private ServerManager serverManager;
 
@@ -130,8 +129,8 @@ public class AssignmentManager {
   private final int maximumAttempts;
 
   /**
-   * The sleep time for which the assignment will wait before retrying in case of hbase:meta assignment
-   * failure due to lack of availability of region plan or bad region plan
+   * The sleep time for which the assignment will wait before retrying in case of
+   * hbase:meta assignment failure due to lack of availability of region plan or bad region plan
    */
   private final long sleepTimeBeforeRetryingMetaAssignment;
 
@@ -209,7 +208,7 @@ public class AssignmentManager {
    * @param tableLockManager TableLock manager
    * @throws IOException
    */
-  public AssignmentManager(Server server, ServerManager serverManager,
+  public AssignmentManager(MasterServices server, ServerManager serverManager,
       final LoadBalancer balancer,
       final ExecutorService service, MetricsMaster metricsMaster,
       final TableLockManager tableLockManager,
@@ -1570,7 +1569,7 @@ public class AssignmentManager {
             TableState.State.ENABLING);
 
     // Region assignment from META
-    List<Result> results = MetaTableAccessor.fullScanOfMeta(server.getConnection());
+    List<Result> results = MetaTableAccessor.fullScanRegions(server.getConnection());
     // Get any new but slow to checkin region server that joined the cluster
     Set<ServerName> onlineServers = serverManager.getOnlineServers().keySet();
     // Set of offline servers to be returned

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/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 020d6fb..61a1c66 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
@@ -430,6 +430,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
     return connector.getLocalPort();
   }
 
+  @Override
+  protected TableDescriptors getFsTableDescriptors() throws IOException {
+    return super.getFsTableDescriptors();
+  }
+
   /**
    * For compatibility, if failed with regionserver credentials, try the master one
    */
@@ -629,9 +634,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
 
     // Invalidate all write locks held previously
     this.tableLockManager.reapWriteLocks();
-
     this.tableStateManager = new TableStateManager(this);
-    this.tableStateManager.start();
 
     status.setStatus("Initializing ZK system trackers");
     initializeZKBasedSystemTrackers();
@@ -869,7 +872,10 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
       assigned++;
     }
 
-    if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) enableMeta(TableName.META_TABLE_NAME);
+    if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID)
+      getTableStateManager().setTableState(TableName.META_TABLE_NAME, TableState.State.ENABLED);
+    // TODO: should we prevent from using state manager before meta was initialized?
+    // tableStateManager.start();
 
     if ((RecoveryMode.LOG_REPLAY == this.getMasterFileSystem().getLogRecoveryMode())
         && (!previouslyFailedMetaRSs.isEmpty())) {
@@ -878,6 +884,9 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
       this.fileSystemManager.splitMetaLog(previouslyFailedMetaRSs);
     }
 
+    this.assignmentManager.setEnabledTable(TableName.META_TABLE_NAME);
+    tableStateManager.start();
+
     // Make sure a hbase:meta location is set. We need to enable SSH here since
     // if the meta region server is died at this time, we need it to be re-assigned
     // by SSH so that system tables can be assigned.
@@ -934,13 +943,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
     }
   }
 
-  private void enableMeta(TableName metaTableName) {
-    if (!this.tableStateManager.isTableState(metaTableName,
-            TableState.State.ENABLED)) {
-      this.assignmentManager.setEnabledTable(metaTableName);
-    }
-  }
-
   /**
    * This function returns a set of region server names under hbase:meta recovering region ZK node
    * @return Set of meta server names which were recorded in ZK
@@ -1173,7 +1175,8 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
           if (rpCount < plans.size() &&
               // if performing next balance exceeds cutoff time, exit the loop
               (System.currentTimeMillis() + (totalRegPlanExecTime / rpCount)) > cutoffTime) {
-            //TODO: After balance, there should not be a cutoff time (keeping it as a security net for now)
+            //TODO: After balance, there should not be a cutoff time (keeping it as
+            // a security net for now)
             LOG.debug("No more balancing till next balance run; maximumBalanceTime=" +
               maximumBalanceTime);
             break;
@@ -1463,7 +1466,8 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
           LOG.fatal("Failed to become active master", t);
           // HBASE-5680: Likely hadoop23 vs hadoop 20.x/1.x incompatibility
           if (t instanceof NoClassDefFoundError &&
-              t.getMessage().contains("org/apache/hadoop/hdfs/protocol/FSConstants$SafeModeAction")) {
+              t.getMessage()
+                  .contains("org/apache/hadoop/hdfs/protocol/FSConstants$SafeModeAction")) {
             // improved error message for this special case
             abort("HBase is having a problem with its Hadoop jars.  You may need to "
               + "recompile HBase against Hadoop version "
@@ -2192,15 +2196,18 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
         }
 
         for (HTableDescriptor desc: htds) {
-          if (includeSysTables || !desc.getTableName().isSystemTable()) {
+          if (tableStateManager.isTablePresent(desc.getTableName())
+              && (includeSysTables || !desc.getTableName().isSystemTable())) {
             descriptors.add(desc);
           }
         }
       } else {
         for (TableName s: tableNameList) {
-          HTableDescriptor desc = tableDescriptors.get(s);
-          if (desc != null) {
-            descriptors.add(desc);
+          if (tableStateManager.isTablePresent(s)) {
+            HTableDescriptor desc = tableDescriptors.get(s);
+            if (desc != null) {
+              descriptors.add(desc);
+            }
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java
index f979403..c4eecfa 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java
@@ -18,6 +18,7 @@
  */
 package org.apache.hadoop.hbase.master;
 
+import javax.annotation.Nullable;
 import java.util.List;
 import java.util.Map;
 
@@ -88,6 +89,7 @@ public interface LoadBalancer extends Configurable, Stoppable, ConfigurationObse
    * @param servers
    * @return List of plans
    */
+  @Nullable
   Map<ServerName, List<HRegionInfo>> retainAssignment(
     Map<HRegionInfo, ServerName> regions,
     List<ServerName> servers

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
index 4d72312..78e4c11 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
@@ -471,7 +471,7 @@ public class MasterFileSystem {
     // we should get them from registry.
     FSTableDescriptors fsd = new FSTableDescriptors(c, fs, rd);
     fsd.createTableDescriptor(
-        new TableDescriptor(fsd.get(TableName.META_TABLE_NAME), TableState.State.ENABLING));
+        new TableDescriptor(fsd.get(TableName.META_TABLE_NAME)));
 
     return rd;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index 0e81461..4af53a4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -850,8 +850,6 @@ public class MasterRpcServices extends RSRpcServices
       TableName tableName = ProtobufUtil.toTableName(request.getTableName());
       TableState.State state = master.getTableStateManager()
               .getTableState(tableName);
-      if (state == null)
-        throw new TableNotFoundException(tableName);
       MasterProtos.GetTableStateResponse.Builder builder =
               MasterProtos.GetTableStateResponse.newBuilder();
       builder.setTableState(new TableState(tableName, state).convert());

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java
index 9dd412c..df61b45 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java
@@ -230,7 +230,8 @@ public class RegionStateStore {
         }
       }
       // Called when meta is not on master
-      multiHConnection.processBatchCallback(Arrays.asList(put), TableName.META_TABLE_NAME, null, null);
+      multiHConnection.processBatchCallback(Arrays.asList(put),
+          TableName.META_TABLE_NAME, null, null);
 
     } catch (IOException ioe) {
       LOG.error("Failed to persist region state " + newState, ioe);


[35/50] [abbrv] hbase git commit: HBASE-13009: HBase REST UI inaccessible

Posted by jm...@apache.org.
HBASE-13009: HBase REST UI inaccessible

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


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

Branch: refs/heads/hbase-11339
Commit: 8a6e9827a2b35e4b967dc7690455ae941c49f18b
Parents: 9c66afb
Author: Aditya Kishore <ad...@mapr.com>
Authored: Tue Feb 10 16:07:48 2015 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Tue Feb 10 19:09:08 2015 -0800

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/8a6e9827/hbase-assembly/src/main/assembly/components.xml
----------------------------------------------------------------------
diff --git a/hbase-assembly/src/main/assembly/components.xml b/hbase-assembly/src/main/assembly/components.xml
index 520e85a..c6315ad 100644
--- a/hbase-assembly/src/main/assembly/components.xml
+++ b/hbase-assembly/src/main/assembly/components.xml
@@ -85,6 +85,12 @@
       <fileMode>0644</fileMode>
       <directoryMode>0755</directoryMode>
     </fileSet>
+    <fileSet>
+      <directory>${project.basedir}/../hbase-rest/target/hbase-webapps</directory>
+      <outputDirectory>hbase-webapps/</outputDirectory>
+      <fileMode>0644</fileMode>
+      <directoryMode>0755</directoryMode>
+    </fileSet>
     <!-- Include native libraries -->
     <fileSet>
         <directory>${project.basedir}/../hbase-server/target/native</directory>


[22/50] [abbrv] hbase git commit: HBASE-12984: SSL cannot be used by the InfoPort in branch-1

Posted by jm...@apache.org.
HBASE-12984: SSL cannot be used by the InfoPort in branch-1


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

Branch: refs/heads/hbase-11339
Commit: 1f830bea892df01bd657aeaab7d34926dbc372b4
Parents: 9d6b237
Author: Esteban Gutierrez <es...@cloudera.com>
Authored: Sat Feb 7 00:16:23 2015 -0800
Committer: Enis Soztutar <en...@apache.org>
Committed: Mon Feb 9 16:43:37 2015 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/http/HttpConfig.java    | 25 +++++++++--------
 .../apache/hadoop/hbase/http/InfoServer.java    | 24 +++++++++++-----
 .../hadoop/hbase/TestHBaseTestingUtility.java   | 29 ++++++++++++++++++++
 3 files changed, 60 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/1f830bea/hbase-server/src/main/java/org/apache/hadoop/hbase/http/HttpConfig.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/http/HttpConfig.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/HttpConfig.java
index d6180b5..4ed7fbd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/http/HttpConfig.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/HttpConfig.java
@@ -27,13 +27,13 @@ import org.apache.hadoop.conf.Configuration;
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
 public class HttpConfig {
-  private static Policy policy;
+  private Policy policy;
   public enum Policy {
     HTTP_ONLY,
     HTTPS_ONLY,
     HTTP_AND_HTTPS;
 
-    public static Policy fromString(String value) {
+    public Policy fromString(String value) {
       if (HTTPS_ONLY.name().equalsIgnoreCase(value)) {
         return HTTPS_ONLY;
       } else if (HTTP_AND_HTTPS.name().equalsIgnoreCase(value)) {
@@ -51,27 +51,30 @@ public class HttpConfig {
     }
   }
 
-  static {
-    Configuration conf = new Configuration();
+   public HttpConfig(final Configuration conf) {
     boolean sslEnabled = conf.getBoolean(
-            ServerConfigurationKeys.HBASE_SSL_ENABLED_KEY,
-            ServerConfigurationKeys.HBASE_SSL_ENABLED_DEFAULT);
+      ServerConfigurationKeys.HBASE_SSL_ENABLED_KEY,
+      ServerConfigurationKeys.HBASE_SSL_ENABLED_DEFAULT);
     policy = sslEnabled ? Policy.HTTPS_ONLY : Policy.HTTP_ONLY;
+    if (sslEnabled) {
+      conf.addResource("ssl-server.xml");
+      conf.addResource("ssl-client.xml");
+    }
   }
 
-  public static void setPolicy(Policy policy) {
-    HttpConfig.policy = policy;
+  public void setPolicy(Policy policy) {
+    this.policy = policy;
   }
 
-  public static boolean isSecure() {
+  public boolean isSecure() {
     return policy == Policy.HTTPS_ONLY;
   }
 
-  public static String getSchemePrefix() {
+  public String getSchemePrefix() {
     return (isSecure()) ? "https://" : "http://";
   }
 
-  public static String getScheme(Policy policy) {
+  public String getScheme(Policy policy) {
     return policy == Policy.HTTPS_ONLY ? "https://" : "http://";
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1f830bea/hbase-server/src/main/java/org/apache/hadoop/hbase/http/InfoServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/http/InfoServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/InfoServer.java
index ffaaeaa..e9b76bc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/http/InfoServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/InfoServer.java
@@ -54,15 +54,25 @@ public class InfoServer {
   public InfoServer(String name, String bindAddress, int port, boolean findPort,
       final Configuration c)
   throws IOException {
+    HttpConfig httpConfig = new HttpConfig(c);
     HttpServer.Builder builder =
       new org.apache.hadoop.hbase.http.HttpServer.Builder();
-    builder
-      .setName(name)
-      .addEndpoint(URI.create("http://" + bindAddress + ":" + port))
-      .setAppDir(HBASE_APP_DIR).setFindPort(findPort).setConf(c);
-    String logDir = System.getProperty("hbase.log.dir");
-    if (logDir != null) {
-      builder.setLogDir(logDir);
+
+      builder.setName(name).addEndpoint(URI.create(httpConfig.getSchemePrefix() +
+        bindAddress + ":" +
+        port)).setAppDir(HBASE_APP_DIR).setFindPort(findPort).setConf(c);
+      String logDir = System.getProperty("hbase.log.dir");
+      if (logDir != null) {
+        builder.setLogDir(logDir);
+      }
+    if (httpConfig.isSecure()) {
+    builder.keyPassword(c.get("ssl.server.keystore.keypassword"))
+      .keyStore(c.get("ssl.server.keystore.location"),
+        c.get("ssl.server.keystore.password"),
+        c.get("ssl.server.keystore.type", "jks"))
+      .trustStore(c.get("ssl.server.truststore.location"),
+        c.get("ssl.server.truststore.password"),
+        c.get("ssl.server.truststore.type", "jks"));
     }
     this.httpServer = builder.build();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1f830bea/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java
index 56720a3..f3e3dc2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java
@@ -26,6 +26,7 @@ import static org.junit.Assert.assertTrue;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
@@ -36,8 +37,10 @@ import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hbase.http.ssl.KeyStoreTestUtil;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import java.io.File;
 
 /**
  * Test our testing utility class
@@ -137,6 +140,32 @@ public class TestHBaseTestingUtility {
     }
   }
 
+  @Test
+  public void testMiniClusterWithSSLOn() throws Exception {
+    final String BASEDIR = System.getProperty("test.build.dir",
+        "target/test-dir") + "/" + TestHBaseTestingUtility.class.getSimpleName();
+    String sslConfDir = KeyStoreTestUtil.getClasspathDir(TestHBaseTestingUtility.class);
+    String keystoresDir = new File(BASEDIR).getAbsolutePath();
+
+    HBaseTestingUtility hbt = new HBaseTestingUtility();
+    File base = new File(BASEDIR);
+    FileUtil.fullyDelete(base);
+    base.mkdirs();
+
+    KeyStoreTestUtil.setupSSLConfig(keystoresDir, sslConfDir, hbt.getConfiguration(), false);
+
+    hbt.getConfiguration().set("hbase.ssl.enabled", "true");
+    hbt.getConfiguration().addResource("ssl-server.xml");
+    hbt.getConfiguration().addResource("ssl-client.xml");
+
+    MiniHBaseCluster cluster = hbt.startMiniCluster();
+    try {
+      assertEquals(1, cluster.getLiveRegionServerThreads().size());
+    } finally {
+      hbt.shutdownMiniCluster();
+    }
+  }
+
   /**
    *  Test that we can start and stop multiple time a cluster
    *   with the same HBaseTestingUtility.


[05/50] [abbrv] hbase git commit: HBASE-12961 Fix negative values in read and write region server metrics.

Posted by jm...@apache.org.
HBASE-12961 Fix negative values in read and write region server metrics.


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

Branch: refs/heads/hbase-11339
Commit: 4f472062a436726f686608023126dd8d5cc9c9bc
Parents: 37ef793
Author: Victoria Dudin <vd...@vdudin-mbp.local>
Authored: Tue Feb 3 09:47:46 2015 -0800
Committer: Elliott Clark <ec...@apache.org>
Committed: Wed Feb 4 10:32:09 2015 -0800

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/hbase/ServerLoad.java |  8 ++++----
 .../java/org/apache/hadoop/hbase/TestServerLoad.java  | 14 +++++++++++---
 2 files changed, 15 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/4f472062/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java
index 18e5d67..9141659 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java
@@ -45,8 +45,8 @@ public class ServerLoad {
   private int storefileSizeMB = 0;
   private int memstoreSizeMB = 0;
   private int storefileIndexSizeMB = 0;
-  private int readRequestsCount = 0;
-  private int writeRequestsCount = 0;
+  private long readRequestsCount = 0;
+  private long writeRequestsCount = 0;
   private int rootIndexSizeKB = 0;
   private int totalStaticIndexSizeKB = 0;
   private int totalStaticBloomSizeKB = 0;
@@ -138,11 +138,11 @@ public class ServerLoad {
     return storefileIndexSizeMB;
   }
 
-  public int getReadRequestsCount() {
+  public long getReadRequestsCount() {
     return readRequestsCount;
   }
 
-  public int getWriteRequestsCount() {
+  public long getWriteRequestsCount() {
     return writeRequestsCount;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/4f472062/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerLoad.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerLoad.java
index 97b518a..5c56e9a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerLoad.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerLoad.java
@@ -43,7 +43,7 @@ public class TestServerLoad {
     assertEquals(504, sl.getRootIndexSizeKB());
     assertEquals(820, sl.getStorefileSizeInMB());
     assertEquals(82, sl.getStorefileIndexSizeInMB());
-    assertEquals(0, sl.getReadRequestsCount());
+    assertEquals(((long)Integer.MAX_VALUE)*2, sl.getReadRequestsCount());
     
   }
  
@@ -59,6 +59,14 @@ public class TestServerLoad {
     assertTrue(slToString.contains("coprocessors=[]"));
   }
 
+  @Test
+  public void testRegionLoadWrapAroundAggregation() {
+	  ServerLoad sl = new ServerLoad(createServerLoadProto());
+	  long totalCount = ((long)Integer.MAX_VALUE)*2;
+	  assertEquals(totalCount, sl.getReadRequestsCount());
+	  assertEquals(totalCount, sl.getWriteRequestsCount());
+  }
+  
   private ClusterStatusProtos.ServerLoad createServerLoadProto() {
     HBaseProtos.RegionSpecifier rSpecOne =
         HBaseProtos.RegionSpecifier.newBuilder()
@@ -72,11 +80,11 @@ public class TestServerLoad {
     ClusterStatusProtos.RegionLoad rlOne =
         ClusterStatusProtos.RegionLoad.newBuilder().setRegionSpecifier(rSpecOne).setStores(10)
             .setStorefiles(101).setStoreUncompressedSizeMB(106).setStorefileSizeMB(520)
-            .setStorefileIndexSizeMB(42).setRootIndexSizeKB(201).build();
+            .setStorefileIndexSizeMB(42).setRootIndexSizeKB(201).setReadRequestsCount(Integer.MAX_VALUE).setWriteRequestsCount(Integer.MAX_VALUE).build();
     ClusterStatusProtos.RegionLoad rlTwo =
         ClusterStatusProtos.RegionLoad.newBuilder().setRegionSpecifier(rSpecTwo).setStores(3)
             .setStorefiles(13).setStoreUncompressedSizeMB(23).setStorefileSizeMB(300)
-            .setStorefileIndexSizeMB(40).setRootIndexSizeKB(303).build();
+            .setStorefileIndexSizeMB(40).setRootIndexSizeKB(303).setReadRequestsCount(Integer.MAX_VALUE).setWriteRequestsCount(Integer.MAX_VALUE).build();
 
     ClusterStatusProtos.ServerLoad sl =
         ClusterStatusProtos.ServerLoad.newBuilder().addRegionLoads(rlOne).


[33/50] [abbrv] hbase git commit: HBASE-9910 TestHFilePerformance and HFilePerformanceEvaluation should be merged in a single HFile performance test class (Vikas Vishwakarma)

Posted by jm...@apache.org.
HBASE-9910 TestHFilePerformance and HFilePerformanceEvaluation should be merged in a single HFile performance test class (Vikas Vishwakarma)

Amending-Author: Andrew Purtell <ap...@apache.org>


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

Branch: refs/heads/hbase-11339
Commit: f9cf565f1ddcd9120fe26e5e92760662825f13f9
Parents: a7d9315
Author: Andrew Purtell <ap...@apache.org>
Authored: Tue Feb 10 15:56:14 2015 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Tue Feb 10 16:11:08 2015 -0800

----------------------------------------------------------------------
 .../hbase/HFilePerformanceEvaluation.java       | 175 ++++++-
 .../hbase/io/hfile/TestHFilePerformance.java    | 455 -------------------
 2 files changed, 150 insertions(+), 480 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/f9cf565f/hbase-server/src/test/java/org/apache/hadoop/hbase/HFilePerformanceEvaluation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HFilePerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HFilePerformanceEvaluation.java
index 8336543..ea10f60 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HFilePerformanceEvaluation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HFilePerformanceEvaluation.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hbase;
 
 import java.io.IOException;
+import java.security.SecureRandom;
 import java.util.Random;
 
 import org.apache.commons.logging.Log;
@@ -30,6 +31,10 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.io.crypto.Encryption;
+import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting;
+import org.apache.hadoop.hbase.io.crypto.aes.AES;
+import org.apache.hadoop.hbase.io.hfile.AbstractHFileWriter;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.HFileContext;
@@ -45,7 +50,16 @@ public class HFilePerformanceEvaluation {
   private static final int ROW_LENGTH = 10;
   private static final int ROW_COUNT = 1000000;
   private static final int RFILE_BLOCKSIZE = 8 * 1024;
-
+  private static StringBuilder testSummary = new StringBuilder();
+  
+  // Disable verbose INFO logging from org.apache.hadoop.io.compress.CodecPool
+  static {
+    System.setProperty("org.apache.commons.logging.Log", 
+      "org.apache.commons.logging.impl.SimpleLog");
+    System.setProperty("org.apache.commons.logging.simplelog.log.org.apache.hadoop.io.compress.CodecPool",
+      "WARN");
+  }
+  
   static final Log LOG =
     LogFactory.getLog(HFilePerformanceEvaluation.class.getName());
 
@@ -82,70 +96,154 @@ public class HFilePerformanceEvaluation {
     return CellUtil.createCell(keyRow, value);
   }
 
+  /**
+   * Add any supported codec or cipher to test the HFile read/write performance. 
+   * Specify "none" to disable codec or cipher or both.  
+   * @throws Exception
+   */
   private void runBenchmarks() throws Exception {
     final Configuration conf = new Configuration();
     final FileSystem fs = FileSystem.get(conf);
     final Path mf = fs.makeQualified(new Path("performanceevaluation.mapfile"));
+    
+    // codec=none cipher=none
+    runWriteBenchmark(conf, fs, mf, "none", "none");
+    runReadBenchmark(conf, fs, mf, "none", "none");
+    
+    // codec=gz cipher=none
+    runWriteBenchmark(conf, fs, mf, "gz", "none");
+    runReadBenchmark(conf, fs, mf, "gz", "none");
+
+    // Add configuration for AES cipher
+    final Configuration aesconf = new Configuration();
+    aesconf.set(HConstants.CRYPTO_KEYPROVIDER_CONF_KEY, KeyProviderForTesting.class.getName());
+    aesconf.set(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY, "hbase");
+    aesconf.setInt("hfile.format.version", 3);
+    final FileSystem aesfs = FileSystem.get(aesconf);
+    final Path aesmf = aesfs.makeQualified(new Path("performanceevaluation.aes.mapfile"));
+
+    // codec=none cipher=aes
+    runWriteBenchmark(aesconf, aesfs, aesmf, "none", "aes");
+    runReadBenchmark(aesconf, aesfs, aesmf, "none", "aes");
+
+    // codec=gz cipher=aes
+    runWriteBenchmark(aesconf, aesfs, aesmf, "gz", "aes");
+    runReadBenchmark(aesconf, aesfs, aesmf, "gz", "aes");
+
+    // cleanup test files
     if (fs.exists(mf)) {
       fs.delete(mf, true);
     }
+    if (aesfs.exists(aesmf)) {
+      aesfs.delete(aesmf, true);
+    }
+    
+    // Print Result Summary
+    LOG.info("\n***************\n" + "Result Summary" + "\n***************\n");
+    LOG.info(testSummary.toString());
+
+  }
+
+  /**
+   * Write a test HFile with the given codec & cipher
+   * @param conf
+   * @param fs
+   * @param mf
+   * @param codec "none", "lzo", "gz", "snappy"
+   * @param cipher "none", "aes"
+   * @throws Exception
+   */
+  private void runWriteBenchmark(Configuration conf, FileSystem fs, Path mf, String codec,
+      String cipher) throws Exception {
+    if (fs.exists(mf)) {
+      fs.delete(mf, true);
+    }
+
+    runBenchmark(new SequentialWriteBenchmark(conf, fs, mf, ROW_COUNT, codec, cipher),
+        ROW_COUNT, codec, cipher);
+
+  }
 
-    runBenchmark(new SequentialWriteBenchmark(conf, fs, mf, ROW_COUNT),
-        ROW_COUNT);
+  /**
+   * Run all the read benchmarks for the test HFile 
+   * @param conf
+   * @param fs
+   * @param mf
+   * @param codec "none", "lzo", "gz", "snappy"
+   * @param cipher "none", "aes"
+   */
+  private void runReadBenchmark(final Configuration conf, final FileSystem fs, final Path mf,
+      final String codec, final String cipher) {
     PerformanceEvaluationCommons.concurrentReads(new Runnable() {
       @Override
       public void run() {
         try {
           runBenchmark(new UniformRandomSmallScan(conf, fs, mf, ROW_COUNT),
-            ROW_COUNT);
+            ROW_COUNT, codec, cipher);
         } catch (Exception e) {
+          testSummary.append("UniformRandomSmallScan failed " + e.getMessage());
           e.printStackTrace();
         }
       }
     });
+    
     PerformanceEvaluationCommons.concurrentReads(new Runnable() {
       @Override
       public void run() {
         try {
           runBenchmark(new UniformRandomReadBenchmark(conf, fs, mf, ROW_COUNT),
-              ROW_COUNT);
+              ROW_COUNT, codec, cipher);
         } catch (Exception e) {
+          testSummary.append("UniformRandomReadBenchmark failed " + e.getMessage());
           e.printStackTrace();
         }
       }
     });
+    
     PerformanceEvaluationCommons.concurrentReads(new Runnable() {
       @Override
       public void run() {
         try {
           runBenchmark(new GaussianRandomReadBenchmark(conf, fs, mf, ROW_COUNT),
-              ROW_COUNT);
+              ROW_COUNT, codec, cipher);
         } catch (Exception e) {
+          testSummary.append("GaussianRandomReadBenchmark failed " + e.getMessage());
           e.printStackTrace();
         }
       }
     });
+    
     PerformanceEvaluationCommons.concurrentReads(new Runnable() {
       @Override
       public void run() {
         try {
           runBenchmark(new SequentialReadBenchmark(conf, fs, mf, ROW_COUNT),
-              ROW_COUNT);
+              ROW_COUNT, codec, cipher);
         } catch (Exception e) {
+          testSummary.append("SequentialReadBenchmark failed " + e.getMessage());
           e.printStackTrace();
         }
       }
-    });
+    });    
 
   }
-
-  protected void runBenchmark(RowOrientedBenchmark benchmark, int rowCount)
-    throws Exception {
-    LOG.info("Running " + benchmark.getClass().getSimpleName() + " for " +
-        rowCount + " rows.");
+  
+  protected void runBenchmark(RowOrientedBenchmark benchmark, int rowCount,
+      String codec, String cipher) throws Exception {
+    LOG.info("Running " + benchmark.getClass().getSimpleName() + " with codec[" + 
+        codec + "] " + "cipher[" + cipher + "] for " + rowCount + " rows.");
+    
     long elapsedTime = benchmark.run();
-    LOG.info("Running " + benchmark.getClass().getSimpleName() + " for " +
-        rowCount + " rows took " + elapsedTime + "ms.");
+    
+    LOG.info("Running " + benchmark.getClass().getSimpleName() + " with codec[" + 
+        codec + "] " + "cipher[" + cipher + "] for " + rowCount + " rows took " + 
+        elapsedTime + "ms.");
+    
+    // Store results to print summary at the end
+    testSummary.append("Running ").append(benchmark.getClass().getSimpleName())
+        .append(" with codec[").append(codec).append("] cipher[").append(cipher)
+        .append("] for ").append(rowCount).append(" rows took ").append(elapsedTime)
+        .append("ms.").append("\n");
   }
 
   static abstract class RowOrientedBenchmark {
@@ -154,6 +252,18 @@ public class HFilePerformanceEvaluation {
     protected final FileSystem fs;
     protected final Path mf;
     protected final int totalRows;
+    protected String codec = "none";
+    protected String cipher = "none";
+
+    public RowOrientedBenchmark(Configuration conf, FileSystem fs, Path mf,
+        int totalRows, String codec, String cipher) {
+      this.conf = conf;
+      this.fs = fs;
+      this.mf = mf;
+      this.totalRows = totalRows;
+      this.codec = codec;
+      this.cipher = cipher;
+    }
 
     public RowOrientedBenchmark(Configuration conf, FileSystem fs, Path mf,
         int totalRows) {
@@ -208,21 +318,36 @@ public class HFilePerformanceEvaluation {
     private byte[] bytes = new byte[ROW_LENGTH];
 
     public SequentialWriteBenchmark(Configuration conf, FileSystem fs, Path mf,
-        int totalRows) {
-      super(conf, fs, mf, totalRows);
+        int totalRows, String codec, String cipher) {
+      super(conf, fs, mf, totalRows, codec, cipher);
     }
 
     @Override
     void setUp() throws Exception {
-      HFileContext hFileContext = new HFileContextBuilder().withBlockSize(RFILE_BLOCKSIZE).build();
-      writer =
-        HFile.getWriterFactoryNoCache(conf)
-            .withPath(fs, mf)
-            .withFileContext(hFileContext)
-            .withComparator(new KeyValue.RawBytesComparator())
-            .create();
-    }
 
+      HFileContextBuilder builder = new HFileContextBuilder()
+          .withCompression(AbstractHFileWriter.compressionByName(codec))
+          .withBlockSize(RFILE_BLOCKSIZE);
+      
+      if (cipher == "aes") {
+        byte[] cipherKey = new byte[AES.KEY_LENGTH];
+        new SecureRandom().nextBytes(cipherKey);
+        builder.withEncryptionContext(Encryption.newContext(conf)
+            .setCipher(Encryption.getCipher(conf, cipher))
+            .setKey(cipherKey));
+      } else if (!"none".equals(cipher)) {
+        throw new IOException("Cipher " + cipher + " not supported.");
+      }
+      
+      HFileContext hFileContext = builder.build();
+
+      writer = HFile.getWriterFactoryNoCache(conf)
+          .withPath(fs, mf)
+          .withFileContext(hFileContext)
+          .withComparator(new KeyValue.RawBytesComparator())
+          .create();
+    }
+    
     @Override
     void doRow(int i) throws Exception {
       writer.append(createCell(i, generateValue()));

http://git-wip-us.apache.org/repos/asf/hbase/blob/f9cf565f/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFilePerformance.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFilePerformance.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFilePerformance.java
deleted file mode 100644
index 78de413..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFilePerformance.java
+++ /dev/null
@@ -1,455 +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.io.hfile;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.security.SecureRandom;
-import java.text.DateFormat;
-import java.text.SimpleDateFormat;
-import java.util.Random;
-
-import org.apache.commons.cli.CommandLine;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.io.crypto.Encryption;
-import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting;
-import org.apache.hadoop.hbase.io.crypto.aes.AES;
-import org.apache.hadoop.hbase.util.AbstractHBaseTool;
-import org.apache.hadoop.io.BytesWritable;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.GzipCodec;
-import org.apache.hadoop.util.ToolRunner;
-
-/**
- *  Set of long-running tests to measure performance of HFile.
- * <p>
- * Copied from
- * <a href="https://issues.apache.org/jira/browse/HADOOP-3315">hadoop-3315 tfile</a>.
- * Remove after tfile is committed and use the tfile version of this class
- * instead.</p>
- */
-public class TestHFilePerformance extends AbstractHBaseTool {
-  private HBaseTestingUtility TEST_UTIL;
-  private static String ROOT_DIR;
-  private FileSystem fs;
-  private long startTimeEpoch;
-  private long finishTimeEpoch;
-  private DateFormat formatter;
-
-  @Override
-  public void setConf(Configuration conf) {
-    super.setConf(conf);
-    try {
-      fs = FileSystem.get(conf);
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-    conf.set(HConstants.CRYPTO_KEYPROVIDER_CONF_KEY, KeyProviderForTesting.class.getName());
-    conf.set(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY, "hbase");
-    formatter = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
-    TEST_UTIL = new HBaseTestingUtility(conf);
-    ROOT_DIR = TEST_UTIL.getDataTestDir("TestHFilePerformance").toString();
-  }
-
-  public void startTime() {
-    startTimeEpoch = System.currentTimeMillis();
-    System.out.println(formatTime() + " Started timing.");
-  }
-
-  public void stopTime() {
-    finishTimeEpoch = System.currentTimeMillis();
-    System.out.println(formatTime() + " Stopped timing.");
-  }
-
-  public long getIntervalMillis() {
-    return finishTimeEpoch - startTimeEpoch;
-  }
-
-  public void printlnWithTimestamp(String message) {
-    System.out.println(formatTime() + "  " +  message);
-  }
-
-  /*
-   * Format millis into minutes and seconds.
-   */
-  public String formatTime(long milis){
-    return formatter.format(milis);
-  }
-
-  public String formatTime(){
-    return formatTime(System.currentTimeMillis());
-  }
-
-  private FSDataOutputStream createFSOutput(Path name) throws IOException {
-    if (fs.exists(name))
-      fs.delete(name, true);
-    FSDataOutputStream fout = fs.create(name);
-    return fout;
-  }
-
-  //TODO have multiple ways of generating key/value e.g. dictionary words
-  //TODO to have a sample compressable data, for now, made 1 out of 3 values random
-  //     keys are all random.
-
-  private static class KeyValueGenerator {
-    Random keyRandomizer;
-    Random valueRandomizer;
-    long randomValueRatio = 3; // 1 out of randomValueRatio generated values will be random.
-    long valueSequence = 0 ;
-
-
-    KeyValueGenerator() {
-      keyRandomizer = new Random(0L); //TODO with seed zero
-      valueRandomizer = new Random(1L); //TODO with seed one
-    }
-
-    // Key is always random now.
-    void getKey(byte[] key) {
-      keyRandomizer.nextBytes(key);
-    }
-
-    void getValue(byte[] value) {
-      if (valueSequence % randomValueRatio == 0)
-          valueRandomizer.nextBytes(value);
-      valueSequence++;
-    }
-  }
-
-  /**
-   *
-   * @param fileType "HFile" or "SequenceFile"
-   * @param keyLength
-   * @param valueLength
-   * @param codecName "none", "lzo", "gz", "snappy"
-   * @param cipherName "none", "aes"
-   * @param rows number of rows to be written.
-   * @param writeMethod used for HFile only.
-   * @param minBlockSize used for HFile only.
-   * @throws IOException
-   */
-   //TODO writeMethod: implement multiple ways of writing e.g. A) known length (no chunk) B) using a buffer and streaming (for many chunks).
-  public void timeWrite(String fileType, int keyLength, int valueLength,
-    String codecName, String cipherName, long rows, String writeMethod, int minBlockSize)
-  throws IOException {
-    System.out.println("File Type: " + fileType);
-    System.out.println("Writing " + fileType + " with codecName: " + codecName +
-      " cipherName: " + cipherName);
-    long totalBytesWritten = 0;
-
-
-    //Using separate randomizer for key/value with seeds matching Sequence File.
-    byte[] key = new byte[keyLength];
-    byte[] value = new byte[valueLength];
-    KeyValueGenerator generator = new KeyValueGenerator();
-
-    startTime();
-
-    Path path = new Path(ROOT_DIR, fileType + ".Performance");
-    System.out.println(ROOT_DIR + Path.SEPARATOR + path.getName());
-    FSDataOutputStream fout =  createFSOutput(path);
-
-    if ("HFile".equals(fileType)){
-        HFileContextBuilder builder = new HFileContextBuilder()
-          .withCompression(AbstractHFileWriter.compressionByName(codecName))
-          .withBlockSize(minBlockSize);
-        if (cipherName != "none") {
-          byte[] cipherKey = new byte[AES.KEY_LENGTH];
-          new SecureRandom().nextBytes(cipherKey);
-          builder.withEncryptionContext(
-            Encryption.newContext(conf)
-              .setCipher(Encryption.getCipher(conf, cipherName))
-              .setKey(cipherKey));
-        }
-        HFileContext context = builder.build();
-        System.out.println("HFile write method: ");
-        HFile.Writer writer = HFile.getWriterFactoryNoCache(conf)
-            .withOutputStream(fout)
-            .withFileContext(context)
-            .withComparator(new KeyValue.RawBytesComparator())
-            .create();
-
-        // Writing value in one shot.
-        for (long l=0; l<rows; l++ ) {
-          generator.getKey(key);
-          generator.getValue(value);
-          writer.append(CellUtil.createCell(key, value));
-          totalBytesWritten += key.length;
-          totalBytesWritten += value.length;
-         }
-        writer.close();
-    } else if ("SequenceFile".equals(fileType)){
-        CompressionCodec codec = null;
-        if ("gz".equals(codecName))
-          codec = new GzipCodec();
-        else if (!"none".equals(codecName))
-          throw new IOException("Codec not supported.");
-
-        SequenceFile.Writer writer;
-
-        //TODO
-        //JobConf conf = new JobConf();
-
-        if (!"none".equals(codecName))
-          writer = SequenceFile.createWriter(conf, fout, BytesWritable.class,
-            BytesWritable.class, SequenceFile.CompressionType.BLOCK, codec);
-        else
-          writer = SequenceFile.createWriter(conf, fout, BytesWritable.class,
-            BytesWritable.class, SequenceFile.CompressionType.NONE, null);
-
-        BytesWritable keyBsw;
-        BytesWritable valBsw;
-        for (long l=0; l<rows; l++ ) {
-
-           generator.getKey(key);
-           keyBsw = new BytesWritable(key);
-           totalBytesWritten += keyBsw.getSize();
-
-           generator.getValue(value);
-           valBsw = new BytesWritable(value);
-           writer.append(keyBsw, valBsw);
-           totalBytesWritten += valBsw.getSize();
-        }
-
-        writer.close();
-    } else
-       throw new IOException("File Type is not supported");
-
-    fout.close();
-    stopTime();
-
-    printlnWithTimestamp("Data written: ");
-    printlnWithTimestamp("  rate  = " +
-      totalBytesWritten / getIntervalMillis() * 1000 / 1024 / 1024 + "MB/s");
-    printlnWithTimestamp("  total = " + totalBytesWritten + "B");
-
-    printlnWithTimestamp("File written: ");
-    printlnWithTimestamp("  rate  = " +
-      fs.getFileStatus(path).getLen() / getIntervalMillis() * 1000 / 1024 / 1024 + "MB/s");
-    printlnWithTimestamp("  total = " + fs.getFileStatus(path).getLen() + "B");
-  }
-
-  public void timeReading(String fileType, int keyLength, int valueLength,
-      long rows, int method) throws IOException {
-    System.out.println("Reading file of type: " + fileType);
-    Path path = new Path(ROOT_DIR, fileType + ".Performance");
-    System.out.println("Input file size: " + fs.getFileStatus(path).getLen());
-    long totalBytesRead = 0;
-
-
-    ByteBuffer val;
-
-    ByteBuffer key;
-
-    startTime();
-    FSDataInputStream fin = fs.open(path);
-
-    if ("HFile".equals(fileType)){
-        HFile.Reader reader = HFile.createReaderFromStream(path, fs.open(path),
-          fs.getFileStatus(path).getLen(), new CacheConfig(conf), conf);
-        reader.loadFileInfo();
-        switch (method) {
-
-          case 0:
-          case 1:
-          default:
-            {
-              HFileScanner scanner = reader.getScanner(false, false);
-              scanner.seekTo();
-              for (long l=0; l<rows; l++ ) {
-                key = scanner.getKey();
-                val = scanner.getValue();
-                totalBytesRead += key.limit() + val.limit();
-                scanner.next();
-              }
-            }
-            break;
-        }
-      reader.close();
-    } else if("SequenceFile".equals(fileType)){
-
-        SequenceFile.Reader reader;
-        reader = new SequenceFile.Reader(fs, path, new Configuration());
-
-        if (reader.getCompressionCodec() != null) {
-            printlnWithTimestamp("Compression codec class: " + reader.getCompressionCodec().getClass());
-        } else
-            printlnWithTimestamp("Compression codec class: " + "none");
-
-        BytesWritable keyBsw = new BytesWritable();
-        BytesWritable valBsw = new BytesWritable();
-
-        for (long l=0; l<rows; l++ ) {
-          reader.next(keyBsw, valBsw);
-          totalBytesRead += keyBsw.getSize() + valBsw.getSize();
-        }
-        reader.close();
-
-        //TODO make a tests for other types of SequenceFile reading scenarios
-
-    } else {
-        throw new IOException("File Type not supported.");
-    }
-
-
-    //printlnWithTimestamp("Closing reader");
-    fin.close();
-    stopTime();
-    //printlnWithTimestamp("Finished close");
-
-    printlnWithTimestamp("Finished in " + getIntervalMillis() + "ms");
-    printlnWithTimestamp("Data read: ");
-    printlnWithTimestamp("  rate  = " +
-      totalBytesRead / getIntervalMillis() * 1000 / 1024 / 1024 + "MB/s");
-    printlnWithTimestamp("  total = " + totalBytesRead + "B");
-
-    printlnWithTimestamp("File read: ");
-    printlnWithTimestamp("  rate  = " +
-      fs.getFileStatus(path).getLen() / getIntervalMillis() * 1000 / 1024 / 1024 + "MB/s");
-    printlnWithTimestamp("  total = " + fs.getFileStatus(path).getLen() + "B");
-
-    //TODO uncomment this for final committing so test files is removed.
-    //fs.delete(path, true);
-  }
-
-  public void testRunComparisons() throws IOException {
-
-    int keyLength = 100; // 100B
-    int valueLength = 5*1024; // 5KB
-    int minBlockSize = 10*1024*1024; // 10MB
-    int rows = 10000;
-
-    System.out.println("****************************** Sequence File *****************************");
-
-    timeWrite("SequenceFile", keyLength, valueLength, "none", "none", rows, null, minBlockSize);
-    System.out.println("\n+++++++\n");
-    timeReading("SequenceFile", keyLength, valueLength, rows, -1);
-
-    System.out.println("");
-    System.out.println("----------------------");
-    System.out.println("");
-
-    /* DISABLED LZO
-    timeWrite("SequenceFile", keyLength, valueLength, "lzo", rows, null, minBlockSize);
-    System.out.println("\n+++++++\n");
-    timeReading("SequenceFile", keyLength, valueLength, rows, -1);
-
-    System.out.println("");
-    System.out.println("----------------------");
-    System.out.println("");
-
-    /* Sequence file can only use native hadoop libs gzipping so commenting out.
-     */
-    try {
-      timeWrite("SequenceFile", keyLength, valueLength, "gz", "none", rows, null,
-        minBlockSize);
-      System.out.println("\n+++++++\n");
-      timeReading("SequenceFile", keyLength, valueLength, rows, -1);
-    } catch (IllegalArgumentException e) {
-      System.out.println("Skipping sequencefile gz: " + e.getMessage());
-    }
-
-
-    System.out.println("\n\n\n");
-    System.out.println("****************************** HFile *****************************");
-
-    timeWrite("HFile", keyLength, valueLength, "none", "none", rows, null, minBlockSize);
-    System.out.println("\n+++++++\n");
-    timeReading("HFile", keyLength, valueLength, rows, 0 );
-
-    System.out.println("");
-    System.out.println("----------------------");
-    System.out.println("");
-
-    timeWrite("HFile", keyLength, valueLength, "none", "aes", rows, null, minBlockSize);
-    System.out.println("\n+++++++\n");
-    timeReading("HFile", keyLength, valueLength, rows, 0 );
-
-    System.out.println("");
-    System.out.println("----------------------");
-    System.out.println("");
-
-/* DISABLED LZO
-    timeWrite("HFile", keyLength, valueLength, "lzo", rows, null, minBlockSize);
-    System.out.println("\n+++++++\n");
-    timeReading("HFile", keyLength, valueLength, rows, 0 );
-    System.out.println("\n+++++++\n");
-    timeReading("HFile", keyLength, valueLength, rows, 1 );
-    System.out.println("\n+++++++\n");
-    timeReading("HFile", keyLength, valueLength, rows, 2 );
-
-    System.out.println("");
-    System.out.println("----------------------");
-    System.out.println("");
-*/
-
-    timeWrite("HFile", keyLength, valueLength, "gz", "none", rows, null, minBlockSize);
-    System.out.println("\n+++++++\n");
-    timeReading("HFile", keyLength, valueLength, rows, 0 );
-
-    System.out.println("");
-    System.out.println("----------------------");
-    System.out.println("");
-
-    timeWrite("HFile", keyLength, valueLength, "gz", "aes", rows, null, minBlockSize);
-    System.out.println("\n+++++++\n");
-    timeReading("HFile", keyLength, valueLength, rows, 0 );
-
-    System.out.println("\n\n\n\nNotes: ");
-    System.out.println(" * Timing includes open/closing of files.");
-    System.out.println(" * Timing includes reading both Key and Value");
-    System.out.println(" * Data is generated as random bytes. Other methods e.g. using " +
-            "dictionary with care for distributation of words is under development.");
-    System.out.println(" * Timing of write currently, includes random value/key generations. " +
-            "Which is the same for Sequence File and HFile. Another possibility is to generate " +
-            "test data beforehand");
-    System.out.println(" * We need to mitigate cache effect on benchmark. We can apply several " +
-            "ideas, for next step we do a large dummy read between benchmark read to dismantle " +
-            "caching of data. Renaming of file may be helpful. We can have a loop that reads with" +
-            " the same method several times and flood cache every time and average it to get a" +
-            " better number.");
-  }
-
-  @Override
-  protected void addOptions() {
-  }
-
-  @Override
-  protected void processOptions(CommandLine cmd) {
-  }
-
-  @Override
-  protected int doWork() throws Exception {
-    testRunComparisons();
-    return 0;
-  }
-
-  public static void main(String[] args) throws Exception {
-    int ret = ToolRunner.run(HBaseConfiguration.create(), new TestHFilePerformance(), args);
-    System.exit(ret);
-  }
-}