You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by sy...@apache.org on 2016/03/30 18:03:03 UTC

[01/50] [abbrv] hbase git commit: HBASE-15464 Flush / Compaction metrics revisited

Repository: hbase
Updated Branches:
  refs/heads/hbase-12439 8f6e29785 -> f1fc5208a


HBASE-15464 Flush / Compaction metrics revisited


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

Branch: refs/heads/hbase-12439
Commit: 797562e6c3a2131f93a1ab5d777abf1867d91383
Parents: 75252af
Author: Enis Soztutar <en...@apache.org>
Authored: Mon Mar 21 17:50:02 2016 -0700
Committer: Enis Soztutar <en...@apache.org>
Committed: Mon Mar 21 17:50:02 2016 -0700

----------------------------------------------------------------------
 .../regionserver/MetricsRegionServerSource.java | 107 ++++++++++++++-
 .../MetricsRegionServerSourceImpl.java          | 130 ++++++++++++++++++-
 .../procedure/flush/FlushTableSubprocedure.java |   1 +
 .../hbase/regionserver/DefaultStoreEngine.java  |   6 -
 .../hadoop/hbase/regionserver/HRegion.java      |  11 +-
 .../hbase/regionserver/HRegionServer.java       |   6 +
 .../hadoop/hbase/regionserver/HStore.java       |  38 +++++-
 .../hbase/regionserver/MemStoreFlusher.java     |  17 +--
 .../hbase/regionserver/MetricsRegionServer.java |  13 +-
 .../hbase/regionserver/RSRpcServices.java       |  27 +---
 .../regionserver/RegionServerServices.java      |   5 +
 .../apache/hadoop/hbase/regionserver/Store.java |   7 +-
 .../hbase/regionserver/StoreFlushContext.java   |   5 +
 .../hbase/regionserver/StripeStoreEngine.java   |   7 -
 .../compactions/CompactionContext.java          |   7 -
 .../compactions/CompactionRequest.java          |   3 +
 .../hadoop/hbase/MockRegionServerServices.java  |   7 +
 .../hadoop/hbase/master/MockRegionServer.java   |   7 +
 .../hbase/regionserver/TestCompaction.java      |  19 +--
 .../regionserver/TestMetricsRegionServer.java   |  64 +++++++++
 .../regionserver/TestStripeStoreEngine.java     |   2 +-
 21 files changed, 401 insertions(+), 88 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/797562e6/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java
index ee3e847..f097296 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java
@@ -146,6 +146,53 @@ public interface MetricsRegionServerSource extends BaseSource {
    */
   void updateFlushTime(long t);
 
+  /**
+   * Update the flush memstore size histogram
+   * @param bytes the number of bytes in the memstore
+   */
+  void updateFlushMemstoreSize(long bytes);
+
+  /**
+   * Update the flush output file size histogram
+   * @param bytes the number of bytes in the output file
+   */
+  void updateFlushOutputSize(long bytes);
+
+  /**
+   * Update the compaction time histogram, both major and minor
+   * @param isMajor whether compaction is a major compaction
+   * @param t time it took, in milliseconds
+   */
+  void updateCompactionTime(boolean isMajor, long t);
+
+  /**
+   * Update the compaction input number of files histogram
+   * @param isMajor whether compaction is a major compaction
+   * @param c number of files
+   */
+  void updateCompactionInputFileCount(boolean isMajor, long c);
+
+  /**
+   * Update the compaction total input file size histogram
+   * @param isMajor whether compaction is a major compaction
+   * @param bytes the number of bytes of the compaction input file
+   */
+  void updateCompactionInputSize(boolean isMajor, long bytes);
+
+  /**
+   * Update the compaction output number of files histogram
+   * @param isMajor whether compaction is a major compaction
+   * @param c number of files
+   */
+  void updateCompactionOutputFileCount(boolean isMajor, long c);
+
+  /**
+   * Update the compaction total output file size
+   * @param isMajor whether compaction is a major compaction
+   * @param bytes the number of bytes of the compaction input file
+   */
+  void updateCompactionOutputSize(boolean isMajor, long bytes);
+
   // Strings used for exporting to metrics system.
   String REGION_COUNT = "regionCount";
   String REGION_COUNT_DESC = "Number of regions";
@@ -212,6 +259,10 @@ public interface MetricsRegionServerSource extends BaseSource {
   String LARGE_COMPACTION_QUEUE_LENGTH = "largeCompactionQueueLength";
   String SMALL_COMPACTION_QUEUE_LENGTH = "smallCompactionQueueLength";
   String COMPACTION_QUEUE_LENGTH_DESC = "Length of the queue for compactions.";
+  String LARGE_COMPACTION_QUEUE_LENGTH_DESC = "Length of the queue for compactions with input size "
+      + "larger than throttle threshold (2.5GB by default)";
+  String SMALL_COMPACTION_QUEUE_LENGTH_DESC = "Length of the queue for compactions with input size "
+      + "smaller than throttle threshold (2.5GB by default)";
   String FLUSH_QUEUE_LENGTH = "flushQueueLength";
   String FLUSH_QUEUE_LENGTH_DESC = "Length of the queue for region flushes";
   String BLOCK_CACHE_FREE_SIZE = "blockCacheFreeSize";
@@ -345,7 +396,61 @@ public interface MetricsRegionServerSource extends BaseSource {
   String SPLIT_REQUEST_DESC = "Number of splits requested";
   String SPLIT_SUCCESS_KEY = "splitSuccessCount";
   String SPLIT_SUCCESS_DESC = "Number of successfully executed splits";
-  String FLUSH_KEY = "flushTime";
+
+  String FLUSH_TIME = "flushTime";
+  String FLUSH_TIME_DESC = "Histogram for the time in millis for memstore flush";
+  String FLUSH_MEMSTORE_SIZE = "flushMemstoreSize";
+  String FLUSH_MEMSTORE_SIZE_DESC = "Histogram for number of bytes in the memstore for a flush";
+  String FLUSH_OUTPUT_SIZE = "flushOutputSize";
+  String FLUSH_OUTPUT_SIZE_DESC = "Histogram for number of bytes in the resulting file for a flush";
+  String FLUSHED_OUTPUT_BYTES = "flushedOutputBytes";
+  String FLUSHED_OUTPUT_BYTES_DESC = "Total number of bytes written from flush";
+  String FLUSHED_MEMSTORE_BYTES = "flushedMemstoreBytes";
+  String FLUSHED_MEMSTORE_BYTES_DESC = "Total number of bytes of cells in memstore from flush";
+
+  String COMPACTION_TIME = "compactionTime";
+  String COMPACTION_TIME_DESC
+    = "Histogram for the time in millis for compaction, both major and minor";
+  String COMPACTION_INPUT_FILE_COUNT = "compactionInputFileCount";
+  String COMPACTION_INPUT_FILE_COUNT_DESC
+    = "Histogram for the compaction input number of files, both major and minor";
+  String COMPACTION_INPUT_SIZE = "compactionInputSize";
+  String COMPACTION_INPUT_SIZE_DESC
+    = "Histogram for the compaction total input file sizes, both major and minor";
+  String COMPACTION_OUTPUT_FILE_COUNT = "compactionOutputFileCount";
+  String COMPACTION_OUTPUT_FILE_COUNT_DESC
+    = "Histogram for the compaction output number of files, both major and minor";
+  String COMPACTION_OUTPUT_SIZE = "compactionOutputSize";
+  String COMPACTION_OUTPUT_SIZE_DESC
+    = "Histogram for the compaction total output file sizes, both major and minor";
+  String COMPACTED_INPUT_BYTES = "compactedInputBytes";
+  String COMPACTED_INPUT_BYTES_DESC
+    = "Total number of bytes that is read for compaction, both major and minor";
+  String COMPACTED_OUTPUT_BYTES = "compactedOutputBytes";
+  String COMPACTED_OUTPUT_BYTES_DESC
+    = "Total number of bytes that is output from compaction, both major and minor";
+
+  String MAJOR_COMPACTION_TIME = "majorCompactionTime";
+  String MAJOR_COMPACTION_TIME_DESC
+    = "Histogram for the time in millis for compaction, major only";
+  String MAJOR_COMPACTION_INPUT_FILE_COUNT = "majorCompactionInputFileCount";
+  String MAJOR_COMPACTION_INPUT_FILE_COUNT_DESC
+    = "Histogram for the compaction input number of files, major only";
+  String MAJOR_COMPACTION_INPUT_SIZE = "majorCompactionInputSize";
+  String MAJOR_COMPACTION_INPUT_SIZE_DESC
+    = "Histogram for the compaction total input file sizes, major only";
+  String MAJOR_COMPACTION_OUTPUT_FILE_COUNT = "majorCompactionOutputFileCount";
+  String MAJOR_COMPACTION_OUTPUT_FILE_COUNT_DESC
+    = "Histogram for the compaction output number of files, major only";
+  String MAJOR_COMPACTION_OUTPUT_SIZE = "majorCompactionOutputSize";
+  String MAJOR_COMPACTION_OUTPUT_SIZE_DESC
+    = "Histogram for the compaction total output file sizes, major only";
+  String MAJOR_COMPACTED_INPUT_BYTES = "majorCompactedInputBytes";
+  String MAJOR_COMPACTED_INPUT_BYTES_DESC
+    = "Total number of bytes that is read for compaction, major only";
+  String MAJOR_COMPACTED_OUTPUT_BYTES = "majorCompactedOutputBytes";
+  String MAJOR_COMPACTED_OUTPUT_BYTES_DESC
+    = "Total number of bytes that is output from compaction, major only";
 
   String RPC_GET_REQUEST_COUNT = "rpcGetRequestCount";
   String RPC_GET_REQUEST_COUNT_DESC = "Number of rpc get requests this region server has answered.";

http://git-wip-us.apache.org/repos/asf/hbase/blob/797562e6/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
index f869397..0c24cb4 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
@@ -35,7 +35,6 @@ import org.apache.hadoop.metrics2.lib.MutableFastCounter;
 public class MetricsRegionServerSourceImpl
     extends BaseSourceImpl implements MetricsRegionServerSource {
 
-
   final MetricsRegionServerWrapper rsWrap;
   private final MetricHistogram putHisto;
   private final MetricHistogram deleteHisto;
@@ -55,7 +54,30 @@ public class MetricsRegionServerSourceImpl
   private final MutableFastCounter splitSuccess;
 
   private final MetricHistogram splitTimeHisto;
+
+  // flush related metrics
   private final MetricHistogram flushTimeHisto;
+  private final MetricHistogram flushMemstoreSizeHisto;
+  private final MetricHistogram flushOutputSizeHisto;
+  private final MutableFastCounter flushedMemstoreBytes;
+  private final MutableFastCounter flushedOutputBytes;
+
+  // compaction related metrics
+  private final MetricHistogram compactionTimeHisto;
+  private final MetricHistogram compactionInputFileCountHisto;
+  private final MetricHistogram compactionInputSizeHisto;
+  private final MetricHistogram compactionOutputFileCountHisto;
+  private final MetricHistogram compactionOutputSizeHisto;
+  private final MutableFastCounter compactedInputBytes;
+  private final MutableFastCounter compactedOutputBytes;
+
+  private final MetricHistogram majorCompactionTimeHisto;
+  private final MetricHistogram majorCompactionInputFileCountHisto;
+  private final MetricHistogram majorCompactionInputSizeHisto;
+  private final MetricHistogram majorCompactionOutputFileCountHisto;
+  private final MetricHistogram majorCompactionOutputSizeHisto;
+  private final MutableFastCounter majorCompactedInputBytes;
+  private final MutableFastCounter majorCompactedOutputBytes;
 
   public MetricsRegionServerSourceImpl(MetricsRegionServerWrapper rsWrap) {
     this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT, rsWrap);
@@ -83,14 +105,52 @@ public class MetricsRegionServerSourceImpl
 
     appendHisto = getMetricsRegistry().newTimeHistogram(APPEND_KEY);
     slowAppend = getMetricsRegistry().newCounter(SLOW_APPEND_KEY, SLOW_APPEND_DESC, 0L);
-    
+
     replayHisto = getMetricsRegistry().newTimeHistogram(REPLAY_KEY);
     scanSizeHisto = getMetricsRegistry().newSizeHistogram(SCAN_SIZE_KEY);
     scanTimeHisto = getMetricsRegistry().newTimeHistogram(SCAN_TIME_KEY);
 
-    splitTimeHisto = getMetricsRegistry().newTimeHistogram(SPLIT_KEY);
-    flushTimeHisto = getMetricsRegistry().newTimeHistogram(FLUSH_KEY);
+    flushTimeHisto = getMetricsRegistry().newTimeHistogram(FLUSH_TIME, FLUSH_TIME_DESC);
+    flushMemstoreSizeHisto = getMetricsRegistry()
+        .newSizeHistogram(FLUSH_MEMSTORE_SIZE, FLUSH_MEMSTORE_SIZE_DESC);
+    flushOutputSizeHisto = getMetricsRegistry().newSizeHistogram(FLUSH_OUTPUT_SIZE,
+      FLUSH_OUTPUT_SIZE_DESC);
+    flushedOutputBytes = getMetricsRegistry().newCounter(FLUSHED_OUTPUT_BYTES,
+      FLUSHED_OUTPUT_BYTES_DESC, 0L);
+    flushedMemstoreBytes = getMetricsRegistry().newCounter(FLUSHED_MEMSTORE_BYTES,
+      FLUSHED_MEMSTORE_BYTES_DESC, 0L);
+
+    compactionTimeHisto = getMetricsRegistry()
+        .newTimeHistogram(COMPACTION_TIME, COMPACTION_TIME_DESC);
+    compactionInputFileCountHisto = getMetricsRegistry()
+      .newHistogram(COMPACTION_INPUT_FILE_COUNT, COMPACTION_INPUT_FILE_COUNT_DESC);
+    compactionInputSizeHisto = getMetricsRegistry()
+        .newSizeHistogram(COMPACTION_INPUT_SIZE, COMPACTION_INPUT_SIZE_DESC);
+    compactionOutputFileCountHisto = getMetricsRegistry()
+        .newHistogram(COMPACTION_OUTPUT_FILE_COUNT, COMPACTION_OUTPUT_FILE_COUNT_DESC);
+    compactionOutputSizeHisto = getMetricsRegistry()
+      .newSizeHistogram(COMPACTION_OUTPUT_SIZE, COMPACTION_OUTPUT_SIZE_DESC);
+    compactedInputBytes = getMetricsRegistry()
+        .newCounter(COMPACTED_INPUT_BYTES, COMPACTED_INPUT_BYTES_DESC, 0L);
+    compactedOutputBytes = getMetricsRegistry()
+        .newCounter(COMPACTED_OUTPUT_BYTES, COMPACTED_OUTPUT_BYTES_DESC, 0L);
+
+    majorCompactionTimeHisto = getMetricsRegistry()
+        .newTimeHistogram(MAJOR_COMPACTION_TIME, MAJOR_COMPACTION_TIME_DESC);
+    majorCompactionInputFileCountHisto = getMetricsRegistry()
+      .newHistogram(MAJOR_COMPACTION_INPUT_FILE_COUNT, MAJOR_COMPACTION_INPUT_FILE_COUNT_DESC);
+    majorCompactionInputSizeHisto = getMetricsRegistry()
+        .newSizeHistogram(MAJOR_COMPACTION_INPUT_SIZE, MAJOR_COMPACTION_INPUT_SIZE_DESC);
+    majorCompactionOutputFileCountHisto = getMetricsRegistry()
+        .newHistogram(MAJOR_COMPACTION_OUTPUT_FILE_COUNT, MAJOR_COMPACTION_OUTPUT_FILE_COUNT_DESC);
+    majorCompactionOutputSizeHisto = getMetricsRegistry()
+      .newSizeHistogram(MAJOR_COMPACTION_OUTPUT_SIZE, MAJOR_COMPACTION_OUTPUT_SIZE_DESC);
+    majorCompactedInputBytes = getMetricsRegistry()
+        .newCounter(MAJOR_COMPACTED_INPUT_BYTES, MAJOR_COMPACTED_INPUT_BYTES_DESC, 0L);
+    majorCompactedOutputBytes = getMetricsRegistry()
+        .newCounter(MAJOR_COMPACTED_OUTPUT_BYTES, MAJOR_COMPACTED_OUTPUT_BYTES_DESC, 0L);
 
+    splitTimeHisto = getMetricsRegistry().newTimeHistogram(SPLIT_KEY);
     splitRequest = getMetricsRegistry().newCounter(SPLIT_REQUEST_KEY, SPLIT_REQUEST_DESC, 0L);
     splitSuccess = getMetricsRegistry().newCounter(SPLIT_SUCCESS_KEY, SPLIT_SUCCESS_DESC, 0L);
   }
@@ -180,6 +240,62 @@ public class MetricsRegionServerSourceImpl
     flushTimeHisto.add(t);
   }
 
+  @Override
+  public void updateFlushMemstoreSize(long bytes) {
+    flushMemstoreSizeHisto.add(bytes);
+    flushedMemstoreBytes.incr(bytes);
+  }
+
+  @Override
+  public void updateFlushOutputSize(long bytes) {
+    flushOutputSizeHisto.add(bytes);
+    flushedOutputBytes.incr(bytes);
+  }
+
+  @Override
+  public void updateCompactionTime(boolean isMajor, long t) {
+    compactionTimeHisto.add(t);
+    if (isMajor) {
+      majorCompactionTimeHisto.add(t);
+    }
+  }
+
+  @Override
+  public void updateCompactionInputFileCount(boolean isMajor, long c) {
+    compactionInputFileCountHisto.add(c);
+    if (isMajor) {
+      majorCompactionInputFileCountHisto.add(c);
+    }
+  }
+
+  @Override
+  public void updateCompactionInputSize(boolean isMajor, long bytes) {
+    compactionInputSizeHisto.add(bytes);
+    compactedInputBytes.incr(bytes);
+    if (isMajor) {
+      majorCompactionInputSizeHisto.add(bytes);
+      majorCompactedInputBytes.incr(bytes);
+    }
+  }
+
+  @Override
+  public void updateCompactionOutputFileCount(boolean isMajor, long c) {
+    compactionOutputFileCountHisto.add(c);
+    if (isMajor) {
+      majorCompactionOutputFileCountHisto.add(c);
+    }
+  }
+
+  @Override
+  public void updateCompactionOutputSize(boolean isMajor, long bytes) {
+    compactionOutputSizeHisto.add(bytes);
+    compactedOutputBytes.incr(bytes);
+    if (isMajor) {
+      majorCompactionOutputSizeHisto.add(bytes);
+      majorCompactedOutputBytes.incr(bytes);
+    }
+  }
+
   /**
    * Yes this is a get function that doesn't return anything.  Thanks Hadoop for breaking all
    * expectations of java programmers.  Instead of returning anything Hadoop metrics expects
@@ -252,6 +368,12 @@ public class MetricsRegionServerSourceImpl
               rsWrap.getSplitQueueSize())
           .addGauge(Interns.info(COMPACTION_QUEUE_LENGTH, COMPACTION_QUEUE_LENGTH_DESC),
               rsWrap.getCompactionQueueSize())
+          .addGauge(Interns.info(SMALL_COMPACTION_QUEUE_LENGTH, SMALL_COMPACTION_QUEUE_LENGTH_DESC),
+            rsWrap.getSmallCompactionQueueSize())
+          .addGauge(Interns.info(LARGE_COMPACTION_QUEUE_LENGTH, LARGE_COMPACTION_QUEUE_LENGTH_DESC),
+            rsWrap.getLargeCompactionQueueSize())
+          .addGauge(Interns.info(COMPACTION_QUEUE_LENGTH, COMPACTION_QUEUE_LENGTH_DESC),
+            rsWrap.getCompactionQueueSize())
           .addGauge(Interns.info(FLUSH_QUEUE_LENGTH, FLUSH_QUEUE_LENGTH_DESC),
               rsWrap.getFlushQueueSize())
           .addGauge(Interns.info(BLOCK_CACHE_FREE_SIZE, BLOCK_CACHE_FREE_DESC),

http://git-wip-us.apache.org/repos/asf/hbase/blob/797562e6/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 5723919..9898bb9 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
@@ -66,6 +66,7 @@ public class FlushTableSubprocedure extends Subprocedure {
       try {
         LOG.debug("Flush region " + region.toString() + " started...");
         region.flush(true);
+        // TODO: flush result is not checked?
       } finally {
         LOG.debug("Closing region operation on " + region);
         region.closeRegionOperation();

http://git-wip-us.apache.org/repos/asf/hbase/blob/797562e6/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreEngine.java
----------------------------------------------------------------------
diff --git 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
index e73a456..1a059d7 100644
--- 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
@@ -120,12 +120,6 @@ public class DefaultStoreEngine extends StoreEngine<
     }
 
     @Override
-    public List<Path> compact(ThroughputController throughputController)
-        throws IOException {
-      return compact(throughputController, null);
-    }
-
-    @Override
     public List<Path> compact(ThroughputController throughputController, User user)
         throws IOException {
       return compactor.compact(request, throughputController, user);

http://git-wip-us.apache.org/repos/asf/hbase/blob/797562e6/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 a17df8c..ab19d08 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
@@ -1154,6 +1154,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     return memstoreSize.get();
   }
 
+  @Override
   public RegionServicesForStores getRegionServicesForStores() {
     return regionServicesForStores;
   }
@@ -2444,6 +2445,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     // Otherwise, the snapshot content while backed up in the wal, it will not
     // be part of the current running servers state.
     boolean compactionRequested = false;
+    long flushedOutputFileSize = 0;
     try {
       // A.  Flush memstore to all the HStores.
       // Keep running vector of all store files that includes both old and the
@@ -2470,6 +2472,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         if (storeCommittedFiles == null || storeCommittedFiles.isEmpty()) {
           totalFlushableSizeOfFlushableStores -= prepareResult.storeFlushableSize.get(storeName);
         }
+        flushedOutputFileSize += flush.getOutputFileSize();
       }
       storeFlushCtxs.clear();
 
@@ -2555,10 +2558,14 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     LOG.info(msg);
     status.setStatus(msg);
 
+    if (rsServices != null && rsServices.getMetrics() != null) {
+      rsServices.getMetrics().updateFlush(time - startTime,
+        totalFlushableSizeOfFlushableStores, flushedOutputFileSize);
+    }
+
     return new FlushResultImpl(compactionRequested ?
         FlushResult.Result.FLUSHED_COMPACTION_NEEDED :
-          FlushResult.Result.FLUSHED_NO_COMPACTION_NEEDED,
-        flushOpSeqId);
+          FlushResult.Result.FLUSHED_NO_COMPACTION_NEEDED, flushOpSeqId);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/797562e6/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 4ab2693..1476190 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
@@ -620,6 +620,7 @@ public class HRegionServer extends HasThread implements
 
     if (!SystemUtils.IS_OS_WINDOWS) {
       Signal.handle(new Signal("HUP"), new SignalHandler() {
+        @Override
         public void handle(Signal signal) {
           getConfiguration().reloadConfiguration();
           configurationManager.notifyAllObservers(getConfiguration());
@@ -3418,4 +3419,9 @@ public class HRegionServer extends HasThread implements
     }
     this.flushThroughputController = FlushThroughputControllerFactory.create(this, newConf);
   }
+
+  @Override
+  public MetricsRegionServer getMetrics() {
+    return metricsRegionServer;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/797562e6/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
index 8961d17..7321028 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
@@ -184,6 +184,7 @@ public class HStore implements Store {
   private volatile long compactedCellsCount = 0;
   private volatile long majorCompactedCellsCount = 0;
   private volatile long flushedCellsSize = 0;
+  private volatile long flushedOutputFileSize = 0;
   private volatile long compactedCellsSize = 0;
   private volatile long majorCompactedCellsSize = 0;
 
@@ -1210,6 +1211,7 @@ public class HStore implements Store {
       // Commence the compaction.
       List<Path> newFiles = compaction.compact(throughputController, user);
 
+      long outputBytes = 0L;
       // TODO: get rid of this!
       if (!this.conf.getBoolean("hbase.hstore.compaction.complete", true)) {
         LOG.warn("hbase.hstore.compaction.complete is set to false");
@@ -1235,10 +1237,23 @@ public class HStore implements Store {
         compactedCellsCount += getCompactionProgress().totalCompactingKVs;
         compactedCellsSize += getCompactionProgress().totalCompactedSize;
       }
+
+      for (StoreFile sf : sfs) {
+        outputBytes += sf.getReader().length();
+      }
+
       // At this point the store will use new files for all new scanners.
       completeCompaction(filesToCompact); // update store size.
 
-      logCompactionEndMessage(cr, sfs, compactionStartTime);
+      long now = EnvironmentEdgeManager.currentTime();
+      if (region.getRegionServerServices() != null
+          && region.getRegionServerServices().getMetrics() != null) {
+        region.getRegionServerServices().getMetrics().updateCompaction(cr.isMajor(),
+          now - compactionStartTime, cr.getFiles().size(), newFiles.size(), cr.getSize(),
+          outputBytes);
+      }
+
+      logCompactionEndMessage(cr, sfs, now, compactionStartTime);
       return sfs;
     } finally {
       finishCompactionRequest(cr);
@@ -1330,8 +1345,7 @@ public class HStore implements Store {
    * @param compactionStartTime Start time.
    */
   private void logCompactionEndMessage(
-      CompactionRequest cr, List<StoreFile> sfs, long compactionStartTime) {
-    long now = EnvironmentEdgeManager.currentTime();
+      CompactionRequest cr, List<StoreFile> sfs, long now, long compactionStartTime) {
     StringBuilder message = new StringBuilder(
       "Completed" + (cr.isMajor() ? " major" : "") + " compaction of "
       + cr.getFiles().size() + (cr.isAllFiles() ? " (all)" : "") + " file(s) in "
@@ -2129,6 +2143,7 @@ public class HStore implements Store {
     private List<Path> committedFiles;
     private long cacheFlushCount;
     private long cacheFlushSize;
+    private long outputFileSize;
 
     private StoreFlusherImpl(long cacheFlushSeqNum) {
       this.cacheFlushSeqNum = cacheFlushSeqNum;
@@ -2163,7 +2178,9 @@ public class HStore implements Store {
       List<StoreFile> storeFiles = new ArrayList<StoreFile>(this.tempFiles.size());
       for (Path storeFilePath : tempFiles) {
         try {
-          storeFiles.add(HStore.this.commitFile(storeFilePath, cacheFlushSeqNum, status));
+          StoreFile sf = HStore.this.commitFile(storeFilePath, cacheFlushSeqNum, status);
+          outputFileSize += sf.getReader().length();
+          storeFiles.add(sf);
         } catch (IOException ex) {
           LOG.error("Failed to commit store file " + storeFilePath, ex);
           // Try to delete the files we have committed before.
@@ -2189,12 +2206,18 @@ public class HStore implements Store {
 
       HStore.this.flushedCellsCount += cacheFlushCount;
       HStore.this.flushedCellsSize += cacheFlushSize;
+      HStore.this.flushedOutputFileSize += outputFileSize;
 
       // Add new file to store files.  Clear snapshot too while we have the Store write lock.
       return HStore.this.updateStorefiles(storeFiles, snapshot.getId());
     }
 
     @Override
+    public long getOutputFileSize() {
+      return outputFileSize;
+    }
+
+    @Override
     public List<Path> getCommittedFiles() {
       return committedFiles;
     }
@@ -2257,7 +2280,7 @@ public class HStore implements Store {
   }
 
   public static final long FIXED_OVERHEAD =
-      ClassSize.align(ClassSize.OBJECT + (16 * ClassSize.REFERENCE) + (10 * Bytes.SIZEOF_LONG)
+      ClassSize.align(ClassSize.OBJECT + (16 * ClassSize.REFERENCE) + (11 * Bytes.SIZEOF_LONG)
               + (5 * Bytes.SIZEOF_INT) + (2 * Bytes.SIZEOF_BOOLEAN));
 
   public static final long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD
@@ -2305,6 +2328,11 @@ public class HStore implements Store {
   }
 
   @Override
+  public long getFlushedOutputFileSize() {
+    return flushedOutputFileSize;
+  }
+
+  @Override
   public long getCompactedCellsCount() {
     return compactedCellsCount;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/797562e6/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java
index 40c5046..a69d8c0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java
@@ -486,25 +486,16 @@ class MemStoreFlusher implements FlushRequester {
    */
   private boolean flushRegion(final Region region, final boolean emergencyFlush,
       boolean forceFlushAllStores) {
-    long startTime = 0;
     synchronized (this.regionsInQueue) {
       FlushRegionEntry fqe = this.regionsInQueue.remove(region);
       // Use the start time of the FlushRegionEntry if available
-      if (fqe != null) {
-        startTime = fqe.createTime;
-      }
       if (fqe != null && emergencyFlush) {
         // Need to remove from region from delay queue.  When NOT an
         // emergencyFlush, then item was removed via a flushQueue.poll.
         flushQueue.remove(fqe);
-     }
-    }
-    if (startTime == 0) {
-      // Avoid getting the system time unless we don't have a FlushRegionEntry;
-      // shame we can't capture the time also spent in the above synchronized
-      // block
-      startTime = EnvironmentEdgeManager.currentTime();
+      }
     }
+
     lock.readLock().lock();
     try {
       notifyFlushRequest(region, emergencyFlush);
@@ -518,10 +509,6 @@ class MemStoreFlusher implements FlushRequester {
         server.compactSplitThread.requestSystemCompaction(
             region, Thread.currentThread().getName());
       }
-      if (flushResult.isFlushSucceeded()) {
-        long endTime = EnvironmentEdgeManager.currentTime();
-        server.metricsRegionServer.updateFlushTime(endTime - startTime);
-      }
     } catch (DroppedSnapshotException ex) {
       // Cache flush can fail in a few places. If it fails in a critical
       // section, we get a DroppedSnapshotException and a replay of wal

http://git-wip-us.apache.org/repos/asf/hbase/blob/797562e6/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServer.java
index 7ff9bed..8bca6c5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServer.java
@@ -117,7 +117,18 @@ public class MetricsRegionServer {
     serverSource.incrSplitSuccess();
   }
 
-  public void updateFlushTime(long t) {
+  public void updateFlush(long t, long memstoreSize, long fileSize) {
     serverSource.updateFlushTime(t);
+    serverSource.updateFlushMemstoreSize(memstoreSize);
+    serverSource.updateFlushOutputSize(fileSize);
+  }
+
+  public void updateCompaction(boolean isMajor, long t, int inputFileCount, int outputFileCount,
+      long inputBytes, long outputBytes) {
+    serverSource.updateCompactionTime(isMajor, t);
+    serverSource.updateCompactionInputFileCount(isMajor, inputFileCount);
+    serverSource.updateCompactionOutputFileCount(isMajor, outputFileCount);
+    serverSource.updateCompactionInputSize(isMajor, inputBytes);
+    serverSource.updateCompactionOutputSize(isMajor, outputBytes);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/797562e6/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 f4a2574..2d27219 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
@@ -166,7 +166,6 @@ import org.apache.hadoop.hbase.quotas.RegionServerQuotaManager;
 import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl;
 import org.apache.hadoop.hbase.regionserver.Leases.Lease;
 import org.apache.hadoop.hbase.regionserver.Leases.LeaseStillHeldException;
-import org.apache.hadoop.hbase.regionserver.Region.FlushResult;
 import org.apache.hadoop.hbase.regionserver.Region.Operation;
 import org.apache.hadoop.hbase.regionserver.ScannerContext.LimitScope;
 import org.apache.hadoop.hbase.regionserver.handler.OpenMetaHandler;
@@ -1421,14 +1420,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       if (shouldFlush) {
         boolean writeFlushWalMarker =  request.hasWriteFlushWalMarker() ?
             request.getWriteFlushWalMarker() : false;
-        long startTime = EnvironmentEdgeManager.currentTime();
         // Go behind the curtain so we can manage writing of the flush WAL marker
         HRegion.FlushResultImpl flushResult = (HRegion.FlushResultImpl)
             ((HRegion)region).flushcache(true, writeFlushWalMarker);
-        if (flushResult.isFlushSucceeded()) {
-          long endTime = EnvironmentEdgeManager.currentTime();
-          regionServer.metricsRegionServer.updateFlushTime(endTime - startTime);
-        }
         boolean compactionNeeded = flushResult.isCompactionNeeded();
         if (compactionNeeded) {
           regionServer.compactSplitThread.requestSystemCompaction(region,
@@ -1567,18 +1561,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       }
       LOG.info("Receiving merging request for  " + regionA + ", " + regionB
           + ",forcible=" + forcible);
-      long startTime = EnvironmentEdgeManager.currentTime();
-      FlushResult flushResult = regionA.flush(true);
-      if (flushResult.isFlushSucceeded()) {
-        long endTime = EnvironmentEdgeManager.currentTime();
-        regionServer.metricsRegionServer.updateFlushTime(endTime - startTime);
-      }
-      startTime = EnvironmentEdgeManager.currentTime();
-      flushResult = regionB.flush(true);
-      if (flushResult.isFlushSucceeded()) {
-        long endTime = EnvironmentEdgeManager.currentTime();
-        regionServer.metricsRegionServer.updateFlushTime(endTime - startTime);
-      }
+      regionA.flush(true);
+      regionB.flush(true);
       regionServer.compactSplitThread.requestRegionsMerge(regionA, regionB, forcible,
           masterSystemTime, RpcServer.getRequestUser());
       return MergeRegionsResponse.newBuilder().build();
@@ -1991,12 +1975,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
             + "Replicas are auto-split when their primary is split.");
       }
       LOG.info("Splitting " + region.getRegionInfo().getRegionNameAsString());
-      long startTime = EnvironmentEdgeManager.currentTime();
-      FlushResult flushResult = region.flush(true);
-      if (flushResult.isFlushSucceeded()) {
-        long endTime = EnvironmentEdgeManager.currentTime();
-        regionServer.metricsRegionServer.updateFlushTime(endTime - startTime);
-      }
+      region.flush(true);
       byte[] splitPoint = null;
       if (request.hasSplitPoint()) {
         splitPoint = request.getSplitPoint().toByteArray();

http://git-wip-us.apache.org/repos/asf/hbase/blob/797562e6/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
index 2a71629..c6689a9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
@@ -244,4 +244,9 @@ public interface RegionServerServices extends OnlineRegions, FavoredNodesForRegi
    *         global memstore size already exceeds lower limit.
    */
   double getFlushPressure();
+
+  /**
+   * @return the metrics tracker for the region server
+   */
+  MetricsRegionServer getMetrics();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/797562e6/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
index c167535..1cffaad 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
@@ -124,7 +124,7 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf
    List<KeyValueScanner> getScanners(List<StoreFile> files, boolean cacheBlocks, boolean isGet,
           boolean usePread, boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow,
           byte[] stopRow, long readPt, boolean includeMemstoreScanner) throws IOException;
-  
+
   ScanInfo getScanInfo();
 
   /**
@@ -439,6 +439,11 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf
   long getFlushedCellsSize();
 
   /**
+   * @return The total size of out output files on disk, in bytes
+   */
+  long getFlushedOutputFileSize();
+
+  /**
    * @return The number of cells processed during minor compactions
    */
   long getCompactedCellsCount();

http://git-wip-us.apache.org/repos/asf/hbase/blob/797562e6/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFlushContext.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFlushContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFlushContext.java
index f4f25dd..f62e96e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFlushContext.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFlushContext.java
@@ -85,4 +85,9 @@ interface StoreFlushContext {
    * @return a list of Paths for new files
    */
   List<Path> getCommittedFiles();
+
+  /**
+   * @return the total file size for flush output files, in bytes
+   */
+  long getOutputFileSize();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/797562e6/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreEngine.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreEngine.java
index 1e16ca8..9255634 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreEngine.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreEngine.java
@@ -100,13 +100,6 @@ public class StripeStoreEngine extends StoreEngine<StripeStoreFlusher,
     }
 
     @Override
-    public List<Path> compact(ThroughputController throughputController)
-        throws IOException {
-      Preconditions.checkArgument(this.stripeRequest != null, "Cannot compact without selection");
-      return this.stripeRequest.execute(compactor, throughputController, null);
-    }
-
-    @Override
     public List<Path> compact(ThroughputController throughputController, User user)
         throws IOException {
       Preconditions.checkArgument(this.stripeRequest != null, "Cannot compact without selection");

http://git-wip-us.apache.org/repos/asf/hbase/blob/797562e6/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionContext.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionContext.java
index 6902c40..ef6d7cb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionContext.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionContext.java
@@ -66,13 +66,6 @@ public abstract class CompactionContext {
     this.request = request;
   }
 
-  /**
-   * Runs the compaction based on current selection. select/forceSelect must have been called.
-   * @return The new file paths resulting from compaction.
-   */
-  public abstract List<Path> compact(ThroughputController throughputController)
-      throws IOException;
-
   public abstract List<Path> compact(ThroughputController throughputController, User user)
       throws IOException;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/797562e6/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionRequest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionRequest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionRequest.java
index d52077ba..268bb09 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionRequest.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionRequest.java
@@ -103,6 +103,7 @@ public class CompactionRequest implements Comparable<CompactionRequest> {
     this.regionName = other.regionName;
     this.storeName = other.storeName;
     this.totalSize = other.totalSize;
+    recalculateSize();
     return this;
   }
 
@@ -225,10 +226,12 @@ public class CompactionRequest implements Comparable<CompactionRequest> {
         Collections2.transform(Collections2.filter(
             this.getFiles(),
             new Predicate<StoreFile>() {
+              @Override
               public boolean apply(StoreFile sf) {
                 return sf.getReader() != null;
               }
           }), new Function<StoreFile, String>() {
+            @Override
             public String apply(StoreFile sf) {
               return StringUtils.humanReadableInt(
                 (sf.getReader() == null) ? 0 : sf.getReader().length());

http://git-wip-us.apache.org/repos/asf/hbase/blob/797562e6/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
index e634327..6cd1963 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.regionserver.CompactionRequestor;
 import org.apache.hadoop.hbase.regionserver.FlushRequester;
 import org.apache.hadoop.hbase.regionserver.HeapMemoryManager;
 import org.apache.hadoop.hbase.regionserver.Leases;
+import org.apache.hadoop.hbase.regionserver.MetricsRegionServer;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionServerAccounting;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
@@ -319,6 +320,7 @@ public class MockRegionServerServices implements RegionServerServices {
     return null;
   }
 
+  @Override
   public ThroughputController getFlushThroughputController() {
     return null;
   }
@@ -327,4 +329,9 @@ public class MockRegionServerServices implements RegionServerServices {
   public double getFlushPressure() {
     return 0;
   }
+
+  @Override
+  public MetricsRegionServer getMetrics() {
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/797562e6/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
index 4de4a5f..69f2e35 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
@@ -99,6 +99,7 @@ import org.apache.hadoop.hbase.regionserver.FlushRequester;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HeapMemoryManager;
 import org.apache.hadoop.hbase.regionserver.Leases;
+import org.apache.hadoop.hbase.regionserver.MetricsRegionServer;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionServerAccounting;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
@@ -664,6 +665,7 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
     return null;
   }
 
+  @Override
   public ThroughputController getFlushThroughputController() {
     return null;
   }
@@ -672,4 +674,9 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
   public double getFlushPressure() {
     return 0;
   }
+
+  @Override
+  public MetricsRegionServer getMetrics() {
+    return null;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/797562e6/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
index 06b4c46..72dcbe7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
@@ -86,7 +86,7 @@ public class TestCompaction {
   @Rule public TestName name = new TestName();
   private static final HBaseTestingUtility UTIL = HBaseTestingUtility.createLocalHTU();
   protected Configuration conf = UTIL.getConfiguration();
-  
+
   private HRegion r = null;
   private HTableDescriptor htd = null;
   private static final byte [] COLUMN_FAMILY = fam1;
@@ -158,6 +158,7 @@ public class TestCompaction {
 
       HRegion spyR = spy(r);
       doAnswer(new Answer() {
+        @Override
         public Object answer(InvocationOnMock invocation) throws Throwable {
           r.writestate.writesEnabled = false;
           return invocation.callRealMethod();
@@ -363,12 +364,6 @@ public class TestCompaction {
       }
 
       @Override
-      public List<Path> compact(ThroughputController throughputController)
-          throws IOException {
-        return compact(throughputController, null);
-      }
-
-      @Override
       public List<Path> compact(ThroughputController throughputController, User user)
           throws IOException {
         finishCompaction(this.selectedFiles);
@@ -421,12 +416,6 @@ public class TestCompaction {
       }
 
       @Override
-      public List<Path> compact(ThroughputController throughputController)
-          throws IOException {
-        return compact(throughputController, null);
-      }
-
-      @Override
       public List<Path> compact(ThroughputController throughputController, User user)
           throws IOException {
         try {
@@ -467,6 +456,7 @@ public class TestCompaction {
     @Override
     public void cancelCompaction(Object object) {}
 
+    @Override
     public int getPriority() {
       return Integer.MIN_VALUE; // some invalid value, see createStoreMock
     }
@@ -511,9 +501,10 @@ public class TestCompaction {
     when(
       r.compact(any(CompactionContext.class), any(Store.class),
         any(ThroughputController.class), any(User.class))).then(new Answer<Boolean>() {
+      @Override
       public Boolean answer(InvocationOnMock invocation) throws Throwable {
         invocation.getArgumentAt(0, CompactionContext.class).compact(
-          invocation.getArgumentAt(2, ThroughputController.class));
+          invocation.getArgumentAt(2, ThroughputController.class), null);
         return true;
       }
     });

http://git-wip-us.apache.org/repos/asf/hbase/blob/797562e6/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServer.java
index 4bfa64d..5f56ba9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServer.java
@@ -138,5 +138,69 @@ public class TestMetricsRegionServer {
     HELPER.assertCounter("slowIncrementCount", 15, serverSource);
     HELPER.assertCounter("slowPutCount", 16, serverSource);
   }
+
+  String FLUSH_TIME = "flushTime";
+  String FLUSH_TIME_DESC = "Histogram for the time in millis for memstore flush";
+  String FLUSH_MEMSTORE_SIZE = "flushMemstoreSize";
+  String FLUSH_MEMSTORE_SIZE_DESC = "Histogram for number of bytes in the memstore for a flush";
+  String FLUSH_FILE_SIZE = "flushFileSize";
+  String FLUSH_FILE_SIZE_DESC = "Histogram for number of bytes in the resulting file for a flush";
+  String FLUSHED_OUTPUT_BYTES = "flushedOutputBytes";
+  String FLUSHED_OUTPUT_BYTES_DESC = "Total number of bytes written from flush";
+  String FLUSHED_MEMSTORE_BYTES = "flushedMemstoreBytes";
+  String FLUSHED_MEMSTORE_BYTES_DESC = "Total number of bytes of cells in memstore from flush";
+
+  @Test
+  public void testFlush() {
+    rsm.updateFlush(1, 2, 3);
+    HELPER.assertCounter("flushTime_num_ops", 1, serverSource);
+    HELPER.assertCounter("flushMemstoreSize_num_ops", 1, serverSource);
+    HELPER.assertCounter("flushOutputSize_num_ops", 1, serverSource);
+    HELPER.assertCounter("flushedMemstoreBytes", 2, serverSource);
+    HELPER.assertCounter("flushedOutputBytes", 3, serverSource);
+
+    rsm.updateFlush(10, 20, 30);
+    HELPER.assertCounter("flushTimeNumOps", 2, serverSource);
+    HELPER.assertCounter("flushMemstoreSize_num_ops", 2, serverSource);
+    HELPER.assertCounter("flushOutputSize_num_ops", 2, serverSource);
+    HELPER.assertCounter("flushedMemstoreBytes", 22, serverSource);
+    HELPER.assertCounter("flushedOutputBytes", 33, serverSource);
+  }
+
+  @Test
+  public void testCompaction() {
+    rsm.updateCompaction(false, 1, 2, 3, 4, 5);
+    HELPER.assertCounter("compactionTime_num_ops", 1, serverSource);
+    HELPER.assertCounter("compactionInputFileCount_num_ops", 1, serverSource);
+    HELPER.assertCounter("compactionInputSize_num_ops", 1, serverSource);
+    HELPER.assertCounter("compactionOutputFileCount_num_ops", 1, serverSource);
+    HELPER.assertCounter("compactedInputBytes", 4, serverSource);
+    HELPER.assertCounter("compactedoutputBytes", 5, serverSource);
+
+    rsm.updateCompaction(false, 10, 20, 30, 40, 50);
+    HELPER.assertCounter("compactionTime_num_ops", 2, serverSource);
+    HELPER.assertCounter("compactionInputFileCount_num_ops", 2, serverSource);
+    HELPER.assertCounter("compactionInputSize_num_ops", 2, serverSource);
+    HELPER.assertCounter("compactionOutputFileCount_num_ops", 2, serverSource);
+    HELPER.assertCounter("compactedInputBytes", 44, serverSource);
+    HELPER.assertCounter("compactedoutputBytes", 55, serverSource);
+
+    // do major compaction
+    rsm.updateCompaction(true, 100, 200, 300, 400, 500);
+
+    HELPER.assertCounter("compactionTime_num_ops", 3, serverSource);
+    HELPER.assertCounter("compactionInputFileCount_num_ops", 3, serverSource);
+    HELPER.assertCounter("compactionInputSize_num_ops", 3, serverSource);
+    HELPER.assertCounter("compactionOutputFileCount_num_ops", 3, serverSource);
+    HELPER.assertCounter("compactedInputBytes", 444, serverSource);
+    HELPER.assertCounter("compactedoutputBytes", 555, serverSource);
+
+    HELPER.assertCounter("majorCompactionTime_num_ops", 1, serverSource);
+    HELPER.assertCounter("majorCompactionInputFileCount_num_ops", 1, serverSource);
+    HELPER.assertCounter("majorCompactionInputSize_num_ops", 1, serverSource);
+    HELPER.assertCounter("majorCompactionOutputFileCount_num_ops", 1, serverSource);
+    HELPER.assertCounter("majorCompactedInputBytes", 400, serverSource);
+    HELPER.assertCounter("majorCompactedoutputBytes", 500, serverSource);
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/797562e6/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreEngine.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreEngine.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreEngine.java
index 6b641c1..635e5b4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreEngine.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreEngine.java
@@ -96,7 +96,7 @@ public class TestStripeStoreEngine {
     assertEquals(2, compaction.getRequest().getFiles().size());
     assertFalse(compaction.getRequest().getFiles().contains(sf));
     // Make sure the correct method it called on compactor.
-    compaction.compact(NoLimitThroughputController.INSTANCE);
+    compaction.compact(NoLimitThroughputController.INSTANCE, null);
     verify(mockCompactor, times(1)).compact(compaction.getRequest(), targetCount, 0L,
       StripeStoreFileManager.OPEN_KEY, StripeStoreFileManager.OPEN_KEY, null, null,
       NoLimitThroughputController.INSTANCE, null);


[43/50] [abbrv] hbase git commit: HBASE-15295 MutateTableAccess.multiMutate() does not get high priority causing a deadlock

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java
index 8b2b733..45093bb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java
@@ -35,6 +35,8 @@ import org.apache.hadoop.hbase.MasterNotRunningException;
 import org.apache.hadoop.hbase.PleaseHoldException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalanceRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest;
@@ -310,6 +312,14 @@ public class TestHBaseAdminNoCluster {
           }
         });
     Mockito.when(connection.getKeepAliveMasterService()).thenReturn(masterAdmin);
+    RpcControllerFactory rpcControllerFactory = Mockito.mock(RpcControllerFactory.class);
+    Mockito.when(connection.getRpcControllerFactory()).thenReturn(rpcControllerFactory);
+    Mockito.when(rpcControllerFactory.newController()).thenReturn(
+      Mockito.mock(PayloadCarryingRpcController.class));
+
+    // we need a real retrying caller
+    RpcRetryingCallerFactory callerFactory = new RpcRetryingCallerFactory(configuration);
+    Mockito.when(connection.getRpcRetryingCallerFactory()).thenReturn(callerFactory);
 
     Admin admin = null;
     try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
index 77771ba..d4d319a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
@@ -285,7 +285,7 @@ public class TestScannersFromClientSide {
   private void verifyExpectedCounts(Table table, Scan scan, int expectedRowCount,
       int expectedCellCount) throws Exception {
     ResultScanner scanner = table.getScanner(scan);
-    
+
     int rowCount = 0;
     int cellCount = 0;
     Result r = null;
@@ -609,7 +609,7 @@ public class TestScannersFromClientSide {
     byte[] regionName = hri.getRegionName();
     int i = cluster.getServerWith(regionName);
     HRegionServer rs = cluster.getRegionServer(i);
-    ProtobufUtil.closeRegion(
+    ProtobufUtil.closeRegion(null,
       rs.getRSRpcServices(), rs.getServerName(), regionName);
     long startTime = EnvironmentEdgeManager.currentTime();
     long timeOut = 300000;
@@ -627,7 +627,7 @@ public class TestScannersFromClientSide {
     RegionStates states = master.getAssignmentManager().getRegionStates();
     states.regionOffline(hri);
     states.updateRegionState(hri, State.OPENING);
-    ProtobufUtil.openRegion(rs.getRSRpcServices(), rs.getServerName(), hri);
+    ProtobufUtil.openRegion(null, rs.getRSRpcServices(), rs.getServerName(), hri);
     startTime = EnvironmentEdgeManager.currentTime();
     while (true) {
       if (rs.getOnlineRegion(regionName) != null) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/DelegatingRpcScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/DelegatingRpcScheduler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/DelegatingRpcScheduler.java
new file mode 100644
index 0000000..b1b3b23
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/DelegatingRpcScheduler.java
@@ -0,0 +1,76 @@
+/**
+ * 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.ipc;
+
+import java.io.IOException;
+
+public class DelegatingRpcScheduler extends RpcScheduler {
+  protected RpcScheduler delegate;
+
+  public DelegatingRpcScheduler(RpcScheduler delegate) {
+    this.delegate = delegate;
+  }
+
+  @Override
+  public void stop() {
+    delegate.stop();
+  }
+  @Override
+  public void start() {
+    delegate.start();
+  }
+  @Override
+  public void init(Context context) {
+    delegate.init(context);
+  }
+  @Override
+  public int getReplicationQueueLength() {
+    return delegate.getReplicationQueueLength();
+  }
+
+  @Override
+  public int getPriorityQueueLength() {
+    return delegate.getPriorityQueueLength();
+  }
+
+  @Override
+  public int getGeneralQueueLength() {
+    return delegate.getGeneralQueueLength();
+  }
+
+  @Override
+  public int getActiveRpcHandlerCount() {
+    return delegate.getActiveRpcHandlerCount();
+  }
+
+  @Override
+  public boolean dispatch(CallRunner task) throws IOException, InterruptedException {
+    return delegate.dispatch(task);
+  }
+
+  @Override
+  public long getNumGeneralCallsDropped() {
+    return delegate.getNumGeneralCallsDropped();
+  }
+
+  @Override
+  public long getNumLifoModeSwitches() {
+    return delegate.getNumLifoModeSwitches();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java
index 052e05c..32e3058 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java
@@ -196,7 +196,7 @@ public class TestLoadIncrementalHFilesSplitRecovery {
           ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices())) {
         if (hri.getTable().equals(table)) {
           // splitRegion doesn't work if startkey/endkey are null
-          ProtobufUtil.split(hrs.getRSRpcServices(), hri, rowkey(ROWCOUNT / 2)); // hard code split
+          ProtobufUtil.split(null, hrs.getRSRpcServices(), hri, rowkey(ROWCOUNT / 2));
         }
       }
 
@@ -480,6 +480,7 @@ public class TestLoadIncrementalHFilesSplitRecovery {
     final AtomicInteger countedLqis = new AtomicInteger();
     LoadIncrementalHFiles loader = new LoadIncrementalHFiles(util.getConfiguration()) {
 
+      @Override
       protected List<LoadQueueItem> groupOrSplit(
           Multimap<ByteBuffer, LoadQueueItem> regionGroups,
           final LoadQueueItem item, final Table htable,

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClockSkewDetection.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClockSkewDetection.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClockSkewDetection.java
index 142437c..b2ffc3e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClockSkewDetection.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClockSkewDetection.java
@@ -19,6 +19,8 @@
 package org.apache.hadoop.hbase.master;
 
 import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 import java.net.InetAddress;
 
@@ -32,6 +34,7 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
@@ -81,7 +84,7 @@ public class TestClockSkewDetection {
 
       @Override
       public void abort(String why, Throwable e) {}
-      
+
       @Override
       public boolean isAborted() {
         return false;
@@ -103,10 +106,11 @@ public class TestClockSkewDetection {
 
       @Override
       public ClusterConnection getClusterConnection() {
-        // TODO Auto-generated method stub
-        return null;
+        ClusterConnection conn = mock(ClusterConnection.class);
+        when(conn.getRpcControllerFactory()).thenReturn(mock(RpcControllerFactory.class));
+        return conn;
       }
-    }, null, false);
+    }, null, true);
 
     LOG.debug("regionServerStartup 1");
     InetAddress ia1 = InetAddress.getLocalHost();
@@ -135,7 +139,7 @@ public class TestClockSkewDetection {
       //we want an exception
       LOG.info("Recieved expected exception: "+e);
     }
-    
+
     try {
       // Master Time < Region Server Time
       LOG.debug("Test: Master Time < Region Server Time");
@@ -151,7 +155,7 @@ public class TestClockSkewDetection {
       // we want an exception
       LOG.info("Recieved expected exception: " + e);
     }
-    
+
     // make sure values above warning threshold but below max threshold don't kill
     LOG.debug("regionServerStartup 4");
     InetAddress ia4 = InetAddress.getLocalHost();
@@ -160,7 +164,7 @@ public class TestClockSkewDetection {
     request.setServerStartCode(-1);
     request.setServerCurrentTime(System.currentTimeMillis() - warningSkew * 2);
     sm.regionServerStartup(request.build(), ia4);
-    
+
     // make sure values above warning threshold but below max threshold don't kill
     LOG.debug("regionServerStartup 5");
     InetAddress ia5 = InetAddress.getLocalHost();
@@ -169,7 +173,7 @@ public class TestClockSkewDetection {
     request.setServerStartCode(-1);
     request.setServerCurrentTime(System.currentTimeMillis() + warningSkew * 2);
     sm.regionServerStartup(request.build(), ia5);
-    
+
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java
index 03b43f1..0ee75a8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java
@@ -114,7 +114,7 @@ public class TestRegionServerNoMaster {
       return;
     }
 
-    ProtobufUtil.openRegion(hrs.getRSRpcServices(),
+    ProtobufUtil.openRegion(null, hrs.getRSRpcServices(),
       hrs.getServerName(), HRegionInfo.FIRST_META_REGIONINFO);
     while (true) {
       sn = mtl.getMetaRegionLocation(zkw);

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
index cbc1a90..22a9748 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
@@ -372,7 +372,7 @@ public class SecureTestUtil {
             BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW);
             AccessControlService.BlockingInterface protocol =
                 AccessControlService.newBlockingStub(service);
-            ProtobufUtil.grant(protocol, user, actions);
+            ProtobufUtil.grant(null, protocol, user, actions);
           }
         }
         return null;
@@ -395,7 +395,7 @@ public class SecureTestUtil {
             BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW);
             AccessControlService.BlockingInterface protocol =
                 AccessControlService.newBlockingStub(service);
-            ProtobufUtil.revoke(protocol, user, actions);
+            ProtobufUtil.revoke(null, protocol, user, actions);
           }
         }
         return null;
@@ -418,7 +418,7 @@ public class SecureTestUtil {
             BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW);
             AccessControlService.BlockingInterface protocol =
                 AccessControlService.newBlockingStub(service);
-            ProtobufUtil.grant(protocol, user, namespace, actions);
+            ProtobufUtil.grant(null, protocol, user, namespace, actions);
           }
         }
         return null;
@@ -483,7 +483,7 @@ public class SecureTestUtil {
             BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW);
             AccessControlService.BlockingInterface protocol =
                 AccessControlService.newBlockingStub(service);
-            ProtobufUtil.revoke(protocol, user, namespace, actions);
+            ProtobufUtil.revoke(null, protocol, user, namespace, actions);
           }
         }
         return null;
@@ -507,7 +507,7 @@ public class SecureTestUtil {
             BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW);
             AccessControlService.BlockingInterface protocol =
                 AccessControlService.newBlockingStub(service);
-            ProtobufUtil.grant(protocol, user, table, family, qualifier, actions);
+            ProtobufUtil.grant(null, protocol, user, table, family, qualifier, actions);
           }
         }
         return null;
@@ -573,7 +573,7 @@ public class SecureTestUtil {
             BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW);
             AccessControlService.BlockingInterface protocol =
                 AccessControlService.newBlockingStub(service);
-            ProtobufUtil.revoke(protocol, user, table, family, qualifier, actions);
+            ProtobufUtil.revoke(null, protocol, user, table, family, qualifier, actions);
           }
         }
         return null;

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/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 7b1454d..37c42a0 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
@@ -1165,7 +1165,7 @@ public class TestAccessController extends SecureTestUtil {
           BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE.getName());
           AccessControlService.BlockingInterface protocol =
             AccessControlService.newBlockingStub(service);
-          ProtobufUtil.grant(protocol, USER_RO.getShortName(), TEST_TABLE, TEST_FAMILY, null,
+          ProtobufUtil.grant(null, protocol, USER_RO.getShortName(), TEST_TABLE, TEST_FAMILY, null,
             Action.READ);
         }
         return null;
@@ -1180,7 +1180,7 @@ public class TestAccessController extends SecureTestUtil {
           BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE.getName());
           AccessControlService.BlockingInterface protocol =
             AccessControlService.newBlockingStub(service);
-          ProtobufUtil.revoke(protocol, USER_RO.getShortName(), TEST_TABLE, TEST_FAMILY, null,
+          ProtobufUtil.revoke(null, protocol, USER_RO.getShortName(), TEST_TABLE, TEST_FAMILY, null,
             Action.READ);
         }
         return null;
@@ -1195,7 +1195,7 @@ public class TestAccessController extends SecureTestUtil {
           BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE.getName());
           AccessControlService.BlockingInterface protocol =
               AccessControlService.newBlockingStub(service);
-          ProtobufUtil.getUserPermissions(protocol, TEST_TABLE);
+          ProtobufUtil.getUserPermissions(null, protocol, TEST_TABLE);
         }
         return null;
       }
@@ -1209,7 +1209,7 @@ public class TestAccessController extends SecureTestUtil {
           BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW);
           AccessControlService.BlockingInterface protocol =
             AccessControlService.newBlockingStub(service);
-          ProtobufUtil.getUserPermissions(protocol);
+          ProtobufUtil.getUserPermissions(null, protocol);
         }
         return null;
       }
@@ -1620,7 +1620,7 @@ public class TestAccessController extends SecureTestUtil {
         BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
         AccessControlService.BlockingInterface protocol =
             AccessControlService.newBlockingStub(service);
-        perms = ProtobufUtil.getUserPermissions(protocol, tableName);
+        perms = ProtobufUtil.getUserPermissions(null, protocol, tableName);
       } finally {
         acl.close();
       }
@@ -1647,7 +1647,7 @@ public class TestAccessController extends SecureTestUtil {
         BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
         AccessControlService.BlockingInterface protocol =
             AccessControlService.newBlockingStub(service);
-        perms = ProtobufUtil.getUserPermissions(protocol, tableName);
+        perms = ProtobufUtil.getUserPermissions(null, protocol, tableName);
       } finally {
         acl.close();
       }
@@ -1671,7 +1671,7 @@ public class TestAccessController extends SecureTestUtil {
         BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
         AccessControlService.BlockingInterface protocol =
             AccessControlService.newBlockingStub(service);
-        perms = ProtobufUtil.getUserPermissions(protocol, tableName);
+        perms = ProtobufUtil.getUserPermissions(null, protocol, tableName);
       } finally {
         acl.close();
       }
@@ -1691,7 +1691,7 @@ public class TestAccessController extends SecureTestUtil {
         BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
         AccessControlService.BlockingInterface protocol =
             AccessControlService.newBlockingStub(service);
-        perms = ProtobufUtil.getUserPermissions(protocol, tableName);
+        perms = ProtobufUtil.getUserPermissions(null, protocol, tableName);
       } finally {
         acl.close();
       }
@@ -1711,7 +1711,7 @@ public class TestAccessController extends SecureTestUtil {
         BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
         AccessControlService.BlockingInterface protocol =
             AccessControlService.newBlockingStub(service);
-        perms = ProtobufUtil.getUserPermissions(protocol, tableName);
+        perms = ProtobufUtil.getUserPermissions(null, protocol, tableName);
       } finally {
         acl.close();
       }
@@ -1734,7 +1734,7 @@ public class TestAccessController extends SecureTestUtil {
       BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW);
       AccessControlService.BlockingInterface protocol =
         AccessControlService.newBlockingStub(service);
-      perms = ProtobufUtil.getUserPermissions(protocol);
+      perms = ProtobufUtil.getUserPermissions(null, protocol);
     } finally {
       acl.close();
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java
index 96cd299..d5834fd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java
@@ -233,6 +233,7 @@ public class TestNamespaceCommands extends SecureTestUtil {
   @Test
   public void testModifyNamespace() throws Exception {
     AccessTestAction modifyNamespace = new AccessTestAction() {
+      @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER.preModifyNamespace(ObserverContext.createAndPrepare(CP_ENV, null),
           NamespaceDescriptor.create(TEST_NAMESPACE).addConfiguration("abc", "156").build());
@@ -359,7 +360,7 @@ public class TestNamespaceCommands extends SecureTestUtil {
               acl.coprocessorService(HConstants.EMPTY_START_ROW);
           AccessControlService.BlockingInterface protocol =
             AccessControlService.newBlockingStub(service);
-          ProtobufUtil.grant(protocol, testUser, TEST_NAMESPACE, Action.WRITE);
+          ProtobufUtil.grant(null, protocol, testUser, TEST_NAMESPACE, Action.WRITE);
         } finally {
           acl.close();
           connection.close();
@@ -376,7 +377,7 @@ public class TestNamespaceCommands extends SecureTestUtil {
           BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW);
           AccessControlService.BlockingInterface protocol =
             AccessControlService.newBlockingStub(service);
-          ProtobufUtil.grant(protocol, USER_GROUP_NS_ADMIN.getShortName(),
+          ProtobufUtil.grant(null, protocol, USER_GROUP_NS_ADMIN.getShortName(),
             TEST_NAMESPACE, Action.READ);
         }
         return null;
@@ -384,6 +385,7 @@ public class TestNamespaceCommands extends SecureTestUtil {
     };
 
     AccessTestAction revokeAction = new AccessTestAction() {
+      @Override
       public Object run() throws Exception {
         Connection connection = ConnectionFactory.createConnection(conf);
         Table acl = connection.getTable(AccessControlLists.ACL_TABLE_NAME);
@@ -392,7 +394,7 @@ public class TestNamespaceCommands extends SecureTestUtil {
               acl.coprocessorService(HConstants.EMPTY_START_ROW);
           AccessControlService.BlockingInterface protocol =
             AccessControlService.newBlockingStub(service);
-          ProtobufUtil.revoke(protocol, testUser, TEST_NAMESPACE, Action.WRITE);
+          ProtobufUtil.revoke(null, protocol, testUser, TEST_NAMESPACE, Action.WRITE);
         } finally {
           acl.close();
           connection.close();
@@ -402,6 +404,7 @@ public class TestNamespaceCommands extends SecureTestUtil {
     };
 
     AccessTestAction revokeNamespaceAction = new AccessTestAction() {
+      @Override
       public Object run() throws Exception {
         Connection connection = ConnectionFactory.createConnection(conf);
         Table acl = connection.getTable(AccessControlLists.ACL_TABLE_NAME);
@@ -410,7 +413,7 @@ public class TestNamespaceCommands extends SecureTestUtil {
               acl.coprocessorService(HConstants.EMPTY_START_ROW);
           AccessControlService.BlockingInterface protocol =
             AccessControlService.newBlockingStub(service);
-          ProtobufUtil.revoke(protocol, USER_GROUP_NS_ADMIN.getShortName(),
+          ProtobufUtil.revoke(null, protocol, USER_GROUP_NS_ADMIN.getShortName(),
             TEST_NAMESPACE, Action.READ);
         } finally {
           acl.close();
@@ -429,7 +432,7 @@ public class TestNamespaceCommands extends SecureTestUtil {
           BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW);
           AccessControlService.BlockingInterface protocol =
             AccessControlService.newBlockingStub(service);
-          ProtobufUtil.getUserPermissions(protocol, Bytes.toBytes(TEST_NAMESPACE));
+          ProtobufUtil.getUserPermissions(null, protocol, Bytes.toBytes(TEST_NAMESPACE));
         } finally {
           acl.close();
           connection.close();


[45/50] [abbrv] hbase git commit: HBASE-15295 MutateTableAccess.multiMutate() does not get high priority causing a deadlock

Posted by sy...@apache.org.
HBASE-15295 MutateTableAccess.multiMutate() does not get high priority causing a deadlock


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

Branch: refs/heads/hbase-12439
Commit: 05200976110135abb60f9b879b9b830671c07141
Parents: cbf9c1e
Author: Enis Soztutar <en...@apache.org>
Authored: Wed Mar 23 12:30:41 2016 -0700
Committer: Enis Soztutar <en...@apache.org>
Committed: Mon Mar 28 17:56:32 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/HRegionInfo.java    |   1 +
 .../hbase/client/BufferedMutatorImpl.java       |   2 +-
 .../hadoop/hbase/client/ClusterConnection.java  |  20 +-
 .../hbase/client/ConnectionConfiguration.java   | 144 ++++++
 .../hbase/client/ConnectionImplementation.java  |  69 ++-
 .../apache/hadoop/hbase/client/HBaseAdmin.java  | 511 +++++++++++++------
 .../org/apache/hadoop/hbase/client/HTable.java  |  55 +-
 .../hadoop/hbase/client/TableConfiguration.java | 144 ------
 .../hadoop/hbase/ipc/AbstractRpcClient.java     |   2 +-
 .../hadoop/hbase/ipc/CoprocessorRpcChannel.java |  11 +-
 .../hbase/ipc/MasterCoprocessorRpcChannel.java  |  18 +-
 .../hbase/ipc/RegionCoprocessorRpcChannel.java  |  46 +-
 .../ipc/RegionServerCoprocessorRpcChannel.java  |  10 +-
 .../hadoop/hbase/protobuf/ProtobufUtil.java     | 168 +++---
 .../security/access/AccessControlClient.java    |  46 +-
 .../hbase/zookeeper/MetaTableLocator.java       |  18 +-
 .../hbase/client/TestSnapshotFromAdmin.java     |  31 +-
 .../hadoop/hbase/DistributedHBaseCluster.java   |   4 +-
 .../hbase/tmpl/regionserver/RSStatusTmpl.jamon  |   2 +-
 .../org/apache/hadoop/hbase/ipc/CallRunner.java |   5 +-
 .../apache/hadoop/hbase/ipc/RpcScheduler.java   |   2 +-
 .../org/apache/hadoop/hbase/ipc/RpcServer.java  |  12 +-
 .../hbase/master/RegionPlacementMaintainer.java |   2 +-
 .../hadoop/hbase/master/ServerManager.java      |  34 +-
 .../apache/hadoop/hbase/MiniHBaseCluster.java   |   4 +-
 .../hadoop/hbase/TestGlobalMemStoreSize.java    |  14 +-
 .../hadoop/hbase/TestMetaTableAccessor.java     |  79 +++
 .../hadoop/hbase/TestMetaTableLocator.java      |   8 +-
 .../hbase/client/HConnectionTestingUtility.java |   5 +
 .../apache/hadoop/hbase/client/TestAdmin1.java  |   4 +-
 .../hadoop/hbase/client/TestFromClientSide.java |   6 +-
 .../hbase/client/TestFromClientSide3.java       |  10 +-
 .../hbase/client/TestHBaseAdminNoCluster.java   |  10 +
 .../client/TestScannersFromClientSide.java      |   6 +-
 .../hbase/ipc/DelegatingRpcScheduler.java       |  76 +++
 .../TestLoadIncrementalHFilesSplitRecovery.java |   3 +-
 .../hbase/master/TestClockSkewDetection.java    |  20 +-
 .../regionserver/TestRegionServerNoMaster.java  |   2 +-
 .../hbase/security/access/SecureTestUtil.java   |  12 +-
 .../security/access/TestAccessController.java   |  20 +-
 .../security/access/TestNamespaceCommands.java  |  13 +-
 41 files changed, 1095 insertions(+), 554 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
index 13ba23d..71f87f7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
@@ -202,6 +202,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
   public final static byte[] HIDDEN_START_KEY = Bytes.toBytes("hidden-start-key");
 
   /** HRegionInfo for first meta region */
+  // TODO: How come Meta regions still do not have encoded region names? Fix.
   public static final HRegionInfo FIRST_META_REGIONINFO =
       new HRegionInfo(1L, TableName.META_TABLE_NAME);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java
index ef3f7e9..01aaec5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java
@@ -88,7 +88,7 @@ public class BufferedMutatorImpl implements BufferedMutator {
     this.pool = params.getPool();
     this.listener = params.getListener();
 
-    TableConfiguration tableConf = new TableConfiguration(conf);
+    ConnectionConfiguration tableConf = new ConnectionConfiguration(conf);
     this.writeBufferSize = params.getWriteBufferSize() != BufferedMutatorParams.UNSET ?
         params.getWriteBufferSize() : tableConf.getWriteBufferSize();
     this.maxKeyValueSize = params.getMaxKeyValueSize() != BufferedMutatorParams.UNSET ?

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java
index 45589be..d348ffc 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java
@@ -31,11 +31,12 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicy;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterService;
 
-/** Internal methods on HConnection that should not be used by user code. */
+/** Internal methods on Connection that should not be used by user code. */
 @InterfaceAudience.Private
 // NOTE: Although this class is public, this class is meant to be used directly from internal
 // classes and unit tests only.
@@ -287,7 +288,22 @@ public interface ClusterConnection extends HConnection {
    * @return RpcRetryingCallerFactory
    */
   RpcRetryingCallerFactory getNewRpcRetryingCallerFactory(Configuration conf);
-  
+
+  /**
+   * @return Connection's RpcRetryingCallerFactory instance
+   */
+  RpcRetryingCallerFactory getRpcRetryingCallerFactory();
+
+  /**
+   * @return Connection's RpcControllerFactory instance
+   */
+  RpcControllerFactory getRpcControllerFactory();
+
+  /**
+   * @return a ConnectionConfiguration object holding parsed configuration values
+   */
+  ConnectionConfiguration getConnectionConfiguration();
+
   /**
    * @return the current statistics tracker associated with this connection
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionConfiguration.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionConfiguration.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionConfiguration.java
new file mode 100644
index 0000000..35bebae
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionConfiguration.java
@@ -0,0 +1,144 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required by applicable
+ * law or agreed to in writing, software distributed under the License is distributed on an "AS IS"
+ * BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License
+ * for the specific language governing permissions and limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.client;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Configuration parameters for the connection.
+ * Configuration is a heavy weight registry that does a lot of string operations and regex matching.
+ * Method calls into Configuration account for high CPU usage and have huge performance impact.
+ * This class caches connection-related configuration values in the  ConnectionConfiguration
+ * object so that expensive conf.getXXX() calls are avoided every time HTable, etc is instantiated.
+ * see HBASE-12128
+ */
+@InterfaceAudience.Private
+public class ConnectionConfiguration {
+
+  public static final String WRITE_BUFFER_SIZE_KEY = "hbase.client.write.buffer";
+  public static final long WRITE_BUFFER_SIZE_DEFAULT = 2097152;
+  public static final String MAX_KEYVALUE_SIZE_KEY = "hbase.client.keyvalue.maxsize";
+  public static final int MAX_KEYVALUE_SIZE_DEFAULT = -1;
+
+  private final long writeBufferSize;
+  private final int metaOperationTimeout;
+  private final int operationTimeout;
+  private final int scannerCaching;
+  private final long scannerMaxResultSize;
+  private final int primaryCallTimeoutMicroSecond;
+  private final int replicaCallTimeoutMicroSecondScan;
+  private final int retries;
+  private final int maxKeyValueSize;
+
+    // toggle for async/sync prefetch
+  private final boolean clientScannerAsyncPrefetch;
+
+    /**
+   * Constructor
+   * @param conf Configuration object
+   */
+  ConnectionConfiguration(Configuration conf) {
+    this.writeBufferSize = conf.getLong(WRITE_BUFFER_SIZE_KEY, WRITE_BUFFER_SIZE_DEFAULT);
+
+    this.metaOperationTimeout = conf.getInt(
+      HConstants.HBASE_CLIENT_META_OPERATION_TIMEOUT,
+      HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
+
+    this.operationTimeout = conf.getInt(
+      HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
+
+    this.scannerCaching = conf.getInt(
+      HConstants.HBASE_CLIENT_SCANNER_CACHING, HConstants.DEFAULT_HBASE_CLIENT_SCANNER_CACHING);
+
+    this.scannerMaxResultSize =
+        conf.getLong(HConstants.HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE_KEY,
+          HConstants.DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE);
+
+    this.primaryCallTimeoutMicroSecond =
+        conf.getInt("hbase.client.primaryCallTimeout.get", 10000); // 10ms
+
+    this.replicaCallTimeoutMicroSecondScan =
+        conf.getInt("hbase.client.replicaCallTimeout.scan", 1000000); // 1000 ms
+
+    this.retries = conf.getInt(
+       HConstants.HBASE_CLIENT_RETRIES_NUMBER, HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
+
+    this.clientScannerAsyncPrefetch = conf.getBoolean(
+       Scan.HBASE_CLIENT_SCANNER_ASYNC_PREFETCH, Scan.DEFAULT_HBASE_CLIENT_SCANNER_ASYNC_PREFETCH);
+
+    this.maxKeyValueSize = conf.getInt(MAX_KEYVALUE_SIZE_KEY, MAX_KEYVALUE_SIZE_DEFAULT);
+  }
+
+  /**
+   * Constructor
+   * This is for internal testing purpose (using the default value).
+   * In real usage, we should read the configuration from the Configuration object.
+   */
+  @VisibleForTesting
+  protected ConnectionConfiguration() {
+    this.writeBufferSize = WRITE_BUFFER_SIZE_DEFAULT;
+    this.metaOperationTimeout = HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT;
+    this.operationTimeout = HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT;
+    this.scannerCaching = HConstants.DEFAULT_HBASE_CLIENT_SCANNER_CACHING;
+    this.scannerMaxResultSize = HConstants.DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE;
+    this.primaryCallTimeoutMicroSecond = 10000;
+    this.replicaCallTimeoutMicroSecondScan = 1000000;
+    this.retries = HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER;
+    this.clientScannerAsyncPrefetch = Scan.DEFAULT_HBASE_CLIENT_SCANNER_ASYNC_PREFETCH;
+    this.maxKeyValueSize = MAX_KEYVALUE_SIZE_DEFAULT;
+  }
+
+  public long getWriteBufferSize() {
+    return writeBufferSize;
+  }
+
+  public int getMetaOperationTimeout() {
+    return metaOperationTimeout;
+  }
+
+  public int getOperationTimeout() {
+    return operationTimeout;
+  }
+
+  public int getScannerCaching() {
+    return scannerCaching;
+  }
+
+  public int getPrimaryCallTimeoutMicroSecond() {
+    return primaryCallTimeoutMicroSecond;
+  }
+
+  public int getReplicaCallTimeoutMicroSecondScan() {
+    return replicaCallTimeoutMicroSecondScan;
+  }
+
+  public int getRetriesNumber() {
+    return retries;
+  }
+
+  public int getMaxKeyValueSize() {
+    return maxKeyValueSize;
+  }
+
+  public long getScannerMaxResultSize() {
+    return scannerMaxResultSize;
+  }
+
+  public boolean isClientScannerAsyncPrefetch() {
+    return clientScannerAsyncPrefetch;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
index fd4dc6d..ecaf18b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
@@ -25,6 +25,26 @@ import com.google.protobuf.BlockingRpcChannel;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
 
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.lang.reflect.UndeclaredThrowableException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+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.AtomicInteger;
+
+import javax.annotation.Nullable;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -78,25 +98,6 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.zookeeper.KeeperException;
 
-import javax.annotation.Nullable;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.lang.reflect.UndeclaredThrowableException;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.util.ArrayList;
-import java.util.Date;
-import java.util.List;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-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.AtomicInteger;
 
 /**
  * Main implementation of {@link Connection} and {@link ClusterConnection} interfaces.
@@ -158,7 +159,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
 
   // cache the configuration value for tables so that we can avoid calling
   // the expensive Configuration to fetch the value multiple times.
-  private final TableConfiguration tableConfig;
+  private final ConnectionConfiguration connectionConfig;
 
   // Client rpc instance.
   private RpcClient rpcClient;
@@ -190,14 +191,14 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
     this.conf = conf;
     this.user = user;
     this.batchPool = pool;
-    this.tableConfig = new TableConfiguration(conf);
+    this.connectionConfig = new ConnectionConfiguration(conf);
     this.closed = false;
     this.pause = conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
         HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
     this.useMetaReplicas = conf.getBoolean(HConstants.USE_META_REPLICAS,
       HConstants.DEFAULT_USE_META_REPLICAS);
     // how many times to try, one more than max *retry* time
-    this.numTries = tableConfig.getRetriesNumber() + 1;
+    this.numTries = connectionConfig.getRetriesNumber() + 1;
     this.rpcTimeout = conf.getInt(
         HConstants.HBASE_RPC_TIMEOUT_KEY,
         HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
@@ -306,7 +307,8 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
 
   @Override
   public HTableInterface getTable(TableName tableName, ExecutorService pool) throws IOException {
-    return new HTable(tableName, this, tableConfig, rpcCallerFactory, rpcControllerFactory, pool);
+    return new HTable(tableName, this, connectionConfig,
+      rpcCallerFactory, rpcControllerFactory, pool);
   }
 
   @Override
@@ -318,10 +320,10 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
       params.pool(HTable.getDefaultExecutor(getConfiguration()));
     }
     if (params.getWriteBufferSize() == BufferedMutatorParams.UNSET) {
-      params.writeBufferSize(tableConfig.getWriteBufferSize());
+      params.writeBufferSize(connectionConfig.getWriteBufferSize());
     }
     if (params.getMaxKeyValueSize() == BufferedMutatorParams.UNSET) {
-      params.maxKeyValueSize(tableConfig.getMaxKeyValueSize());
+      params.maxKeyValueSize(connectionConfig.getMaxKeyValueSize());
     }
     return new BufferedMutatorImpl(this, rpcCallerFactory, rpcControllerFactory, params);
   }
@@ -2281,4 +2283,19 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
   public boolean hasCellBlockSupport() {
     return this.rpcClient.hasCellBlockSupport();
   }
-}
\ No newline at end of file
+
+  @Override
+  public ConnectionConfiguration getConnectionConfiguration() {
+    return this.connectionConfig;
+  }
+
+  @Override
+  public RpcRetryingCallerFactory getRpcRetryingCallerFactory() {
+    return this.rpcCallerFactory;
+  }
+
+  @Override
+  public RpcControllerFactory getRpcControllerFactory() {
+    return this.rpcControllerFactory;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/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 c2a0bb8..c1d07ae 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
@@ -68,7 +68,9 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
 import org.apache.hadoop.hbase.ipc.MasterCoprocessorRpcChannel;
+import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
 import org.apache.hadoop.hbase.ipc.RegionServerCoprocessorRpcChannel;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
@@ -217,6 +219,7 @@ public class HBaseAdmin implements Admin {
   private int operationTimeout;
 
   private RpcRetryingCallerFactory rpcCallerFactory;
+  private RpcControllerFactory rpcControllerFactory;
 
   private NonceGenerator ng;
 
@@ -229,6 +232,7 @@ public class HBaseAdmin implements Admin {
     this.conf = connection.getConfiguration();
     this.connection = connection;
 
+    // TODO: receive ConnectionConfiguration here rather than re-parsing these configs every time.
     this.pause = this.conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
         HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
     this.numRetries = this.conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
@@ -240,7 +244,8 @@ public class HBaseAdmin implements Admin {
     this.syncWaitTimeout = this.conf.getInt(
       "hbase.client.sync.wait.timeout.msec", 10 * 60000); // 10min
 
-    this.rpcCallerFactory = RpcRetryingCallerFactory.instantiate(this.conf);
+    this.rpcCallerFactory = connection.getRpcRetryingCallerFactory();
+    this.rpcControllerFactory = connection.getRpcControllerFactory();
 
     this.ng = this.connection.getNonceGenerator();
   }
@@ -266,17 +271,19 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public Future<Boolean> abortProcedureAsync(
-    final long procId,
-    final boolean mayInterruptIfRunning) throws IOException {
+      final long procId,
+      final boolean mayInterruptIfRunning) throws IOException {
     Boolean abortProcResponse = executeCallable(
       new MasterCallable<AbortProcedureResponse>(getConnection()) {
-    @Override
-    public AbortProcedureResponse call(int callTimeout) throws ServiceException {
-      AbortProcedureRequest abortProcRequest =
-          AbortProcedureRequest.newBuilder().setProcId(procId).build();
-      return master.abortProcedure(null,abortProcRequest);
-      }
-    }).getIsProcedureAborted();
+        @Override
+        public AbortProcedureResponse call(int callTimeout) throws ServiceException {
+          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+          controller.setCallTimeout(callTimeout);
+          AbortProcedureRequest abortProcRequest =
+              AbortProcedureRequest.newBuilder().setProcId(procId).build();
+          return master.abortProcedure(controller, abortProcRequest);
+        }
+      }).getIsProcedureAborted();
 
     AbortProcedureFuture abortProcFuture =
         new AbortProcedureFuture(this, procId, abortProcResponse);
@@ -342,9 +349,11 @@ public class HBaseAdmin implements Admin {
     return executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection()) {
       @Override
       public HTableDescriptor[] call(int callTimeout) throws ServiceException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
         GetTableDescriptorsRequest req =
             RequestConverter.buildGetTableDescriptorsRequest(pattern, includeSysTables);
-        return ProtobufUtil.getHTableDescriptorArray(master.getTableDescriptors(null, req));
+        return ProtobufUtil.getHTableDescriptorArray(master.getTableDescriptors(controller, req));
       }
     });
   }
@@ -376,9 +385,11 @@ public class HBaseAdmin implements Admin {
     return executeCallable(new MasterCallable<TableName[]>(getConnection()) {
       @Override
       public TableName[] call(int callTimeout) throws ServiceException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
         GetTableNamesRequest req =
             RequestConverter.buildGetTableNamesRequest(pattern, includeSysTables);
-        return ProtobufUtil.getTableNameArray(master.getTableNames(null, req)
+        return ProtobufUtil.getTableNameArray(master.getTableNames(controller, req)
             .getTableNamesList());
       }
     });
@@ -392,19 +403,23 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public HTableDescriptor getTableDescriptor(final TableName tableName) throws IOException {
-     return getTableDescriptor(tableName, getConnection(), rpcCallerFactory, operationTimeout);
+     return getTableDescriptor(tableName, getConnection(), rpcCallerFactory, rpcControllerFactory,
+       operationTimeout);
   }
 
   static HTableDescriptor getTableDescriptor(final TableName tableName, HConnection connection,
-      RpcRetryingCallerFactory rpcCallerFactory, int operationTimeout) throws IOException {
+      RpcRetryingCallerFactory rpcCallerFactory, final RpcControllerFactory rpcControllerFactory,
+      int operationTimeout) throws IOException {
       if (tableName == null) return null;
       HTableDescriptor htd = executeCallable(new MasterCallable<HTableDescriptor>(connection) {
         @Override
         public HTableDescriptor call(int callTimeout) throws ServiceException {
+          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+          controller.setCallTimeout(callTimeout);
           GetTableDescriptorsResponse htds;
           GetTableDescriptorsRequest req =
                   RequestConverter.buildGetTableDescriptorsRequest(tableName);
-          htds = master.getTableDescriptors(null, req);
+          htds = master.getTableDescriptors(controller, req);
 
           if (!htds.getTableSchemaList().isEmpty()) {
             return HTableDescriptor.convert(htds.getTableSchemaList().get(0));
@@ -483,14 +498,17 @@ public class HBaseAdmin implements Admin {
     }
 
     CreateTableResponse response = executeCallable(
-        new MasterCallable<CreateTableResponse>(getConnection()) {
-      @Override
-      public CreateTableResponse call(int callTimeout) throws ServiceException {
-        CreateTableRequest request = RequestConverter.buildCreateTableRequest(
-          desc, splitKeys, ng.getNonceGroup(), ng.newNonce());
-        return master.createTable(null, request);
-      }
-    });
+      new MasterCallable<CreateTableResponse>(getConnection()) {
+        @Override
+        public CreateTableResponse call(int callTimeout) throws ServiceException {
+          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+          controller.setCallTimeout(callTimeout);
+          controller.setPriority(desc.getTableName());
+          CreateTableRequest request = RequestConverter.buildCreateTableRequest(
+            desc, splitKeys, ng.getNonceGroup(), ng.newNonce());
+          return master.createTable(controller, request);
+        }
+      });
     return new CreateTableFuture(this, desc, splitKeys, response);
   }
 
@@ -532,14 +550,17 @@ public class HBaseAdmin implements Admin {
   @Override
   public Future<Void> deleteTableAsync(final TableName tableName) throws IOException {
     DeleteTableResponse response = executeCallable(
-        new MasterCallable<DeleteTableResponse>(getConnection()) {
-      @Override
-      public DeleteTableResponse call(int callTimeout) throws ServiceException {
-        DeleteTableRequest req =
-            RequestConverter.buildDeleteTableRequest(tableName, ng.getNonceGroup(), ng.newNonce());
-        return master.deleteTable(null,req);
-      }
-    });
+      new MasterCallable<DeleteTableResponse>(getConnection()) {
+        @Override
+        public DeleteTableResponse call(int callTimeout) throws ServiceException {
+          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+          controller.setCallTimeout(callTimeout);
+          controller.setPriority(tableName);
+          DeleteTableRequest req =
+              RequestConverter.buildDeleteTableRequest(tableName, ng.getNonceGroup(),ng.newNonce());
+          return master.deleteTable(controller,req);
+        }
+      });
     return new DeleteTableFuture(this, tableName, response);
   }
 
@@ -614,10 +635,13 @@ public class HBaseAdmin implements Admin {
         executeCallable(new MasterCallable<TruncateTableResponse>(getConnection()) {
           @Override
           public TruncateTableResponse call(int callTimeout) throws ServiceException {
+            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+            controller.setCallTimeout(callTimeout);
+            controller.setPriority(tableName);
             LOG.info("Started truncating " + tableName);
             TruncateTableRequest req = RequestConverter.buildTruncateTableRequest(
               tableName, preserveSplits, ng.getNonceGroup(), ng.newNonce());
-            return master.truncateTable(null, req);
+            return master.truncateTable(controller, req);
           }
         });
     return new TruncateTableFuture(this, tableName, preserveSplits, response);
@@ -714,15 +738,19 @@ public class HBaseAdmin implements Admin {
   public Future<Void> enableTableAsync(final TableName tableName) throws IOException {
     TableName.isLegalFullyQualifiedTableName(tableName.getName());
     EnableTableResponse response = executeCallable(
-        new MasterCallable<EnableTableResponse>(getConnection()) {
-      @Override
-      public EnableTableResponse call(int callTimeout) throws ServiceException {
-        LOG.info("Started enable of " + tableName);
-        EnableTableRequest req =
-            RequestConverter.buildEnableTableRequest(tableName, ng.getNonceGroup(), ng.newNonce());
-        return master.enableTable(null,req);
-      }
-    });
+      new MasterCallable<EnableTableResponse>(getConnection()) {
+        @Override
+        public EnableTableResponse call(int callTimeout) throws ServiceException {
+          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+          controller.setCallTimeout(callTimeout);
+          controller.setPriority(tableName);
+
+          LOG.info("Started enable of " + tableName);
+          EnableTableRequest req =
+              RequestConverter.buildEnableTableRequest(tableName, ng.getNonceGroup(),ng.newNonce());
+          return master.enableTable(controller,req);
+        }
+      });
     return new EnableTableFuture(this, tableName, response);
   }
 
@@ -776,15 +804,20 @@ public class HBaseAdmin implements Admin {
   public Future<Void> disableTableAsync(final TableName tableName) throws IOException {
     TableName.isLegalFullyQualifiedTableName(tableName.getName());
     DisableTableResponse response = executeCallable(
-        new MasterCallable<DisableTableResponse>(getConnection()) {
-      @Override
-      public DisableTableResponse call(int callTimeout) throws ServiceException {
-        LOG.info("Started disable of " + tableName);
-        DisableTableRequest req =
-            RequestConverter.buildDisableTableRequest(tableName, ng.getNonceGroup(), ng.newNonce());
-        return master.disableTable(null, req);
-      }
-    });
+      new MasterCallable<DisableTableResponse>(getConnection()) {
+        @Override
+        public DisableTableResponse call(int callTimeout) throws ServiceException {
+          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+          controller.setCallTimeout(callTimeout);
+          controller.setPriority(tableName);
+
+          LOG.info("Started disable of " + tableName);
+          DisableTableRequest req =
+              RequestConverter.buildDisableTableRequest(
+                tableName, ng.getNonceGroup(), ng.newNonce());
+          return master.disableTable(controller, req);
+        }
+      });
     return new DisableTableFuture(this, tableName, response);
   }
 
@@ -863,9 +896,13 @@ public class HBaseAdmin implements Admin {
     return executeCallable(new MasterCallable<Pair<Integer, Integer>>(getConnection()) {
       @Override
       public Pair<Integer, Integer> call(int callTimeout) throws ServiceException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+        controller.setPriority(tableName);
+
         GetSchemaAlterStatusRequest req = RequestConverter
             .buildGetSchemaAlterStatusRequest(tableName);
-        GetSchemaAlterStatusResponse ret = master.getSchemaAlterStatus(null, req);
+        GetSchemaAlterStatusResponse ret = master.getSchemaAlterStatus(controller, req);
         Pair<Integer, Integer> pair = new Pair<>(ret.getYetToUpdateRegions(),
             ret.getTotalRegions());
         return pair;
@@ -897,10 +934,14 @@ public class HBaseAdmin implements Admin {
         executeCallable(new MasterCallable<AddColumnResponse>(getConnection()) {
           @Override
           public AddColumnResponse call(int callTimeout) throws ServiceException {
+            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+            controller.setCallTimeout(callTimeout);
+            controller.setPriority(tableName);
+
             AddColumnRequest req =
                 RequestConverter.buildAddColumnRequest(tableName, columnFamily, ng.getNonceGroup(),
                   ng.newNonce());
-            return master.addColumn(null, req);
+            return master.addColumn(controller, req);
           }
         });
     return new AddColumnFamilyFuture(this, tableName, response);
@@ -938,10 +979,14 @@ public class HBaseAdmin implements Admin {
         executeCallable(new MasterCallable<DeleteColumnResponse>(getConnection()) {
           @Override
           public DeleteColumnResponse call(int callTimeout) throws ServiceException {
+            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+            controller.setCallTimeout(callTimeout);
+            controller.setPriority(tableName);
+
             DeleteColumnRequest req =
                 RequestConverter.buildDeleteColumnRequest(tableName, columnFamily,
                   ng.getNonceGroup(), ng.newNonce());
-            master.deleteColumn(null, req);
+            master.deleteColumn(controller, req);
             return null;
           }
         });
@@ -980,10 +1025,14 @@ public class HBaseAdmin implements Admin {
         executeCallable(new MasterCallable<ModifyColumnResponse>(getConnection()) {
           @Override
           public ModifyColumnResponse call(int callTimeout) throws ServiceException {
+            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+            controller.setCallTimeout(callTimeout);
+            controller.setPriority(tableName);
+
             ModifyColumnRequest req =
                 RequestConverter.buildModifyColumnRequest(tableName, columnFamily,
                   ng.getNonceGroup(), ng.newNonce());
-            master.modifyColumn(null, req);
+            master.modifyColumn(controller, req);
             return null;
           }
         });
@@ -1042,7 +1091,10 @@ public class HBaseAdmin implements Admin {
     CloseRegionRequest request =
       RequestConverter.buildCloseRegionRequest(sn, encodedRegionName);
     try {
-      CloseRegionResponse response = admin.closeRegion(null, request);
+      PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+
+      // TODO: this does not do retries, it should. Set priority and timeout in controller
+      CloseRegionResponse response = admin.closeRegion(controller, request);
       boolean isRegionClosed = response.getClosed();
       if (false == isRegionClosed) {
         LOG.error("Not able to close the region " + encodedRegionName + ".");
@@ -1056,14 +1108,17 @@ public class HBaseAdmin implements Admin {
   @Override
   public void closeRegion(final ServerName sn, final HRegionInfo hri) throws IOException {
     AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
+    PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+
     // Close the region without updating zk state.
-    ProtobufUtil.closeRegion(admin, sn, hri.getRegionName());
+    ProtobufUtil.closeRegion(controller, admin, sn, hri.getRegionName());
   }
 
   @Override
   public List<HRegionInfo> getOnlineRegions(final ServerName sn) throws IOException {
     AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
-    return ProtobufUtil.getOnlineRegions(admin);
+    PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+    return ProtobufUtil.getOnlineRegions(controller, admin);
   }
 
   @Override
@@ -1088,23 +1143,15 @@ public class HBaseAdmin implements Admin {
     }
     HRegionInfo hRegionInfo = regionServerPair.getFirst();
     ServerName serverName = regionServerPair.getSecond();
+
+    PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+
     AdminService.BlockingInterface admin = this.connection.getAdmin(serverName);
     FlushRegionRequest request =
         RequestConverter.buildFlushRegionRequest(hRegionInfo.getRegionName());
     try {
-      admin.flushRegion(null, request);
-    } catch (ServiceException se) {
-      throw ProtobufUtil.getRemoteException(se);
-    }
-  }
-
-  private void flush(final ServerName sn, final HRegionInfo hri)
-  throws IOException {
-    AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
-    FlushRegionRequest request =
-      RequestConverter.buildFlushRegionRequest(hri.getRegionName());
-    try {
-      admin.flushRegion(null, request);
+      // TODO: this does not do retries, it should. Set priority and timeout in controller
+      admin.flushRegion(controller, request);
     } catch (ServiceException se) {
       throw ProtobufUtil.getRemoteException(se);
     }
@@ -1258,11 +1305,13 @@ public class HBaseAdmin implements Admin {
   private void compact(final ServerName sn, final HRegionInfo hri,
       final boolean major, final byte [] family)
   throws IOException {
+    PayloadCarryingRpcController controller = rpcControllerFactory.newController();
     AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
     CompactRegionRequest request =
       RequestConverter.buildCompactRegionRequest(hri.getRegionName(), major, family);
     try {
-      admin.compactRegion(null, request);
+      // TODO: this does not do retries, it should. Set priority and timeout in controller
+      admin.compactRegion(controller, request);
     } catch (ServiceException se) {
       throw ProtobufUtil.getRemoteException(se);
     }
@@ -1275,10 +1324,17 @@ public class HBaseAdmin implements Admin {
     executeCallable(new MasterCallable<Void>(getConnection()) {
       @Override
       public Void call(int callTimeout) throws ServiceException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+        // Hard to know the table name, at least check if meta
+        if (isMetaRegion(encodedRegionName)) {
+          controller.setPriority(TableName.META_TABLE_NAME);
+        }
+
         try {
           MoveRegionRequest request =
               RequestConverter.buildMoveRegionRequest(encodedRegionName, destServerName);
-            master.moveRegion(null, request);
+            master.moveRegion(controller, request);
         } catch (DeserializationException de) {
           LOG.error("Could not parse destination server name: " + de);
           throw new ServiceException(new DoNotRetryIOException(de));
@@ -1288,6 +1344,11 @@ public class HBaseAdmin implements Admin {
     });
   }
 
+  private boolean isMetaRegion(final byte[] regionName) {
+    return Bytes.equals(regionName, HRegionInfo.FIRST_META_REGIONINFO.getRegionName())
+        || Bytes.equals(regionName, HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes());
+  }
+
   @Override
   public void assign(final byte[] regionName) throws MasterNotRunningException,
       ZooKeeperConnectionException, IOException {
@@ -1295,9 +1356,16 @@ public class HBaseAdmin implements Admin {
     executeCallable(new MasterCallable<Void>(getConnection()) {
       @Override
       public Void call(int callTimeout) throws ServiceException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+        // Hard to know the table name, at least check if meta
+        if (isMetaRegion(regionName)) {
+          controller.setPriority(TableName.META_TABLE_NAME);
+        }
+
         AssignRegionRequest request =
           RequestConverter.buildAssignRegionRequest(toBeAssigned);
-        master.assignRegion(null,request);
+        master.assignRegion(controller,request);
         return null;
       }
     });
@@ -1310,9 +1378,15 @@ public class HBaseAdmin implements Admin {
     executeCallable(new MasterCallable<Void>(getConnection()) {
       @Override
       public Void call(int callTimeout) throws ServiceException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+        // Hard to know the table name, at least check if meta
+        if (isMetaRegion(regionName)) {
+          controller.setPriority(TableName.META_TABLE_NAME);
+        }
         UnassignRegionRequest request =
           RequestConverter.buildUnassignRegionRequest(toBeUnassigned, force);
-        master.unassignRegion(null, request);
+        master.unassignRegion(controller, request);
         return null;
       }
     });
@@ -1324,7 +1398,13 @@ public class HBaseAdmin implements Admin {
     executeCallable(new MasterCallable<Void>(getConnection()) {
       @Override
       public Void call(int callTimeout) throws ServiceException {
-        master.offlineRegion(null,RequestConverter.buildOfflineRegionRequest(regionName));
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+        // Hard to know the table name, at least check if meta
+        if (isMetaRegion(regionName)) {
+          controller.setPriority(TableName.META_TABLE_NAME);
+        }
+        master.offlineRegion(controller, RequestConverter.buildOfflineRegionRequest(regionName));
         return null;
       }
     });
@@ -1336,9 +1416,12 @@ public class HBaseAdmin implements Admin {
     return executeCallable(new MasterCallable<Boolean>(getConnection()) {
       @Override
       public Boolean call(int callTimeout) throws ServiceException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+
         SetBalancerRunningRequest req =
             RequestConverter.buildSetBalancerRunningRequest(on, synchronous);
-        return master.setBalancerRunning(null, req).getPrevBalanceValue();
+        return master.setBalancerRunning(controller, req).getPrevBalanceValue();
       }
     });
   }
@@ -1348,7 +1431,11 @@ public class HBaseAdmin implements Admin {
     return executeCallable(new MasterCallable<Boolean>(getConnection()) {
       @Override
       public Boolean call(int callTimeout) throws ServiceException {
-        return master.balance(null, RequestConverter.buildBalanceRequest(false)).getBalancerRan();
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+
+        return master.balance(controller,
+          RequestConverter.buildBalanceRequest(false)).getBalancerRan();
       }
     });
   }
@@ -1358,7 +1445,11 @@ public class HBaseAdmin implements Admin {
     return executeCallable(new MasterCallable<Boolean>(getConnection()) {
       @Override
       public Boolean call(int callTimeout) throws ServiceException {
-        return master.balance(null, RequestConverter.buildBalanceRequest(force)).getBalancerRan();
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+
+        return master.balance(controller,
+          RequestConverter.buildBalanceRequest(force)).getBalancerRan();
       }
     });
   }
@@ -1368,8 +1459,11 @@ public class HBaseAdmin implements Admin {
     return executeCallable(new MasterCallable<Boolean>(getConnection()) {
       @Override
       public Boolean call(int callTimeout) throws ServiceException {
-        return master.isBalancerEnabled(null, RequestConverter.buildIsBalancerEnabledRequest())
-            .getEnabled();
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+
+        return master.isBalancerEnabled(controller,
+          RequestConverter.buildIsBalancerEnabledRequest()).getEnabled();
       }
     });
   }
@@ -1379,7 +1473,10 @@ public class HBaseAdmin implements Admin {
     return executeCallable(new MasterCallable<Boolean>(getConnection()) {
       @Override
       public Boolean call(int callTimeout) throws ServiceException {
-        return master.normalize(null,
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+
+        return master.normalize(controller,
           RequestConverter.buildNormalizeRequest()).getNormalizerRan();
       }
     });
@@ -1390,7 +1487,10 @@ public class HBaseAdmin implements Admin {
     return executeCallable(new MasterCallable<Boolean>(getConnection()) {
       @Override
       public Boolean call(int callTimeout) throws ServiceException {
-        return master.isNormalizerEnabled(null,
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+
+        return master.isNormalizerEnabled(controller,
           RequestConverter.buildIsNormalizerEnabledRequest()).getEnabled();
       }
     });
@@ -1401,9 +1501,12 @@ public class HBaseAdmin implements Admin {
     return executeCallable(new MasterCallable<Boolean>(getConnection()) {
       @Override
       public Boolean call(int callTimeout) throws ServiceException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+
         SetNormalizerRunningRequest req =
           RequestConverter.buildSetNormalizerRunningRequest(on);
-        return master.setNormalizerRunning(null, req).getPrevNormalizerValue();
+        return master.setNormalizerRunning(controller, req).getPrevNormalizerValue();
       }
     });
   }
@@ -1413,7 +1516,10 @@ public class HBaseAdmin implements Admin {
     return executeCallable(new MasterCallable<Boolean>(getConnection()) {
       @Override
       public Boolean call(int callTimeout) throws ServiceException {
-        return master.enableCatalogJanitor(null,
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+
+        return master.enableCatalogJanitor(controller,
           RequestConverter.buildEnableCatalogJanitorRequest(enable)).getPrevValue();
       }
     });
@@ -1424,7 +1530,10 @@ public class HBaseAdmin implements Admin {
     return executeCallable(new MasterCallable<Integer>(getConnection()) {
       @Override
       public Integer call(int callTimeout) throws ServiceException {
-        return master.runCatalogScan(null,
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+
+        return master.runCatalogScan(controller,
           RequestConverter.buildCatalogScanRequest()).getScanResult();
       }
     });
@@ -1435,7 +1544,10 @@ public class HBaseAdmin implements Admin {
     return executeCallable(new MasterCallable<Boolean>(getConnection()) {
       @Override
       public Boolean call(int callTimeout) throws ServiceException {
-        return master.isCatalogJanitorEnabled(null,
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+
+        return master.isCatalogJanitorEnabled(controller,
           RequestConverter.buildIsCatalogJanitorEnabledRequest()).getValue();
       }
     });
@@ -1480,11 +1592,14 @@ public class HBaseAdmin implements Admin {
     executeCallable(new MasterCallable<Void>(getConnection()) {
       @Override
       public Void call(int callTimeout) throws ServiceException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+
         try {
           DispatchMergingRegionsRequest request = RequestConverter
               .buildDispatchMergingRegionsRequest(encodedNameOfRegionA,
                 encodedNameOfRegionB, forcible);
-          master.dispatchMergingRegions(null, request);
+          master.dispatchMergingRegions(controller, request);
         } catch (DeserializationException de) {
           LOG.error("Could not parse destination server name: " + de);
         }
@@ -1562,28 +1677,35 @@ public class HBaseAdmin implements Admin {
          Bytes.compareTo(hri.getStartKey(), splitPoint) == 0) {
        throw new IOException("should not give a splitkey which equals to startkey!");
     }
-    // TODO: This is not executed via retries
+    PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+    controller.setPriority(hri.getTable());
+
+    // TODO: this does not do retries, it should. Set priority and timeout in controller
     AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
-    ProtobufUtil.split(admin, hri, splitPoint);
+    ProtobufUtil.split(controller, admin, hri, splitPoint);
   }
 
   @Override
   public Future<Void> modifyTable(final TableName tableName, final HTableDescriptor htd)
-  throws IOException {
+      throws IOException {
     if (!tableName.equals(htd.getTableName())) {
       throw new IllegalArgumentException("the specified table name '" + tableName +
         "' doesn't match with the HTD one: " + htd.getTableName());
     }
 
     ModifyTableResponse response = executeCallable(
-        new MasterCallable<ModifyTableResponse>(getConnection()) {
-      @Override
-      public ModifyTableResponse call(int callTimeout) throws ServiceException {
-        ModifyTableRequest request = RequestConverter.buildModifyTableRequest(
-          tableName, htd, ng.getNonceGroup(), ng.newNonce());
-        return master.modifyTable(null, request);
-      }
-    });
+      new MasterCallable<ModifyTableResponse>(getConnection()) {
+        @Override
+        public ModifyTableResponse call(int callTimeout) throws ServiceException {
+          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+          controller.setCallTimeout(callTimeout);
+          controller.setPriority(tableName);
+
+          ModifyTableRequest request = RequestConverter.buildModifyTableRequest(
+            tableName, htd, ng.getNonceGroup(), ng.newNonce());
+          return master.modifyTable(controller, request);
+        }
+      });
 
     return new ModifyTableFuture(this, tableName, response);
   }
@@ -1715,7 +1837,10 @@ public class HBaseAdmin implements Admin {
     executeCallable(new MasterCallable<Void>(getConnection()) {
       @Override
       public Void call(int callTimeout) throws ServiceException {
-        master.shutdown(null,ShutdownRequest.newBuilder().build());
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+        controller.setPriority(HConstants.HIGH_QOS);
+        master.shutdown(controller, ShutdownRequest.newBuilder().build());
         return null;
       }
     });
@@ -1726,7 +1851,10 @@ public class HBaseAdmin implements Admin {
     executeCallable(new MasterCallable<Void>(getConnection()) {
       @Override
       public Void call(int callTimeout) throws ServiceException {
-        master.stopMaster(null, StopMasterRequest.newBuilder().build());
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+        controller.setPriority(HConstants.HIGH_QOS);
+        master.stopMaster(controller, StopMasterRequest.newBuilder().build());
         return null;
       }
     });
@@ -1741,8 +1869,12 @@ public class HBaseAdmin implements Admin {
       this.connection.getAdmin(ServerName.valueOf(hostname, port, 0));
     StopServerRequest request = RequestConverter.buildStopServerRequest(
       "Called by admin client " + this.connection.toString());
+    PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+
+    controller.setPriority(HConstants.HIGH_QOS);
     try {
-      admin.stopServer(null, request);
+      // TODO: this does not do retries, it should. Set priority and timeout in controller
+      admin.stopServer(controller, request);
     } catch (ServiceException se) {
       throw ProtobufUtil.getRemoteException(se);
     }
@@ -1753,8 +1885,10 @@ public class HBaseAdmin implements Admin {
     return executeCallable(new MasterCallable<ClusterStatus>(getConnection()) {
       @Override
       public ClusterStatus call(int callTimeout) throws ServiceException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
         GetClusterStatusRequest req = RequestConverter.buildGetClusterStatusRequest();
-        return ClusterStatus.convert(master.getClusterStatus(null, req).getClusterStatus());
+        return ClusterStatus.convert(master.getClusterStatus(controller, req).getClusterStatus());
       }
     });
   }
@@ -1793,18 +1927,21 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public Future<Void> createNamespaceAsync(final NamespaceDescriptor descriptor)
-  throws IOException {
+      throws IOException {
     CreateNamespaceResponse response =
         executeCallable(new MasterCallable<CreateNamespaceResponse>(getConnection()) {
-      @Override
-      public CreateNamespaceResponse call(int callTimeout) throws Exception {
-        return master.createNamespace(null,
-          CreateNamespaceRequest.newBuilder()
-            .setNamespaceDescriptor(ProtobufUtil
-              .toProtoNamespaceDescriptor(descriptor)).build()
-        );
-      }
-    });
+          @Override
+          public CreateNamespaceResponse call(int callTimeout) throws Exception {
+            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+            controller.setCallTimeout(callTimeout);
+            // TODO: set priority based on NS?
+            return master.createNamespace(controller,
+              CreateNamespaceRequest.newBuilder()
+              .setNamespaceDescriptor(ProtobufUtil
+                .toProtoNamespaceDescriptor(descriptor)).build()
+                );
+          }
+        });
     return new NamespaceFuture(this, descriptor.getName(), response.getProcId()) {
       @Override
       public String getOperationType() {
@@ -1821,15 +1958,18 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public Future<Void> modifyNamespaceAsync(final NamespaceDescriptor descriptor)
-  throws IOException {
+      throws IOException {
     ModifyNamespaceResponse response =
         executeCallable(new MasterCallable<ModifyNamespaceResponse>(getConnection()) {
-      @Override
-      public ModifyNamespaceResponse call(int callTimeout) throws Exception {
-        return master.modifyNamespace(null, ModifyNamespaceRequest.newBuilder().
-          setNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(descriptor)).build());
-      }
-    });
+          @Override
+          public ModifyNamespaceResponse call(int callTimeout) throws Exception {
+            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+            controller.setCallTimeout(callTimeout);
+            // TODO: set priority based on NS?
+            return master.modifyNamespace(controller, ModifyNamespaceRequest.newBuilder().
+              setNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(descriptor)).build());
+          }
+        });
     return new NamespaceFuture(this, descriptor.getName(), response.getProcId()) {
       @Override
       public String getOperationType() {
@@ -1846,15 +1986,18 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public Future<Void> deleteNamespaceAsync(final String name)
-  throws IOException {
+      throws IOException {
     DeleteNamespaceResponse response =
         executeCallable(new MasterCallable<DeleteNamespaceResponse>(getConnection()) {
-      @Override
-      public DeleteNamespaceResponse call(int callTimeout) throws Exception {
-        return master.deleteNamespace(null, DeleteNamespaceRequest.newBuilder().
-          setNamespaceName(name).build());
-      }
-    });
+          @Override
+          public DeleteNamespaceResponse call(int callTimeout) throws Exception {
+            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+            controller.setCallTimeout(callTimeout);
+            // TODO: set priority based on NS?
+            return master.deleteNamespace(controller, DeleteNamespaceRequest.newBuilder().
+              setNamespaceName(name).build());
+          }
+        });
     return new NamespaceFuture(this, name, response.getProcId()) {
       @Override
       public String getOperationType() {
@@ -1869,8 +2012,10 @@ public class HBaseAdmin implements Admin {
         executeCallable(new MasterCallable<NamespaceDescriptor>(getConnection()) {
           @Override
           public NamespaceDescriptor call(int callTimeout) throws Exception {
+            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+            controller.setCallTimeout(callTimeout);
             return ProtobufUtil.toNamespaceDescriptor(
-              master.getNamespaceDescriptor(null, GetNamespaceDescriptorRequest.newBuilder().
+              master.getNamespaceDescriptor(controller, GetNamespaceDescriptorRequest.newBuilder().
                 setNamespaceName(name).build()).getNamespaceDescriptor());
           }
         });
@@ -1882,9 +2027,12 @@ public class HBaseAdmin implements Admin {
         executeCallable(new MasterCallable<NamespaceDescriptor[]>(getConnection()) {
           @Override
           public NamespaceDescriptor[] call(int callTimeout) throws Exception {
+            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+            controller.setCallTimeout(callTimeout);
             List<HBaseProtos.NamespaceDescriptor> list =
-              master.listNamespaceDescriptors(null, ListNamespaceDescriptorsRequest.newBuilder().
-                build()).getNamespaceDescriptorList();
+                master.listNamespaceDescriptors(controller,
+                  ListNamespaceDescriptorsRequest.newBuilder().build())
+                .getNamespaceDescriptorList();
             NamespaceDescriptor[] res = new NamespaceDescriptor[list.size()];
             for(int i = 0; i < list.size(); i++) {
               res[i] = ProtobufUtil.toNamespaceDescriptor(list.get(i));
@@ -1900,8 +2048,10 @@ public class HBaseAdmin implements Admin {
         executeCallable(new MasterCallable<ProcedureInfo[]>(getConnection()) {
           @Override
           public ProcedureInfo[] call(int callTimeout) throws Exception {
+            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+            controller.setCallTimeout(callTimeout);
             List<ProcedureProtos.Procedure> procList = master.listProcedures(
-              null, ListProceduresRequest.newBuilder().build()).getProcedureList();
+              controller, ListProceduresRequest.newBuilder().build()).getProcedureList();
             ProcedureInfo[] procInfoList = new ProcedureInfo[procList.size()];
             for (int i = 0; i < procList.size(); i++) {
               procInfoList[i] = ProcedureInfo.convert(procList.get(i));
@@ -1917,9 +2067,12 @@ public class HBaseAdmin implements Admin {
         executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection()) {
           @Override
           public HTableDescriptor[] call(int callTimeout) throws Exception {
+            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+            controller.setCallTimeout(callTimeout);
             List<TableSchema> list =
-              master.listTableDescriptorsByNamespace(null, ListTableDescriptorsByNamespaceRequest.
-                newBuilder().setNamespaceName(name).build()).getTableSchemaList();
+                master.listTableDescriptorsByNamespace(controller,
+                  ListTableDescriptorsByNamespaceRequest.newBuilder().setNamespaceName(name)
+                  .build()).getTableSchemaList();
             HTableDescriptor[] res = new HTableDescriptor[list.size()];
             for(int i=0; i < list.size(); i++) {
 
@@ -1936,8 +2089,10 @@ public class HBaseAdmin implements Admin {
         executeCallable(new MasterCallable<TableName[]>(getConnection()) {
           @Override
           public TableName[] call(int callTimeout) throws Exception {
+            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+            controller.setCallTimeout(callTimeout);
             List<HBaseProtos.TableName> tableNames =
-              master.listTableNamesByNamespace(null, ListTableNamesByNamespaceRequest.
+              master.listTableNamesByNamespace(controller, ListTableNamesByNamespaceRequest.
                 newBuilder().setNamespaceName(name).build())
                 .getTableNameList();
             TableName[] result = new TableName[tableNames.size()];
@@ -2017,9 +2172,11 @@ public class HBaseAdmin implements Admin {
     return executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection()) {
       @Override
       public HTableDescriptor[] call(int callTimeout) throws Exception {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
         GetTableDescriptorsRequest req =
             RequestConverter.buildGetTableDescriptorsRequest(tableNames);
-          return ProtobufUtil.getHTableDescriptorArray(master.getTableDescriptors(null, req));
+          return ProtobufUtil.getHTableDescriptorArray(master.getTableDescriptors(controller, req));
       }
     });
   }
@@ -2059,8 +2216,11 @@ public class HBaseAdmin implements Admin {
       FailedLogCloseException {
     AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
     RollWALWriterRequest request = RequestConverter.buildRollWALWriterRequest();
+    PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+
     try {
-      return admin.rollWALWriter(null, request);
+      // TODO: this does not do retries, it should. Set priority and timeout in controller
+      return admin.rollWALWriter(controller, request);
     } catch (ServiceException se) {
       throw ProtobufUtil.getRemoteException(se);
     }
@@ -2142,7 +2302,9 @@ public class HBaseAdmin implements Admin {
       AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
       GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
         regionServerPair.getFirst().getRegionName(), true);
-      GetRegionInfoResponse response = admin.getRegionInfo(null, request);
+      PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+      // TODO: this does not do retries, it should. Set priority and timeout in controller
+      GetRegionInfoResponse response = admin.getRegionInfo(controller, request);
       return response.getCompactionState();
     } catch (ServiceException se) {
       throw ProtobufUtil.getRemoteException(se);
@@ -2204,7 +2366,9 @@ public class HBaseAdmin implements Admin {
       done = executeCallable(new MasterCallable<IsSnapshotDoneResponse>(getConnection()) {
         @Override
         public IsSnapshotDoneResponse call(int callTimeout) throws ServiceException {
-          return master.isSnapshotDone(null, request);
+          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+          controller.setCallTimeout(callTimeout);
+          return master.isSnapshotDone(controller, request);
         }
       });
     }
@@ -2224,7 +2388,9 @@ public class HBaseAdmin implements Admin {
     return executeCallable(new MasterCallable<SnapshotResponse>(getConnection()) {
       @Override
       public SnapshotResponse call(int callTimeout) throws ServiceException {
-        return master.snapshot(null, request);
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+        return master.snapshot(controller, request);
       }
     });
   }
@@ -2236,7 +2402,9 @@ public class HBaseAdmin implements Admin {
     return executeCallable(new MasterCallable<IsSnapshotDoneResponse>(getConnection()) {
       @Override
       public IsSnapshotDoneResponse call(int callTimeout) throws ServiceException {
-        return master.isSnapshotDone(null,
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+        return master.isSnapshotDone(controller,
           IsSnapshotDoneRequest.newBuilder().setSnapshot(snapshot).build());
       }
     }).getDone();
@@ -2370,7 +2538,9 @@ public class HBaseAdmin implements Admin {
         getConnection()) {
       @Override
       public ExecProcedureResponse call(int callTimeout) throws ServiceException {
-        return master.execProcedureWithRet(null, request);
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+        return master.execProcedureWithRet(controller, request);
       }
     });
 
@@ -2395,7 +2565,9 @@ public class HBaseAdmin implements Admin {
         getConnection()) {
       @Override
       public ExecProcedureResponse call(int callTimeout) throws ServiceException {
-        return master.execProcedure(null, request);
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+        return master.execProcedure(controller, request);
       }
     });
 
@@ -2442,7 +2614,9 @@ public class HBaseAdmin implements Admin {
         new MasterCallable<IsProcedureDoneResponse>(getConnection()) {
           @Override
           public IsProcedureDoneResponse call(int callTimeout) throws ServiceException {
-            return master.isProcedureDone(null, IsProcedureDoneRequest
+            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+            controller.setCallTimeout(callTimeout);
+            return master.isProcedureDone(controller, IsProcedureDoneRequest
                 .newBuilder().setProcedure(desc).build());
           }
         }).getDone();
@@ -2488,7 +2662,9 @@ public class HBaseAdmin implements Admin {
           getConnection()) {
         @Override
         public IsRestoreSnapshotDoneResponse call(int callTimeout) throws ServiceException {
-          return master.isRestoreSnapshotDone(null, request);
+          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+          controller.setCallTimeout(callTimeout);
+          return master.isRestoreSnapshotDone(controller, request);
         }
       });
     }
@@ -2518,7 +2694,9 @@ public class HBaseAdmin implements Admin {
     return executeCallable(new MasterCallable<RestoreSnapshotResponse>(getConnection()) {
       @Override
       public RestoreSnapshotResponse call(int callTimeout) throws ServiceException {
-        return master.restoreSnapshot(null, request);
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+        return master.restoreSnapshot(controller, request);
       }
     });
   }
@@ -2528,8 +2706,10 @@ public class HBaseAdmin implements Admin {
     return executeCallable(new MasterCallable<List<SnapshotDescription>>(getConnection()) {
       @Override
       public List<SnapshotDescription> call(int callTimeout) throws ServiceException {
-        return master.getCompletedSnapshots(null, GetCompletedSnapshotsRequest.newBuilder().build())
-            .getSnapshotsList();
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+        return master.getCompletedSnapshots(controller,
+          GetCompletedSnapshotsRequest.newBuilder().build()).getSnapshotsList();
       }
     });
   }
@@ -2587,7 +2767,9 @@ public class HBaseAdmin implements Admin {
     executeCallable(new MasterCallable<Void>(getConnection()) {
       @Override
       public Void call(int callTimeout) throws ServiceException {
-        master.deleteSnapshot(null,
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+        master.deleteSnapshot(controller,
           DeleteSnapshotRequest.newBuilder().
             setSnapshot(SnapshotDescription.newBuilder().setName(snapshotName).build()).build()
         );
@@ -2619,8 +2801,10 @@ public class HBaseAdmin implements Admin {
     executeCallable(new MasterCallable<Void>(getConnection()) {
       @Override
       public Void call(int callTimeout) throws ServiceException {
-        this.master.deleteSnapshot(null, DeleteSnapshotRequest.newBuilder().setSnapshot(snapshot)
-            .build());
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+        this.master.deleteSnapshot(controller, DeleteSnapshotRequest.newBuilder()
+          .setSnapshot(snapshot).build());
         return null;
       }
     });
@@ -2651,7 +2835,9 @@ public class HBaseAdmin implements Admin {
     executeCallable(new MasterCallable<Void>(getConnection()) {
       @Override
       public Void call(int callTimeout) throws ServiceException {
-        this.master.setQuota(null, QuotaSettings.buildSetQuotaRequestProto(quota));
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+        this.master.setQuota(controller, QuotaSettings.buildSetQuotaRequestProto(quota));
         return null;
       }
     });
@@ -2750,10 +2936,12 @@ public class HBaseAdmin implements Admin {
     return executeCallable(new MasterCallable<Long>(getConnection()) {
       @Override
       public Long call(int callTimeout) throws ServiceException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
         MajorCompactionTimestampRequest req =
             MajorCompactionTimestampRequest.newBuilder()
                 .setTableName(ProtobufUtil.toProtoTableName(tableName)).build();
-        return master.getLastMajorCompactionTimestamp(null, req).getCompactionTimestamp();
+        return master.getLastMajorCompactionTimestamp(controller, req).getCompactionTimestamp();
       }
     });
   }
@@ -2763,13 +2951,16 @@ public class HBaseAdmin implements Admin {
     return executeCallable(new MasterCallable<Long>(getConnection()) {
       @Override
       public Long call(int callTimeout) throws ServiceException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
         MajorCompactionTimestampForRegionRequest req =
             MajorCompactionTimestampForRegionRequest
                 .newBuilder()
                 .setRegion(
                   RequestConverter
                       .buildRegionSpecifier(RegionSpecifierType.REGION_NAME, regionName)).build();
-        return master.getLastMajorCompactionTimestampForRegion(null, req).getCompactionTimestamp();
+        return master.getLastMajorCompactionTimestampForRegion(controller, req)
+            .getCompactionTimestamp();
       }
     });
   }
@@ -2818,6 +3009,7 @@ public class HBaseAdmin implements Admin {
     CompactType compactType) throws IOException {
     CompactionState state = CompactionState.NONE;
     checkTableExists(tableName);
+    PayloadCarryingRpcController controller = rpcControllerFactory.newController();
     switch (compactType) {
       case MOB:
         try {
@@ -2826,7 +3018,7 @@ public class HBaseAdmin implements Admin {
           GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
                   info.getRegionName(), true);
           GetRegionInfoResponse response = this.connection.getAdmin(master)
-                  .getRegionInfo(null, request);
+                  .getRegionInfo(controller, request);
           state = response.getCompactionState();
         } catch (ServiceException se) {
           throw ProtobufUtil.getRemoteException(se);
@@ -2852,7 +3044,7 @@ public class HBaseAdmin implements Admin {
               AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
               GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
                       pair.getFirst().getRegionName(), true);
-              GetRegionInfoResponse response = admin.getRegionInfo(null, request);
+              GetRegionInfoResponse response = admin.getRegionInfo(controller, request);
               switch (response.getCompactionState()) {
                 case MAJOR_AND_MINOR:
                   return CompactionState.MAJOR_AND_MINOR;
@@ -2952,7 +3144,9 @@ public class HBaseAdmin implements Admin {
           admin.getConnection()) {
         @Override
         public AbortProcedureResponse call(int callTimeout) throws ServiceException {
-          return master.abortProcedure(null, request);
+          PayloadCarryingRpcController controller = admin.getRpcControllerFactory().newController();
+          controller.setCallTimeout(callTimeout);
+          return master.abortProcedure(controller, request);
         }
       });
     }
@@ -3366,9 +3560,11 @@ public class HBaseAdmin implements Admin {
       return executeCallable(new MasterCallable<List<SecurityCapability>>(getConnection()) {
         @Override
         public List<SecurityCapability> call(int callTimeout) throws ServiceException {
+          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+          controller.setCallTimeout(callTimeout);
           SecurityCapabilitiesRequest req = SecurityCapabilitiesRequest.newBuilder().build();
           return ProtobufUtil.toSecurityCapabilityList(
-            master.getSecurityCapabilities(null, req).getCapabilitiesList());
+            master.getSecurityCapabilities(controller, req).getCapabilitiesList());
         }
       });
     } catch (IOException e) {
@@ -3414,4 +3610,7 @@ public class HBaseAdmin implements Admin {
             HConstants.EMPTY_END_ROW, false, 0);
   }
 
+  private RpcControllerFactory getRpcControllerFactory() {
+    return rpcControllerFactory;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
index 33fd94e..befc671 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
@@ -105,7 +105,7 @@ public class HTable implements HTableInterface {
   protected ClusterConnection connection;
   private final TableName tableName;
   private volatile Configuration configuration;
-  private TableConfiguration tableConfiguration;
+  private ConnectionConfiguration connConfiguration;
   protected BufferedMutatorImpl mutator;
   private boolean autoFlush = true;
   private boolean closed = false;
@@ -154,7 +154,7 @@ public class HTable implements HTableInterface {
    */
   @InterfaceAudience.Private
   protected HTable(TableName tableName, final ClusterConnection connection,
-      final TableConfiguration tableConfig,
+      final ConnectionConfiguration tableConfig,
       final RpcRetryingCallerFactory rpcCallerFactory,
       final RpcControllerFactory rpcControllerFactory,
       final ExecutorService pool) throws IOException {
@@ -165,7 +165,7 @@ public class HTable implements HTableInterface {
     this.cleanupConnectionOnClose = false;
     this.connection = connection;
     this.configuration = connection.getConfiguration();
-    this.tableConfiguration = tableConfig;
+    this.connConfiguration = tableConfig;
     this.pool = pool;
     if (pool == null) {
       this.pool = getDefaultExecutor(this.configuration);
@@ -188,7 +188,7 @@ public class HTable implements HTableInterface {
   protected HTable(ClusterConnection conn, BufferedMutatorParams params) throws IOException {
     connection = conn;
     tableName = params.getTableName();
-    tableConfiguration = new TableConfiguration(connection.getConfiguration());
+    connConfiguration = new ConnectionConfiguration(connection.getConfiguration());
     cleanupPoolOnClose = false;
     cleanupConnectionOnClose = false;
     // used from tests, don't trust the connection is real
@@ -206,14 +206,14 @@ public class HTable implements HTableInterface {
    * setup this HTable's parameter based on the passed configuration
    */
   private void finishSetup() throws IOException {
-    if (tableConfiguration == null) {
-      tableConfiguration = new TableConfiguration(configuration);
+    if (connConfiguration == null) {
+      connConfiguration = new ConnectionConfiguration(configuration);
     }
 
     this.operationTimeout = tableName.isSystemTable() ?
-        tableConfiguration.getMetaOperationTimeout() : tableConfiguration.getOperationTimeout();
-    this.scannerCaching = tableConfiguration.getScannerCaching();
-    this.scannerMaxResultSize = tableConfiguration.getScannerMaxResultSize();
+        connConfiguration.getMetaOperationTimeout() : connConfiguration.getOperationTimeout();
+    this.scannerCaching = connConfiguration.getScannerCaching();
+    this.scannerMaxResultSize = connConfiguration.getScannerMaxResultSize();
     if (this.rpcCallerFactory == null) {
       this.rpcCallerFactory = connection.getNewRpcRetryingCallerFactory(configuration);
     }
@@ -265,23 +265,14 @@ public class HTable implements HTableInterface {
    */
   @Override
   public HTableDescriptor getTableDescriptor() throws IOException {
-    HTableDescriptor htd = HBaseAdmin.getTableDescriptor(tableName, connection,
-        rpcCallerFactory, operationTimeout);
+    HTableDescriptor htd = HBaseAdmin.getTableDescriptor(tableName, connection, rpcCallerFactory,
+      rpcControllerFactory, operationTimeout);
     if (htd != null) {
       return new UnmodifyableHTableDescriptor(htd);
     }
     return null;
   }
 
-  private <V> V executeMasterCallable(MasterCallable<V> callable) throws IOException {
-    RpcRetryingCaller<V> caller = rpcCallerFactory.newCaller();
-    try {
-      return caller.callWithRetries(callable, operationTimeout);
-    } finally {
-      callable.close();
-    }
-  }
-
   /**
    * Get the corresponding start keys and regions for an arbitrary range of
    * keys.
@@ -354,34 +345,34 @@ public class HTable implements HTableInterface {
 
     Boolean async = scan.isAsyncPrefetch();
     if (async == null) {
-      async = tableConfiguration.isClientScannerAsyncPrefetch();
+      async = connConfiguration.isClientScannerAsyncPrefetch();
     }
 
     if (scan.isReversed()) {
       if (scan.isSmall()) {
         return new ClientSmallReversedScanner(getConfiguration(), scan, getName(),
             this.connection, this.rpcCallerFactory, this.rpcControllerFactory,
-            pool, tableConfiguration.getReplicaCallTimeoutMicroSecondScan());
+            pool, connConfiguration.getReplicaCallTimeoutMicroSecondScan());
       } else {
         return new ReversedClientScanner(getConfiguration(), scan, getName(),
             this.connection, this.rpcCallerFactory, this.rpcControllerFactory,
-            pool, tableConfiguration.getReplicaCallTimeoutMicroSecondScan());
+            pool, connConfiguration.getReplicaCallTimeoutMicroSecondScan());
       }
     }
 
     if (scan.isSmall()) {
       return new ClientSmallScanner(getConfiguration(), scan, getName(),
           this.connection, this.rpcCallerFactory, this.rpcControllerFactory,
-          pool, tableConfiguration.getReplicaCallTimeoutMicroSecondScan());
+          pool, connConfiguration.getReplicaCallTimeoutMicroSecondScan());
     } else {
       if (async) {
         return new ClientAsyncPrefetchScanner(getConfiguration(), scan, getName(), this.connection,
             this.rpcCallerFactory, this.rpcControllerFactory,
-            pool, tableConfiguration.getReplicaCallTimeoutMicroSecondScan());
+            pool, connConfiguration.getReplicaCallTimeoutMicroSecondScan());
       } else {
         return new ClientSimpleScanner(getConfiguration(), scan, getName(), this.connection,
             this.rpcCallerFactory, this.rpcControllerFactory,
-            pool, tableConfiguration.getReplicaCallTimeoutMicroSecondScan());
+            pool, connConfiguration.getReplicaCallTimeoutMicroSecondScan());
       }
     }
   }
@@ -454,9 +445,9 @@ public class HTable implements HTableInterface {
     // Call that takes into account the replica
     RpcRetryingCallerWithReadReplicas callable = new RpcRetryingCallerWithReadReplicas(
         rpcControllerFactory, tableName, this.connection, get, pool,
-        tableConfiguration.getRetriesNumber(),
+        connConfiguration.getRetriesNumber(),
         operationTimeout,
-        tableConfiguration.getPrimaryCallTimeoutMicroSecond());
+        connConfiguration.getPrimaryCallTimeoutMicroSecond());
     return callable.call();
   }
 
@@ -1039,7 +1030,7 @@ public class HTable implements HTableInterface {
 
   // validate for well-formedness
   public void validatePut(final Put put) throws IllegalArgumentException {
-    validatePut(put, tableConfiguration.getMaxKeyValueSize());
+    validatePut(put, connConfiguration.getMaxKeyValueSize());
   }
 
   // validate for well-formedness
@@ -1092,7 +1083,7 @@ public class HTable implements HTableInterface {
   @Override
   public long getWriteBufferSize() {
     if (mutator == null) {
-      return tableConfiguration.getWriteBufferSize();
+      return connConfiguration.getWriteBufferSize();
     } else {
       return mutator.getWriteBufferSize();
     }
@@ -1344,8 +1335,8 @@ public class HTable implements HTableInterface {
       this.mutator = (BufferedMutatorImpl) connection.getBufferedMutator(
           new BufferedMutatorParams(tableName)
               .pool(pool)
-              .writeBufferSize(tableConfiguration.getWriteBufferSize())
-              .maxKeyValueSize(tableConfiguration.getMaxKeyValueSize())
+              .writeBufferSize(connConfiguration.getWriteBufferSize())
+              .maxKeyValueSize(connConfiguration.getMaxKeyValueSize())
       );
     }
     return mutator;


[48/50] [abbrv] hbase git commit: HBASE-11393 Replication TableCfs should be a PB object rather than a string

Posted by sy...@apache.org.
HBASE-11393 Replication TableCfs should be a PB object rather than a string


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

Branch: refs/heads/hbase-12439
Commit: 7f39baf0f4572ff209837d7de5d37554851ecbb7
Parents: 0520097
Author: chenheng <ch...@apache.org>
Authored: Fri Mar 25 14:16:47 2016 +0800
Committer: chenheng <ch...@apache.org>
Committed: Tue Mar 29 10:25:29 2016 +0800

----------------------------------------------------------------------
 .../client/replication/ReplicationAdmin.java    |  170 +--
 .../replication/ReplicationSerDeHelper.java     |  315 +++++
 .../hbase/replication/ReplicationPeer.java      |    1 +
 .../replication/ReplicationPeerConfig.java      |   16 +-
 .../replication/ReplicationPeerZKImpl.java      |   76 +-
 .../hbase/replication/ReplicationPeers.java     |   19 +-
 .../replication/ReplicationPeersZKImpl.java     |  163 +--
 .../replication/ReplicationStateZKBase.java     |   19 +
 .../protobuf/generated/ZooKeeperProtos.java     | 1155 +++++++++++++++++-
 .../src/main/protobuf/ZooKeeper.proto           |    6 +
 .../org/apache/hadoop/hbase/master/HMaster.java |   10 +-
 .../replication/master/TableCFsUpdater.java     |  122 ++
 .../hbase/client/TestReplicaWithCluster.java    |    6 +-
 .../replication/TestReplicationAdmin.java       |  195 +--
 .../cleaner/TestReplicationHFileCleaner.java    |    2 +-
 .../replication/TestMasterReplication.java      |   11 +-
 .../replication/TestMultiSlaveReplication.java  |   10 +-
 .../replication/TestPerTableCFReplication.java  |  158 ++-
 .../hbase/replication/TestReplicationBase.java  |    4 +-
 .../replication/TestReplicationSmallTests.java  |    5 +-
 .../replication/TestReplicationStateBasic.java  |   14 +-
 .../replication/TestReplicationSyncUpTool.java  |    4 +-
 .../TestReplicationTrackerZKImpl.java           |   10 +-
 .../replication/TestReplicationWithTags.java    |    6 +-
 .../replication/master/TestTableCFsUpdater.java |  210 ++++
 ...sibilityLabelReplicationWithExpAsString.java |    9 +-
 .../TestVisibilityLabelsReplication.java        |    5 +-
 .../hadoop/hbase/util/TestHBaseFsckOneRS.java   |    5 +-
 .../src/main/ruby/hbase/replication_admin.rb    |   44 +-
 .../src/main/ruby/shell/commands/add_peer.rb    |    4 +-
 .../ruby/shell/commands/append_peer_tableCFs.rb |    2 +-
 .../ruby/shell/commands/remove_peer_tableCFs.rb |    4 +-
 .../ruby/shell/commands/set_peer_tableCFs.rb    |    5 +-
 33 files changed, 2309 insertions(+), 476 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/7f39baf0/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
index dcf1957..8ee3a22 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
@@ -32,7 +32,6 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 
-import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -48,6 +47,7 @@ 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.RegionLocator;
+import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationPeer;
@@ -166,27 +166,6 @@ public class ReplicationAdmin implements Closeable {
   }
 
   /**
-   * Add a new peer cluster to replicate to.
-   * @param id a short name that identifies the cluster
-   * @param clusterKey the concatenation of the slave cluster's
-   * <code>hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent</code>
-   * @throws IllegalStateException if there's already one slave since
-   * multi-slave isn't supported yet.
-   * @deprecated Use addPeer(String, ReplicationPeerConfig, Map) instead.
-   */
-  @Deprecated
-  public void addPeer(String id, String clusterKey) throws ReplicationException {
-    this.addPeer(id, new ReplicationPeerConfig().setClusterKey(clusterKey), null);
-  }
-
-  @Deprecated
-  public void addPeer(String id, String clusterKey, String tableCFs)
-    throws ReplicationException {
-    this.replicationPeers.addPeer(id,
-      new ReplicationPeerConfig().setClusterKey(clusterKey), tableCFs);
-  }
-  
-  /**
    * Add a new remote slave cluster for replication.
    * @param id a short name that identifies the cluster
    * @param peerConfig configuration for the replication slave cluster
@@ -194,83 +173,36 @@ public class ReplicationAdmin implements Closeable {
    * A map from tableName to column family names. An empty collection can be passed
    * to indicate replicating all column families. Pass null for replicating all table and column
    * families
+   * @deprecated as release of 2.0.0, and it will be removed in 3.0.0,
+   * use {@link #addPeer(String, ReplicationPeerConfig)} instead.
    */
+  @Deprecated
   public void addPeer(String id, ReplicationPeerConfig peerConfig,
       Map<TableName, ? extends Collection<String>> tableCfs) throws ReplicationException {
-    this.replicationPeers.addPeer(id, peerConfig, getTableCfsStr(tableCfs));
-  }
-
-  public static Map<TableName, List<String>> parseTableCFsFromConfig(String tableCFsConfig) {
-    if (tableCFsConfig == null || tableCFsConfig.trim().length() == 0) {
-      return null;
-    }
-
-    Map<TableName, List<String>> tableCFsMap = null;
-    // TODO: This should be a PB object rather than a String to be parsed!! See HBASE-11393
-    // parse out (table, cf-list) pairs from tableCFsConfig
-    // format: "table1:cf1,cf2;table2:cfA,cfB"
-    String[] tables = tableCFsConfig.split(";");
-    for (String tab : tables) {
-      // 1 ignore empty table config
-      tab = tab.trim();
-      if (tab.length() == 0) {
-        continue;
-      }
-      // 2 split to "table" and "cf1,cf2"
-      //   for each table: "table:cf1,cf2" or "table"
-      String[] pair = tab.split(":");
-      String tabName = pair[0].trim();
-      if (pair.length > 2 || tabName.length() == 0) {
-        LOG.error("ignore invalid tableCFs setting: " + tab);
-        continue;
-      }
-
-      // 3 parse "cf1,cf2" part to List<cf>
-      List<String> cfs = null;
-      if (pair.length == 2) {
-        String[] cfsList = pair[1].split(",");
-        for (String cf : cfsList) {
-          String cfName = cf.trim();
-          if (cfName.length() > 0) {
-            if (cfs == null) {
-              cfs = new ArrayList<String>();
-            }
-            cfs.add(cfName);
-          }
-        }
-      }
-
-      // 4 put <table, List<cf>> to map
-      if (tableCFsMap == null) {
-        tableCFsMap = new HashMap<TableName, List<String>>();
-      }
-      tableCFsMap.put(TableName.valueOf(tabName), cfs);
+    if (tableCfs != null) {
+      peerConfig.setTableCFsMap(tableCfs);
     }
-    return tableCFsMap;
+    this.replicationPeers.addPeer(id, peerConfig);
   }
 
-  @VisibleForTesting
-  static String getTableCfsStr(Map<TableName, ? extends Collection<String>> tableCfs) {
-    String tableCfsStr = null;
-    if (tableCfs != null) {
-      // Format: table1:cf1,cf2;table2:cfA,cfB;table3
-      StringBuilder builder = new StringBuilder();
-      for (Entry<TableName, ? extends Collection<String>> entry : tableCfs.entrySet()) {
-        if (builder.length() > 0) {
-          builder.append(";");
-        }
-        builder.append(entry.getKey());
-        if (entry.getValue() != null && !entry.getValue().isEmpty()) {
-          builder.append(":");
-          builder.append(StringUtils.join(entry.getValue(), ","));
-        }
-      }
-      tableCfsStr = builder.toString();
-    }
-    return tableCfsStr;
+  /**
+   * Add a new remote slave cluster for replication.
+   * @param id a short name that identifies the cluster
+   * @param peerConfig configuration for the replication slave cluster
+   */
+  public void addPeer(String id, ReplicationPeerConfig peerConfig) throws ReplicationException {
+    this.replicationPeers.addPeer(id, peerConfig);
   }
 
   /**
+   *  @deprecated as release of 2.0.0, and it will be removed in 3.0.0
+   * */
+  @Deprecated
+  public static Map<TableName, List<String>> parseTableCFsFromConfig(String tableCFsConfig) {
+    return ReplicationSerDeHelper.parseTableCFsFromConfig(tableCFsConfig);
+  }
+  
+  /**
    * Removes a peer cluster and stops the replication to it.
    * @param id a short name that identifies the cluster
    */
@@ -302,22 +234,6 @@ public class ReplicationAdmin implements Closeable {
     return this.replicationPeers.getAllPeerIds().size();
   }
 
-  /**
-   * Map of this cluster's peers for display.
-   * @return A map of peer ids to peer cluster keys
-   * @deprecated use {@link #listPeerConfigs()}
-   */
-  @Deprecated
-  public Map<String, String> listPeers() {
-    Map<String, ReplicationPeerConfig> peers = this.listPeerConfigs();
-    Map<String, String> ret = new HashMap<String, String>(peers.size());
-
-    for (Map.Entry<String, ReplicationPeerConfig> entry : peers.entrySet()) {
-      ret.put(entry.getKey(), entry.getValue().getClusterKey());
-    }
-    return ret;
-  }
-
   public Map<String, ReplicationPeerConfig> listPeerConfigs() {
     return this.replicationPeers.getAllPeerConfigs();
   }
@@ -329,19 +245,12 @@ public class ReplicationAdmin implements Closeable {
   /**
    * Get the replicable table-cf config of the specified peer.
    * @param id a short name that identifies the cluster
-   */
-  public String getPeerTableCFs(String id) throws ReplicationException {
-    return this.replicationPeers.getPeerTableCFsConfig(id);
-  }
-
-  /**
-   * Set the replicable table-cf config of the specified peer
-   * @param id a short name that identifies the cluster
-   * @deprecated use {@link #setPeerTableCFs(String, Map)}
-   */
+   * @deprecated as release of 2.0.0, and it will be removed in 3.0.0,
+   * use {@link #getPeerConfig(String)} instead.
+   * */
   @Deprecated
-  public void setPeerTableCFs(String id, String tableCFs) throws ReplicationException {
-    this.replicationPeers.setPeerTableCFsConfig(id, tableCFs);
+  public String getPeerTableCFs(String id) throws ReplicationException {
+    return ReplicationSerDeHelper.convertToString(this.replicationPeers.getPeerTableCFsConfig(id));
   }
 
   /**
@@ -349,9 +258,12 @@ public class ReplicationAdmin implements Closeable {
    * @param id a short that identifies the cluster
    * @param tableCfs table-cfs config str
    * @throws ReplicationException
+   * @deprecated as release of 2.0.0, and it will be removed in 3.0.0,
+   * use {@link #appendPeerTableCFs(String, Map)} instead.
    */
+  @Deprecated
   public void appendPeerTableCFs(String id, String tableCfs) throws ReplicationException {
-    appendPeerTableCFs(id, parseTableCFsFromConfig(tableCfs));
+    appendPeerTableCFs(id, ReplicationSerDeHelper.parseTableCFsFromConfig(tableCfs));
   }
 
   /**
@@ -365,12 +277,11 @@ public class ReplicationAdmin implements Closeable {
     if (tableCfs == null) {
       throw new ReplicationException("tableCfs is null");
     }
-    Map<TableName, List<String>> preTableCfs = parseTableCFsFromConfig(getPeerTableCFs(id));
+    Map<TableName, List<String>> preTableCfs = this.replicationPeers.getPeerTableCFsConfig(id);
     if (preTableCfs == null) {
       setPeerTableCFs(id, tableCfs);
       return;
     }
-
     for (Map.Entry<TableName, ? extends Collection<String>> entry : tableCfs.entrySet()) {
       TableName table = entry.getKey();
       Collection<String> appendCfs = entry.getValue();
@@ -382,6 +293,7 @@ public class ReplicationAdmin implements Closeable {
           Set<String> cfSet = new HashSet<String>(cfs);
           cfSet.addAll(appendCfs);
           preTableCfs.put(table, Lists.newArrayList(cfSet));
+
         }
       } else {
         if (appendCfs == null || appendCfs.isEmpty()) {
@@ -399,9 +311,12 @@ public class ReplicationAdmin implements Closeable {
    * @param id a short name that identifies the cluster
    * @param tableCf table-cfs config str
    * @throws ReplicationException
+   * @deprecated as release of 2.0.0, and it will be removed in 3.0.0,
+   * use {@link #removePeerTableCFs(String, Map)} instead.
    */
+  @Deprecated
   public void removePeerTableCFs(String id, String tableCf) throws ReplicationException {
-    removePeerTableCFs(id, parseTableCFsFromConfig(tableCf));
+    removePeerTableCFs(id, ReplicationSerDeHelper.parseTableCFsFromConfig(tableCf));
   }
 
   /**
@@ -415,12 +330,12 @@ public class ReplicationAdmin implements Closeable {
     if (tableCfs == null) {
       throw new ReplicationException("tableCfs is null");
     }
-
-    Map<TableName, List<String>> preTableCfs = parseTableCFsFromConfig(getPeerTableCFs(id));
+    Map<TableName, List<String>> preTableCfs = this.replicationPeers.getPeerTableCFsConfig(id);
     if (preTableCfs == null) {
       throw new ReplicationException("Table-Cfs for peer" + id + " is null");
     }
     for (Map.Entry<TableName, ? extends Collection<String>> entry: tableCfs.entrySet()) {
+
       TableName table = entry.getKey();
       Collection<String> removeCfs = entry.getValue();
       if (preTableCfs.containsKey(table)) {
@@ -444,6 +359,7 @@ public class ReplicationAdmin implements Closeable {
         }
       } else {
         throw new ReplicationException("No table: " + table + " in table-cfs config of peer: " + id);
+
       }
     }
     setPeerTableCFs(id, preTableCfs);
@@ -459,7 +375,7 @@ public class ReplicationAdmin implements Closeable {
    */
   public void setPeerTableCFs(String id, Map<TableName, ? extends Collection<String>> tableCfs)
       throws ReplicationException {
-    this.replicationPeers.setPeerTableCFsConfig(id, getTableCfsStr(tableCfs));
+    this.replicationPeers.setPeerTableCFsConfig(id, tableCfs);
   }
 
   /**
@@ -645,8 +561,8 @@ public class ReplicationAdmin implements Closeable {
       try {
         Pair<ReplicationPeerConfig, Configuration> pair = this.replicationPeers.getPeerConf(peerId);
         Configuration peerConf = pair.getSecond();
-        ReplicationPeer peer = new ReplicationPeerZKImpl(peerConf, peerId, pair.getFirst(),
-            parseTableCFsFromConfig(this.getPeerTableCFs(peerId)));
+        ReplicationPeer peer = new ReplicationPeerZKImpl(zkw, pair.getSecond(),
+          peerId, pair.getFirst(), this.connection);
         listOfPeers.add(peer);
       } catch (ReplicationException e) {
         LOG.warn("Failed to get valid replication peers. "

http://git-wip-us.apache.org/repos/asf/hbase/blob/7f39baf0/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationSerDeHelper.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationSerDeHelper.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationSerDeHelper.java
new file mode 100644
index 0000000..9682f89
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationSerDeHelper.java
@@ -0,0 +1,315 @@
+/**
+ *
+ * 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.replication;
+
+import com.google.protobuf.ByteString;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Strings;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.ArrayList;
+
+/**
+ * Helper for TableCFs Operations.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Stable
+public final class ReplicationSerDeHelper {
+
+  private static final Log LOG = LogFactory.getLog(ReplicationSerDeHelper.class);
+
+  private ReplicationSerDeHelper() {}
+
+  /** convert map to TableCFs Object */
+  public static ZooKeeperProtos.TableCF[] convert(
+      Map<TableName, ? extends Collection<String>> tableCfs) {
+    if (tableCfs == null) {
+      return null;
+    }
+    List<ZooKeeperProtos.TableCF> tableCFList = new ArrayList<>();
+    ZooKeeperProtos.TableCF.Builder tableCFBuilder =  ZooKeeperProtos.TableCF.newBuilder();
+    for (Map.Entry<TableName, ? extends Collection<String>> entry : tableCfs.entrySet()) {
+      tableCFBuilder.clear();
+      tableCFBuilder.setTableName(ProtobufUtil.toProtoTableName(entry.getKey()));
+      Collection<String> v = entry.getValue();
+      if (v != null && !v.isEmpty()) {
+        for (String value : entry.getValue()) {
+          tableCFBuilder.addFamilies(ByteString.copyFromUtf8(value));
+        }
+      }
+      tableCFList.add(tableCFBuilder.build());
+    }
+    return tableCFList.toArray(new ZooKeeperProtos.TableCF[tableCFList.size()]);
+  }
+
+  public static String convertToString(Map<TableName, ? extends Collection<String>> tableCfs) {
+    if (tableCfs == null) {
+      return null;
+    }
+    return convert(convert(tableCfs));
+  }
+
+  /**
+   *  Convert string to TableCFs Object.
+   *  This is only for read TableCFs information from TableCF node.
+   *  Input String Format: ns1.table1:cf1,cf2;ns2.table2:cfA,cfB;ns3.table3.
+   * */
+  public static ZooKeeperProtos.TableCF[] convert(String tableCFsConfig) {
+    if (tableCFsConfig == null || tableCFsConfig.trim().length() == 0) {
+      return null;
+    }
+    List<ZooKeeperProtos.TableCF> tableCFList = new ArrayList<>();
+    ZooKeeperProtos.TableCF.Builder tableCFBuilder = ZooKeeperProtos.TableCF.newBuilder();
+
+    String[] tables = tableCFsConfig.split(";");
+    for (String tab : tables) {
+      // 1 ignore empty table config
+      tab = tab.trim();
+      if (tab.length() == 0) {
+        continue;
+      }
+      // 2 split to "table" and "cf1,cf2"
+      //   for each table: "table#cf1,cf2" or "table"
+      String[] pair = tab.split(":");
+      String tabName = pair[0].trim();
+      if (pair.length > 2 || tabName.length() == 0) {
+        LOG.info("incorrect format:" + tableCFsConfig);
+        continue;
+      }
+
+      tableCFBuilder.clear();
+      // split namespace from tableName
+      String ns = "default";
+      String tName = tabName;
+      String[] dbs = tabName.split("\\.");
+      if (dbs != null && dbs.length == 2) {
+        ns = dbs[0];
+        tName = dbs[1];
+      }
+      tableCFBuilder.setTableName(
+        ProtobufUtil.toProtoTableName(TableName.valueOf(ns, tName)));
+
+      // 3 parse "cf1,cf2" part to List<cf>
+      if (pair.length == 2) {
+        String[] cfsList = pair[1].split(",");
+        for (String cf : cfsList) {
+          String cfName = cf.trim();
+          if (cfName.length() > 0) {
+            tableCFBuilder.addFamilies(ByteString.copyFromUtf8(cfName));
+          }
+        }
+      }
+      tableCFList.add(tableCFBuilder.build());
+    }
+    return tableCFList.toArray(new ZooKeeperProtos.TableCF[tableCFList.size()]);
+  }
+
+  /**
+   *  Convert TableCFs Object to String.
+   *  Output String Format: ns1.table1:cf1,cf2;ns2.table2:cfA,cfB;table3
+   * */
+  public static String convert(ZooKeeperProtos.TableCF[] tableCFs) {
+    StringBuilder sb = new StringBuilder();
+    for (int i = 0, n = tableCFs.length; i < n; i++) {
+      ZooKeeperProtos.TableCF tableCF = tableCFs[i];
+      String namespace = tableCF.getTableName().getNamespace().toStringUtf8();
+      if (!Strings.isEmpty(namespace)) {
+        sb.append(namespace).append(".").
+            append(tableCF.getTableName().getQualifier().toStringUtf8())
+            .append(":");
+      } else {
+        sb.append(tableCF.getTableName().toString()).append(":");
+      }
+      for (int j = 0; j < tableCF.getFamiliesCount(); j++) {
+        sb.append(tableCF.getFamilies(j).toStringUtf8()).append(",");
+      }
+      sb.deleteCharAt(sb.length() - 1).append(";");
+    }
+    if (sb.length() > 0) {
+      sb.deleteCharAt(sb.length() - 1);
+    }
+    return sb.toString();
+  }
+
+  /**
+   *  Get TableCF in TableCFs, if not exist, return null.
+   * */
+  public static ZooKeeperProtos.TableCF getTableCF(ZooKeeperProtos.TableCF[] tableCFs,
+                                           String table) {
+    for (int i = 0, n = tableCFs.length; i < n; i++) {
+      ZooKeeperProtos.TableCF tableCF = tableCFs[i];
+      if (tableCF.getTableName().getQualifier().toStringUtf8().equals(table)) {
+        return tableCF;
+      }
+    }
+    return null;
+  }
+
+  /**
+   *  Parse bytes into TableCFs.
+   *  It is used for backward compatibility.
+   *  Old format bytes have no PB_MAGIC Header
+   * */
+  public static ZooKeeperProtos.TableCF[] parseTableCFs(byte[] bytes) throws IOException {
+    if (bytes == null) {
+      return null;
+    }
+    return ReplicationSerDeHelper.convert(Bytes.toString(bytes));
+  }
+
+  /**
+   *  Convert tableCFs string into Map.
+   * */
+  public static Map<TableName, List<String>> parseTableCFsFromConfig(String tableCFsConfig) {
+    ZooKeeperProtos.TableCF[] tableCFs = convert(tableCFsConfig);
+    return convert2Map(tableCFs);
+  }
+
+  /**
+   *  Convert tableCFs Object to Map.
+   * */
+  public static Map<TableName, List<String>> convert2Map(ZooKeeperProtos.TableCF[] tableCFs) {
+    if (tableCFs == null || tableCFs.length == 0) {
+      return null;
+    }
+    Map<TableName, List<String>> tableCFsMap = new HashMap<TableName, List<String>>();
+    for (int i = 0, n = tableCFs.length; i < n; i++) {
+      ZooKeeperProtos.TableCF tableCF = tableCFs[i];
+      List<String> families = new ArrayList<>();
+      for (int j = 0, m = tableCF.getFamiliesCount(); j < m; j++) {
+        families.add(tableCF.getFamilies(j).toStringUtf8());
+      }
+      if (families.size() > 0) {
+        tableCFsMap.put(ProtobufUtil.toTableName(tableCF.getTableName()), families);
+      } else {
+        tableCFsMap.put(ProtobufUtil.toTableName(tableCF.getTableName()), null);
+      }
+    }
+
+    return tableCFsMap;
+  }
+
+  /**
+   * @param bytes Content of a peer znode.
+   * @return ClusterKey parsed from the passed bytes.
+   * @throws DeserializationException
+   */
+  public static ReplicationPeerConfig parsePeerFrom(final byte[] bytes)
+      throws DeserializationException {
+    if (ProtobufUtil.isPBMagicPrefix(bytes)) {
+      int pblen = ProtobufUtil.lengthOfPBMagic();
+      ZooKeeperProtos.ReplicationPeer.Builder builder =
+          ZooKeeperProtos.ReplicationPeer.newBuilder();
+      ZooKeeperProtos.ReplicationPeer peer;
+      try {
+        ProtobufUtil.mergeFrom(builder, bytes, pblen, bytes.length - pblen);
+        peer = builder.build();
+      } catch (IOException e) {
+        throw new DeserializationException(e);
+      }
+      return convert(peer);
+    } else {
+      if (bytes.length > 0) {
+        return new ReplicationPeerConfig().setClusterKey(Bytes.toString(bytes));
+      }
+      return new ReplicationPeerConfig().setClusterKey("");
+    }
+  }
+
+  public static ReplicationPeerConfig convert(ZooKeeperProtos.ReplicationPeer peer) {
+    ReplicationPeerConfig peerConfig = new ReplicationPeerConfig();
+    if (peer.hasClusterkey()) {
+      peerConfig.setClusterKey(peer.getClusterkey());
+    }
+    if (peer.hasReplicationEndpointImpl()) {
+      peerConfig.setReplicationEndpointImpl(peer.getReplicationEndpointImpl());
+    }
+
+    for (HBaseProtos.BytesBytesPair pair : peer.getDataList()) {
+      peerConfig.getPeerData().put(pair.getFirst().toByteArray(), pair.getSecond().toByteArray());
+    }
+
+    for (HBaseProtos.NameStringPair pair : peer.getConfigurationList()) {
+      peerConfig.getConfiguration().put(pair.getName(), pair.getValue());
+    }
+    Map<TableName, ? extends Collection<String>> tableCFsMap = convert2Map(
+      peer.getTableCfsList().toArray(new ZooKeeperProtos.TableCF[peer.getTableCfsCount()]));
+    if (tableCFsMap != null) {
+      peerConfig.setTableCFsMap(tableCFsMap);
+    }
+    return peerConfig;
+  }
+
+  public static ZooKeeperProtos.ReplicationPeer convert(ReplicationPeerConfig  peerConfig) {
+    ZooKeeperProtos.ReplicationPeer.Builder builder = ZooKeeperProtos.ReplicationPeer.newBuilder();
+    if (peerConfig.getClusterKey() != null) {
+      builder.setClusterkey(peerConfig.getClusterKey());
+    }
+    if (peerConfig.getReplicationEndpointImpl() != null) {
+      builder.setReplicationEndpointImpl(peerConfig.getReplicationEndpointImpl());
+    }
+
+    for (Map.Entry<byte[], byte[]> entry : peerConfig.getPeerData().entrySet()) {
+      builder.addData(HBaseProtos.BytesBytesPair.newBuilder()
+          .setFirst(ByteString.copyFrom(entry.getKey()))
+          .setSecond(ByteString.copyFrom(entry.getValue()))
+          .build());
+    }
+
+    for (Map.Entry<String, String> entry : peerConfig.getConfiguration().entrySet()) {
+      builder.addConfiguration(HBaseProtos.NameStringPair.newBuilder()
+          .setName(entry.getKey())
+          .setValue(entry.getValue())
+          .build());
+    }
+    ZooKeeperProtos.TableCF[] tableCFs = convert(peerConfig.getTableCFsMap());
+    if (tableCFs != null) {
+      for (int i = 0; i < tableCFs.length; i++) {
+        builder.addTableCfs(tableCFs[i]);
+      }
+    }
+    return builder.build();
+  }
+
+  /**
+   * @param peerConfig
+   * @return Serialized protobuf of <code>peerConfig</code> with pb magic prefix prepended suitable
+   *         for use as content of a this.peersZNode; i.e. the content of PEER_ID znode under
+   *         /hbase/replication/peers/PEER_ID
+   */
+  public static byte[] toByteArray(final ReplicationPeerConfig peerConfig) {
+    byte[] bytes = convert(peerConfig).toByteArray();
+    return ProtobufUtil.prependPBMagic(bytes);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/7f39baf0/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java
index b8b5b22..920eea6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 
+
 /**
  * ReplicationPeer manages enabled / disabled state for the peer.
  */

http://git-wip-us.apache.org/repos/asf/hbase/blob/7f39baf0/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java
index 043b38f..8d05fa0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java
@@ -18,12 +18,16 @@
 
 package org.apache.hadoop.hbase.replication;
 
+import java.util.Collection;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
 
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
@@ -37,6 +41,7 @@ public class ReplicationPeerConfig {
   private String replicationEndpointImpl;
   private final Map<byte[], byte[]> peerData;
   private final Map<String, String> configuration;
+  private Map<TableName, ? extends Collection<String>> tableCFsMap = null;
 
 
   public ReplicationPeerConfig() {
@@ -78,10 +83,19 @@ public class ReplicationPeerConfig {
     return configuration;
   }
 
+  public Map<TableName, List<String>> getTableCFsMap() {
+    return (Map<TableName, List<String>>) tableCFsMap;
+  }
+
+  public void setTableCFsMap(Map<TableName,? extends Collection<String>> tableCFsMap) {
+    this.tableCFsMap = tableCFsMap;
+  }
+
   @Override
   public String toString() {
     StringBuilder builder = new StringBuilder("clusterKey=").append(clusterKey).append(",");
-    builder.append("replicationEndpointImpl=").append(replicationEndpointImpl);
+    builder.append("replicationEndpointImpl=").append(replicationEndpointImpl).append(",")
+        .append("tableCFs=").append(tableCFsMap.toString());
     return builder.toString();
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/7f39baf0/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
index 39f6ebc..f7a2411 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
@@ -30,11 +30,10 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
+import org.apache.hadoop.hbase.client.replication.ReplicationSerDeHelper;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@@ -42,17 +41,18 @@ import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.NodeExistsException;
 
 @InterfaceAudience.Private
-public class ReplicationPeerZKImpl implements ReplicationPeer, Abortable, Closeable {
+public class ReplicationPeerZKImpl extends ReplicationStateZKBase
+    implements ReplicationPeer, Abortable, Closeable {
   private static final Log LOG = LogFactory.getLog(ReplicationPeerZKImpl.class);
 
-  private final ReplicationPeerConfig peerConfig;
+  private ReplicationPeerConfig peerConfig;
   private final String id;
   private volatile PeerState peerState;
   private volatile Map<TableName, List<String>> tableCFs = new HashMap<TableName, List<String>>();
   private final Configuration conf;
-
   private PeerStateTracker peerStateTracker;
-  private TableCFsTracker tableCFsTracker;
+  private PeerConfigTracker peerConfigTracker;
+
 
   /**
    * Constructor that takes all the objects required to communicate with the specified peer, except
@@ -61,39 +61,25 @@ public class ReplicationPeerZKImpl implements ReplicationPeer, Abortable, Closea
    * @param id string representation of this peer's identifier
    * @param peerConfig configuration for the replication peer
    */
-  public ReplicationPeerZKImpl(Configuration conf, String id, ReplicationPeerConfig peerConfig)
+  public ReplicationPeerZKImpl(ZooKeeperWatcher zkWatcher, Configuration conf,
+                               String id, ReplicationPeerConfig peerConfig,
+                               Abortable abortable)
       throws ReplicationException {
+    super(zkWatcher, conf, abortable);
     this.conf = conf;
     this.peerConfig = peerConfig;
     this.id = id;
   }
-  
-  /**
-   * Constructor that takes all the objects required to communicate with the specified peer, except
-   * for the region server addresses.
-   * @param conf configuration object to this peer
-   * @param id string representation of this peer's identifier
-   * @param peerConfig configuration for the replication peer
-   * @param tableCFs table-cf configuration for this peer
-   */
-  public ReplicationPeerZKImpl(Configuration conf, String id, ReplicationPeerConfig peerConfig,
-      Map<TableName, List<String>> tableCFs) throws ReplicationException {
-    this.conf = conf;
-    this.peerConfig = peerConfig;
-    this.id = id;
-    this.tableCFs = tableCFs;
-  }
 
   /**
    * start a state tracker to check whether this peer is enabled or not
    *
-   * @param zookeeper zk watcher for the local cluster
    * @param peerStateNode path to zk node which stores peer state
    * @throws KeeperException
    */
-  public void startStateTracker(ZooKeeperWatcher zookeeper, String peerStateNode)
+  public void startStateTracker(String peerStateNode)
       throws KeeperException {
-    ensurePeerEnabled(zookeeper, peerStateNode);
+    ensurePeerEnabled(peerStateNode);
     this.peerStateTracker = new PeerStateTracker(peerStateNode, zookeeper, this);
     this.peerStateTracker.start();
     try {
@@ -112,22 +98,26 @@ public class ReplicationPeerZKImpl implements ReplicationPeer, Abortable, Closea
 
   /**
    * start a table-cfs tracker to listen the (table, cf-list) map change
-   *
-   * @param zookeeper zk watcher for the local cluster
-   * @param tableCFsNode path to zk node which stores table-cfs
+   * @param peerConfigNode path to zk node which stores table-cfs
    * @throws KeeperException
    */
-  public void startTableCFsTracker(ZooKeeperWatcher zookeeper, String tableCFsNode)
+  public void startPeerConfigTracker(String peerConfigNode)
     throws KeeperException {
-    this.tableCFsTracker = new TableCFsTracker(tableCFsNode, zookeeper,
+    this.peerConfigTracker = new PeerConfigTracker(peerConfigNode, zookeeper,
         this);
-    this.tableCFsTracker.start();
-    this.readTableCFsZnode();
+    this.peerConfigTracker.start();
+    this.readPeerConfig();
   }
 
-  private void readTableCFsZnode() {
-    String currentTableCFs = Bytes.toString(tableCFsTracker.getData(false));
-    this.tableCFs = ReplicationAdmin.parseTableCFsFromConfig(currentTableCFs);
+  private void readPeerConfig() {
+    try {
+      byte[] data = peerConfigTracker.getData(false);
+      if (data != null) {
+        this.peerConfig = ReplicationSerDeHelper.parsePeerFrom(data);
+      }
+    } catch (DeserializationException e) {
+      LOG.error("", e);
+    }
   }
 
   @Override
@@ -168,6 +158,7 @@ public class ReplicationPeerZKImpl implements ReplicationPeer, Abortable, Closea
    */
   @Override
   public Map<TableName, List<String>> getTableCFs() {
+    this.tableCFs = peerConfig.getTableCFsMap();
     return this.tableCFs;
   }
 
@@ -223,13 +214,12 @@ public class ReplicationPeerZKImpl implements ReplicationPeer, Abortable, Closea
 
   /**
    * Utility method to ensure an ENABLED znode is in place; if not present, we create it.
-   * @param zookeeper
    * @param path Path to znode to check
    * @return True if we created the znode.
    * @throws NodeExistsException
    * @throws KeeperException
    */
-  private static boolean ensurePeerEnabled(final ZooKeeperWatcher zookeeper, final String path)
+  private boolean ensurePeerEnabled(final String path)
       throws NodeExistsException, KeeperException {
     if (ZKUtil.checkExists(zookeeper, path) == -1) {
       // There is a race b/w PeerWatcher and ReplicationZookeeper#add method to create the
@@ -266,20 +256,20 @@ public class ReplicationPeerZKImpl implements ReplicationPeer, Abortable, Closea
   }
 
   /**
-   * Tracker for (table, cf-list) map of this peer
+   * Tracker for PeerConfigNode of this peer
    */
-  public class TableCFsTracker extends ZooKeeperNodeTracker {
+  public class PeerConfigTracker extends ZooKeeperNodeTracker {
 
-    public TableCFsTracker(String tableCFsZNode, ZooKeeperWatcher watcher,
+    public PeerConfigTracker(String peerConfigNode, ZooKeeperWatcher watcher,
         Abortable abortable) {
-      super(watcher, tableCFsZNode, abortable);
+      super(watcher, peerConfigNode, abortable);
     }
     
     @Override
     public synchronized void nodeCreated(String path) {
       if (path.equals(node)) {
         super.nodeCreated(path);
-        readTableCFsZnode();
+        readPeerConfig();
       }
     }
 

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/7f39baf0/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
index e14f2c6..367c688 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.replication;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -29,19 +30,16 @@ import java.util.concurrent.ConcurrentMap;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.CompoundConfiguration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.replication.ReplicationSerDeHelper;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair;
-import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
 import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp;
@@ -49,8 +47,6 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.NoNodeException;
 
-import com.google.protobuf.ByteString;
-
 /**
  * This class provides an implementation of the ReplicationPeers interface using Zookeeper. The
  * peers znode contains a list of all peer replication clusters and the current replication state of
@@ -82,15 +78,15 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
 
   // Map of peer clusters keyed by their id
   private Map<String, ReplicationPeerZKImpl> peerClusters;
-  private final String tableCFsNodeName;
   private final ReplicationQueuesClient queuesClient;
+  private Abortable abortable;
 
   private static final Log LOG = LogFactory.getLog(ReplicationPeersZKImpl.class);
 
   public ReplicationPeersZKImpl(final ZooKeeperWatcher zk, final Configuration conf,
       final ReplicationQueuesClient queuesClient, Abortable abortable) {
     super(zk, conf, abortable);
-    this.tableCFsNodeName = conf.get("zookeeper.znode.replication.peers.tableCFs", "tableCFs");
+    this.abortable = abortable;
     this.peerClusters = new ConcurrentHashMap<String, ReplicationPeerZKImpl>();
     this.queuesClient = queuesClient;
   }
@@ -108,7 +104,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
   }
 
   @Override
-  public void addPeer(String id, ReplicationPeerConfig peerConfig, String tableCFs)
+  public void addPeer(String id, ReplicationPeerConfig peerConfig)
       throws ReplicationException {
     try {
       if (peerExists(id)) {
@@ -136,18 +132,15 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
       }
 
       List<ZKUtilOp> listOfOps = new ArrayList<ZKUtil.ZKUtilOp>();
-      ZKUtilOp op1 = ZKUtilOp.createAndFailSilent(ZKUtil.joinZNode(this.peersZNode, id),
-        toByteArray(peerConfig));
+      ZKUtilOp op1 = ZKUtilOp.createAndFailSilent(getPeerNode(id),
+        ReplicationSerDeHelper.toByteArray(peerConfig));
       // There is a race (if hbase.zookeeper.useMulti is false)
       // b/w PeerWatcher and ReplicationZookeeper#add method to create the
       // peer-state znode. This happens while adding a peer
       // The peer state data is set as "ENABLED" by default.
       ZKUtilOp op2 = ZKUtilOp.createAndFailSilent(getPeerStateNode(id), ENABLED_ZNODE_BYTES);
-      String tableCFsStr = (tableCFs == null) ? "" : tableCFs;
-      ZKUtilOp op3 = ZKUtilOp.createAndFailSilent(getTableCFsNode(id), Bytes.toBytes(tableCFsStr));
       listOfOps.add(op1);
       listOfOps.add(op2);
-      listOfOps.add(op3);
       ZKUtil.multiOrSequential(this.zookeeper, listOfOps, false);
       // A peer is enabled by default
     } catch (KeeperException e) {
@@ -192,13 +185,17 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
   }
 
   @Override
-  public String getPeerTableCFsConfig(String id) throws ReplicationException {
+  public Map<TableName, List<String>> getPeerTableCFsConfig(String id) throws ReplicationException {
     try {
       if (!peerExists(id)) {
         throw new IllegalArgumentException("peer " + id + " doesn't exist");
       }
       try {
-        return Bytes.toString(ZKUtil.getData(this.zookeeper, getTableCFsNode(id)));
+        ReplicationPeerConfig rpc = getReplicationPeerConfig(id);
+        if (rpc == null) {
+          throw new ReplicationException("Unable to get tableCFs of the peer with id=" + id);
+        }
+        return rpc.getTableCFsMap();
       } catch (Exception e) {
         throw new ReplicationException(e);
       }
@@ -208,35 +205,29 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
   }
 
   @Override
-  public void setPeerTableCFsConfig(String id, String tableCFsStr) throws ReplicationException {
+  public void setPeerTableCFsConfig(String id,
+                                    Map<TableName, ? extends Collection<String>>  tableCFs)
+      throws ReplicationException {
     try {
       if (!peerExists(id)) {
         throw new IllegalArgumentException("Cannot set peer tableCFs because id=" + id
             + " does not exist.");
       }
-      String tableCFsZKNode = getTableCFsNode(id);
-      byte[] tableCFs = Bytes.toBytes(tableCFsStr);
-      if (ZKUtil.checkExists(this.zookeeper, tableCFsZKNode) != -1) {
-        ZKUtil.setData(this.zookeeper, tableCFsZKNode, tableCFs);
-      } else {
-        ZKUtil.createAndWatch(this.zookeeper, tableCFsZKNode, tableCFs);
+      ReplicationPeerConfig rpc = getReplicationPeerConfig(id);
+      if (rpc == null) {
+        throw new ReplicationException("Unable to get tableCFs of the peer with id=" + id);
       }
-      LOG.info("Peer tableCFs with id= " + id + " is now " + tableCFsStr);
+      rpc.setTableCFsMap(tableCFs);
+      ZKUtil.setData(this.zookeeper, getPeerNode(id),
+          ReplicationSerDeHelper.toByteArray(rpc));
+      LOG.info("Peer tableCFs with id= " + id + " is now " +
+        ReplicationSerDeHelper.convertToString(tableCFs));
     } catch (KeeperException e) {
       throw new ReplicationException("Unable to change tableCFs of the peer with id=" + id, e);
     }
   }
 
   @Override
-  public Map<TableName, List<String>> getTableCFs(String id) throws IllegalArgumentException {
-    ReplicationPeer replicationPeer = this.peerClusters.get(id);
-    if (replicationPeer == null) {
-      throw new IllegalArgumentException("Peer with id= " + id + " is not connected");
-    }
-    return replicationPeer.getTableCFs();
-  }
-
-  @Override
   public boolean getStatusOfPeer(String id) {
     ReplicationPeer replicationPeer = this.peerClusters.get(id);
     if (replicationPeer == null) {
@@ -306,7 +297,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
   @Override
   public ReplicationPeerConfig getReplicationPeerConfig(String peerId)
       throws ReplicationException {
-    String znode = ZKUtil.joinZNode(this.peersZNode, peerId);
+    String znode = getPeerNode(peerId);
     byte[] data = null;
     try {
       data = ZKUtil.getData(this.zookeeper, znode);
@@ -325,7 +316,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
     }
 
     try {
-      return parsePeerFrom(data);
+      return ReplicationSerDeHelper.parsePeerFrom(data);
     } catch (DeserializationException e) {
       LOG.warn("Failed to parse cluster key from peerId=" + peerId
           + ", specifically the content from the following znode: " + znode);
@@ -438,14 +429,6 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
     return true;
   }
 
-  private String getTableCFsNode(String id) {
-    return ZKUtil.joinZNode(this.peersZNode, ZKUtil.joinZNode(id, this.tableCFsNodeName));
-  }
-
-  private String getPeerStateNode(String id) {
-    return ZKUtil.joinZNode(this.peersZNode, ZKUtil.joinZNode(id, this.peerStateNodeName));
-  }
-
   /**
    * Update the state znode of a peer cluster.
    * @param id
@@ -486,16 +469,17 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
     }
     Configuration peerConf = pair.getSecond();
 
-    ReplicationPeerZKImpl peer = new ReplicationPeerZKImpl(peerConf, peerId, pair.getFirst());
+    ReplicationPeerZKImpl peer = new ReplicationPeerZKImpl(zookeeper,
+        peerConf, peerId, pair.getFirst(), abortable);
     try {
-      peer.startStateTracker(this.zookeeper, this.getPeerStateNode(peerId));
+      peer.startStateTracker(this.getPeerStateNode(peerId));
     } catch (KeeperException e) {
       throw new ReplicationException("Error starting the peer state tracker for peerId=" +
           peerId, e);
     }
 
     try {
-      peer.startTableCFsTracker(this.zookeeper, this.getTableCFsNode(peerId));
+      peer.startPeerConfigTracker(this.getPeerNode(peerId));
     } catch (KeeperException e) {
       throw new ReplicationException("Error starting the peer tableCFs tracker for peerId=" +
           peerId, e);
@@ -504,89 +488,6 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
     return peer;
   }
 
-  /**
-   * @param bytes Content of a peer znode.
-   * @return ClusterKey parsed from the passed bytes.
-   * @throws DeserializationException
-   */
-  private static ReplicationPeerConfig parsePeerFrom(final byte[] bytes)
-      throws DeserializationException {
-    if (ProtobufUtil.isPBMagicPrefix(bytes)) {
-      int pblen = ProtobufUtil.lengthOfPBMagic();
-      ZooKeeperProtos.ReplicationPeer.Builder builder =
-          ZooKeeperProtos.ReplicationPeer.newBuilder();
-      ZooKeeperProtos.ReplicationPeer peer;
-      try {
-        ProtobufUtil.mergeFrom(builder, bytes, pblen, bytes.length - pblen);
-        peer = builder.build();
-      } catch (IOException e) {
-        throw new DeserializationException(e);
-      }
-      return convert(peer);
-    } else {
-      if (bytes.length > 0) {
-        return new ReplicationPeerConfig().setClusterKey(Bytes.toString(bytes));
-      }
-      return new ReplicationPeerConfig().setClusterKey("");
-    }
-  }
-
-  private static ReplicationPeerConfig convert(ZooKeeperProtos.ReplicationPeer peer) {
-    ReplicationPeerConfig peerConfig = new ReplicationPeerConfig();
-    if (peer.hasClusterkey()) {
-      peerConfig.setClusterKey(peer.getClusterkey());
-    }
-    if (peer.hasReplicationEndpointImpl()) {
-      peerConfig.setReplicationEndpointImpl(peer.getReplicationEndpointImpl());
-    }
-
-    for (BytesBytesPair pair : peer.getDataList()) {
-      peerConfig.getPeerData().put(pair.getFirst().toByteArray(), pair.getSecond().toByteArray());
-    }
-
-    for (NameStringPair pair : peer.getConfigurationList()) {
-      peerConfig.getConfiguration().put(pair.getName(), pair.getValue());
-    }
-    return peerConfig;
-  }
-
-  private static ZooKeeperProtos.ReplicationPeer convert(ReplicationPeerConfig  peerConfig) {
-    ZooKeeperProtos.ReplicationPeer.Builder builder = ZooKeeperProtos.ReplicationPeer.newBuilder();
-    if (peerConfig.getClusterKey() != null) {
-      builder.setClusterkey(peerConfig.getClusterKey());
-    }
-    if (peerConfig.getReplicationEndpointImpl() != null) {
-      builder.setReplicationEndpointImpl(peerConfig.getReplicationEndpointImpl());
-    }
-
-    for (Map.Entry<byte[], byte[]> entry : peerConfig.getPeerData().entrySet()) {
-      builder.addData(BytesBytesPair.newBuilder()
-        .setFirst(ByteString.copyFrom(entry.getKey()))
-        .setSecond(ByteString.copyFrom(entry.getValue()))
-          .build());
-    }
-
-    for (Map.Entry<String, String> entry : peerConfig.getConfiguration().entrySet()) {
-      builder.addConfiguration(NameStringPair.newBuilder()
-        .setName(entry.getKey())
-        .setValue(entry.getValue())
-        .build());
-    }
-
-    return builder.build();
-  }
-
-  /**
-   * @param peerConfig
-   * @return Serialized protobuf of <code>peerConfig</code> with pb magic prefix prepended suitable
-   *         for use as content of a this.peersZNode; i.e. the content of PEER_ID znode under
-   *         /hbase/replication/peers/PEER_ID
-   */
-  private static byte[] toByteArray(final ReplicationPeerConfig peerConfig) {
-    byte[] bytes = convert(peerConfig).toByteArray();
-    return ProtobufUtil.prependPBMagic(bytes);
-  }
-
   private void checkQueuesDeleted(String peerId) throws ReplicationException {
     if (queuesClient == null) return;
     try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/7f39baf0/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
index a1dc1c8..79853a8 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.replication;
 
 import java.util.List;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -52,6 +53,9 @@ public abstract class ReplicationStateZKBase {
   protected final String hfileRefsZNode;
   /** The cluster key of the local cluster */
   protected final String ourClusterKey;
+  /** The name of the znode that contains tableCFs */
+  protected final String tableCFsNodeName;
+
   protected final ZooKeeperWatcher zookeeper;
   protected final Configuration conf;
   protected final Abortable abortable;
@@ -77,6 +81,7 @@ public abstract class ReplicationStateZKBase {
     String hfileRefsZNodeName = conf.get(ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_KEY,
       ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_DEFAULT);
     this.peerStateNodeName = conf.get("zookeeper.znode.replication.peers.state", "peer-state");
+    this.tableCFsNodeName = conf.get("zookeeper.znode.replication.peers.tableCFs", "tableCFs");
     this.ourClusterKey = ZKConfig.getZooKeeperClusterKey(this.conf);
     this.replicationZNode = ZKUtil.joinZNode(this.zookeeper.baseZNode, replicationZNodeName);
     this.peersZNode = ZKUtil.joinZNode(replicationZNode, peersZNodeName);
@@ -118,4 +123,18 @@ public abstract class ReplicationStateZKBase {
   protected boolean isPeerPath(String path) {
     return path.split("/").length == peersZNode.split("/").length + 1;
   }
+
+  @VisibleForTesting
+  protected String getTableCFsNode(String id) {
+    return ZKUtil.joinZNode(this.peersZNode, ZKUtil.joinZNode(id, this.tableCFsNodeName));
+  }
+
+  @VisibleForTesting
+  protected String getPeerStateNode(String id) {
+    return ZKUtil.joinZNode(this.peersZNode, ZKUtil.joinZNode(id, this.peerStateNodeName));
+  }
+  @VisibleForTesting
+  protected String getPeerNode(String id) {
+    return ZKUtil.joinZNode(this.peersZNode, id);
+  }
 }


[30/50] [abbrv] hbase git commit: HBASE-15515 Improve LocalityBasedCandidateGenerator in Balancer (Guanghao Zhang)

Posted by sy...@apache.org.
HBASE-15515 Improve LocalityBasedCandidateGenerator in Balancer (Guanghao Zhang)


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

Branch: refs/heads/hbase-12439
Commit: ad3feaa44800f10d102255a240c38ccf23a82d49
Parents: 7c93098
Author: tedyu <yu...@gmail.com>
Authored: Thu Mar 24 11:10:58 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Thu Mar 24 11:10:58 2016 -0700

----------------------------------------------------------------------
 .../hbase/master/balancer/BaseLoadBalancer.java | 30 ++++++++++++++++----
 .../master/balancer/RegionLocationFinder.java   | 16 +++++++++++
 .../master/balancer/StochasticLoadBalancer.java |  6 ++--
 3 files changed, 42 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ad3feaa4/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 bde5c61..6a28006 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
@@ -828,7 +828,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
     int getLowestLocalityRegionOnServer(int serverIndex) {
       if (regionFinder != null) {
         float lowestLocality = 1.0f;
-        int lowestLocalityRegionIndex = 0;
+        int lowestLocalityRegionIndex = -1;
         if (regionsPerServer[serverIndex].length == 0) {
           // No regions on that region server
           return -1;
@@ -838,15 +838,24 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
           HDFSBlocksDistribution distribution = regionFinder
               .getBlockDistribution(regions[regionIndex]);
           float locality = distribution.getBlockLocalityIndex(servers[serverIndex].getHostname());
+          // skip empty region
+          if (distribution.getUniqueBlocksTotalWeight() == 0) {
+            continue;
+          }
           if (locality < lowestLocality) {
             lowestLocality = locality;
             lowestLocalityRegionIndex = j;
           }
         }
+        if (lowestLocalityRegionIndex == -1) {
+          return -1;
+        }
         if (LOG.isTraceEnabled()) {
-          LOG.trace(" Lowest locality region index is " + lowestLocalityRegionIndex
-            + " and its region server contains " + regionsPerServer[serverIndex].length
-            + " regions");
+          LOG.trace("Lowest locality region is "
+              + regions[regionsPerServer[serverIndex][lowestLocalityRegionIndex]]
+                  .getRegionNameAsString() + " with locality " + lowestLocality
+              + " and its region server contains " + regionsPerServer[serverIndex].length
+              + " regions");
         }
         return regionsPerServer[serverIndex][lowestLocalityRegionIndex];
       } else {
@@ -863,9 +872,14 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
       }
     }
 
-    int getLeastLoadedTopServerForRegion(int region) {
+    /**
+     * Returns a least loaded server which has better locality for this region
+     * than the current server.
+     */
+    int getLeastLoadedTopServerForRegion(int region, int currentServer) {
       if (regionFinder != null) {
-        List<ServerName> topLocalServers = regionFinder.getTopBlockLocations(regions[region]);
+        List<ServerName> topLocalServers = regionFinder.getTopBlockLocations(regions[region],
+          servers[currentServer].getHostname());
         int leastLoadedServerIndex = -1;
         int load = Integer.MAX_VALUE;
         for (ServerName sn : topLocalServers) {
@@ -882,6 +896,10 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
             load = tempLoad;
           }
         }
+        if (leastLoadedServerIndex != -1) {
+          LOG.debug("Pick the least loaded server " + servers[leastLoadedServerIndex].getHostname()
+            + " with better locality for region " + regions[region]);
+        }
         return leastLoadedServerIndex;
       } else {
         return -1;

http://git-wip-us.apache.org/repos/asf/hbase/blob/ad3feaa4/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionLocationFinder.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionLocationFinder.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionLocationFinder.java
index 9657e53..a6724ee 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionLocationFinder.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionLocationFinder.java
@@ -171,6 +171,22 @@ class RegionLocationFinder {
   }
 
   /**
+   * Returns an ordered list of hosts which have better locality for this region
+   * than the current host.
+   */
+  protected List<ServerName> getTopBlockLocations(HRegionInfo region, String currentHost) {
+    HDFSBlocksDistribution blocksDistribution = getBlockDistribution(region);
+    List<String> topHosts = new ArrayList<String>();
+    for (String host : blocksDistribution.getTopHosts()) {
+      if (host.equals(currentHost)) {
+        break;
+      }
+      topHosts.add(host);
+    }
+    return mapHostNameToServerName(topHosts);
+  }
+
+  /**
    * Returns an ordered list of hosts that are hosting the blocks for this
    * region. The weight of each host is the sum of the block lengths of all
    * files on that host, so the first host in the list is the server which holds

http://git-wip-us.apache.org/repos/asf/hbase/blob/ad3feaa4/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
index add3242..d01f510 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
@@ -705,9 +705,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
         return pickRandomRegions(cluster, thisServer, otherServer);
       }
 
-      cluster.calculateRegionServerLocalities();
-      // Pick server with lowest locality
-      int thisServer = pickLowestLocalityServer(cluster);
+      int thisServer = pickRandomServer(cluster);
       int thisRegion;
       if (thisServer == -1) {
         LOG.warn("Could not pick lowest locality region server");
@@ -722,7 +720,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
       }
 
       // Pick the least loaded server with good locality for the region
-      int otherServer = cluster.getLeastLoadedTopServerForRegion(thisRegion);
+      int otherServer = cluster.getLeastLoadedTopServerForRegion(thisRegion, thisServer);
 
       if (otherServer == -1) {
         return Cluster.NullAction;


[27/50] [abbrv] hbase git commit: HBASE-15486 Avoid multiple disable/enable balancer calls while running rolling-restart.sh --graceful (Samir Ahmic)

Posted by sy...@apache.org.
HBASE-15486 Avoid multiple disable/enable balancer calls while running rolling-restart.sh --graceful (Samir Ahmic)


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

Branch: refs/heads/hbase-12439
Commit: 52fd70500e0a00e273e2ec0c09d7c914b89432ce
Parents: db3ba65
Author: tedyu <yu...@gmail.com>
Authored: Thu Mar 24 06:55:45 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Thu Mar 24 06:55:45 2016 -0700

----------------------------------------------------------------------
 bin/graceful_stop.sh   | 21 ++++++++++++++++-----
 bin/rolling-restart.sh | 10 +++++++++-
 2 files changed, 25 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/52fd7050/bin/graceful_stop.sh
----------------------------------------------------------------------
diff --git a/bin/graceful_stop.sh b/bin/graceful_stop.sh
index f1f04d6..510411e 100755
--- a/bin/graceful_stop.sh
+++ b/bin/graceful_stop.sh
@@ -22,7 +22,7 @@
 # Turn off the balancer before running this script.
 function usage {
   echo "Usage: graceful_stop.sh [--config <conf-dir>] [-e] [--restart [--reload]] [--thrift] \
-[--rest] <hostname>"
+[--rest]  [-nob |--nobalancer ] <hostname>"
   echo " thrift         If we should stop/start thrift before/after the hbase stop/start"
   echo " rest           If we should stop/start rest before/after the hbase stop/start"
   echo " restart        If we should restart after graceful stop"
@@ -34,6 +34,8 @@ moving regions"
 exit with error. Default value is INT_MAX."
   echo " hostname       Hostname of server we are to stop"
   echo " e|failfast     Set -e so exit immediately if any command exits with non-zero status"
+  echo " nob| nobalancer Do not manage balancer states. This is only used as optimization in \
+rolling_restart.sh to avoid multiple calls to hbase shell"
   exit 1
 }
 
@@ -54,6 +56,7 @@ rest=
 movetimeout=2147483647
 maxthreads=1
 failfast=
+nob=false
 while [ $# -gt 0 ]
 do
   case "$1" in
@@ -65,6 +68,7 @@ do
     --noack | -n)  noack="--noack"; shift;;
     --maxthreads) shift; maxthreads=$1; shift;;
     --movetimeout) shift; movetimeout=$1; shift;;
+    --nobalancer | -nob) nob=true; shift;;
     --) shift; break;;
     -*) usage ;;
     *)  break;;	# terminate while loop
@@ -97,9 +101,14 @@ if [ "$localhostname" == "$hostname" ]; then
   local=true
 fi
 
-log "Disabling load balancer"
-HBASE_BALANCER_STATE=`echo 'balance_switch false' | "$bin"/hbase --config ${HBASE_CONF_DIR} shell | tail -3 | head -1`
-log "Previous balancer state was $HBASE_BALANCER_STATE"
+if [ $nob == "true"  ]; then
+  log "[ $0 ] skipping disabling balancer -nob argument is used"
+  HBASE_BALANCER_STATE=false
+else
+  log "Disabling load balancer"
+  HBASE_BALANCER_STATE=$(echo 'balance_switch false' | "$bin"/hbase --config "${HBASE_CONF_DIR}" shell | tail -3 | head -1)
+  log "Previous balancer state was $HBASE_BALANCER_STATE"
+fi
 
 log "Unloading $hostname region(s)"
 HBASE_NOEXEC=true "$bin"/hbase --config ${HBASE_CONF_DIR} org.apache.hadoop.hbase.util.RegionMover \
@@ -166,9 +175,11 @@ if [ "$restart" != "" ]; then
 fi
 
 # Restore balancer state
-if [ $HBASE_BALANCER_STATE != "false" ]; then
+if [ $HBASE_BALANCER_STATE != "false" ] && [ $nob != "true"  ]; then
   log "Restoring balancer state to $HBASE_BALANCER_STATE"
   echo "balance_switch $HBASE_BALANCER_STATE" | "$bin"/hbase --config ${HBASE_CONF_DIR} shell &> /dev/null
+else
+  log "[ $0 ] skipping restoring balancer"
 fi
 
 # Cleanup tmp files.

http://git-wip-us.apache.org/repos/asf/hbase/blob/52fd7050/bin/rolling-restart.sh
----------------------------------------------------------------------
diff --git a/bin/rolling-restart.sh b/bin/rolling-restart.sh
index e6d25ed..530b76b 100755
--- a/bin/rolling-restart.sh
+++ b/bin/rolling-restart.sh
@@ -188,6 +188,10 @@ else
     if [ "$zkrs" == "null" ]; then zkrs="rs"; fi
     zkrs="$zparent/$zkrs"
     online_regionservers=`$bin/hbase zkcli ls $zkrs 2>&1 | tail -1 | sed "s/\[//" | sed "s/\]//"`
+    echo "Disabling load balancer"
+    HBASE_BALANCER_STATE=$(echo 'balance_switch false' | "$bin"/hbase --config "${HBASE_CONF_DIR}" shell | tail -3 | head -1)
+    echo "Previous balancer state was $HBASE_BALANCER_STATE"
+
     for rs in $online_regionservers
     do
         rs_parts=(${rs//,/ })
@@ -198,10 +202,14 @@ else
           continue
         else
           echo "Gracefully restarting: $hostname"
-          "$bin"/graceful_stop.sh --config ${HBASE_CONF_DIR} --restart --reload --maxthreads \
+          "$bin"/graceful_stop.sh --config ${HBASE_CONF_DIR} --restart --reload -nob --maxthreads  \
 		${RR_MAXTHREADS} ${RR_NOACK} --movetimeout ${RR_MOVE_TIMEOUT} $hostname
           sleep 1
         fi
     done
+    if [ "$HBASE_BALANCER_STATE" != "false" ]; then
+      echo "Restoring balancer state to $HBASE_BALANCER_STATE"
+      echo "balance_switch $HBASE_BALANCER_STATE" | "$bin"/hbase --config "${HBASE_CONF_DIR}" shell &> /dev/null
+    fi
   fi
 fi


[46/50] [abbrv] hbase git commit: HBASE-11393 Replication TableCfs should be a PB object rather than a string

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/7f39baf0/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java
index 29a052b..8b7c0a5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java
@@ -19,13 +19,9 @@
  */
 package org.apache.hadoop.hbase.replication;
 
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
@@ -47,7 +43,9 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
+import org.apache.hadoop.hbase.client.replication.ReplicationSerDeHelper;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
 import org.apache.hadoop.hbase.testclassification.FlakeyTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -58,6 +56,8 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import static org.junit.Assert.*;
+
 @Category({FlakeyTests.class, LargeTests.class})
 public class TestPerTableCFReplication {
 
@@ -184,13 +184,13 @@ public class TestPerTableCFReplication {
     Map<TableName, List<String>> tabCFsMap = null;
 
     // 1. null or empty string, result should be null
-    tabCFsMap = ReplicationAdmin.parseTableCFsFromConfig(null);
+    tabCFsMap = ReplicationSerDeHelper.parseTableCFsFromConfig(null);
     assertEquals(null, tabCFsMap);
 
-    tabCFsMap = ReplicationAdmin.parseTableCFsFromConfig("");
+    tabCFsMap = ReplicationSerDeHelper.parseTableCFsFromConfig("");
     assertEquals(null, tabCFsMap);
 
-    tabCFsMap = ReplicationAdmin.parseTableCFsFromConfig("   ");
+    tabCFsMap = ReplicationSerDeHelper.parseTableCFsFromConfig("   ");
     assertEquals(null, tabCFsMap);
 
     TableName tab1 = TableName.valueOf("tab1");
@@ -198,20 +198,20 @@ public class TestPerTableCFReplication {
     TableName tab3 = TableName.valueOf("tab3");
 
     // 2. single table: "tab1" / "tab2:cf1" / "tab3:cf1,cf3"
-    tabCFsMap = ReplicationAdmin.parseTableCFsFromConfig("tab1");
+    tabCFsMap = ReplicationSerDeHelper.parseTableCFsFromConfig("tab1");
     assertEquals(1, tabCFsMap.size()); // only one table
     assertTrue(tabCFsMap.containsKey(tab1));   // its table name is "tab1"
     assertFalse(tabCFsMap.containsKey(tab2));  // not other table
     assertEquals(null, tabCFsMap.get(tab1));   // null cf-list,
 
-    tabCFsMap = ReplicationAdmin.parseTableCFsFromConfig("tab2:cf1");
+    tabCFsMap = ReplicationSerDeHelper.parseTableCFsFromConfig("tab2:cf1");
     assertEquals(1, tabCFsMap.size()); // only one table
     assertTrue(tabCFsMap.containsKey(tab2));   // its table name is "tab2"
     assertFalse(tabCFsMap.containsKey(tab1));  // not other table
     assertEquals(1, tabCFsMap.get(tab2).size());   // cf-list contains only 1 cf
     assertEquals("cf1", tabCFsMap.get(tab2).get(0));// the only cf is "cf1"
 
-    tabCFsMap = ReplicationAdmin.parseTableCFsFromConfig("tab3 : cf1 , cf3");
+    tabCFsMap = ReplicationSerDeHelper.parseTableCFsFromConfig("tab3 : cf1 , cf3");
     assertEquals(1, tabCFsMap.size()); // only one table
     assertTrue(tabCFsMap.containsKey(tab3));   // its table name is "tab2"
     assertFalse(tabCFsMap.containsKey(tab1));  // not other table
@@ -220,7 +220,7 @@ public class TestPerTableCFReplication {
     assertTrue(tabCFsMap.get(tab3).contains("cf3"));// contains "cf3"
 
     // 3. multiple tables: "tab1 ; tab2:cf1 ; tab3:cf1,cf3"
-    tabCFsMap = ReplicationAdmin.parseTableCFsFromConfig("tab1 ; tab2:cf1 ; tab3:cf1,cf3");
+    tabCFsMap = ReplicationSerDeHelper.parseTableCFsFromConfig("tab1 ; tab2:cf1 ; tab3:cf1,cf3");
     // 3.1 contains 3 tables : "tab1", "tab2" and "tab3"
     assertEquals(3, tabCFsMap.size());
     assertTrue(tabCFsMap.containsKey(tab1));
@@ -238,7 +238,7 @@ public class TestPerTableCFReplication {
 
     // 4. contiguous or additional ";"(table delimiter) or ","(cf delimiter) can be tolerated
     // still use the example of multiple tables: "tab1 ; tab2:cf1 ; tab3:cf1,cf3"
-    tabCFsMap = ReplicationAdmin.parseTableCFsFromConfig(
+    tabCFsMap = ReplicationSerDeHelper.parseTableCFsFromConfig(
       "tab1 ; ; tab2:cf1 ; tab3:cf1,,cf3 ;");
     // 4.1 contains 3 tables : "tab1", "tab2" and "tab3"
     assertEquals(3, tabCFsMap.size());
@@ -257,7 +257,7 @@ public class TestPerTableCFReplication {
 
     // 5. invalid format "tab1:tt:cf1 ; tab2::cf1 ; tab3:cf1,cf3"
     //    "tab1:tt:cf1" and "tab2::cf1" are invalid and will be ignored totally
-    tabCFsMap = ReplicationAdmin.parseTableCFsFromConfig(
+    tabCFsMap = ReplicationSerDeHelper.parseTableCFsFromConfig(
       "tab1:tt:cf1 ; tab2::cf1 ; tab3:cf1,cf3");
     // 5.1 no "tab1" and "tab2", only "tab3"
     assertEquals(1, tabCFsMap.size()); // only one table
@@ -270,6 +270,99 @@ public class TestPerTableCFReplication {
     assertTrue(tabCFsMap.get(tab3).contains("cf3"));
  }
 
+  @Test
+  public void testTableCFsHelperConverter() {
+
+    ZooKeeperProtos.TableCF[] tableCFs = null;
+    Map<TableName, List<String>> tabCFsMap = null;
+
+    // 1. null or empty string, result should be null
+    assertNull(ReplicationSerDeHelper.convert(tabCFsMap));
+
+    tabCFsMap = new HashMap<TableName, List<String>>();
+    tableCFs = ReplicationSerDeHelper.convert(tabCFsMap);
+    assertEquals(0, tableCFs.length);
+
+    TableName tab1 = TableName.valueOf("tab1");
+    TableName tab2 = TableName.valueOf("tab2");
+    TableName tab3 = TableName.valueOf("tab3");
+
+    // 2. single table: "tab1" / "tab2:cf1" / "tab3:cf1,cf3"
+    tabCFsMap.clear();
+    tabCFsMap.put(tab1, null);
+    tableCFs = ReplicationSerDeHelper.convert(tabCFsMap);
+    assertEquals(1, tableCFs.length); // only one table
+    assertEquals(tab1.toString(),
+        tableCFs[0].getTableName().getQualifier().toStringUtf8());
+    assertEquals(0, tableCFs[0].getFamiliesCount());
+
+    tabCFsMap.clear();
+    tabCFsMap.put(tab2, new ArrayList<String>());
+    tabCFsMap.get(tab2).add("cf1");
+    tableCFs = ReplicationSerDeHelper.convert(tabCFsMap);
+    assertEquals(1, tableCFs.length); // only one table
+    assertEquals(tab2.toString(),
+        tableCFs[0].getTableName().getQualifier().toStringUtf8());
+    assertEquals(1, tableCFs[0].getFamiliesCount());
+    assertEquals("cf1", tableCFs[0].getFamilies(0).toStringUtf8());
+
+    tabCFsMap.clear();
+    tabCFsMap.put(tab3, new ArrayList<String>());
+    tabCFsMap.get(tab3).add("cf1");
+    tabCFsMap.get(tab3).add("cf3");
+    tableCFs = ReplicationSerDeHelper.convert(tabCFsMap);
+    assertEquals(1, tableCFs.length);
+    assertEquals(tab3.toString(),
+        tableCFs[0].getTableName().getQualifier().toStringUtf8());
+    assertEquals(2, tableCFs[0].getFamiliesCount());
+    assertEquals("cf1", tableCFs[0].getFamilies(0).toStringUtf8());
+    assertEquals("cf3", tableCFs[0].getFamilies(1).toStringUtf8());
+
+    tabCFsMap.clear();
+    tabCFsMap.put(tab1, null);
+    tabCFsMap.put(tab2, new ArrayList<String>());
+    tabCFsMap.get(tab2).add("cf1");
+    tabCFsMap.put(tab3, new ArrayList<String>());
+    tabCFsMap.get(tab3).add("cf1");
+    tabCFsMap.get(tab3).add("cf3");
+
+    tableCFs = ReplicationSerDeHelper.convert(tabCFsMap);
+    assertEquals(3, tableCFs.length);
+    assertNotNull(ReplicationSerDeHelper.getTableCF(tableCFs, tab1.toString()));
+    assertNotNull(ReplicationSerDeHelper.getTableCF(tableCFs, tab2.toString()));
+    assertNotNull(ReplicationSerDeHelper.getTableCF(tableCFs, tab3.toString()));
+
+    assertEquals(0,
+        ReplicationSerDeHelper.getTableCF(tableCFs, tab1.toString()).getFamiliesCount());
+
+    assertEquals(1,
+        ReplicationSerDeHelper.getTableCF(tableCFs, tab2.toString()).getFamiliesCount());
+    assertEquals("cf1",
+        ReplicationSerDeHelper.getTableCF(tableCFs, tab2.toString()).getFamilies(0).toStringUtf8());
+
+    assertEquals(2,
+        ReplicationSerDeHelper.getTableCF(tableCFs, tab3.toString()).getFamiliesCount());
+    assertEquals("cf1",
+        ReplicationSerDeHelper.getTableCF(tableCFs, tab3.toString()).getFamilies(0).toStringUtf8());
+    assertEquals("cf3",
+        ReplicationSerDeHelper.getTableCF(tableCFs, tab3.toString()).getFamilies(1).toStringUtf8());
+
+    tabCFsMap = ReplicationSerDeHelper.convert2Map(tableCFs);
+    assertEquals(3, tabCFsMap.size());
+    assertTrue(tabCFsMap.containsKey(tab1));
+    assertTrue(tabCFsMap.containsKey(tab2));
+    assertTrue(tabCFsMap.containsKey(tab3));
+    // 3.2 table "tab1" : null cf-list
+    assertEquals(null, tabCFsMap.get(tab1));
+    // 3.3 table "tab2" : cf-list contains a single cf "cf1"
+    assertEquals(1, tabCFsMap.get(tab2).size());
+    assertEquals("cf1", tabCFsMap.get(tab2).get(0));
+    // 3.4 table "tab3" : cf-list contains "cf1" and "cf3"
+    assertEquals(2, tabCFsMap.get(tab3).size());
+    assertTrue(tabCFsMap.get(tab3).contains("cf1"));
+    assertTrue(tabCFsMap.get(tab3).contains("cf3"));
+  }
+
   @Test(timeout=300000)
   public void testPerTableCFReplication() throws Exception {
     LOG.info("testPerTableCFReplication");
@@ -305,8 +398,23 @@ public class TestPerTableCFReplication {
       Table htab3C = connection3.getTable(tabCName);
 
       // A. add cluster2/cluster3 as peers to cluster1
-      replicationAdmin.addPeer("2", utility2.getClusterKey(), "TC;TB:f1,f3");
-      replicationAdmin.addPeer("3", utility3.getClusterKey(), "TA;TB:f1,f2");
+      ReplicationPeerConfig rpc2 = new ReplicationPeerConfig();
+      rpc2.setClusterKey(utility2.getClusterKey());
+      Map<TableName, List<String>> tableCFs = new HashMap<>();
+      tableCFs.put(tabCName, null);
+      tableCFs.put(tabBName, new ArrayList<String>());
+      tableCFs.get(tabBName).add("f1");
+      tableCFs.get(tabBName).add("f3");
+      replicationAdmin.addPeer("2", rpc2, tableCFs);
+
+      ReplicationPeerConfig rpc3 = new ReplicationPeerConfig();
+      rpc3.setClusterKey(utility3.getClusterKey());
+      tableCFs.clear();
+      tableCFs.put(tabAName, null);
+      tableCFs.put(tabBName, new ArrayList<String>());
+      tableCFs.get(tabBName).add("f1");
+      tableCFs.get(tabBName).add("f2");
+      replicationAdmin.addPeer("3", rpc3, tableCFs);
 
       // A1. tableA can only replicated to cluster3
       putAndWaitWithFamily(row1, f1Name, htab1A, htab3A);
@@ -349,8 +457,20 @@ public class TestPerTableCFReplication {
       deleteAndWaitWithFamily(row1, f3Name, htab1C, htab2C);
 
       // B. change peers' replicable table-cf config
-      replicationAdmin.setPeerTableCFs("2", "TA:f1,f2; TC:f2,f3");
-      replicationAdmin.setPeerTableCFs("3", "TB; TC:f3");
+      tableCFs.clear();
+      tableCFs.put(tabAName, new ArrayList<String>());
+      tableCFs.get(tabAName).add("f1");
+      tableCFs.get(tabAName).add("f2");
+      tableCFs.put(tabCName, new ArrayList<String>());
+      tableCFs.get(tabCName).add("f2");
+      tableCFs.get(tabCName).add("f3");
+      replicationAdmin.setPeerTableCFs("2", tableCFs);
+
+      tableCFs.clear();
+      tableCFs.put(tabBName, null);
+      tableCFs.put(tabCName, new ArrayList<String>());
+      tableCFs.get(tabCName).add("f3");
+      replicationAdmin.setPeerTableCFs("3", tableCFs);
 
       // B1. cf 'f1' of tableA can only replicated to cluster2
       putAndWaitWithFamily(row2, f1Name, htab1A, htab2A);

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

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/7f39baf0/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
index 41c3240..a768e93 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
@@ -172,7 +172,7 @@ public abstract class TestReplicationStateBasic {
     files1.add("file_3");
     assertNull(rqc.getReplicableHFiles(ID_ONE));
     assertEquals(0, rqc.getAllPeersFromHFileRefsQueue().size());
-    rp.addPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE), null);
+    rp.addPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE));
     rq1.addHFileRefs(ID_ONE, files1);
     assertEquals(1, rqc.getAllPeersFromHFileRefsQueue().size());
     assertEquals(3, rqc.getReplicableHFiles(ID_ONE).size());
@@ -193,8 +193,8 @@ public abstract class TestReplicationStateBasic {
     rqc.init();
 
     rp.init();
-    rp.addPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE), null);
-    rp.addPeer(ID_TWO, new ReplicationPeerConfig().setClusterKey(KEY_TWO), null);
+    rp.addPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE));
+    rp.addPeer(ID_TWO, new ReplicationPeerConfig().setClusterKey(KEY_TWO));
 
     List<String> files1 = new ArrayList<String>(3);
     files1.add("file_1");
@@ -248,9 +248,9 @@ public abstract class TestReplicationStateBasic {
     assertNumberOfPeers(0);
 
     // Add some peers
-    rp.addPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE), null);
+    rp.addPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE));
     assertNumberOfPeers(1);
-    rp.addPeer(ID_TWO, new ReplicationPeerConfig().setClusterKey(KEY_TWO), null);
+    rp.addPeer(ID_TWO, new ReplicationPeerConfig().setClusterKey(KEY_TWO));
     assertNumberOfPeers(2);
 
     // Test methods with a peer that is added but not connected
@@ -265,7 +265,7 @@ public abstract class TestReplicationStateBasic {
     assertNumberOfPeers(1);
 
     // Add one peer
-    rp.addPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE), null);
+    rp.addPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE));
     rp.peerAdded(ID_ONE);
     assertNumberOfPeers(2);
     assertTrue(rp.getStatusOfPeer(ID_ONE));
@@ -325,7 +325,7 @@ public abstract class TestReplicationStateBasic {
         rq3.addLog("qId" + i, "filename" + j);
       }
       //Add peers for the corresponding queues so they are not orphans
-      rp.addPeer("qId" + i, new ReplicationPeerConfig().setClusterKey("bogus" + i), null);
+      rp.addPeer("qId" + i, new ReplicationPeerConfig().setClusterKey("bogus" + i));
     }
   }
 }

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

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/7f39baf0/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java
index 8bfdc2a..b2a9611 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java
@@ -127,12 +127,14 @@ public class TestReplicationWithTags {
     conf2.setBoolean("hbase.tests.use.shortcircuit.reads", false);
     conf2.setStrings(HConstants.REPLICATION_CODEC_CONF_KEY, KeyValueCodecWithTags.class.getName());
     conf2.setStrings(CoprocessorHost.USER_REGION_COPROCESSOR_CONF_KEY,
-        TestCoprocessorForTagsAtSink.class.getName());
+            TestCoprocessorForTagsAtSink.class.getName());
 
     utility2 = new HBaseTestingUtility(conf2);
     utility2.setZkCluster(miniZK);
 
-    replicationAdmin.addPeer("2", utility2.getClusterKey());
+    ReplicationPeerConfig rpc = new ReplicationPeerConfig();
+    rpc.setClusterKey(utility2.getClusterKey());
+    replicationAdmin.addPeer("2", rpc, null);
 
     LOG.info("Setup second Zk");
     utility1.startMiniCluster(2);

http://git-wip-us.apache.org/repos/asf/hbase/blob/7f39baf0/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestTableCFsUpdater.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestTableCFsUpdater.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestTableCFsUpdater.java
new file mode 100644
index 0000000..0a10ab0
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestTableCFsUpdater.java
@@ -0,0 +1,210 @@
+/**
+ * Copyright The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.replication.master;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.replication.ReplicationSerDeHelper;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.zookeeper.KeeperException;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+@Category({ReplicationTests.class, SmallTests.class})
+public class TestTableCFsUpdater extends TableCFsUpdater {
+
+  private static final Log LOG = LogFactory.getLog(TestTableCFsUpdater.class);
+  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+  private static ZooKeeperWatcher zkw = null;
+  private static Abortable abortable = null;
+
+  public TestTableCFsUpdater() {
+    super(zkw, TEST_UTIL.getConfiguration(), abortable);
+  }
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    TEST_UTIL.startMiniZKCluster();
+    Configuration conf = TEST_UTIL.getConfiguration();
+    abortable = new Abortable() {
+      @Override
+      public void abort(String why, Throwable e) {
+        LOG.info(why, e);
+      }
+
+      @Override
+      public boolean isAborted() {
+        return false;
+      }
+    };
+    zkw = new ZooKeeperWatcher(conf, "TableCFs", abortable, true);
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    TEST_UTIL.shutdownMiniZKCluster();
+  }
+
+  @Test
+  public void testUpgrade() throws KeeperException, InterruptedException,
+      DeserializationException {
+    String peerId = "1";
+    TableName tab1 = TableName.valueOf("table1");
+    TableName tab2 = TableName.valueOf("table2");
+    TableName tab3 = TableName.valueOf("table3");
+
+    ReplicationPeerConfig rpc = new ReplicationPeerConfig();
+    rpc.setClusterKey(zkw.getQuorum());
+    String peerNode = getPeerNode(peerId);
+    ZKUtil.createWithParents(zkw, peerNode, ReplicationSerDeHelper.toByteArray(rpc));
+
+    String tableCFs = "table1:cf1,cf2;table2:cf3;table3";
+    String tableCFsNode = getTableCFsNode(peerId);
+    LOG.info("create tableCFs :" + tableCFsNode + " for peerId=" + peerId);
+    ZKUtil.createWithParents(zkw, tableCFsNode , Bytes.toBytes(tableCFs));
+
+    ReplicationPeerConfig actualRpc = ReplicationSerDeHelper.parsePeerFrom(ZKUtil.getData(zkw, peerNode));
+    String actualTableCfs = Bytes.toString(ZKUtil.getData(zkw, tableCFsNode));
+
+    assertEquals(rpc.getClusterKey(), actualRpc.getClusterKey());
+    assertNull(actualRpc.getTableCFsMap());
+    assertEquals(tableCFs, actualTableCfs);
+
+    peerId = "2";
+    rpc = new ReplicationPeerConfig();
+    rpc.setClusterKey(zkw.getQuorum());
+    peerNode = getPeerNode(peerId);
+    ZKUtil.createWithParents(zkw, peerNode, ReplicationSerDeHelper.toByteArray(rpc));
+
+    tableCFs = "table1:cf1,cf3;table2:cf2";
+    tableCFsNode = getTableCFsNode(peerId);
+    LOG.info("create tableCFs :" + tableCFsNode + " for peerId=" + peerId);
+    ZKUtil.createWithParents(zkw, tableCFsNode , Bytes.toBytes(tableCFs));
+
+    actualRpc = ReplicationSerDeHelper.parsePeerFrom(ZKUtil.getData(zkw, peerNode));
+    actualTableCfs = Bytes.toString(ZKUtil.getData(zkw, tableCFsNode));
+
+    assertEquals(rpc.getClusterKey(), actualRpc.getClusterKey());
+    assertNull(actualRpc.getTableCFsMap());
+    assertEquals(tableCFs, actualTableCfs);
+
+    peerId = "3";
+    rpc = new ReplicationPeerConfig();
+    rpc.setClusterKey(zkw.getQuorum());
+    peerNode = getPeerNode(peerId);
+    ZKUtil.createWithParents(zkw, peerNode, ReplicationSerDeHelper.toByteArray(rpc));
+
+    tableCFs = "";
+    tableCFsNode = getTableCFsNode(peerId);
+    LOG.info("create tableCFs :" + tableCFsNode + " for peerId=" + peerId);
+    ZKUtil.createWithParents(zkw, tableCFsNode , Bytes.toBytes(tableCFs));
+
+    actualRpc = ReplicationSerDeHelper.parsePeerFrom(ZKUtil.getData(zkw, peerNode));
+    actualTableCfs = Bytes.toString(ZKUtil.getData(zkw, tableCFsNode));
+
+    assertEquals(rpc.getClusterKey(), actualRpc.getClusterKey());
+    assertNull(actualRpc.getTableCFsMap());
+    assertEquals(tableCFs, actualTableCfs);
+
+    peerId = "4";
+    rpc = new ReplicationPeerConfig();
+    rpc.setClusterKey(zkw.getQuorum());
+    peerNode = getPeerNode(peerId);
+    ZKUtil.createWithParents(zkw, peerNode, ReplicationSerDeHelper.toByteArray(rpc));
+
+    tableCFsNode = getTableCFsNode(peerId);
+    actualRpc = ReplicationSerDeHelper.parsePeerFrom(ZKUtil.getData(zkw, peerNode));
+    actualTableCfs = Bytes.toString(ZKUtil.getData(zkw, tableCFsNode));
+
+    assertEquals(rpc.getClusterKey(), actualRpc.getClusterKey());
+    assertNull(actualRpc.getTableCFsMap());
+    assertNull(actualTableCfs);
+
+    update();
+
+    peerId = "1";
+    peerNode = getPeerNode(peerId);
+    actualRpc = ReplicationSerDeHelper.parsePeerFrom(ZKUtil.getData(zkw, peerNode));
+    assertEquals(rpc.getClusterKey(), actualRpc.getClusterKey());
+    Map<TableName, List<String>> tableNameListMap = actualRpc.getTableCFsMap();
+    assertEquals(3, tableNameListMap.size());
+    assertTrue(tableNameListMap.containsKey(tab1));
+    assertTrue(tableNameListMap.containsKey(tab2));
+    assertTrue(tableNameListMap.containsKey(tab3));
+    assertEquals(2, tableNameListMap.get(tab1).size());
+    assertEquals("cf1", tableNameListMap.get(tab1).get(0));
+    assertEquals("cf2", tableNameListMap.get(tab1).get(1));
+    assertEquals(1, tableNameListMap.get(tab2).size());
+    assertEquals("cf3", tableNameListMap.get(tab2).get(0));
+    assertNull(tableNameListMap.get(tab3));
+
+
+    peerId = "2";
+    peerNode = getPeerNode(peerId);
+    actualRpc = ReplicationSerDeHelper.parsePeerFrom(ZKUtil.getData(zkw, peerNode));
+    assertEquals(rpc.getClusterKey(), actualRpc.getClusterKey());
+    tableNameListMap = actualRpc.getTableCFsMap();
+    assertEquals(2, tableNameListMap.size());
+    assertTrue(tableNameListMap.containsKey(tab1));
+    assertTrue(tableNameListMap.containsKey(tab2));
+    assertEquals(2, tableNameListMap.get(tab1).size());
+    assertEquals("cf1", tableNameListMap.get(tab1).get(0));
+    assertEquals("cf3", tableNameListMap.get(tab1).get(1));
+    assertEquals(1, tableNameListMap.get(tab2).size());
+    assertEquals("cf2", tableNameListMap.get(tab2).get(0));
+
+    peerId = "3";
+    peerNode = getPeerNode(peerId);
+    actualRpc = ReplicationSerDeHelper.parsePeerFrom(ZKUtil.getData(zkw, peerNode));
+    assertEquals(rpc.getClusterKey(), actualRpc.getClusterKey());
+    tableNameListMap = actualRpc.getTableCFsMap();
+    assertNull(tableNameListMap);
+
+    peerId = "4";
+    peerNode = getPeerNode(peerId);
+    actualRpc = ReplicationSerDeHelper.parsePeerFrom(ZKUtil.getData(zkw, peerNode));
+    assertEquals(rpc.getClusterKey(), actualRpc.getClusterKey());
+    tableNameListMap = actualRpc.getTableCFsMap();
+    assertNull(tableNameListMap);
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/7f39baf0/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java
index 2140a5c..18a1088 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java
@@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
 import org.apache.hadoop.hbase.codec.KeyValueCodecWithTags;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.visibility.VisibilityController.VisibilityReplication;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -103,7 +104,7 @@ public class TestVisibilityLabelReplicationWithExpAsString extends TestVisibilit
     // Have to reset conf1 in case zk cluster location different
     // than default
     conf.setClass(VisibilityUtils.VISIBILITY_LABEL_GENERATOR_CLASS, SimpleScanLabelGenerator.class,
-        ScanLabelGenerator.class);
+            ScanLabelGenerator.class);
     conf.set("hbase.superuser", "admin");
     conf.set("hbase.superuser", User.getCurrent().getShortName());
     SUPERUSER = User.createUserForTesting(conf, User.getCurrent().getShortName(),
@@ -127,12 +128,14 @@ public class TestVisibilityLabelReplicationWithExpAsString extends TestVisibilit
     conf.setBoolean(HConstants.REPLICATION_ENABLE_KEY, HConstants.REPLICATION_ENABLE_DEFAULT);
     conf1.setStrings(HConstants.REPLICATION_CODEC_CONF_KEY, KeyValueCodecWithTags.class.getName());
     conf1.setStrings(CoprocessorHost.USER_REGION_COPROCESSOR_CONF_KEY,
-        TestCoprocessorForTagsAtSink.class.getName());
+            TestCoprocessorForTagsAtSink.class.getName());
     setVisibilityLabelServiceImpl(conf1, ExpAsStringVisibilityLabelServiceImpl.class);
     TEST_UTIL1 = new HBaseTestingUtility(conf1);
     TEST_UTIL1.setZkCluster(miniZK);
     zkw2 = new ZooKeeperWatcher(conf1, "cluster2", null, true);
-    replicationAdmin.addPeer("2", TEST_UTIL1.getClusterKey());
+    ReplicationPeerConfig rpc = new ReplicationPeerConfig();
+    rpc.setClusterKey(TEST_UTIL1.getClusterKey());
+    replicationAdmin.addPeer("2", rpc, null);
 
     TEST_UTIL.startMiniCluster(1);
     // Wait for the labels table to become available

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/7f39baf0/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java
index b1864d2..b722feb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java
@@ -55,6 +55,7 @@ import org.apache.hadoop.hbase.regionserver.SplitTransactionImpl;
 import org.apache.hadoop.hbase.regionserver.SplitTransactionFactory;
 import org.apache.hadoop.hbase.regionserver.TestEndToEndSplitTransaction;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationQueues;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
@@ -1527,7 +1528,9 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
     Assert.assertEquals(0, replicationAdmin.getPeersCount());
     int zkPort = conf.getInt(HConstants.ZOOKEEPER_CLIENT_PORT,
       HConstants.DEFAULT_ZOOKEPER_CLIENT_PORT);
-    replicationAdmin.addPeer("1", "127.0.0.1:" + zkPort + ":/hbase");
+    ReplicationPeerConfig rpc = new ReplicationPeerConfig();
+    rpc.setClusterKey("127.0.0.1:" + zkPort + ":/hbase");
+    replicationAdmin.addPeer("1", rpc, null);
     replicationAdmin.getPeersCount();
     Assert.assertEquals(1, replicationAdmin.getPeersCount());
     

http://git-wip-us.apache.org/repos/asf/hbase/blob/7f39baf0/hbase-shell/src/main/ruby/hbase/replication_admin.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/replication_admin.rb b/hbase-shell/src/main/ruby/hbase/replication_admin.rb
index 1c64f09..a026d09 100644
--- a/hbase-shell/src/main/ruby/hbase/replication_admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/replication_admin.rb
@@ -40,11 +40,7 @@ module Hbase
     #----------------------------------------------------------------------------------------------
     # Add a new peer cluster to replicate to
     def add_peer(id, args = {}, peer_tableCFs = nil)
-      # make add_peer backwards compatible to take in string for clusterKey and peer_tableCFs
-      if args.is_a?(String)
-        cluster_key = args
-        @replication_admin.addPeer(id, cluster_key, peer_tableCFs)
-      elsif args.is_a?(Hash)
+      if args.is_a?(Hash)
         unless peer_tableCFs.nil?
           raise(ArgumentError, "peer_tableCFs should be specified as TABLE_CFS in args")
         end
@@ -90,7 +86,14 @@ module Hbase
           }
         end
 
-        @replication_admin.add_peer(id, replication_peer_config, table_cfs)
+        unless table_cfs.nil?
+          # convert table_cfs to TableName
+          map = java.util.HashMap.new
+          table_cfs.each{|key, val|
+            map.put(org.apache.hadoop.hbase.TableName.valueOf(key), val)
+          }
+        end
+        @replication_admin.add_peer(id, replication_peer_config, map)
       else
         raise(ArgumentError, "args must be either a String or Hash")
       end
@@ -114,7 +117,7 @@ module Hbase
     #----------------------------------------------------------------------------------------------
     # List all peer clusters
     def list_peers
-      @replication_admin.listPeers
+      @replication_admin.listPeerConfigs
     end
 
     #----------------------------------------------------------------------------------------------
@@ -144,19 +147,40 @@ module Hbase
     #----------------------------------------------------------------------------------------------
     # Set new tableCFs config for the specified peer
     def set_peer_tableCFs(id, tableCFs)
-      @replication_admin.setPeerTableCFs(id, tableCFs)
+      unless tableCFs.nil?
+        # convert tableCFs to TableName
+        map = java.util.HashMap.new
+        tableCFs.each{|key, val|
+          map.put(org.apache.hadoop.hbase.TableName.valueOf(key), val)
+        }
+      end
+      @replication_admin.setPeerTableCFs(id, map)
     end
 
     #----------------------------------------------------------------------------------------------
     # Append a tableCFs config for the specified peer
     def append_peer_tableCFs(id, tableCFs)
-      @replication_admin.appendPeerTableCFs(id, tableCFs)
+      unless tableCFs.nil?
+        # convert tableCFs to TableName
+        map = java.util.HashMap.new
+        tableCFs.each{|key, val|
+          map.put(org.apache.hadoop.hbase.TableName.valueOf(key), val)
+        }
+      end
+      @replication_admin.appendPeerTableCFs(id, map)
     end
 
     #----------------------------------------------------------------------------------------------
     # Remove some tableCFs from the tableCFs config of the specified peer
     def remove_peer_tableCFs(id, tableCFs)
-      @replication_admin.removePeerTableCFs(id, tableCFs)
+      unless tableCFs.nil?
+        # convert tableCFs to TableName
+        map = java.util.HashMap.new
+        tableCFs.each{|key, val|
+          map.put(org.apache.hadoop.hbase.TableName.valueOf(key), val)
+        }
+      end
+      @replication_admin.removePeerTableCFs(id, map)
     end
     #----------------------------------------------------------------------------------------------
     # Enables a table's replication switch

http://git-wip-us.apache.org/repos/asf/hbase/blob/7f39baf0/hbase-shell/src/main/ruby/shell/commands/add_peer.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/add_peer.rb b/hbase-shell/src/main/ruby/shell/commands/add_peer.rb
index be01041..c0d6c6d 100644
--- a/hbase-shell/src/main/ruby/shell/commands/add_peer.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/add_peer.rb
@@ -51,10 +51,10 @@ the key TABLE_CFS.
   hbase> add_peer '9', ENDPOINT_CLASSNAME => 'org.apache.hadoop.hbase.MyReplicationEndpoint',
     DATA => { "key1" => 1 }, CONFIG => { "config1" => "value1", "config2" => "value2" },
   hbase> add_peer '10', ENDPOINT_CLASSNAME => 'org.apache.hadoop.hbase.MyReplicationEndpoint',
-    TABLE_CFS => { "table1" => [], "table2" => ["cf1"], "table3" => ["cf1", "cf2"] }
+    TABLE_CFS => { "table1" => [], "ns2:table2" => ["cf1"], "ns3:table3" => ["cf1", "cf2"] }
   hbase> add_peer '11', ENDPOINT_CLASSNAME => 'org.apache.hadoop.hbase.MyReplicationEndpoint',
     DATA => { "key1" => 1 }, CONFIG => { "config1" => "value1", "config2" => "value2" },
-    TABLE_CFS => { "table1" => [], "table2" => ["cf1"], "table3" => ["cf1", "cf2"] }
+    TABLE_CFS => { "table1" => [], "ns2:table2" => ["cf1"], "ns3:table3" => ["cf1", "cf2"] }
 
 Note: Either CLUSTER_KEY or ENDPOINT_CLASSNAME must be specified but not both.
 EOF

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

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

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


[39/50] [abbrv] hbase git commit: HBASE-15265 Implement an asynchronous FSHLog

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/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 dfbdae5..67c2b93 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
@@ -17,62 +17,44 @@
  */
 package org.apache.hadoop.hbase.regionserver.wal;
 
-import static org.apache.hadoop.hbase.wal.DefaultWALProvider.WAL_FILE_NAME_DELIMITER;
-
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.io.InterruptedIOException;
 import java.io.OutputStream;
-import java.lang.management.ManagementFactory;
-import java.lang.management.MemoryUsage;
-import java.lang.reflect.InvocationTargetException;
-import java.net.URLEncoder;
-import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Comparator;
 import java.util.List;
-import java.util.Map;
-import java.util.NavigableMap;
-import java.util.Set;
 import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentSkipListMap;
-import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.locks.ReentrantLock;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.lmax.disruptor.BlockingWaitStrategy;
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.ExceptionHandler;
+import com.lmax.disruptor.LifecycleAware;
+import com.lmax.disruptor.TimeoutException;
+import com.lmax.disruptor.dsl.Disruptor;
+import com.lmax.disruptor.dsl.ProducerType;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.PathFilter;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.io.util.HeapMemorySizeUtil;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ClassSize;
-import org.apache.hadoop.hbase.util.DrainBarrier;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HasThread;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.wal.DefaultWALProvider;
-import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALFactory;
 import org.apache.hadoop.hbase.wal.WALKey;
 import org.apache.hadoop.hbase.wal.WALPrettyPrinter;
@@ -81,95 +63,59 @@ import org.apache.hadoop.hbase.wal.WALSplitter;
 import org.apache.hadoop.hdfs.DFSOutputStream;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.util.StringUtils;
 import org.apache.htrace.NullScope;
 import org.apache.htrace.Span;
 import org.apache.htrace.Trace;
 import org.apache.htrace.TraceScope;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.lmax.disruptor.BlockingWaitStrategy;
-import com.lmax.disruptor.EventHandler;
-import com.lmax.disruptor.ExceptionHandler;
-import com.lmax.disruptor.LifecycleAware;
-import com.lmax.disruptor.TimeoutException;
-import com.lmax.disruptor.dsl.Disruptor;
-import com.lmax.disruptor.dsl.ProducerType;
-
 /**
- * Implementation of {@link WAL} to go against {@link FileSystem}; i.e. keep WALs in HDFS.
- * Only one WAL is ever being written at a time.  When a WAL hits a configured maximum size,
- * it is rolled.  This is done internal to the implementation.
- *
- * <p>As data is flushed from the MemStore to other on-disk structures (files sorted by
- * key, hfiles), a WAL becomes obsolete. We can let go of all the log edits/entries for a given
- * HRegion-sequence id.  A bunch of work in the below is done keeping account of these region
- * sequence ids -- what is flushed out to hfiles, and what is yet in WAL and in memory only.
- *
- * <p>It is only practical to delete entire files. Thus, we delete an entire on-disk file
- * <code>F</code> when all of the edits in <code>F</code> have a log-sequence-id that's older
- * (smaller) than the most-recent flush.
- *
- * <p>To read an WAL, call {@link WALFactory#createReader(org.apache.hadoop.fs.FileSystem,
- * org.apache.hadoop.fs.Path)}.
- *
- * <h2>Failure Semantic</h2>
- * If an exception on append or sync, roll the WAL because the current WAL is now a lame duck;
- * any more appends or syncs will fail also with the same original exception. If we have made
- * successful appends to the WAL and we then are unable to sync them, our current semantic is to
- * return error to the client that the appends failed but also to abort the current context,
- * usually the hosting server. We need to replay the WALs. TODO: Change this semantic. A roll of
- * WAL may be sufficient as long as we have flagged client that the append failed. TODO:
- * replication may pick up these last edits though they have been marked as failed append (Need to
- * keep our own file lengths, not rely on HDFS).
+ * The default implementation of FSWAL.
  */
 @InterfaceAudience.Private
-public class FSHLog implements WAL {
+public class FSHLog extends AbstractFSWAL<Writer> {
   // IMPLEMENTATION NOTES:
   //
-  // At the core is a ring buffer.  Our ring buffer is the LMAX Disruptor.  It tries to
+  // At the core is a ring buffer. Our ring buffer is the LMAX Disruptor. It tries to
   // minimize synchronizations and volatile writes when multiple contending threads as is the case
-  // here appending and syncing on a single WAL.  The Disruptor is configured to handle multiple
+  // here appending and syncing on a single WAL. The Disruptor is configured to handle multiple
   // producers but it has one consumer only (the producers in HBase are IPC Handlers calling append
-  // and then sync).  The single consumer/writer pulls the appends and syncs off the ring buffer.
+  // and then sync). The single consumer/writer pulls the appends and syncs off the ring buffer.
   // When a handler calls sync, it is given back a future. The producer 'blocks' on the future so
-  // it does not return until the sync completes.  The future is passed over the ring buffer from
+  // it does not return until the sync completes. The future is passed over the ring buffer from
   // the producer/handler to the consumer thread where it does its best to batch up the producer
-  // syncs so one WAL sync actually spans multiple producer sync invocations.  How well the
+  // syncs so one WAL sync actually spans multiple producer sync invocations. How well the
   // batching works depends on the write rate; i.e. we tend to batch more in times of
   // high writes/syncs.
   //
   // Calls to append now also wait until the append has been done on the consumer side of the
-  // disruptor.  We used to not wait but it makes the implemenation easier to grok if we have
+  // disruptor. We used to not wait but it makes the implementation easier to grok if we have
   // the region edit/sequence id after the append returns.
   //
-  // TODO: Handlers need to coordinate appending AND syncing.  Can we have the threads contend
-  // once only?  Probably hard given syncs take way longer than an append.
+  // TODO: Handlers need to coordinate appending AND syncing. Can we have the threads contend
+  // once only? Probably hard given syncs take way longer than an append.
   //
   // The consumer threads pass the syncs off to multiple syncing threads in a round robin fashion
   // to ensure we keep up back-to-back FS sync calls (FS sync calls are the long poll writing the
-  // WAL).  The consumer thread passes the futures to the sync threads for it to complete
+  // WAL). The consumer thread passes the futures to the sync threads for it to complete
   // the futures when done.
   //
-  // The 'sequence' in the below is the sequence of the append/sync on the ringbuffer.  It
-  // acts as a sort-of transaction id.  It is always incrementing.
+  // The 'sequence' in the below is the sequence of the append/sync on the ringbuffer. It
+  // acts as a sort-of transaction id. It is always incrementing.
   //
-  // The RingBufferEventHandler class hosts the ring buffer consuming code.  The threads that
-  // do the actual FS sync are implementations of SyncRunner.  SafePointZigZagLatch is a
-  // synchronization class used to halt the consumer at a safe point --  just after all outstanding
+  // The RingBufferEventHandler class hosts the ring buffer consuming code. The threads that
+  // do the actual FS sync are implementations of SyncRunner. SafePointZigZagLatch is a
+  // synchronization class used to halt the consumer at a safe point -- just after all outstanding
   // syncs and appends have completed -- so the log roller can swap the WAL out under it.
-
+  //
+  // We use ring buffer sequence as txid of FSWALEntry and SyncFuture.
   private static final Log LOG = LogFactory.getLog(FSHLog.class);
 
-  private static final int DEFAULT_SLOW_SYNC_TIME_MS = 100; // in ms
-
   /**
-   * The nexus at which all incoming handlers meet.  Does appends and sync with an ordering.
-   * Appends and syncs are each put on the ring which means handlers need to
-   * smash up against the ring twice (can we make it once only? ... maybe not since time to append
-   * is so different from time to sync and sometimes we don't want to sync or we want to async
-   * the sync).  The ring is where we make sure of our ordering and it is also where we do
-   * batching up of handler sync calls.
+   * The nexus at which all incoming handlers meet. Does appends and sync with an ordering. Appends
+   * and syncs are each put on the ring which means handlers need to smash up against the ring twice
+   * (can we make it once only? ... maybe not since time to append is so different from time to sync
+   * and sometimes we don't want to sync or we want to async the sync). The ring is where we make
+   * sure of our ordering and it is also where we do batching up of handler sync calls.
    */
   private final Disruptor<RingBufferTruck> disruptor;
 
@@ -180,95 +126,13 @@ public class FSHLog implements WAL {
 
   /**
    * This fellow is run by the above appendExecutor service but it is all about batching up appends
-   * and syncs; it may shutdown without cleaning out the last few appends or syncs.  To guard
-   * against this, keep a reference to this handler and do explicit close on way out to make sure
-   * all flushed out before we exit.
+   * and syncs; it may shutdown without cleaning out the last few appends or syncs. To guard against
+   * this, keep a reference to this handler and do explicit close on way out to make sure all
+   * flushed out before we exit.
    */
   private final RingBufferEventHandler ringBufferEventHandler;
 
   /**
-   * Map of {@link SyncFuture}s keyed by Handler objects.  Used so we reuse SyncFutures.
-   * TODO: Reus FSWALEntry's rather than create them anew each time as we do SyncFutures here.
-   * TODO: Add a FSWalEntry and SyncFuture as thread locals on handlers rather than have them
-   * get them from this Map?
-   */
-  private final Map<Thread, SyncFuture> syncFuturesByHandler;
-
-  /**
-   * The highest known outstanding unsync'd WALEdit sequence number where sequence number is the
-   * ring buffer sequence.  Maintained by the ring buffer consumer.
-   */
-  private volatile long highestUnsyncedSequence = -1;
-
-  /**
-   * Updated to the ring buffer sequence of the last successful sync call.  This can be less than
-   * {@link #highestUnsyncedSequence} for case where we have an append where a sync has not yet
-   * come in for it.  Maintained by the syncing threads.
-   */
-  private final AtomicLong highestSyncedSequence = new AtomicLong(0);
-
-  /**
-   * file system instance
-   */
-  protected final FileSystem fs;
-
-  /**
-   * WAL directory, where all WAL files would be placed.
-   */
-  private final Path fullPathLogDir;
-
-  /**
-   * dir path where old logs are kept.
-   */
-  private final Path fullPathArchiveDir;
-
-  /**
-   * Matches just those wal files that belong to this wal instance.
-   */
-  private final PathFilter ourFiles;
-
-  /**
-   * Prefix of a WAL file, usually the region server name it is hosted on.
-   */
-  private final String logFilePrefix;
-
-  /**
-   * Suffix included on generated wal file names
-   */
-  private final String logFileSuffix;
-
-  /**
-   * Prefix used when checking for wal membership.
-   */
-  private final String prefixPathStr;
-
-  private final WALCoprocessorHost coprocessorHost;
-
-  /**
-   * conf object
-   */
-  protected final Configuration conf;
-
-  /** Listeners that are called on WAL events. */
-  private final List<WALActionsListener> listeners =
-    new CopyOnWriteArrayList<WALActionsListener>();
-
-  @Override
-  public void registerWALActionsListener(final WALActionsListener listener) {
-    this.listeners.add(listener);
-  }
-
-  @Override
-  public boolean unregisterWALActionsListener(final WALActionsListener listener) {
-    return this.listeners.remove(listener);
-  }
-
-  @Override
-  public WALCoprocessorHost getCoprocessorHost() {
-    return coprocessorHost;
-  }
-
-  /**
    * FSDataOutputStream associated with the current SequenceFile.writer
    */
   private FSDataOutputStream hdfs_out;
@@ -278,8 +142,6 @@ public class FSHLog implements WAL {
   // Minimum tolerable replicas, if the actual value is lower than it, rollWriter will be triggered
   private final int minTolerableReplication;
 
-  private final int slowSyncNs;
-
   // If live datanode count is lower than the default replicas value,
   // RollWriter will be triggered in each sync(So the RollWriter will be
   // triggered one by one in a short time). Using it as a workaround to slow
@@ -293,84 +155,14 @@ public class FSHLog implements WAL {
   // Enable it if the replications recover.
   private volatile boolean lowReplicationRollEnabled = true;
 
-  /**
-   * Class that does accounting of sequenceids in WAL subsystem. Holds oldest outstanding
-   * sequence id as yet not flushed as well as the most recent edit sequence id appended to the
-   * WAL. Has facility for answering questions such as "Is it safe to GC a WAL?".
-   */
-  private SequenceIdAccounting sequenceIdAccounting = new SequenceIdAccounting();
-
-  /**
-   * Current log file.
-   */
-  volatile Writer writer;
-
-  /** The barrier used to ensure that close() waits for all log rolls and flushes to finish. */
-  private final DrainBarrier closeBarrier = new DrainBarrier();
-
-  /**
-   * This lock makes sure only one log roll runs at a time. Should not be taken while any other
-   * lock is held. We don't just use synchronized because that results in bogus and tedious
-   * findbugs warning when it thinks synchronized controls writer thread safety.  It is held when
-   * we are actually rolling the log.  It is checked when we are looking to see if we should roll
-   * the log or not.
-   */
-  private final ReentrantLock rollWriterLock = new ReentrantLock(true);
-
-  private volatile boolean closed = false;
-  private final AtomicBoolean shutdown = new AtomicBoolean(false);
-
-  // The timestamp (in ms) when the log file was created.
-  private final AtomicLong filenum = new AtomicLong(-1);
-
-  // Number of transactions in the current Wal.
-  private final AtomicInteger numEntries = new AtomicInteger(0);
-
-  // If > than this size, roll the log.
-  private final long logrollsize;
-
-  /**
-   * The total size of wal
-   */
-  private AtomicLong totalLogSize = new AtomicLong(0);
-
-  /*
-   * If more than this many logs, force flush of oldest region to oldest edit
-   * goes to disk.  If too many and we crash, then will take forever replaying.
-   * Keep the number of logs tidy.
-   */
-  private final int maxLogs;
-
   /** Number of log close errors tolerated before we abort */
   private final int closeErrorsTolerated;
 
   private final AtomicInteger closeErrorCount = new AtomicInteger();
 
-
   /**
-   * WAL Comparator; it compares the timestamp (log filenum), present in the log file name.
-   * Throws an IllegalArgumentException if used to compare paths from different wals.
-   */
-  final Comparator<Path> LOG_NAME_COMPARATOR = new Comparator<Path>() {
-    @Override
-    public int compare(Path o1, Path o2) {
-      long t1 = getFileNumFromFileName(o1);
-      long t2 = getFileNumFromFileName(o2);
-      if (t1 == t2) return 0;
-      return (t1 > t2) ? 1 : -1;
-    }
-  };
-
-  /**
-   * Map of WAL log file to the latest sequence ids of all regions it has entries of.
-   * The map is sorted by the log file creation timestamp (contained in the log file name).
-   */
-  private NavigableMap<Path, Map<byte[], Long>> byWalRegionSequenceIds =
-    new ConcurrentSkipListMap<Path, Map<byte[], Long>>(LOG_NAME_COMPARATOR);
-
-  /**
-   * Exception handler to pass the disruptor ringbuffer.  Same as native implementation only it
-   * logs using our logger instead of java native logger.
+   * Exception handler to pass the disruptor ringbuffer. Same as native implementation only it logs
+   * using our logger instead of java native logger.
    */
   static class RingBufferExceptionHandler implements ExceptionHandler {
     @Override
@@ -394,12 +186,10 @@ public class FSHLog implements WAL {
 
   /**
    * Constructor.
-   *
    * @param fs filesystem handle
    * @param root path for stored and archived wals
    * @param logDir dir where wals are stored
    * @param conf configuration to use
-   * @throws IOException
    */
   public FSHLog(final FileSystem fs, final Path root, final String logDir, final Configuration conf)
       throws IOException {
@@ -407,252 +197,79 @@ public class FSHLog implements WAL {
   }
 
   /**
-   * Create an edit log at the given <code>dir</code> location.
-   *
-   * You should never have to load an existing log. If there is a log at
-   * startup, it should have already been processed and deleted by the time the
-   * WAL object is started up.
-   *
+   * Create an edit log at the given <code>dir</code> location. You should never have to load an
+   * existing log. If there is a log at startup, it should have already been processed and deleted
+   * by the time the WAL object is started up.
    * @param fs filesystem handle
    * @param rootDir path to where logs and oldlogs
    * @param logDir dir where wals are stored
    * @param archiveDir dir where wals are archived
    * @param conf configuration to use
-   * @param listeners Listeners on WAL events. Listeners passed here will
-   * be registered before we do anything else; e.g. the
-   * Constructor {@link #rollWriter()}.
-   * @param failIfWALExists If true IOException will be thrown if files related to this wal
-   *        already exist.
-   * @param prefix should always be hostname and port in distributed env and
-   *        it will be URL encoded before being used.
-   *        If prefix is null, "wal" will be used
+   * @param listeners Listeners on WAL events. Listeners passed here will be registered before we do
+   *          anything else; e.g. the Constructor {@link #rollWriter()}.
+   * @param failIfWALExists If true IOException will be thrown if files related to this wal already
+   *          exist.
+   * @param prefix should always be hostname and port in distributed env and it will be URL encoded
+   *          before being used. If prefix is null, "wal" will be used
    * @param suffix will be url encoded. null is treated as empty. non-empty must start with
-   *        {@link DefaultWALProvider#WAL_FILE_NAME_DELIMITER}
-   * @throws IOException
+   *          {@link DefaultWALProvider#WAL_FILE_NAME_DELIMITER}
    */
   public FSHLog(final FileSystem fs, final Path rootDir, final String logDir,
-      final String archiveDir, final Configuration conf,
-      final List<WALActionsListener> listeners,
-      final boolean failIfWALExists, final String prefix, final String suffix)
-      throws IOException {
-    this.fs = fs;
-    this.fullPathLogDir = new Path(rootDir, logDir);
-    this.fullPathArchiveDir = new Path(rootDir, archiveDir);
-    this.conf = conf;
-
-    if (!fs.exists(fullPathLogDir) && !fs.mkdirs(fullPathLogDir)) {
-      throw new IOException("Unable to mkdir " + fullPathLogDir);
-    }
-
-    if (!fs.exists(this.fullPathArchiveDir)) {
-      if (!fs.mkdirs(this.fullPathArchiveDir)) {
-        throw new IOException("Unable to mkdir " + this.fullPathArchiveDir);
-      }
-    }
-
-    // If prefix is null||empty then just name it wal
-    this.logFilePrefix =
-      prefix == null || prefix.isEmpty() ? "wal" : URLEncoder.encode(prefix, "UTF8");
-    // we only correctly differentiate suffices when numeric ones start with '.'
-    if (suffix != null && !(suffix.isEmpty()) && !(suffix.startsWith(WAL_FILE_NAME_DELIMITER))) {
-      throw new IllegalArgumentException("WAL suffix must start with '" + WAL_FILE_NAME_DELIMITER +
-          "' but instead was '" + suffix + "'");
-    }
-    // Now that it exists, set the storage policy for the entire directory of wal files related to
-    // this FSHLog instance
-    FSUtils.setStoragePolicy(fs, conf, this.fullPathLogDir, HConstants.WAL_STORAGE_POLICY,
-      HConstants.DEFAULT_WAL_STORAGE_POLICY);
-    this.logFileSuffix = (suffix == null) ? "" : URLEncoder.encode(suffix, "UTF8");
-    this.prefixPathStr = new Path(fullPathLogDir,
-        logFilePrefix + WAL_FILE_NAME_DELIMITER).toString();
-
-    this.ourFiles = new PathFilter() {
-      @Override
-      public boolean accept(final Path fileName) {
-        // The path should start with dir/<prefix> and end with our suffix
-        final String fileNameString = fileName.toString();
-        if (!fileNameString.startsWith(prefixPathStr)) {
-          return false;
-        }
-        if (logFileSuffix.isEmpty()) {
-          // in the case of the null suffix, we need to ensure the filename ends with a timestamp.
-          return org.apache.commons.lang.StringUtils.isNumeric(
-              fileNameString.substring(prefixPathStr.length()));
-        } else if (!fileNameString.endsWith(logFileSuffix)) {
-          return false;
-        }
-        return true;
-      }
-    };
-
-    if (failIfWALExists) {
-      final FileStatus[] walFiles = FSUtils.listStatus(fs, fullPathLogDir, ourFiles);
-      if (null != walFiles && 0 != walFiles.length) {
-        throw new IOException("Target WAL already exists within directory " + fullPathLogDir);
-      }
-    }
-
-    // Register listeners.  TODO: Should this exist anymore?  We have CPs?
-    if (listeners != null) {
-      for (WALActionsListener i: listeners) {
-        registerWALActionsListener(i);
-      }
-    }
-    this.coprocessorHost = new WALCoprocessorHost(this, conf);
-
-    // Get size to roll log at. Roll at 95% of HDFS block size so we avoid crossing HDFS blocks
-    // (it costs a little x'ing bocks)
-    final long blocksize = this.conf.getLong("hbase.regionserver.hlog.blocksize",
-        FSUtils.getDefaultBlockSize(this.fs, this.fullPathLogDir));
-    this.logrollsize =
-      (long)(blocksize * conf.getFloat("hbase.regionserver.logroll.multiplier", 0.95f));
-
-    float memstoreRatio = conf.getFloat(HeapMemorySizeUtil.MEMSTORE_SIZE_KEY,
-      conf.getFloat(HeapMemorySizeUtil.MEMSTORE_SIZE_OLD_KEY,
-        HeapMemorySizeUtil.DEFAULT_MEMSTORE_SIZE));
-    boolean maxLogsDefined = conf.get("hbase.regionserver.maxlogs") != null;
-    if(maxLogsDefined){
-      LOG.warn("'hbase.regionserver.maxlogs' was deprecated.");
-    }
-    this.maxLogs = conf.getInt("hbase.regionserver.maxlogs",
-        Math.max(32, calculateMaxLogFiles(memstoreRatio, logrollsize)));
+      final String archiveDir, final Configuration conf, final List<WALActionsListener> listeners,
+      final boolean failIfWALExists, final String prefix, final String suffix) throws IOException {
+    super(fs, rootDir, logDir, archiveDir, conf, listeners, failIfWALExists, prefix, suffix);
     this.minTolerableReplication = conf.getInt("hbase.regionserver.hlog.tolerable.lowreplication",
-        FSUtils.getDefaultReplication(fs, this.fullPathLogDir));
-    this.lowReplicationRollLimit =
-      conf.getInt("hbase.regionserver.hlog.lowreplication.rolllimit", 5);
+      FSUtils.getDefaultReplication(fs, this.walDir));
+    this.lowReplicationRollLimit = conf.getInt("hbase.regionserver.hlog.lowreplication.rolllimit",
+      5);
     this.closeErrorsTolerated = conf.getInt("hbase.regionserver.logroll.errors.tolerated", 0);
-    int maxHandlersCount = conf.getInt(HConstants.REGION_SERVER_HANDLER_COUNT, 200);
-
-    LOG.info("WAL configuration: blocksize=" + StringUtils.byteDesc(blocksize) +
-      ", rollsize=" + StringUtils.byteDesc(this.logrollsize) +
-      ", prefix=" + this.logFilePrefix + ", suffix=" + logFileSuffix + ", logDir=" +
-      this.fullPathLogDir + ", archiveDir=" + this.fullPathArchiveDir);
 
     // rollWriter sets this.hdfs_out if it can.
     rollWriter();
 
-    this.slowSyncNs =
-        1000000 * conf.getInt("hbase.regionserver.hlog.slowsync.ms",
-          DEFAULT_SLOW_SYNC_TIME_MS);
-
-    // This is the 'writer' -- a single threaded executor.  This single thread 'consumes' what is
+    // This is the 'writer' -- a single threaded executor. This single thread 'consumes' what is
     // put on the ring buffer.
     String hostingThreadName = Thread.currentThread().getName();
-    this.appendExecutor = Executors.
-      newSingleThreadExecutor(Threads.getNamedThreadFactory(hostingThreadName + ".append"));
-    // Preallocate objects to use on the ring buffer.  The way that appends and syncs work, we will
+    this.appendExecutor = Executors
+        .newSingleThreadExecutor(Threads.getNamedThreadFactory(hostingThreadName + ".append"));
+    // Preallocate objects to use on the ring buffer. The way that appends and syncs work, we will
     // be stuck and make no progress if the buffer is filled with appends only and there is no
     // sync. If no sync, then the handlers will be outstanding just waiting on sync completion
     // before they return.
-    final int preallocatedEventCount =
-      this.conf.getInt("hbase.regionserver.wal.disruptor.event.count", 1024 * 16);
-    // Using BlockingWaitStrategy.  Stuff that is going on here takes so long it makes no sense
+    final int preallocatedEventCount = this.conf
+        .getInt("hbase.regionserver.wal.disruptor.event.count", 1024 * 16);
+    // Using BlockingWaitStrategy. Stuff that is going on here takes so long it makes no sense
     // spinning as other strategies do.
-    this.disruptor =
-      new Disruptor<RingBufferTruck>(RingBufferTruck.EVENT_FACTORY, preallocatedEventCount,
-        this.appendExecutor, ProducerType.MULTI, new BlockingWaitStrategy());
+    this.disruptor = new Disruptor<RingBufferTruck>(RingBufferTruck.EVENT_FACTORY,
+        preallocatedEventCount, this.appendExecutor, ProducerType.MULTI,
+        new BlockingWaitStrategy());
     // Advance the ring buffer sequence so that it starts from 1 instead of 0,
     // because SyncFuture.NOT_DONE = 0.
     this.disruptor.getRingBuffer().next();
-    this.ringBufferEventHandler =
-      new RingBufferEventHandler(conf.getInt("hbase.regionserver.hlog.syncer.count", 5),
-        maxHandlersCount);
+    int maxHandlersCount = conf.getInt(HConstants.REGION_SERVER_HANDLER_COUNT, 200);
+    this.ringBufferEventHandler = new RingBufferEventHandler(
+        conf.getInt("hbase.regionserver.hlog.syncer.count", 5), maxHandlersCount);
     this.disruptor.handleExceptionsWith(new RingBufferExceptionHandler());
-    this.disruptor.handleEventsWith(new RingBufferEventHandler [] {this.ringBufferEventHandler});
-    // Presize our map of SyncFutures by handler objects.
-    this.syncFuturesByHandler = new ConcurrentHashMap<Thread, SyncFuture>(maxHandlersCount);
+    this.disruptor.handleEventsWith(new RingBufferEventHandler[] { this.ringBufferEventHandler });
     // Starting up threads in constructor is a no no; Interface should have an init call.
     this.disruptor.start();
   }
 
-  private int calculateMaxLogFiles(float memstoreSizeRatio, long logRollSize) {
-    MemoryUsage mu = ManagementFactory.getMemoryMXBean().getHeapMemoryUsage();
-    int maxLogs = Math.round(mu.getMax() * memstoreSizeRatio * 2 / logRollSize);
-    return maxLogs;
-  }
-
-  /**
-   * Get the backing files associated with this WAL.
-   * @return may be null if there are no files.
-   */
-  protected FileStatus[] getFiles() throws IOException {
-    return FSUtils.listStatus(fs, fullPathLogDir, ourFiles);
-  }
-
   /**
-   * Currently, we need to expose the writer's OutputStream to tests so that they can manipulate
-   * the default behavior (such as setting the maxRecoveryErrorCount value for example (see
+   * Currently, we need to expose the writer's OutputStream to tests so that they can manipulate the
+   * default behavior (such as setting the maxRecoveryErrorCount value for example (see
    * {@link TestWALReplay#testReplayEditsWrittenIntoWAL()}). This is done using reflection on the
-   * underlying HDFS OutputStream.
-   * NOTE: This could be removed once Hadoop1 support is removed.
+   * underlying HDFS OutputStream. NOTE: This could be removed once Hadoop1 support is removed.
    * @return null if underlying stream is not ready.
    */
   @VisibleForTesting
   OutputStream getOutputStream() {
     FSDataOutputStream fsdos = this.hdfs_out;
-    if (fsdos == null) return null;
-    return fsdos.getWrappedStream();
-  }
-
-  @Override
-  public byte [][] rollWriter() throws FailedLogCloseException, IOException {
-    return rollWriter(false);
-  }
-
-  /**
-   * retrieve the next path to use for writing.
-   * Increments the internal filenum.
-   */
-  private Path getNewPath() throws IOException {
-    this.filenum.set(System.currentTimeMillis());
-    Path newPath = getCurrentFileName();
-    while (fs.exists(newPath)) {
-      this.filenum.incrementAndGet();
-      newPath = getCurrentFileName();
-    }
-    return newPath;
-  }
-
-  Path getOldPath() {
-    long currentFilenum = this.filenum.get();
-    Path oldPath = null;
-    if (currentFilenum > 0) {
-      // ComputeFilename  will take care of meta wal filename
-      oldPath = computeFilename(currentFilenum);
-    } // I presume if currentFilenum is <= 0, this is first file and null for oldPath if fine?
-    return oldPath;
-  }
-
-  /**
-   * Tell listeners about pre log roll.
-   * @throws IOException
-   */
-  private void tellListenersAboutPreLogRoll(final Path oldPath, final Path newPath)
-  throws IOException {
-    if (!this.listeners.isEmpty()) {
-      for (WALActionsListener i : this.listeners) {
-        i.preLogRoll(oldPath, newPath);
-      }
-    }
-  }
-
-  /**
-   * Tell listeners about post log roll.
-   * @throws IOException
-   */
-  private void tellListenersAboutPostLogRoll(final Path oldPath, final Path newPath)
-  throws IOException {
-    if (!this.listeners.isEmpty()) {
-      for (WALActionsListener i : this.listeners) {
-        i.postLogRoll(oldPath, newPath);
-      }
-    }
+    return fsdos != null ? fsdos.getWrappedStream() : null;
   }
 
   /**
    * Run a sync after opening to set up the pipeline.
-   * @param nextWriter
-   * @param startTimeNanos
    */
   private void preemptiveSync(final ProtobufLogWriter nextWriter) {
     long startTimeNanos = System.nanoTime();
@@ -665,115 +282,17 @@ public class FSHLog implements WAL {
     }
   }
 
-  @Override
-  public byte [][] rollWriter(boolean force) throws FailedLogCloseException, IOException {
-    rollWriterLock.lock();
-    try {
-      // Return if nothing to flush.
-      if (!force && (this.writer != null && this.numEntries.get() <= 0)) return null;
-      byte [][] regionsToFlush = null;
-      if (this.closed) {
-        LOG.debug("WAL closed. Skipping rolling of writer");
-        return regionsToFlush;
-      }
-      if (!closeBarrier.beginOp()) {
-        LOG.debug("WAL closing. Skipping rolling of writer");
-        return regionsToFlush;
-      }
-      TraceScope scope = Trace.startSpan("FSHLog.rollWriter");
-      try {
-        Path oldPath = getOldPath();
-        Path newPath = getNewPath();
-        // Any exception from here on is catastrophic, non-recoverable so we currently abort.
-        Writer nextWriter = this.createWriterInstance(newPath);
-        FSDataOutputStream nextHdfsOut = null;
-        if (nextWriter instanceof ProtobufLogWriter) {
-          nextHdfsOut = ((ProtobufLogWriter)nextWriter).getStream();
-          // If a ProtobufLogWriter, go ahead and try and sync to force setup of pipeline.
-          // If this fails, we just keep going.... it is an optimization, not the end of the world.
-          preemptiveSync((ProtobufLogWriter)nextWriter);
-        }
-        tellListenersAboutPreLogRoll(oldPath, newPath);
-        // NewPath could be equal to oldPath if replaceWriter fails.
-        newPath = replaceWriter(oldPath, newPath, nextWriter, nextHdfsOut);
-        tellListenersAboutPostLogRoll(oldPath, newPath);
-        // Can we delete any of the old log files?
-        if (getNumRolledLogFiles() > 0) {
-          cleanOldLogs();
-          regionsToFlush = findRegionsToForceFlush();
-        }
-      } finally {
-        closeBarrier.endOp();
-        assert scope == NullScope.INSTANCE || !scope.isDetached();
-        scope.close();
-      }
-      return regionsToFlush;
-    } finally {
-      rollWriterLock.unlock();
-    }
-  }
-
   /**
-   * This method allows subclasses to inject different writers without having to
-   * extend other methods like rollWriter().
-   *
+   * This method allows subclasses to inject different writers without having to extend other
+   * methods like rollWriter().
    * @return Writer instance
    */
   protected Writer createWriterInstance(final Path path) throws IOException {
-    return DefaultWALProvider.createWriter(conf, fs, path, false);
-  }
-
-  /**
-   * Archive old logs. A WAL is eligible for archiving if all its WALEdits have been flushed.
-   * @throws IOException
-   */
-  private void cleanOldLogs() throws IOException {
-    List<Path> logsToArchive = null;
-    // For each log file, look at its Map of regions to highest sequence id; if all sequence ids
-    // are older than what is currently in memory, the WAL can be GC'd.
-    for (Map.Entry<Path, Map<byte[], Long>> e : this.byWalRegionSequenceIds.entrySet()) {
-      Path log = e.getKey();
-      Map<byte[], Long> sequenceNums = e.getValue();
-      if (this.sequenceIdAccounting.areAllLower(sequenceNums)) {
-        if (logsToArchive == null) logsToArchive = new ArrayList<Path>();
-        logsToArchive.add(log);
-        if (LOG.isTraceEnabled()) LOG.trace("WAL file ready for archiving " + log);
-      }
-    }
-    if (logsToArchive != null) {
-      for (Path p : logsToArchive) {
-        this.totalLogSize.addAndGet(-this.fs.getFileStatus(p).getLen());
-        archiveLogFile(p);
-        this.byWalRegionSequenceIds.remove(p);
-      }
+    Writer writer = DefaultWALProvider.createWriter(conf, fs, path, false);
+    if (writer instanceof ProtobufLogWriter) {
+      preemptiveSync((ProtobufLogWriter) writer);
     }
-  }
-
-  /**
-   * If the number of un-archived WAL files is greater than maximum allowed, check the first
-   * (oldest) WAL file, and returns those regions which should be flushed so that it can
-   * be archived.
-   * @return regions (encodedRegionNames) to flush in order to archive oldest WAL file.
-   * @throws IOException
-   */
-  byte[][] findRegionsToForceFlush() throws IOException {
-    byte [][] regions = null;
-    int logCount = getNumRolledLogFiles();
-    if (logCount > this.maxLogs && logCount > 0) {
-      Map.Entry<Path, Map<byte[], Long>> firstWALEntry =
-        this.byWalRegionSequenceIds.firstEntry();
-      regions = this.sequenceIdAccounting.findLower(firstWALEntry.getValue());
-    }
-    if (regions != null) {
-      StringBuilder sb = new StringBuilder();
-      for (int i = 0; i < regions.length; i++) {
-        if (i > 0) sb.append(", ");
-        sb.append(Bytes.toStringBinary(regions[i]));
-      }
-      LOG.info("Too many WALs; count=" + logCount + ", max=" + this.maxLogs +
-        "; forcing flush of " + regions.length + " regions(s): " + sb.toString());
-    }
-    return regions;
+    return writer;
   }
 
   /**
@@ -781,51 +300,39 @@ public class FSHLog implements WAL {
    * @see #beforeWaitOnSafePoint()
    */
   @VisibleForTesting
-  protected void afterCreatingZigZagLatch() {}
+  protected void afterCreatingZigZagLatch() {
+  }
 
   /**
    * @see #afterCreatingZigZagLatch()
    */
   @VisibleForTesting
-  protected void beforeWaitOnSafePoint() {};
+  protected void beforeWaitOnSafePoint() {
+  };
 
-  /**
-   * Cleans up current writer closing it and then puts in place the passed in
-   * <code>nextWriter</code>.
-   *
-   * In the case of creating a new WAL, oldPath will be null.
-   *
-   * In the case of rolling over from one file to the next, none of the params will be null.
-   *
-   * In the case of closing out this FSHLog with no further use newPath, nextWriter, and
-   * nextHdfsOut will be null.
-   *
-   * @param oldPath may be null
-   * @param newPath may be null
-   * @param nextWriter may be null
-   * @param nextHdfsOut may be null
-   * @return the passed in <code>newPath</code>
-   * @throws IOException if there is a problem flushing or closing the underlying FS
-   */
-  Path replaceWriter(final Path oldPath, final Path newPath, Writer nextWriter,
-      final FSDataOutputStream nextHdfsOut)
-  throws IOException {
-    // Ask the ring buffer writer to pause at a safe point.  Once we do this, the writer
+  @Override
+  protected void doAppend(Writer writer, FSWALEntry entry) throws IOException {
+    writer.append(entry);
+  }
+
+  @Override
+  protected long doReplaceWriter(Path oldPath, Path newPath, Writer nextWriter) throws IOException {
+    // Ask the ring buffer writer to pause at a safe point. Once we do this, the writer
     // thread will eventually pause. An error hereafter needs to release the writer thread
-    // regardless -- hence the finally block below.  Note, this method is called from the FSHLog
+    // regardless -- hence the finally block below. Note, this method is called from the FSHLog
     // constructor BEFORE the ring buffer is set running so it is null on first time through
     // here; allow for that.
     SyncFuture syncFuture = null;
-    SafePointZigZagLatch zigzagLatch = (this.ringBufferEventHandler == null)?
-      null: this.ringBufferEventHandler.attainSafePoint();
+    SafePointZigZagLatch zigzagLatch = (this.ringBufferEventHandler == null) ? null
+        : this.ringBufferEventHandler.attainSafePoint();
     afterCreatingZigZagLatch();
-    TraceScope scope = Trace.startSpan("FSHFile.replaceWriter");
+    long oldFileLen = 0L;
     try {
-      // Wait on the safe point to be achieved.  Send in a sync in case nothing has hit the
+      // Wait on the safe point to be achieved. Send in a sync in case nothing has hit the
       // ring buffer between the above notification of writer that we want it to go to
-      // 'safe point' and then here where we are waiting on it to attain safe point.  Use
+      // 'safe point' and then here where we are waiting on it to attain safe point. Use
       // 'sendSync' instead of 'sync' because we do not want this thread to block waiting on it
-      // to come back.  Cleanup this syncFuture down below after we are ready to run again.
+      // to come back. Cleanup this syncFuture down below after we are ready to run again.
       try {
         if (zigzagLatch != null) {
           Trace.addTimelineAnnotation("awaiting safepoint");
@@ -833,44 +340,37 @@ public class FSHLog implements WAL {
         }
       } catch (FailedSyncBeforeLogCloseException e) {
         // If unflushed/unsynced entries on close, it is reason to abort.
-        if (isUnflushedEntries()) throw e;
-        LOG.warn("Failed sync-before-close but no outstanding appends; closing WAL: " +
-          e.getMessage());
+        if (isUnflushedEntries()) {
+          throw e;
+        }
+        LOG.warn(
+          "Failed sync-before-close but no outstanding appends; closing WAL" + e.getMessage());
       }
-
-      // It is at the safe point.  Swap out writer from under the blocked writer thread.
-      // TODO: This is close is inline with critical section.  Should happen in background?
-      try {
-        if (this.writer != null) {
+      // It is at the safe point. Swap out writer from under the blocked writer thread.
+      // TODO: This is close is inline with critical section. Should happen in background?
+      if (this.writer != null) {
+        oldFileLen = this.writer.getLength();
+        try {
           Trace.addTimelineAnnotation("closing writer");
           this.writer.close();
           Trace.addTimelineAnnotation("writer closed");
-        }
-        this.closeErrorCount.set(0);
-      } catch (IOException ioe) {
-        int errors = closeErrorCount.incrementAndGet();
-        if (!isUnflushedEntries() && (errors <= this.closeErrorsTolerated)) {
-          LOG.warn("Riding over failed WAL close of " + oldPath + ", cause=\"" +
-            ioe.getMessage() + "\", errors=" + errors +
-            "; THIS FILE WAS NOT CLOSED BUT ALL EDITS SYNCED SO SHOULD BE OK");
-        } else {
-          throw ioe;
+          this.closeErrorCount.set(0);
+        } catch (IOException ioe) {
+          int errors = closeErrorCount.incrementAndGet();
+          if (!isUnflushedEntries() && (errors <= this.closeErrorsTolerated)) {
+            LOG.warn("Riding over failed WAL close of " + oldPath + ", cause=\"" + ioe.getMessage()
+                + "\", errors=" + errors
+                + "; THIS FILE WAS NOT CLOSED BUT ALL EDITS SYNCED SO SHOULD BE OK");
+          } else {
+            throw ioe;
+          }
         }
       }
       this.writer = nextWriter;
-      this.hdfs_out = nextHdfsOut;
-      int oldNumEntries = this.numEntries.get();
-      this.numEntries.set(0);
-      final String newPathString = (null == newPath ? null : FSUtils.getPath(newPath));
-      if (oldPath != null) {
-        this.byWalRegionSequenceIds.put(oldPath, this.sequenceIdAccounting.resetHighest());
-        long oldFileLen = this.fs.getFileStatus(oldPath).getLen();
-        this.totalLogSize.addAndGet(oldFileLen);
-        LOG.info("Rolled WAL " + FSUtils.getPath(oldPath) + " with entries=" + oldNumEntries +
-          ", filesize=" + StringUtils.byteDesc(oldFileLen) + "; new WAL " +
-          newPathString);
+      if (nextWriter != null && nextWriter instanceof ProtobufLogWriter) {
+        this.hdfs_out = ((ProtobufLogWriter) nextWriter).getStream();
       } else {
-        LOG.info("New WAL " + newPathString);
+        this.hdfs_out = null;
       }
     } catch (InterruptedException ie) {
       // Perpetuate the interrupt
@@ -880,223 +380,84 @@ public class FSHLog implements WAL {
       LOG.error("Failed close of WAL writer " + oldPath + ", unflushedEntries=" + count, e);
       throw new FailedLogCloseException(oldPath + ", unflushedEntries=" + count, e);
     } finally {
-      try {
-        // Let the writer thread go regardless, whether error or not.
-        if (zigzagLatch != null) {
-          zigzagLatch.releaseSafePoint();
-          // syncFuture will be null if we failed our wait on safe point above. Otherwise, if
-          // latch was obtained successfully, the sync we threw in either trigger the latch or it
-          // got stamped with an exception because the WAL was damaged and we could not sync. Now
-          // the write pipeline has been opened up again by releasing the safe point, process the
-          // syncFuture we got above. This is probably a noop but it may be stale exception from
-          // when old WAL was in place. Catch it if so.
-          if (syncFuture != null) {
-            try {
-              blockOnSync(syncFuture);
-            } catch (IOException ioe) {
-              if (LOG.isTraceEnabled()) LOG.trace("Stale sync exception", ioe);
+      // Let the writer thread go regardless, whether error or not.
+      if (zigzagLatch != null) {
+        zigzagLatch.releaseSafePoint();
+        // syncFuture will be null if we failed our wait on safe point above. Otherwise, if
+        // latch was obtained successfully, the sync we threw in either trigger the latch or it
+        // got stamped with an exception because the WAL was damaged and we could not sync. Now
+        // the write pipeline has been opened up again by releasing the safe point, process the
+        // syncFuture we got above. This is probably a noop but it may be stale exception from
+        // when old WAL was in place. Catch it if so.
+        if (syncFuture != null) {
+          try {
+            blockOnSync(syncFuture);
+          } catch (IOException ioe) {
+            if (LOG.isTraceEnabled()) {
+              LOG.trace("Stale sync exception", ioe);
             }
           }
         }
-      } finally {
-        scope.close();
       }
     }
-    return newPath;
-  }
-
-  long getUnflushedEntriesCount() {
-    long highestSynced = this.highestSyncedSequence.get();
-    return highestSynced > this.highestUnsyncedSequence?
-      0: this.highestUnsyncedSequence - highestSynced;
-  }
-
-  boolean isUnflushedEntries() {
-    return getUnflushedEntriesCount() > 0;
+    return oldFileLen;
   }
 
-  /*
-   * only public so WALSplitter can use.
-   * @return archived location of a WAL file with the given path p
-   */
-  public static Path getWALArchivePath(Path archiveDir, Path p) {
-    return new Path(archiveDir, p.getName());
-  }
-
-  private void archiveLogFile(final Path p) throws IOException {
-    Path newPath = getWALArchivePath(this.fullPathArchiveDir, p);
-    // Tell our listeners that a log is going to be archived.
-    if (!this.listeners.isEmpty()) {
-      for (WALActionsListener i : this.listeners) {
-        i.preLogArchive(p, newPath);
-      }
-    }
-    LOG.info("Archiving " + p + " to " + newPath);
-    if (!FSUtils.renameAndSetModifyTime(this.fs, p, newPath)) {
-      throw new IOException("Unable to rename " + p + " to " + newPath);
-    }
-    // Tell our listeners that a log has been archived.
-    if (!this.listeners.isEmpty()) {
-      for (WALActionsListener i : this.listeners) {
-        i.postLogArchive(p, newPath);
+  @Override
+  protected void doShutdown() throws IOException {
+    // Shutdown the disruptor. Will stop after all entries have been processed. Make sure we
+    // have stopped incoming appends before calling this else it will not shutdown. We are
+    // conservative below waiting a long time and if not elapsed, then halting.
+    if (this.disruptor != null) {
+      long timeoutms = conf.getLong("hbase.wal.disruptor.shutdown.timeout.ms", 60000);
+      try {
+        this.disruptor.shutdown(timeoutms, TimeUnit.MILLISECONDS);
+      } catch (TimeoutException e) {
+        LOG.warn("Timed out bringing down disruptor after " + timeoutms + "ms; forcing halt "
+            + "(It is a problem if this is NOT an ABORT! -- DATALOSS!!!!)");
+        this.disruptor.halt();
+        this.disruptor.shutdown();
       }
     }
-  }
-
-  /**
-   * This is a convenience method that computes a new filename with a given
-   * file-number.
-   * @param filenum to use
-   * @return Path
-   */
-  protected Path computeFilename(final long filenum) {
-    if (filenum < 0) {
-      throw new RuntimeException("WAL file number can't be < 0");
+    // With disruptor down, this is safe to let go.
+    if (this.appendExecutor != null) {
+      this.appendExecutor.shutdown();
     }
-    String child = logFilePrefix + WAL_FILE_NAME_DELIMITER + filenum + logFileSuffix;
-    return new Path(fullPathLogDir, child);
-  }
-
-  /**
-   * This is a convenience method that computes a new filename with a given
-   * using the current WAL file-number
-   * @return Path
-   */
-  public Path getCurrentFileName() {
-    return computeFilename(this.filenum.get());
-  }
-
-  @Override
-  public String toString() {
-    return "FSHLog " + logFilePrefix + ":" + logFileSuffix + "(num " + filenum + ")";
-  }
 
-/**
- * A log file has a creation timestamp (in ms) in its file name ({@link #filenum}.
- * This helper method returns the creation timestamp from a given log file.
- * It extracts the timestamp assuming the filename is created with the
- * {@link #computeFilename(long filenum)} method.
- * @param fileName
- * @return timestamp, as in the log file name.
- */
-  protected long getFileNumFromFileName(Path fileName) {
-    if (fileName == null) throw new IllegalArgumentException("file name can't be null");
-    if (!ourFiles.accept(fileName)) {
-      throw new IllegalArgumentException("The log file " + fileName +
-          " doesn't belong to this WAL. (" + toString() + ")");
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Closing WAL writer in " + FSUtils.getPath(walDir));
     }
-    final String fileNameString = fileName.toString();
-    String chompedPath = fileNameString.substring(prefixPathStr.length(),
-        (fileNameString.length() - logFileSuffix.length()));
-    return Long.parseLong(chompedPath);
-  }
-
-  @Override
-  public void close() throws IOException {
-    shutdown();
-    final FileStatus[] files = getFiles();
-    if (null != files && 0 != files.length) {
-      for (FileStatus file : files) {
-        Path p = getWALArchivePath(this.fullPathArchiveDir, file.getPath());
-        // Tell our listeners that a log is going to be archived.
-        if (!this.listeners.isEmpty()) {
-          for (WALActionsListener i : this.listeners) {
-            i.preLogArchive(file.getPath(), p);
-          }
-        }
-
-        if (!FSUtils.renameAndSetModifyTime(fs, file.getPath(), p)) {
-          throw new IOException("Unable to rename " + file.getPath() + " to " + p);
-        }
-        // Tell our listeners that a log was archived.
-        if (!this.listeners.isEmpty()) {
-          for (WALActionsListener i : this.listeners) {
-            i.postLogArchive(file.getPath(), p);
-          }
-        }
-      }
-      LOG.debug("Moved " + files.length + " WAL file(s) to " +
-        FSUtils.getPath(this.fullPathArchiveDir));
+    if (this.writer != null) {
+      this.writer.close();
+      this.writer = null;
     }
-    LOG.info("Closed WAL: " + toString());
   }
 
   @Override
-  public void shutdown() throws IOException {
-    if (shutdown.compareAndSet(false, true)) {
-      try {
-        // Prevent all further flushing and rolling.
-        closeBarrier.stopAndDrainOps();
-      } catch (InterruptedException e) {
-        LOG.error("Exception while waiting for cache flushes and log rolls", e);
-        Thread.currentThread().interrupt();
-      }
-
-      // Shutdown the disruptor.  Will stop after all entries have been processed.  Make sure we
-      // have stopped incoming appends before calling this else it will not shutdown.  We are
-      // conservative below waiting a long time and if not elapsed, then halting.
-      if (this.disruptor != null) {
-        long timeoutms = conf.getLong("hbase.wal.disruptor.shutdown.timeout.ms", 60000);
-        try {
-          this.disruptor.shutdown(timeoutms, TimeUnit.MILLISECONDS);
-        } catch (TimeoutException e) {
-          LOG.warn("Timed out bringing down disruptor after " + timeoutms + "ms; forcing halt " +
-            "(It is a problem if this is NOT an ABORT! -- DATALOSS!!!!)");
-          this.disruptor.halt();
-          this.disruptor.shutdown();
-        }
-      }
-      // With disruptor down, this is safe to let go.
-      if (this.appendExecutor !=  null) this.appendExecutor.shutdown();
-
-      // Tell our listeners that the log is closing
-      if (!this.listeners.isEmpty()) {
-        for (WALActionsListener i : this.listeners) {
-          i.logCloseRequested();
-        }
-      }
-      this.closed = true;
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Closing WAL writer in " + FSUtils.getPath(fullPathLogDir));
-      }
-      if (this.writer != null) {
-        this.writer.close();
-        this.writer = null;
-      }
-    }
+  public String toString() {
+    return "FSHLog " + walFilePrefix + ":" + walFileSuffix + "(num " + filenum + ")";
   }
 
-  /**
-   * NOTE: This append, at a time that is usually after this call returns, starts an
-   * mvcc transaction by calling 'begin' wherein which we assign this update a sequenceid. At
-   * assignment time, we stamp all the passed in Cells inside WALEdit with their sequenceId.
-   * You must 'complete' the transaction this mvcc transaction by calling
-   * MultiVersionConcurrencyControl#complete(...) or a variant otherwise mvcc will get stuck. Do it
-   * in the finally of a try/finally
-   * block within which this append lives and any subsequent operations like sync or
-   * update of memstore, etc. Get the WriteEntry to pass mvcc out of the passed in WALKey
-   * <code>walKey</code> parameter. Be warned that the WriteEntry is not immediately available
-   * on return from this method. It WILL be available subsequent to a sync of this append;
-   * otherwise, you will just have to wait on the WriteEntry to get filled in.
-   */
-  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NP_NULL_ON_SOME_PATH_EXCEPTION",
-      justification="Will never be null")
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "NP_NULL_ON_SOME_PATH_EXCEPTION",
+      justification = "Will never be null")
   @Override
   public long append(final HRegionInfo hri,
       final WALKey key, final WALEdit edits, final boolean inMemstore) throws IOException {
-    if (this.closed) throw new IOException("Cannot append; log is closed");
-    // Make a trace scope for the append.  It is closed on other side of the ring buffer by the
-    // single consuming thread.  Don't have to worry about it.
+    if (this.closed) {
+      throw new IOException("Cannot append; log is closed");
+    }
+    // Make a trace scope for the append. It is closed on other side of the ring buffer by the
+    // single consuming thread. Don't have to worry about it.
     TraceScope scope = Trace.startSpan("FSHLog.append");
 
-    // This is crazy how much it takes to make an edit.  Do we need all this stuff!!!!????  We need
+    // This is crazy how much it takes to make an edit. Do we need all this stuff!!!!???? We need
     // all this to make a key and then below to append the edit, we need to carry htd, info,
     // etc. all over the ring buffer.
     FSWALEntry entry = null;
     long sequence = this.disruptor.getRingBuffer().next();
     try {
       RingBufferTruck truck = this.disruptor.getRingBuffer().get(sequence);
-      // Construction of FSWALEntry sets a latch.  The latch is thrown just after we stamp the
+      // Construction of FSWALEntry sets a latch. The latch is thrown just after we stamp the
       // edit with its edit/sequence id.
       // TODO: reuse FSWALEntry as we do SyncFuture rather create per append.
       entry = new FSWALEntry(sequence, key, edits, hri, inMemstore);
@@ -1108,17 +469,18 @@ public class FSHLog implements WAL {
   }
 
   /**
-   * Thread to runs the hdfs sync call. This call takes a while to complete.  This is the longest
-   * pole adding edits to the WAL and this must complete to be sure all edits persisted.  We run
-   * multiple threads sync'ng rather than one that just syncs in series so we have better
-   * latencies; otherwise, an edit that arrived just after a sync started, might have to wait
-   * almost the length of two sync invocations before it is marked done.
-   * <p>When the sync completes, it marks all the passed in futures done.  On the other end of the
-   * sync future is a blocked thread, usually a regionserver Handler.  There may be more than one
-   * future passed in the case where a few threads arrive at about the same time and all invoke
-   * 'sync'.  In this case we'll batch up the invocations and run one filesystem sync only for a
-   * batch of Handler sync invocations.  Do not confuse these Handler SyncFutures with the futures
-   * an ExecutorService returns when you call submit. We have no use for these in this model. These
+   * Thread to runs the hdfs sync call. This call takes a while to complete. This is the longest
+   * pole adding edits to the WAL and this must complete to be sure all edits persisted. We run
+   * multiple threads sync'ng rather than one that just syncs in series so we have better latencies;
+   * otherwise, an edit that arrived just after a sync started, might have to wait almost the length
+   * of two sync invocations before it is marked done.
+   * <p>
+   * When the sync completes, it marks all the passed in futures done. On the other end of the sync
+   * future is a blocked thread, usually a regionserver Handler. There may be more than one future
+   * passed in the case where a few threads arrive at about the same time and all invoke 'sync'. In
+   * this case we'll batch up the invocations and run one filesystem sync only for a batch of
+   * Handler sync invocations. Do not confuse these Handler SyncFutures with the futures an
+   * ExecutorService returns when you call submit. We have no use for these in this model. These
    * SyncFutures are 'artificial', something to hold the Handler until the filesystem sync
    * completes.
    */
@@ -1130,12 +492,13 @@ public class FSHLog implements WAL {
     /**
      * UPDATE!
      * @param syncs the batch of calls to sync that arrived as this thread was starting; when done,
-     * we will put the result of the actual hdfs sync call as the result.
-     * @param sequence The sequence number on the ring buffer when this thread was set running.
-     * If this actual writer sync completes then all appends up this point have been
-     * flushed/synced/pushed to datanodes.  If we fail, then the passed in <code>syncs</code>
-     * futures will return the exception to their clients; some of the edits may have made it out
-     * to data nodes but we will report all that were part of this session as failed.
+     *          we will put the result of the actual hdfs sync call as the result.
+     * @param sequence The sequence number on the ring buffer when this thread was set running. If
+     *          this actual writer sync completes then all appends up this point have been
+     *          flushed/synced/pushed to datanodes. If we fail, then the passed in
+     *          <code>syncs</code> futures will return the exception to their clients; some of the
+     *          edits may have made it out to data nodes but we will report all that were part of
+     *          this session as failed.
      */
     SyncRunner(final String name, final int maxHandlersCount) {
       super(name);
@@ -1145,17 +508,17 @@ public class FSHLog implements WAL {
       //
       // We could let the capacity be 'open' but bound it so we get alerted in pathological case
       // where we cannot sync and we have a bunch of threads all backed up waiting on their syncs
-      // to come in.  LinkedBlockingQueue actually shrinks when you remove elements so Q should
-      // stay neat and tidy in usual case.  Let the max size be three times the maximum handlers.
+      // to come in. LinkedBlockingQueue actually shrinks when you remove elements so Q should
+      // stay neat and tidy in usual case. Let the max size be three times the maximum handlers.
       // The passed in maxHandlerCount is the user-level handlers which is what we put up most of
       // but HBase has other handlers running too -- opening region handlers which want to write
-      // the meta table when succesful (i.e. sync), closing handlers -- etc.  These are usually
+      // the meta table when succesful (i.e. sync), closing handlers -- etc. These are usually
       // much fewer in number than the user-space handlers so Q-size should be user handlers plus
-      // some space for these other handlers.  Lets multiply by 3 for good-measure.
+      // some space for these other handlers. Lets multiply by 3 for good-measure.
       this.syncFutures = new LinkedBlockingQueue<SyncFuture>(maxHandlersCount * 3);
     }
 
-    void offer(final long sequence, final SyncFuture [] syncFutures, final int syncFutureCount) {
+    void offer(final long sequence, final SyncFuture[] syncFutures, final int syncFutureCount) {
       // Set sequence first because the add to the queue will wake the thread if sleeping.
       this.sequence = sequence;
       for (int i = 0; i < syncFutureCount; ++i) {
@@ -1165,28 +528,28 @@ public class FSHLog implements WAL {
 
     /**
      * Release the passed <code>syncFuture</code>
-     * @param syncFuture
-     * @param currentSequence
-     * @param t
      * @return Returns 1.
      */
     private int releaseSyncFuture(final SyncFuture syncFuture, final long currentSequence,
         final Throwable t) {
-      if (!syncFuture.done(currentSequence, t)) throw new IllegalStateException();
+      if (!syncFuture.done(currentSequence, t)) {
+        throw new IllegalStateException();
+      }
       // This function releases one sync future only.
       return 1;
     }
 
     /**
      * Release all SyncFutures whose sequence is <= <code>currentSequence</code>.
-     * @param currentSequence
      * @param t May be non-null if we are processing SyncFutures because an exception was thrown.
      * @return Count of SyncFutures we let go.
      */
     private int releaseSyncFutures(final long currentSequence, final Throwable t) {
       int syncCount = 0;
       for (SyncFuture syncFuture; (syncFuture = this.syncFutures.peek()) != null;) {
-        if (syncFuture.getRingBufferSequence() > currentSequence) break;
+        if (syncFuture.getTxid() > currentSequence) {
+          break;
+        }
         releaseSyncFuture(syncFuture, currentSequence, t);
         if (!this.syncFutures.remove(syncFuture)) {
           throw new IllegalStateException(syncFuture.toString());
@@ -1204,14 +567,14 @@ public class FSHLog implements WAL {
       long currentHighestSyncedSequence;
       // Set the highestSyncedSequence IFF our current sequence id is the 'highest'.
       do {
-        currentHighestSyncedSequence = highestSyncedSequence.get();
+        currentHighestSyncedSequence = highestSyncedTxid.get();
         if (currentHighestSyncedSequence >= sequence) {
           // Set the sync number to current highwater mark; might be able to let go more
           // queued sync futures
           sequence = currentHighestSyncedSequence;
           break;
         }
-      } while (!highestSyncedSequence.compareAndSet(currentHighestSyncedSequence, sequence));
+      } while (!highestSyncedTxid.compareAndSet(currentHighestSyncedSequence, sequence));
       return sequence;
     }
 
@@ -1225,21 +588,21 @@ public class FSHLog implements WAL {
             // We have to process what we 'take' from the queue
             takeSyncFuture = this.syncFutures.take();
             currentSequence = this.sequence;
-            long syncFutureSequence = takeSyncFuture.getRingBufferSequence();
+            long syncFutureSequence = takeSyncFuture.getTxid();
             if (syncFutureSequence > currentSequence) {
-              throw new IllegalStateException("currentSequence=" + syncFutureSequence +
-                ", syncFutureSequence=" + syncFutureSequence);
+              throw new IllegalStateException("currentSequence=" + syncFutureSequence
+                  + ", syncFutureSequence=" + syncFutureSequence);
             }
             // See if we can process any syncfutures BEFORE we go sync.
-            long currentHighestSyncedSequence = highestSyncedSequence.get();
+            long currentHighestSyncedSequence = highestSyncedTxid.get();
             if (currentSequence < currentHighestSyncedSequence) {
               syncCount += releaseSyncFuture(takeSyncFuture, currentHighestSyncedSequence, null);
-              // Done with the 'take'.  Go around again and do a new 'take'.
+              // Done with the 'take'. Go around again and do a new 'take'.
               continue;
             }
             break;
           }
-          // I got something.  Lets run.  Save off current sequence number in case it changes
+          // I got something. Lets run. Save off current sequence number in case it changes
           // while we run.
           TraceScope scope = Trace.continueSpan(takeSyncFuture.getSpan());
           long start = System.nanoTime();
@@ -1262,8 +625,11 @@ public class FSHLog implements WAL {
             syncCount += releaseSyncFuture(takeSyncFuture, currentSequence, lastException);
             // Can we release other syncs?
             syncCount += releaseSyncFutures(currentSequence, lastException);
-            if (lastException != null) requestLogRoll();
-            else checkLogRoll();
+            if (lastException != null) {
+              requestLogRoll();
+            } else {
+              checkLogRoll();
+            }
           }
           postSync(System.nanoTime() - start, syncCount);
         } catch (InterruptedException e) {
@@ -1281,7 +647,9 @@ public class FSHLog implements WAL {
    */
   void checkLogRoll() {
     // Will return immediately if we are in the middle of a WAL log roll currently.
-    if (!rollWriterLock.tryLock()) return;
+    if (!rollWriterLock.tryLock()) {
+      return;
+    }
     boolean lowReplication;
     try {
       lowReplication = checkLowReplication();
@@ -1297,7 +665,7 @@ public class FSHLog implements WAL {
     }
   }
 
-  /*
+  /**
    * @return true if number of replicas for the WAL is lower than threshold
    */
   private boolean checkLowReplication() {
@@ -1309,11 +677,10 @@ public class FSHLog implements WAL {
       if (numCurrentReplicas != 0 && numCurrentReplicas < this.minTolerableReplication) {
         if (this.lowReplicationRollEnabled) {
           if (this.consecutiveLogRolls.get() < this.lowReplicationRollLimit) {
-            LOG.warn("HDFS pipeline error detected. " + "Found "
-                + numCurrentReplicas + " replicas but expecting no less than "
-                + this.minTolerableReplication + " replicas. "
-                + " Requesting close of WAL. current pipeline: "
-                + Arrays.toString(getPipeLine()));
+            LOG.warn("HDFS pipeline error detected. " + "Found " + numCurrentReplicas
+                + " replicas but expecting no less than " + this.minTolerableReplication
+                + " replicas. " + " 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
@@ -1341,8 +708,7 @@ public class FSHLog implements WAL {
         }
       }
     } catch (Exception e) {
-      LOG.warn("DFSOutputStream.getNumCurrentReplicas failed because of " + e +
-        ", continuing...");
+      LOG.warn("DFSOutputStream.getNumCurrentReplicas failed because of " + e + ", continuing...");
     }
     return logRollNeeded;
   }
@@ -1353,6 +719,7 @@ public class FSHLog implements WAL {
 
   private SyncFuture publishSyncOnRingBuffer(Span span) {
     long sequence = this.disruptor.getRingBuffer().next();
+    // here we use ring buffer sequence as transaction id
     SyncFuture syncFuture = getSyncFuture(sequence, span);
     try {
       RingBufferTruck truck = this.disruptor.getRingBuffer().get(sequence);
@@ -1368,81 +735,17 @@ public class FSHLog implements WAL {
     return blockOnSync(publishSyncOnRingBuffer(span));
   }
 
-  private Span blockOnSync(final SyncFuture syncFuture) throws IOException {
-    // Now we have published the ringbuffer, halt the current thread until we get an answer back.
-    try {
-      syncFuture.get();
-      return syncFuture.getSpan();
-    } catch (InterruptedException ie) {
-      LOG.warn("Interrupted", ie);
-      throw convertInterruptedExceptionToIOException(ie);
-    } catch (ExecutionException e) {
-      throw ensureIOException(e.getCause());
-    }
-  }
-
-  private IOException convertInterruptedExceptionToIOException(final InterruptedException ie) {
-    Thread.currentThread().interrupt();
-    IOException ioe = new InterruptedIOException();
-    ioe.initCause(ie);
-    return ioe;
-  }
-
-  private SyncFuture getSyncFuture(final long sequence, Span span) {
-    SyncFuture syncFuture = this.syncFuturesByHandler.get(Thread.currentThread());
-    if (syncFuture == null) {
-      syncFuture = new SyncFuture();
-      this.syncFuturesByHandler.put(Thread.currentThread(), syncFuture);
-    }
-    return syncFuture.reset(sequence, span);
-  }
-
-  private void postSync(final long timeInNanos, final int handlerSyncs) {
-    if (timeInNanos > this.slowSyncNs) {
-      String msg =
-          new StringBuilder().append("Slow sync cost: ")
-              .append(timeInNanos / 1000000).append(" ms, current pipeline: ")
-              .append(Arrays.toString(getPipeLine())).toString();
-      Trace.addTimelineAnnotation(msg);
-      LOG.info(msg);
-    }
-    if (!listeners.isEmpty()) {
-      for (WALActionsListener listener : listeners) {
-        listener.postSync(timeInNanos, handlerSyncs);
-      }
-    }
-  }
-
-  private long postAppend(final Entry e, final long elapsedTime) {
-    long len = 0;
-    if (!listeners.isEmpty()) {
-      for (Cell cell : e.getEdit().getCells()) {
-        len += CellUtil.estimatedSerializedSizeOf(cell);
-      }
-      for (WALActionsListener listener : listeners) {
-        listener.postAppend(len, elapsedTime);
-      }
-    }
-    return len;
-  }
-
-
   /**
-   * This method gets the datanode replication count for the current WAL.
-   *
-   * If the pipeline isn't started yet or is empty, you will get the default
-   * replication factor.  Therefore, if this function returns 0, it means you
-   * are not properly running with the HDFS-826 patch.
-   * @throws InvocationTargetException
-   * @throws IllegalAccessException
-   * @throws IllegalArgumentException
-   *
-   * @throws Exception
+   * {@inheritDoc}
+   * <p>
+   * If the pipeline isn't started yet or is empty, you will get the default replication factor.
+   * Therefore, if this function returns 0, it means you are not properly running with the HDFS-826
+   * patch.
    */
   @VisibleForTesting
   int getLogReplication() {
     try {
-      //in standalone mode, it will return 0
+      // in standalone mode, it will return 0
       if (this.hdfs_out instanceof HdfsDataOutputStream) {
         return ((HdfsDataOutputStream) this.hdfs_out).getCurrentBlockReplication();
       }
@@ -1465,7 +768,7 @@ public class FSHLog implements WAL {
 
   @Override
   public void sync(long txid) throws IOException {
-    if (this.highestSyncedSequence.get() >= txid){
+    if (this.highestSyncedTxid.get() >= txid) {
       // Already sync'd.
       return;
     }
@@ -1478,70 +781,20 @@ public class FSHLog implements WAL {
     }
   }
 
-  // public only until class moves to o.a.h.h.wal
-  public void requestLogRoll() {
-    requestLogRoll(false);
-  }
-
-  private void requestLogRoll(boolean tooFewReplicas) {
-    if (!this.listeners.isEmpty()) {
-      for (WALActionsListener i: this.listeners) {
-        i.logRollRequested(tooFewReplicas);
-      }
-    }
-  }
-
-  // public only until class moves to o.a.h.h.wal
-  /** @return the number of rolled log files */
-  public int getNumRolledLogFiles() {
-    return byWalRegionSequenceIds.size();
-  }
-
-  // public only until class moves to o.a.h.h.wal
-  /** @return the number of log files in use */
-  public int getNumLogFiles() {
-    // +1 for current use log
-    return getNumRolledLogFiles() + 1;
-  }
-
-  // public only until class moves to o.a.h.h.wal
-  /** @return the size of log files in use */
-  public long getLogFileSize() {
-    return this.totalLogSize.get();
-  }
-
-  @Override
-  public Long startCacheFlush(final byte[] encodedRegionName, Set<byte[]> families) {
-    if (!closeBarrier.beginOp()) {
-      LOG.info("Flush not started for " + Bytes.toString(encodedRegionName) + "; server closing.");
-      return null;
-    }
-    return this.sequenceIdAccounting.startCacheFlush(encodedRegionName, families);
-  }
-
-  @Override
-  public void completeCacheFlush(final byte [] encodedRegionName) {
-    this.sequenceIdAccounting.completeCacheFlush(encodedRegionName);
-    closeBarrier.endOp();
-  }
-
   @Override
-  public void abortCacheFlush(byte[] encodedRegionName) {
-    this.sequenceIdAccounting.abortCacheFlush(encodedRegionName);
-    closeBarrier.endOp();
+  public void logRollerExited() {
   }
 
   @VisibleForTesting
   boolean isLowReplicationRollEnabled() {
-      return lowReplicationRollEnabled;
+    return lowReplicationRollEnabled;
   }
 
-  public static final long FIXED_OVERHEAD = ClassSize.align(
-    ClassSize.OBJECT + (5 * ClassSize.REFERENCE) +
-    ClassSize.ATOMIC_INTEGER + Bytes.SIZEOF_INT + (3 * Bytes.SIZEOF_LONG));
+  public static final long FIXED_OVERHEAD = ClassSize
+      .align(ClassSize.OBJECT + (5 * ClassSize.REFERENCE) + ClassSize.ATOMIC_INTEGER
+          + Bytes.SIZEOF_INT + (3 * Bytes.SIZEOF_LONG));
 
-  private static void split(final Configuration conf, final Path p)
-  throws IOException {
+  private static void split(final Configuration conf, final Path p) throws IOException {
     FileSystem fs = FileSystem.get(conf);
     if (!fs.exists(p)) {
       throw new FileNotFoundException(p.toString());
@@ -1555,51 +808,30 @@ public class FSHLog implements WAL {
     WALSplitter.split(baseDir, p, archiveDir, fs, conf, WALFactory.getInstance(conf));
   }
 
-
-  @Override
-  public long getEarliestMemstoreSeqNum(byte[] encodedRegionName) {
-    // Used by tests. Deprecated as too subtle for general usage.
-    return this.sequenceIdAccounting.getLowestSequenceId(encodedRegionName);
-  }
-
-  @Override
-  public long getEarliestMemstoreSeqNum(byte[] encodedRegionName, byte[] familyName) {
-    // This method is used by tests and for figuring if we should flush or not because our
-    // sequenceids are too old. It is also used reporting the master our oldest sequenceid for use
-    // figuring what edits can be skipped during log recovery. getEarliestMemStoreSequenceId
-    // from this.sequenceIdAccounting is looking first in flushingOldestStoreSequenceIds, the
-    // currently flushing sequence ids, and if anything found there, it is returning these. This is
-    // the right thing to do for the reporting oldest sequenceids to master; we won't skip edits if
-    // we crash during the flush. For figuring what to flush, we might get requeued if our sequence
-    // id is old even though we are currently flushing. This may mean we do too much flushing.
-    return this.sequenceIdAccounting.getLowestSequenceId(encodedRegionName, familyName);
-  }
-
   /**
-   * This class is used coordinating two threads holding one thread at a
-   * 'safe point' while the orchestrating thread does some work that requires the first thread
-   * paused: e.g. holding the WAL writer while its WAL is swapped out from under it by another
-   * thread.
-   *
-   * <p>Thread A signals Thread B to hold when it gets to a 'safe point'.  Thread A wait until
-   * Thread B gets there. When the 'safe point' has been attained, Thread B signals Thread A.
-   * Thread B then holds at the 'safe point'.  Thread A on notification that Thread B is paused,
-   * goes ahead and does the work it needs to do while Thread B is holding.  When Thread A is done,
-   * it flags B and then Thread A and Thread B continue along on their merry way.  Pause and
-   * signalling 'zigzags' between the two participating threads.  We use two latches -- one the
-   * inverse of the other -- pausing and signaling when states are achieved.
-   *
-   * <p>To start up the drama, Thread A creates an instance of this class each time it would do
-   * this zigzag dance and passes it to Thread B (these classes use Latches so it is one shot
-   * only). Thread B notices the new instance (via reading a volatile reference or how ever) and it
-   * starts to work toward the 'safe point'.  Thread A calls {@link #waitSafePoint()} when it
-   * cannot proceed until the Thread B 'safe point' is attained. Thread A will be held inside in
-   * {@link #waitSafePoint()} until Thread B reaches the 'safe point'.  Once there, Thread B
-   * frees Thread A by calling {@link #safePointAttained()}.  Thread A now knows Thread B
-   * is at the 'safe point' and that it is holding there (When Thread B calls
-   * {@link #safePointAttained()} it blocks here until Thread A calls {@link #releaseSafePoint()}).
-   * Thread A proceeds to do what it needs to do while Thread B is paused.  When finished,
-   * it lets Thread B lose by calling {@link #releaseSafePoint()} and away go both Threads again.
+   * This class is used coordinating two threads holding one thread at a 'safe point' while the
+   * orchestrating thread does some work that requires the first thread paused: e.g. holding the WAL
+   * writer while its WAL is swapped out from under it by another thread.
+   * <p>
+   * Thread A signals Thread B to hold when it gets to a 'safe point'. Thread A wait until Thread B
+   * gets there. When the 'safe point' has been attained, Thread B signals Thread A. Thread B then
+   * holds at the 'safe point'. Thread A on notification that Thread B is paused, goes ahead and
+   * does the work it needs to do while Thread B is holding. When Thread A is done, it flags B and
+   * then Thread A and Thread B continue along on their merry way. Pause and signalling 'zigzags'
+   * between the two participating threads. We use two latches -- one the inverse of the other --
+   * pausing and signaling when states are achieved.
+   * <p>
+   * To start up the drama, Thread A creates an instance of this class each time it would do this
+   * zigzag dance and passes it to Thread B (these classes use Latches so it is one shot only).
+   * Thread B notices the new instance (via reading a volatile reference or how ever) and it starts
+   * to work toward the 'safe point'. Thread A calls {@link #waitSafePoint()} when it cannot proceed
+   * until the Thread B 'safe point' is attained. Thread A will be held inside in
+   * {@link #waitSafePoint()} until Thread B reaches the 'safe point'. Once there, Thread B frees
+   * Thread A by calling {@link #safePointAttained()}. Thread A now knows Thread B is at the 'safe
+   * point' and that it is holding there (When Thread B calls {@link #safePointAttained()} it blocks
+   * here until Thread A calls {@link #releaseSafePoint()}). Thread A proceeds to do what it needs
+   * to do while Thread B is paused. When finished, it lets Thread B lose by calling
+   * {@link #releaseSafePoint()} and away go both Threads again.
    */
   static class SafePointZigZagLatch {
     /**
@@ -1607,24 +839,23 @@ public class FSHLog implements WAL {
      */
     private volatile CountDownLatch safePointAttainedLatch = new CountDownLatch(1);
     /**
-     * Latch to wait on.  Will be released when we can proceed.
+     * Latch to wait on. Will be released when we can proceed.
      */
     private volatile CountDownLatch safePointReleasedLatch = new CountDownLatch(1);
 
     /**
-     * For Thread A to call when it is ready to wait on the 'safe point' to be attained.
-     * Thread A will be held in here until Thread B calls {@link #safePointAttained()}
-     * @param syncFuture We need this as barometer on outstanding syncs.  If it comes home with
-     * an exception, then something is up w/ our syncing.
-     * @throws InterruptedException
-     * @throws ExecutionException
+     * For Thread A to call when it is ready to wait on the 'safe point' to be attained. Thread A
+     * will be held in here until Thread B calls {@link #safePointAttained()}
+     * @param syncFuture We need this as barometer on outstanding syncs. If it comes home with an
+     *          exception, then something is up w/ our syncing.
      * @return The passed <code>syncFuture</code>
-     * @throws FailedSyncBeforeLogCloseException
      */
-    SyncFuture waitSafePoint(final SyncFuture syncFuture)
-    throws InterruptedException, FailedSyncBeforeLogCloseException {
+    SyncFuture waitSafePoint(final SyncFuture syncFuture) throws InterruptedException,
+        FailedSyncBeforeLogCloseException {
       while (true) {
-        if (this.safePointAttainedLatch.await(1, TimeUnit.NANOSECONDS)) break;
+        if (this.safePointAttainedLatch.await(1, TimeUnit.NANOSECONDS)) {
+          break;
+        }
         if (syncFuture.isThrowable()) {
           throw new FailedSyncBeforeLogCloseException(syncFuture.getThrowable());
         }
@@ -1633,10 +864,9 @@ public class FSHLog implements WAL {
     }
 
     /**
-     * Called by Thread B when it attains the 'safe point'.  In this method, Thread B signals
-     * Thread A it can proceed. Thread B will be held in here until {@link #releaseSafePoint()}
-     * is called by Thread A.
-     * @throws InterruptedException
+     * Called by Thread B when it attains the 'safe point'. In this method, Thread B signals Thread
+     * A it can proceed. Thread B will be held in here until {@link #releaseSafePoint()} is called
+     * by Thread A.
      */
     void safePointAttained() throws InterruptedException {
       this.safePointAttainedLatch.countDown();
@@ -1644,8 +874,8 @@ public class FSHLog implements WAL {
     }
 
     /**
-     * Called by Thread A when it is done with the work it needs to do while Thread B is
-     * halted.  This will release the Thread B held in a call to {@link #safePointAttained()}
+     * Called by Thread A when it is done with the work it needs to do while Thread B is halted.
+     * This will release the Thread B held in a call to {@link #safePointAttained()}
      */
     void releaseSafePoint() {
       this.safePointReleasedLatch.countDown();
@@ -1655,44 +885,44 @@ public class FSHLog implements WAL {
      * @return True is this is a 'cocked', fresh instance, and not one that has already fired.
      */
     boolean isCocked() {
-      return this.safePointAttainedLatch.getCount() > 0 &&
-        this.safePointReleasedLatch.getCount() > 0;
+      return this.safePointAttainedLatch.getCount() > 0
+          && this.safePointReleasedLatch.getCount() > 0;
     }
   }
 
   /**
    * Handler that is run by the disruptor ringbuffer consumer. Consumer is a SINGLE
-   * 'writer/appender' thread.  Appends edits and starts up sync runs.  Tries its best to batch up
-   * syncs.  There is no discernible benefit batching appends so we just append as they come in
-   * because it simplifies the below implementation.  See metrics for batching effectiveness
-   * (In measurement, at 100 concurrent handlers writing 1k, we are batching > 10 appends and 10
-   * handler sync invocations for every actual dfsclient sync call; at 10 concurrent handlers,
-   * YMMV).
-   * <p>Herein, we have an array into which we store the sync futures as they come in.  When we
-   * have a 'batch', we'll then pass what we have collected to a SyncRunner thread to do the
-   * filesystem sync.  When it completes, it will then call
-   * {@link SyncFuture#done(long, Throwable)} on each of SyncFutures in the batch to release
-   * blocked Handler threads.
-   * <p>I've tried various effects to try and make latencies low while keeping throughput high.
-   * I've tried keeping a single Queue of SyncFutures in this class appending to its tail as the
-   * syncs coming and having sync runner threads poll off the head to 'finish' completed
-   * SyncFutures.  I've tried linkedlist, and various from concurrent utils whether
-   * LinkedBlockingQueue or ArrayBlockingQueue, etc.  The more points of synchronization, the
-   * more 'work' (according to 'perf stats') that has to be done; small increases in stall
-   * percentages seem to have a big impact on throughput/latencies.  The below model where we have
-   * an array into which we stash the syncs and then hand them off to the sync thread seemed like
-   * a decent compromise.  See HBASE-8755 for more detail.
+   * 'writer/appender' thread. Appends edits and starts up sync runs. Tries its best to batch up
+   * syncs. There is no discernible benefit batching appends so we just append as they come in
+   * because it simplifies the below implementation. See metrics for batching effectiveness (In
+   * measurement, at 100 concurrent handlers writing 1k, we are batching > 10 appends and 10 handler
+   * sync invocations for every actual dfsclient sync call; at 10 concurrent handlers, YMMV).
+   * <p>
+   * Herein, we have an array into which we store the sync futures as they come in. When we have a
+   * 'batch', we'll then pass what we have collected to a SyncRunner thread to do the filesystem
+   * sync. When it completes, it will then call {@link SyncFuture#done(long, Throwable)} on each of
+   * SyncFutures in the batch to release blocked Handler threads.
+   * <p>
+   * I've tried various effects to try and make latencies low while keeping throughput high. I've
+   * tried keeping a single Queue of SyncFutures in this class appending to its tail as the syncs
+   * coming and having sync runner threads poll off the head to 'finish' completed SyncFutures. I've
+   * tried linkedlist, and various from concurrent utils whether LinkedBlockingQueue or
+   * ArrayBlockingQueue, etc. The more points of synchronization, the more 'work' (according to
+   * 'perf stats') that has to be done; small increases in stall percentages seem to have a big
+   * impact on throughput/latencies. The below model where we have an array into which we stash the
+   * syncs and then hand them off to the sync thread seemed like a decent compromise. See HBASE-8755
+   * for more detail.
    */
   class RingBufferEventHandler implements EventHandler<RingBufferTruck>, LifecycleAware {
-    private final SyncRunner [] syncRunners;
-    private final SyncFuture [] syncFutures;
-    // Had 'interesting' issues when this was non-volatile.  On occasion, we'd not pass all
+    private final SyncRunner[] syncRunners;
+    private final SyncFuture[] syncFutures;
+    // Had 'interesting' issues when this was non-volatile. On occasion, we'd not pass all
     // syncFutures to the next sync'ing thread.
     private volatile int syncFuturesCount = 0;
     private volatile SafePointZigZagLatch zigzagLatch;
     /**
-     * Set if we get an exception appending or syncing so that all subsequence appends and syncs
-     * on this WAL fail until WAL is replaced.
+     * Set if we get an exception appending or syncing so that all subsequence appends and syncs on
+     * this WAL fail until WAL is replaced.
      */
     private Exception exception = null;
     /**
@@ -1716,7 +946,9 @@ public class FSHLog implements WAL {
 
     private void cleanupOutstandingSyncsOnException(final long sequence, final Exception e) {
       // There could be handler-count syncFutures outstanding.
-      for (int i = 0; i < this.syncFuturesCount; i++) this.syncFutures[i].done(sequence, e);
+      for (int i = 0; i < this.syncFuturesCount; i++) {
+        this.syncFutures[i].done(sequence, e);
+      }
       this.syncFuturesCount = 0;
     }
 
@@ -1725,7 +957,9 @@ public class FSHLog implements WAL {
      */
     private boolean isOutstandingSyncs() {
       for (int i = 0; i < this.syncFuturesCount; i++) {
-        if (!this.syncFutures[i].isDone()) return true;
+        if (!this.syncFutures[i].isDone()) {
+          return true;
+        }
       }
       return false;
     }
@@ -1733,10 +967,10 @@ public class FSHLog implements WAL {
     @Override
     // We can set endOfBatch in the below method if at end of our this.syncFutures array
     public void onEvent(final RingBufferTruck truck, final long sequence, boolean endOfBatch)
-    throws Exception {
-      // Appends and syncs are coming in order off the ringbuffer.  We depend on this fact.  We'll
-      // add appends to dfsclient as they come in.  Batching appends doesn't give any significant


<TRUNCATED>

[08/50] [abbrv] hbase git commit: HBASE-15477 Purge 'next block header' from cached blocks

Posted by sy...@apache.org.
HBASE-15477 Purge 'next block header' from cached blocks

When we read from HDFS, we overread to pick up the next blocks header.
Doing this saves a seek as we move through the hfile; we save having to
do an explicit seek just to read the block header every time we need to
read the body.  We used to read in the next header as part of the
current blocks buffer. This buffer was then what got persisted to
blockcache; so we were over-persisting: our block plus the next blocks'
header (33 bytes).

This patch undoes this over-persisting.

Removes support for version 1 blocks (0.2 was added in hbase-0.92.0).
Not needed any more.

There is an open question on whether checksums should be persisted
when caching. The code seems to say no but if cache is SSD backed or
backed by anything that does not do error correction, we'll want
checksums.

Adds loads of documentation.

M hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java
  (write) Add writing from a ByteBuff.

M hbase-common/src/main/java/org/apache/hadoop/hbase/nio/ByteBuff.java
  (toString) Add one so ByteBuff looks like ByteBuffer when you click on
  it in IDE

M hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
  Remove support for version 1 blocks.

  Cleaned up handling of metadata added when we serialize a block to
  caches. Metadata is smaller now.

  When we serialize (used when caching), do not persist the next blocks
  header if present.

  Removed a bunch of methods, a few of which had overlapping
  functionality and others that exposed too much of our internals.
  Also removed a bunch of constructors and unified the constructors we
  had left over making them share a common init method.
  Shutdown access to defines that should only be used internally here.

  Renamed all to do w/ 'EXTRA' and 'extraSerialization' to instead talk
  about metadata saved to caches; was unclear previously what EXTRA was
  about.

  Renamed static final declarations as all uppercase.

  (readBlockDataInternal): Redid. Couldn't make sense of it previously.
  Undid heavy-duty parse of header by constructing HFileBlock. Other
  cleanups. Its 1/3rd the length it used to be. More to do in here.


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

Branch: refs/heads/hbase-12439
Commit: 000117ad9fd7eb59074c9bb0da2cf1f9544d4bed
Parents: ef94b55
Author: stack <st...@apache.org>
Authored: Thu Mar 17 11:18:06 2016 -0700
Committer: stack <st...@apache.org>
Committed: Tue Mar 22 13:42:38 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hbase/io/hfile/BlockType.java |   4 +
 .../hbase/io/hfile/HFileContextBuilder.java     |  20 +
 .../org/apache/hadoop/hbase/nio/ByteBuff.java   |   6 +
 .../hbase/io/hfile/MemcachedBlockCache.java     |   2 +-
 .../hadoop/hbase/io/hfile/ChecksumUtil.java     |   5 +-
 .../hadoop/hbase/io/hfile/HFileBlock.java       | 997 +++++++++----------
 .../hadoop/hbase/io/hfile/HFileBlockIndex.java  |   2 +-
 .../hadoop/hbase/io/hfile/HFileReaderImpl.java  |  26 +-
 .../hadoop/hbase/io/hfile/HFileScanner.java     |  12 +
 .../hbase/io/hfile/bucket/BucketCache.java      |  15 +-
 .../hbase/regionserver/KeyValueScanner.java     |  12 +-
 .../hadoop/hbase/regionserver/StoreFile.java    |   4 +-
 .../hadoop/hbase/io/hfile/CacheTestUtils.java   |  23 +-
 .../hadoop/hbase/io/hfile/TestCacheOnWrite.java |  10 +-
 .../hadoop/hbase/io/hfile/TestChecksum.java     |  27 +-
 .../hadoop/hbase/io/hfile/TestHFileBlock.java   |  27 +-
 .../io/hfile/TestHFileBlockCompatibility.java   | 750 --------------
 .../hbase/io/hfile/TestHFileBlockIndex.java     |   3 +-
 .../io/hfile/TestHFileDataBlockEncoder.java     |  10 +-
 .../hbase/io/hfile/TestHFileEncryption.java     |   2 +-
 .../hbase/io/hfile/TestHFileWriterV3.java       |   7 +-
 .../hadoop/hbase/io/hfile/TestPrefetch.java     |   9 +-
 .../regionserver/TestCacheOnWriteInSchema.java  |   8 +-
 23 files changed, 607 insertions(+), 1374 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/000117ad/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java
index 4228f57..32eb0b2 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java
@@ -132,6 +132,10 @@ public enum BlockType {
     out.write(magic);
   }
 
+  public void write(ByteBuffer buf) {
+    buf.put(magic);
+  }
+
   public void write(ByteBuff buf) {
     buf.put(magic);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/000117ad/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContextBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContextBuilder.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContextBuilder.java
index 6d3bb13..a6645a6 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContextBuilder.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContextBuilder.java
@@ -55,6 +55,26 @@ public class HFileContextBuilder {
 
   private String hfileName = null;
 
+  public HFileContextBuilder() {}
+
+  /**
+   * Use this constructor if you want to change a few settings only in another context.
+   */
+  public HFileContextBuilder(final HFileContext hfc) {
+    this.usesHBaseChecksum = hfc.isUseHBaseChecksum();
+    this.includesMvcc = hfc.isIncludesMvcc();
+    this.includesTags = hfc.isIncludesTags();
+    this.compression = hfc.getCompression();
+    this.compressTags = hfc.isCompressTags();
+    this.checksumType = hfc.getChecksumType();
+    this.bytesPerChecksum = hfc.getBytesPerChecksum();
+    this.blocksize = hfc.getBlocksize();
+    this.encoding = hfc.getDataBlockEncoding();
+    this.cryptoContext = hfc.getEncryptionContext();
+    this.fileCreateTime = hfc.getFileCreateTime();
+    this.hfileName = hfc.getHFileName();
+  }
+
   public HFileContextBuilder withHBaseCheckSum(boolean useHBaseCheckSum) {
     this.usesHBaseChecksum = useHBaseCheckSum;
     return this;

http://git-wip-us.apache.org/repos/asf/hbase/blob/000117ad/hbase-common/src/main/java/org/apache/hadoop/hbase/nio/ByteBuff.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/nio/ByteBuff.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/nio/ByteBuff.java
index 1e0e957..183a031 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/nio/ByteBuff.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/nio/ByteBuff.java
@@ -496,6 +496,12 @@ public abstract class ByteBuff {
     return -(low + 1); // key not found.
   }
 
+  @Override
+  public String toString() {
+    return this.getClass().getSimpleName() + "[pos=" + position() + ", lim=" + limit() +
+        ", cap= " + capacity() + "]";
+  }
+
   public static String toStringBinary(final ByteBuff b, int off, int len) {
     StringBuilder result = new StringBuilder();
     // Just in case we are passed a 'len' that is > buffer length...

http://git-wip-us.apache.org/repos/asf/hbase/blob/000117ad/hbase-external-blockcache/src/main/java/org/apache/hadoop/hbase/io/hfile/MemcachedBlockCache.java
----------------------------------------------------------------------
diff --git a/hbase-external-blockcache/src/main/java/org/apache/hadoop/hbase/io/hfile/MemcachedBlockCache.java b/hbase-external-blockcache/src/main/java/org/apache/hadoop/hbase/io/hfile/MemcachedBlockCache.java
index 536872e..ae871c4 100644
--- a/hbase-external-blockcache/src/main/java/org/apache/hadoop/hbase/io/hfile/MemcachedBlockCache.java
+++ b/hbase-external-blockcache/src/main/java/org/apache/hadoop/hbase/io/hfile/MemcachedBlockCache.java
@@ -260,7 +260,7 @@ public class MemcachedBlockCache implements BlockCache {
     public HFileBlock decode(CachedData d) {
       try {
         ByteBuff buf = new SingleByteBuff(ByteBuffer.wrap(d.getData()));
-        return (HFileBlock) HFileBlock.blockDeserializer.deserialize(buf, true,
+        return (HFileBlock) HFileBlock.BLOCK_DESERIALIZER.deserialize(buf, true,
           MemoryType.EXCLUSIVE);
       } catch (IOException e) {
         LOG.warn("Error deserializing data from memcached",e);

http://git-wip-us.apache.org/repos/asf/hbase/blob/000117ad/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ChecksumUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ChecksumUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ChecksumUtil.java
index 69f4330..b0b1714 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ChecksumUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ChecksumUtil.java
@@ -91,7 +91,7 @@ public class ChecksumUtil {
 
     // If this is an older version of the block that does not have
     // checksums, then return false indicating that checksum verification
-    // did not succeed. Actually, this methiod should never be called
+    // did not succeed. Actually, this method should never be called
     // when the minorVersion is 0, thus this is a defensive check for a
     // cannot-happen case. Since this is a cannot-happen case, it is
     // better to return false to indicate a checksum validation failure.
@@ -141,8 +141,7 @@ public class ChecksumUtil {
    * @return The number of bytes needed to store the checksum values
    */
   static long numBytes(long datasize, int bytesPerChecksum) {
-    return numChunks(datasize, bytesPerChecksum) *
-                     HFileBlock.CHECKSUM_SIZE;
+    return numChunks(datasize, bytesPerChecksum) * HFileBlock.CHECKSUM_SIZE;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/000117ad/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
index 6268f2e..f3402da 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
@@ -56,50 +56,131 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 
 /**
- * Reads {@link HFile} version 1 and version 2 blocks but writes version 2 blocks only.
- * Version 2 was introduced in hbase-0.92.0. Does read and write out to the filesystem but also
- * the read and write to Cache.
+ * Reads {@link HFile} version 2 blocks to HFiles and via {@link Cacheable} Interface to caches.
+ * Version 2 was introduced in hbase-0.92.0. No longer has support for version 1 blocks since
+ * hbase-1.3.0.
+ *
+ * <p>Version 1 was the original file block. Version 2 was introduced when we changed the hbase file
+ * format to support multi-level block indexes and compound bloom filters (HBASE-3857).
  *
- * <h3>HFileBlock: Version 1</h3>
- * As of this writing, there should be no more version 1 blocks found out in the wild. Version 2
- * as introduced in hbase-0.92.0.
- * In version 1 all blocks are always compressed or uncompressed, as
- * specified by the {@link HFile}'s compression algorithm, with a type-specific
- * magic record stored in the beginning of the compressed data (i.e. one needs
- * to uncompress the compressed block to determine the block type). There is
- * only a single compression algorithm setting for all blocks. Offset and size
- * information from the block index are required to read a block.
  * <h3>HFileBlock: Version 2</h3>
  * In version 2, a block is structured as follows:
  * <ul>
- * <li><b>Header:</b> See Writer#putHeader(); header total size is HFILEBLOCK_HEADER_SIZE)
+ * <li><b>Header:</b> See Writer#putHeader() for where header is written; header total size is
+ * HFILEBLOCK_HEADER_SIZE
  * <ul>
- * <li>Magic record identifying the {@link BlockType} (8 bytes): e.g. <code>DATABLK*</code>
- * <li>Compressed -- a.k.a 'on disk' -- block size, excluding header, but including
- *     tailing checksum bytes (4 bytes)
- * <li>Uncompressed block size, excluding header, and excluding checksum bytes (4 bytes)
- * <li>The offset of the previous block of the same type (8 bytes). This is
+ * <li>0. blockType: Magic record identifying the {@link BlockType} (8 bytes):
+ * e.g. <code>DATABLK*</code>
+ * <li>1. onDiskSizeWithoutHeader: Compressed -- a.k.a 'on disk' -- block size, excluding header,
+ * but including tailing checksum bytes (4 bytes)
+ * <li>2. uncompressedSizeWithoutHeader: Uncompressed block size, excluding header, and excluding
+ * checksum bytes (4 bytes)
+ * <li>3. prevBlockOffset: The offset of the previous block of the same type (8 bytes). This is
  * used to navigate to the previous block without having to go to the block index
- * <li>For minorVersions &gt;=1, the ordinal describing checksum type (1 byte)
- * <li>For minorVersions &gt;=1, the number of data bytes/checksum chunk (4 bytes)
- * <li>For minorVersions &gt;=1, the size of data 'on disk', including header,
- * excluding checksums (4 bytes)
+ * <li>4: For minorVersions &gt;=1, the ordinal describing checksum type (1 byte)
+ * <li>5: For minorVersions &gt;=1, the number of data bytes/checksum chunk (4 bytes)
+ * <li>6: onDiskDataSizeWithHeader: For minorVersions &gt;=1, the size of data 'on disk', including
+ * header, excluding checksums (4 bytes)
  * </ul>
  * </li>
- * <li><b>Raw/Compressed/Encrypted/Encoded data:</b> The compression algorithm is the
- * same for all the blocks in the {@link HFile}, similarly to what was done in
- * version 1. If compression is NONE, this is just raw, serialized Cells.
+ * <li><b>Raw/Compressed/Encrypted/Encoded data:</b> The compression
+ * algorithm is the same for all the blocks in an {@link HFile}. If compression is NONE, this is
+ * just raw, serialized Cells.
  * <li><b>Tail:</b> For minorVersions &gt;=1, a series of 4 byte checksums, one each for
  * the number of bytes specified by bytesPerChecksum.
  * </ul>
- * <p>Be aware that when we read from HDFS, we overread pulling in the next blocks' header too.
- * We do this to save having to do two seeks to read an HFileBlock; a seek to read the header
- * to figure lengths, etc., and then another seek to pull in the data.
+ *
+ * <h3>Caching</h3>
+ * Caches cache whole blocks with trailing checksums if any. We then tag on some metadata, the
+ * content of BLOCK_METADATA_SPACE which will be flag on if we are doing 'hbase'
+ * checksums and then the offset into the file which is needed when we re-make a cache key
+ * when we return the block to the cache as 'done'. See {@link Cacheable#serialize(ByteBuffer)} and
+ * {@link Cacheable#getDeserializer()}.
+ *
+ * <p>TODO: Should we cache the checksums? Down in Writer#getBlockForCaching(CacheConfig) where
+ * we make a block to cache-on-write, there is an attempt at turning off checksums. This is not the
+ * only place we get blocks to cache. We also will cache the raw return from an hdfs read. In this
+ * case, the checksums may be present. If the cache is backed by something that doesn't do ECC,
+ * say an SSD, we might want to preserve checksums. For now this is open question.
+ * <p>TODO: Over in BucketCache, we save a block allocation by doing a custom serialization.
+ * Be sure to change it if serialization changes in here. Could we add a method here that takes an
+ * IOEngine and that then serializes to it rather than expose our internals over in BucketCache?
+ * IOEngine is in the bucket subpackage. Pull it up? Then this class knows about bucketcache. Ugh.
  */
 @InterfaceAudience.Private
 public class HFileBlock implements Cacheable {
   private static final Log LOG = LogFactory.getLog(HFileBlock.class);
 
+  /** Type of block. Header field 0. */
+  private BlockType blockType;
+
+  /**
+   * Size on disk excluding header, including checksum. Header field 1.
+   * @see Writer#putHeader(byte[], int, int, int, int)
+   */
+  private int onDiskSizeWithoutHeader;
+
+  /**
+   * Size of pure data. Does not include header or checksums. Header field 2.
+   * @see Writer#putHeader(byte[], int, int, int, int)
+   */
+  private int uncompressedSizeWithoutHeader;
+
+  /**
+   * The offset of the previous block on disk. Header field 3.
+   * @see Writer#putHeader(byte[], int, int, int, int)
+   */
+  private long prevBlockOffset;
+
+  /**
+   * Size on disk of header + data. Excludes checksum. Header field 6,
+   * OR calculated from {@link #onDiskSizeWithoutHeader} when using HDFS checksum.
+   * @see Writer#putHeader(byte[], int, int, int, int)
+   */
+  private int onDiskDataSizeWithHeader;
+
+
+  /**
+   * The in-memory representation of the hfile block. Can be on or offheap. Can be backed by
+   * a single ByteBuffer or by many. Make no assumptions.
+   *
+   * <p>Be careful reading from this <code>buf</code>. Duplicate and work on the duplicate or if
+   * not, be sure to reset position and limit else trouble down the road.
+   *
+   * <p>TODO: Make this read-only once made.
+   *
+   * <p>We are using the ByteBuff type. ByteBuffer is not extensible yet we need to be able to have
+   * a ByteBuffer-like API across multiple ByteBuffers reading from a cache such as BucketCache.
+   * So, we have this ByteBuff type. Unfortunately, it is spread all about HFileBlock. Would be
+   * good if could be confined to cache-use only but hard-to-do.
+   */
+  private ByteBuff buf;
+
+  /** Meta data that holds meta information on the hfileblock.
+   */
+  private HFileContext fileContext;
+
+  /**
+   * The offset of this block in the file. Populated by the reader for
+   * convenience of access. This offset is not part of the block header.
+   */
+  private long offset = UNSET;
+
+  private MemoryType memType = MemoryType.EXCLUSIVE;
+
+  /**
+   * The on-disk size of the next block, including the header and checksums if present, obtained by
+   * peeking into the first {@link HConstants#HFILEBLOCK_HEADER_SIZE} bytes of the next block's
+   * header, or UNSET if unknown.
+   *
+   * Blocks try to carry the size of the next block to read in this data member. They will even have
+   * this value when served from cache. Could save a seek in the case where we are iterating through
+   * a file and some of the blocks come from cache. If from cache, then having this info to hand
+   * will save us doing a seek to read the header so we can read the body of a block.
+   * TODO: see how effective this is at saving seeks.
+   */
+  private int nextBlockOnDiskSize = UNSET;
+
   /**
    * On a checksum failure, do these many succeeding read requests using hdfs checksums before
    * auto-reenabling hbase checksum verification.
@@ -115,14 +196,18 @@ public class HFileBlock implements Cacheable {
       (int)ClassSize.estimateBase(MultiByteBuff.class, false);
 
   /**
-   * See #blockDeserializer method for more info.
-   * 13 bytes of extra stuff stuck on the end of the HFileBlock that we pull in from HDFS (note,
+   * Space for metadata on a block that gets stored along with the block when we cache it.
+   * There are a few bytes stuck on the end of the HFileBlock that we pull in from HDFS (note,
    * when we read from HDFS, we pull in an HFileBlock AND the header of the next block if one).
-   * The 13 bytes are: usesHBaseChecksum (1 byte) + offset of this block (long) +
-   * nextBlockOnDiskSizeWithHeader (int).
+   * 8 bytes are offset of this block (long) in the file. Offset is important because
+   * used when we remake the CacheKey when we return the block to cache when done. There is also
+   * a flag on whether checksumming is being done by hbase or not. See class comment for note on
+   * uncertain state of checksumming of blocks that come out of cache (should we or should we not?).
+   * Finally there 4 bytes to hold the length of the next block which can save a seek on occasion.
+   * <p>This EXTRA came in with original commit of the bucketcache, HBASE-7404. Was formerly
+   * known as EXTRA_SERIALIZATION_SPACE.
    */
-  public static final int EXTRA_SERIALIZATION_SPACE =
-      Bytes.SIZEOF_BYTE + Bytes.SIZEOF_INT + Bytes.SIZEOF_LONG;
+  static final int BLOCK_METADATA_SPACE = Bytes.SIZEOF_BYTE + Bytes.SIZEOF_LONG + Bytes.SIZEOF_INT;
 
   /**
    * Each checksum value is an integer that can be stored in 4 bytes.
@@ -135,57 +220,47 @@ public class HFileBlock implements Cacheable {
   /**
    * Used deserializing blocks from Cache.
    *
-   * Serializing to cache is a little hard to follow. See Writer#finishBlock for where it is done.
-   * When we start to append to a new HFileBlock,
-   * we skip over where the header should go before we start adding Cells. When the block is
-   * done, we'll then go back and fill in the header and the checksum tail. Be aware that what
-   * gets serialized into the blockcache is a byte array that contains an HFileBlock followed by
-   * its checksums and then the header of the next HFileBlock (needed to help navigate), followed
-   * again by an extra 13 bytes of meta info needed when time to recreate the HFileBlock from cache.
-   *
+   * <code>
    * ++++++++++++++
    * + HFileBlock +
    * ++++++++++++++
-   * + Checksums  +
-   * ++++++++++++++
-   * + NextHeader +
+   * + Checksums  + <= Optional
    * ++++++++++++++
-   * + ExtraMeta! +
+   * + Metadata!  +
    * ++++++++++++++
-   *
-   * TODO: Fix it so we do NOT put the NextHeader into blockcache. It is not necessary.
+   * </code>
+   * @see #serialize(ByteBuffer)
    */
-  static final CacheableDeserializer<Cacheable> blockDeserializer =
+  static final CacheableDeserializer<Cacheable> BLOCK_DESERIALIZER =
       new CacheableDeserializer<Cacheable>() {
         public HFileBlock deserialize(ByteBuff buf, boolean reuse, MemoryType memType)
         throws IOException {
-          // Rewind to just before the EXTRA_SERIALIZATION_SPACE.
-          buf.limit(buf.limit() - HFileBlock.EXTRA_SERIALIZATION_SPACE).rewind();
-          // Get a new buffer to pass the deserialized HFileBlock for it to 'own'.
-          ByteBuff newByteBuffer;
+          // The buf has the file block followed by block metadata.
+          // Set limit to just before the BLOCK_METADATA_SPACE then rewind.
+          buf.limit(buf.limit() - BLOCK_METADATA_SPACE).rewind();
+          // Get a new buffer to pass the HFileBlock for it to 'own'.
+          ByteBuff newByteBuff;
           if (reuse) {
-            newByteBuffer = buf.slice();
+            newByteBuff = buf.slice();
           } else {
             int len = buf.limit();
-            newByteBuffer = new SingleByteBuff(ByteBuffer.allocate(len));
-            newByteBuffer.put(0, buf, buf.position(), len);
+            newByteBuff = new SingleByteBuff(ByteBuffer.allocate(len));
+            newByteBuff.put(0, buf, buf.position(), len);
           }
-          // Read out the EXTRA_SERIALIZATION_SPACE content and shove into our HFileBlock.
+          // Read out the BLOCK_METADATA_SPACE content and shove into our HFileBlock.
           buf.position(buf.limit());
-          buf.limit(buf.limit() + HFileBlock.EXTRA_SERIALIZATION_SPACE);
+          buf.limit(buf.limit() + HFileBlock.BLOCK_METADATA_SPACE);
           boolean usesChecksum = buf.get() == (byte)1;
-          HFileBlock hFileBlock = new HFileBlock(newByteBuffer, usesChecksum, memType);
-          hFileBlock.offset = buf.getLong();
-          hFileBlock.nextBlockOnDiskSizeWithHeader = buf.getInt();
-          if (hFileBlock.hasNextBlockHeader()) {
-            hFileBlock.buf.limit(hFileBlock.buf.limit() - hFileBlock.headerSize());
-          }
+          long offset = buf.getLong();
+          int nextBlockOnDiskSize = buf.getInt();
+          HFileBlock hFileBlock =
+              new HFileBlock(newByteBuff, usesChecksum, memType, offset, nextBlockOnDiskSize, null);
           return hFileBlock;
         }
 
         @Override
         public int getDeserialiserIdentifier() {
-          return deserializerIdentifier;
+          return DESERIALIZER_IDENTIFIER;
         }
 
         @Override
@@ -195,65 +270,36 @@ public class HFileBlock implements Cacheable {
         }
       };
 
-  private static final int deserializerIdentifier;
+  private static final int DESERIALIZER_IDENTIFIER;
   static {
-    deserializerIdentifier = CacheableDeserializerIdManager
-        .registerDeserializer(blockDeserializer);
+    DESERIALIZER_IDENTIFIER =
+        CacheableDeserializerIdManager.registerDeserializer(BLOCK_DESERIALIZER);
   }
 
-  /** Type of block. Header field 0. */
-  private BlockType blockType;
-
-  /**
-   * Size on disk excluding header, including checksum. Header field 1.
-   * @see Writer#putHeader(byte[], int, int, int, int)
-   */
-  private int onDiskSizeWithoutHeader;
-
-  /**
-   * Size of pure data. Does not include header or checksums. Header field 2.
-   * @see Writer#putHeader(byte[], int, int, int, int)
-   */
-  private final int uncompressedSizeWithoutHeader;
-
-  /**
-   * The offset of the previous block on disk. Header field 3.
-   * @see Writer#putHeader(byte[], int, int, int, int)
-   */
-  private final long prevBlockOffset;
-
-  /**
-   * Size on disk of header + data. Excludes checksum. Header field 6,
-   * OR calculated from {@link #onDiskSizeWithoutHeader} when using HDFS checksum.
-   * @see Writer#putHeader(byte[], int, int, int, int)
-   */
-  private final int onDiskDataSizeWithHeader;
-
-  /** The in-memory representation of the hfile block */
-  private ByteBuff buf;
-
-  /** Meta data that holds meta information on the hfileblock */
-  private HFileContext fileContext;
-
   /**
-   * The offset of this block in the file. Populated by the reader for
-   * convenience of access. This offset is not part of the block header.
-   */
-  private long offset = UNSET;
-
-  /**
-   * The on-disk size of the next block, including the header, obtained by
-   * peeking into the first {@link HConstants#HFILEBLOCK_HEADER_SIZE} bytes of the next block's
-   * header, or -1 if unknown.
+   * Copy constructor. Creates a shallow copy of {@code that}'s buffer.
    */
-  private int nextBlockOnDiskSizeWithHeader = UNSET;
-
-  private MemoryType memType = MemoryType.EXCLUSIVE;
+  private HFileBlock(HFileBlock that) {
+    this.blockType = that.blockType;
+    this.onDiskSizeWithoutHeader = that.onDiskSizeWithoutHeader;
+    this.uncompressedSizeWithoutHeader = that.uncompressedSizeWithoutHeader;
+    this.prevBlockOffset = that.prevBlockOffset;
+    this.buf = that.buf.duplicate();
+    this.offset = that.offset;
+    this.onDiskDataSizeWithHeader = that.onDiskDataSizeWithHeader;
+    this.fileContext = that.fileContext;
+    this.nextBlockOnDiskSize = that.nextBlockOnDiskSize;
+  }
 
   /**
    * Creates a new {@link HFile} block from the given fields. This constructor
    * is used when the block data has already been read and uncompressed,
-   * and is sitting in a byte buffer.
+   * and is sitting in a byte buffer and we want to stuff the block into cache.
+   * See {@link Writer#getBlockForCaching(CacheConfig)}.
+   *
+   * <p>TODO: The caller presumes no checksumming
+   * required of this block instance since going into cache; checksum already verified on
+   * underlying block data pulled in from filesystem. Is that correct? What if cache is SSD?
    *
    * @param blockType the type of this block, see {@link BlockType}
    * @param onDiskSizeWithoutHeader see {@link #onDiskSizeWithoutHeader}
@@ -267,86 +313,94 @@ public class HFileBlock implements Cacheable {
    * @param fileContext HFile meta data
    */
   HFileBlock(BlockType blockType, int onDiskSizeWithoutHeader, int uncompressedSizeWithoutHeader,
-      long prevBlockOffset, ByteBuff buf, boolean fillHeader, long offset,
-      int onDiskDataSizeWithHeader, HFileContext fileContext) {
-    this.blockType = blockType;
-    this.onDiskSizeWithoutHeader = onDiskSizeWithoutHeader;
-    this.uncompressedSizeWithoutHeader = uncompressedSizeWithoutHeader;
-    this.prevBlockOffset = prevBlockOffset;
-    this.buf = buf;
-    this.offset = offset;
-    this.onDiskDataSizeWithHeader = onDiskDataSizeWithHeader;
-    this.fileContext = fileContext;
+      long prevBlockOffset, ByteBuffer b, boolean fillHeader, long offset,
+      final int nextBlockOnDiskSize, int onDiskDataSizeWithHeader, HFileContext fileContext) {
+    init(blockType, onDiskSizeWithoutHeader, uncompressedSizeWithoutHeader,
+        prevBlockOffset, offset, onDiskDataSizeWithHeader, nextBlockOnDiskSize, fileContext);
+    this.buf = new SingleByteBuff(b);
     if (fillHeader) {
       overwriteHeader();
     }
     this.buf.rewind();
   }
 
-  HFileBlock(BlockType blockType, int onDiskSizeWithoutHeader, int uncompressedSizeWithoutHeader,
-      long prevBlockOffset, ByteBuffer buf, boolean fillHeader, long offset,
-      int onDiskDataSizeWithHeader, HFileContext fileContext) {
-    this(blockType, onDiskSizeWithoutHeader, uncompressedSizeWithoutHeader, prevBlockOffset,
-        new SingleByteBuff(buf), fillHeader, offset, onDiskDataSizeWithHeader, fileContext);
-  }
-
   /**
-   * Copy constructor. Creates a shallow copy of {@code that}'s buffer.
+   * Creates a block from an existing buffer starting with a header. Rewinds
+   * and takes ownership of the buffer. By definition of rewind, ignores the
+   * buffer position, but if you slice the buffer beforehand, it will rewind
+   * to that point.
+   * @param buf Has header, content, and trailing checksums if present.
    */
-  HFileBlock(HFileBlock that) {
-    this.blockType = that.blockType;
-    this.onDiskSizeWithoutHeader = that.onDiskSizeWithoutHeader;
-    this.uncompressedSizeWithoutHeader = that.uncompressedSizeWithoutHeader;
-    this.prevBlockOffset = that.prevBlockOffset;
-    this.buf = that.buf.duplicate();
-    this.offset = that.offset;
-    this.onDiskDataSizeWithHeader = that.onDiskDataSizeWithHeader;
-    this.fileContext = that.fileContext;
-    this.nextBlockOnDiskSizeWithHeader = that.nextBlockOnDiskSizeWithHeader;
+  HFileBlock(ByteBuff buf, boolean usesHBaseChecksum, MemoryType memType, final long offset,
+      final int nextBlockOnDiskSize, HFileContext fileContext) throws IOException {
+    buf.rewind();
+    final BlockType blockType = BlockType.read(buf);
+    final int onDiskSizeWithoutHeader = buf.getInt();
+    final int uncompressedSizeWithoutHeader = buf.getInt();
+    final long prevBlockOffset = buf.getLong();
+    byte checksumType = buf.get();
+    int bytesPerChecksum = buf.getInt();
+    int onDiskDataSizeWithHeader = buf.getInt();
+    // This constructor is called when we deserialize a block from cache and when we read a block in
+    // from the fs. fileCache is null when deserialized from cache so need to make up one.
+    HFileContextBuilder fileContextBuilder = fileContext != null?
+        new HFileContextBuilder(fileContext): new HFileContextBuilder();
+    fileContextBuilder.withHBaseCheckSum(usesHBaseChecksum);
+    if (usesHBaseChecksum) {
+      // Use the checksum type and bytes per checksum from header, not from filecontext.
+      fileContextBuilder.withChecksumType(ChecksumType.codeToType(checksumType));
+      fileContextBuilder.withBytesPerCheckSum(bytesPerChecksum);
+    } else {
+      fileContextBuilder.withChecksumType(ChecksumType.NULL);
+      fileContextBuilder.withBytesPerCheckSum(0);
+      // Need to fix onDiskDataSizeWithHeader; there are not checksums after-block-data
+      onDiskDataSizeWithHeader = onDiskSizeWithoutHeader + headerSize(usesHBaseChecksum);
+    }
+    fileContext = fileContextBuilder.build();
+    assert usesHBaseChecksum == fileContext.isUseHBaseChecksum();
+    init(blockType, onDiskSizeWithoutHeader, uncompressedSizeWithoutHeader,
+        prevBlockOffset, offset, onDiskDataSizeWithHeader, nextBlockOnDiskSize, fileContext);
+    this.memType = memType;
+    this.offset = offset;
+    this.buf = buf;
+    this.buf.rewind();
   }
 
-  HFileBlock(ByteBuffer b, boolean usesHBaseChecksum) throws IOException {
-    this(new SingleByteBuff(b), usesHBaseChecksum);
+  /**
+   * Called from constructors.
+   */
+  private void init(BlockType blockType, int onDiskSizeWithoutHeader,
+      int uncompressedSizeWithoutHeader, long prevBlockOffset,
+      long offset, int onDiskDataSizeWithHeader, final int nextBlockOnDiskSize,
+      HFileContext fileContext) {
+    this.blockType = blockType;
+    this.onDiskSizeWithoutHeader = onDiskSizeWithoutHeader;
+    this.uncompressedSizeWithoutHeader = uncompressedSizeWithoutHeader;
+    this.prevBlockOffset = prevBlockOffset;
+    this.offset = offset;
+    this.onDiskDataSizeWithHeader = onDiskDataSizeWithHeader;
+    this.nextBlockOnDiskSize = nextBlockOnDiskSize;
+    this.fileContext = fileContext;
   }
 
   /**
-   * Creates a block from an existing buffer starting with a header. Rewinds
-   * and takes ownership of the buffer. By definition of rewind, ignores the
-   * buffer position, but if you slice the buffer beforehand, it will rewind
-   * to that point.
+   * Parse total ondisk size including header and checksum. Its second field in header after
+   * the magic bytes.
+   * @param headerBuf Header ByteBuffer. Presumed exact size of header.
+   * @return Size of the block with header included.
    */
-  HFileBlock(ByteBuff b, boolean usesHBaseChecksum) throws IOException {
-    this(b, usesHBaseChecksum, MemoryType.EXCLUSIVE);
+  private static int getOnDiskSizeWithHeader(final ByteBuffer headerBuf) {
+    // Set hbase checksum to true always calling headerSize.
+    return headerBuf.getInt(BlockType.MAGIC_LENGTH) + headerSize(true);
   }
 
   /**
-   * Creates a block from an existing buffer starting with a header. Rewinds
-   * and takes ownership of the buffer. By definition of rewind, ignores the
-   * buffer position, but if you slice the buffer beforehand, it will rewind
-   * to that point.
+   * @return the on-disk size of the next block (including the header size and any checksums if
+   * present) read by peeking into the next block's header; use as a hint when doing
+   * a read of the next block when scanning or running over a file.
    */
-  HFileBlock(ByteBuff b, boolean usesHBaseChecksum, MemoryType memType) throws IOException {
-    b.rewind();
-    blockType = BlockType.read(b);
-    onDiskSizeWithoutHeader = b.getInt();
-    uncompressedSizeWithoutHeader = b.getInt();
-    prevBlockOffset = b.getLong();
-    HFileContextBuilder contextBuilder = new HFileContextBuilder();
-    contextBuilder.withHBaseCheckSum(usesHBaseChecksum);
-    if (usesHBaseChecksum) {
-      contextBuilder.withChecksumType(ChecksumType.codeToType(b.get()));
-      contextBuilder.withBytesPerCheckSum(b.getInt());
-      this.onDiskDataSizeWithHeader = b.getInt();
-    } else {
-      contextBuilder.withChecksumType(ChecksumType.NULL);
-      contextBuilder.withBytesPerCheckSum(0);
-      this.onDiskDataSizeWithHeader =
-          onDiskSizeWithoutHeader + HConstants.HFILEBLOCK_HEADER_SIZE_NO_CHECKSUM;
-    }
-    this.fileContext = contextBuilder.build();
-    this.memType = memType;
-    buf = b;
-    buf.rewind();
+  public int getNextBlockOnDiskSize() {
+    return nextBlockOnDiskSize;
   }
 
   public BlockType getBlockType() {
@@ -414,49 +468,26 @@ public class HFileBlock implements Cacheable {
    * @return the buffer with header skipped and checksum omitted.
    */
   public ByteBuff getBufferWithoutHeader() {
-    ByteBuff dup = this.buf.duplicate();
-    dup.position(headerSize());
-    dup.limit(buf.limit() - totalChecksumBytes());
-    return dup.slice();
+    ByteBuff dup = getBufferReadOnly();
+    // Now set it up so Buffer spans content only -- no header or no checksums.
+    return dup.position(headerSize()).limit(buf.limit() - totalChecksumBytes()).slice();
   }
 
   /**
-   * Returns the buffer this block stores internally. The clients must not
-   * modify the buffer object. This method has to be public because it is used
+   * Returns a read-only duplicate of the buffer this block stores internally ready to be read.
+   * Clients must not modify the buffer object though they may set position and limit on the
+   * returned buffer since we pass back a duplicate. This method has to be public because it is used
    * in {@link CompoundBloomFilter} to avoid object creation on every Bloom
-   * filter lookup, but has to be used with caution. Checksum data is not
-   * included in the returned buffer but header data is.
+   * filter lookup, but has to be used with caution. Buffer holds header, block content,
+   * and any follow-on checksums if present.
    *
    * @return the buffer of this block for read-only operations
    */
-  ByteBuff getBufferReadOnly() {
-    ByteBuff dup = this.buf.duplicate();
-    dup.limit(buf.limit() - totalChecksumBytes());
-    return dup.slice();
-  }
-
-  /**
-   * Returns the buffer of this block, including header data. The clients must
-   * not modify the buffer object. This method has to be public because it is
-   * used in {@link org.apache.hadoop.hbase.io.hfile.bucket.BucketCache} to avoid buffer copy.
-   *
-   * @return the buffer with header and checksum included for read-only operations
-   */
-  public ByteBuff getBufferReadOnlyWithHeader() {
+  public ByteBuff getBufferReadOnly() {
+    // TODO: ByteBuf does not support asReadOnlyBuffer(). Fix.
     ByteBuff dup = this.buf.duplicate();
-    return dup.slice();
-  }
-
-  /**
-   * Returns a byte buffer of this block, including header data and checksum, positioned at
-   * the beginning of header. The underlying data array is not copied.
-   *
-   * @return the byte buffer with header and checksum included
-   */
-  ByteBuff getBufferWithHeader() {
-    ByteBuff dupBuf = buf.duplicate();
-    dupBuf.rewind();
-    return dupBuf;
+    assert dup.position() == 0;
+    return dup;
   }
 
   private void sanityCheckAssertion(long valueFromBuf, long valueFromField,
@@ -481,39 +512,38 @@ public class HFileBlock implements Cacheable {
    * valid header consistent with the fields. Assumes a packed block structure.
    * This function is primary for testing and debugging, and is not
    * thread-safe, because it alters the internal buffer pointer.
+   * Used by tests only.
    */
+  @VisibleForTesting
   void sanityCheck() throws IOException {
-    buf.rewind();
-
-    sanityCheckAssertion(BlockType.read(buf), blockType);
+    // Duplicate so no side-effects
+    ByteBuff dup = this.buf.duplicate().rewind();
+    sanityCheckAssertion(BlockType.read(dup), blockType);
 
-    sanityCheckAssertion(buf.getInt(), onDiskSizeWithoutHeader,
-        "onDiskSizeWithoutHeader");
+    sanityCheckAssertion(dup.getInt(), onDiskSizeWithoutHeader, "onDiskSizeWithoutHeader");
 
-    sanityCheckAssertion(buf.getInt(), uncompressedSizeWithoutHeader,
+    sanityCheckAssertion(dup.getInt(), uncompressedSizeWithoutHeader,
         "uncompressedSizeWithoutHeader");
 
-    sanityCheckAssertion(buf.getLong(), prevBlockOffset, "prevBlocKOffset");
+    sanityCheckAssertion(dup.getLong(), prevBlockOffset, "prevBlockOffset");
     if (this.fileContext.isUseHBaseChecksum()) {
-      sanityCheckAssertion(buf.get(), this.fileContext.getChecksumType().getCode(), "checksumType");
-      sanityCheckAssertion(buf.getInt(), this.fileContext.getBytesPerChecksum(),
+      sanityCheckAssertion(dup.get(), this.fileContext.getChecksumType().getCode(), "checksumType");
+      sanityCheckAssertion(dup.getInt(), this.fileContext.getBytesPerChecksum(),
           "bytesPerChecksum");
-      sanityCheckAssertion(buf.getInt(), onDiskDataSizeWithHeader, "onDiskDataSizeWithHeader");
+      sanityCheckAssertion(dup.getInt(), onDiskDataSizeWithHeader, "onDiskDataSizeWithHeader");
     }
 
     int cksumBytes = totalChecksumBytes();
     int expectedBufLimit = onDiskDataSizeWithHeader + cksumBytes;
-    if (buf.limit() != expectedBufLimit) {
-      throw new AssertionError("Expected buffer limit " + expectedBufLimit
-          + ", got " + buf.limit());
+    if (dup.limit() != expectedBufLimit) {
+      throw new AssertionError("Expected limit " + expectedBufLimit + ", got " + dup.limit());
     }
 
     // We might optionally allocate HFILEBLOCK_HEADER_SIZE more bytes to read the next
     // block's header, so there are two sensible values for buffer capacity.
     int hdrSize = headerSize();
-    if (buf.capacity() != expectedBufLimit &&
-        buf.capacity() != expectedBufLimit + hdrSize) {
-      throw new AssertionError("Invalid buffer capacity: " + buf.capacity() +
+    if (dup.capacity() != expectedBufLimit && dup.capacity() != expectedBufLimit + hdrSize) {
+      throw new AssertionError("Invalid buffer capacity: " + dup.capacity() +
           ", expected " + expectedBufLimit + " or " + (expectedBufLimit + hdrSize));
     }
   }
@@ -560,30 +590,6 @@ public class HFileBlock implements Cacheable {
   }
 
   /**
-   * Called after reading a block with provided onDiskSizeWithHeader.
-   */
-  private void validateOnDiskSizeWithoutHeader(int expectedOnDiskSizeWithoutHeader)
-  throws IOException {
-    if (onDiskSizeWithoutHeader != expectedOnDiskSizeWithoutHeader) {
-      String dataBegin = null;
-      if (buf.hasArray()) {
-        dataBegin = Bytes.toStringBinary(buf.array(), buf.arrayOffset(), Math.min(32, buf.limit()));
-      } else {
-        ByteBuff bufDup = getBufferReadOnly();
-        byte[] dataBeginBytes = new byte[Math.min(32, bufDup.limit() - bufDup.position())];
-        bufDup.get(dataBeginBytes);
-        dataBegin = Bytes.toStringBinary(dataBeginBytes);
-      }
-      String blockInfoMsg =
-        "Block offset: " + offset + ", data starts with: " + dataBegin;
-      throw new IOException("On-disk size without header provided is "
-          + expectedOnDiskSizeWithoutHeader + ", but block "
-          + "header contains " + onDiskSizeWithoutHeader + ". " +
-          blockInfoMsg);
-    }
-  }
-
-  /**
    * Retrieves the decompressed/decrypted view of this block. An encoded block remains in its
    * encoded structure. Internal structures are shared between instances where applicable.
    */
@@ -607,33 +613,10 @@ public class HFileBlock implements Cacheable {
     ctx.prepareDecoding(unpacked.getOnDiskSizeWithoutHeader(),
       unpacked.getUncompressedSizeWithoutHeader(), unpacked.getBufferWithoutHeader(),
       dup);
-
-    // Preserve the next block's header bytes in the new block if we have them.
-    if (unpacked.hasNextBlockHeader()) {
-      // Both the buffers are limited till checksum bytes and avoid the next block's header.
-      // Below call to copyFromBufferToBuffer() will try positional read/write from/to buffers when
-      // any of the buffer is DBB. So we change the limit on a dup buffer. No copying just create
-      // new BB objects
-      ByteBuff inDup = this.buf.duplicate();
-      inDup.limit(inDup.limit() + headerSize());
-      ByteBuff outDup = unpacked.buf.duplicate();
-      outDup.limit(outDup.limit() + unpacked.headerSize());
-      outDup.put(
-          unpacked.headerSize() + unpacked.uncompressedSizeWithoutHeader
-              + unpacked.totalChecksumBytes(), inDup, this.onDiskDataSizeWithHeader,
-          unpacked.headerSize());
-    }
     return unpacked;
   }
 
   /**
-   * Return true when this buffer includes next block's header.
-   */
-  private boolean hasNextBlockHeader() {
-    return nextBlockOnDiskSizeWithHeader > 0;
-  }
-
-  /**
    * Always allocates a new buffer of the correct size. Copies header bytes
    * from the existing buffer. Does not change header fields.
    * Reserve room to keep checksum bytes too.
@@ -641,8 +624,7 @@ public class HFileBlock implements Cacheable {
   private void allocateBuffer() {
     int cksumBytes = totalChecksumBytes();
     int headerSize = headerSize();
-    int capacityNeeded = headerSize + uncompressedSizeWithoutHeader +
-        cksumBytes + (hasNextBlockHeader() ? headerSize : 0);
+    int capacityNeeded = headerSize + uncompressedSizeWithoutHeader + cksumBytes;
 
     // TODO we need consider allocating offheap here?
     ByteBuffer newBuf = ByteBuffer.allocate(capacityNeeded);
@@ -670,9 +652,8 @@ public class HFileBlock implements Cacheable {
   }
 
   /** An additional sanity-check in case no compression or encryption is being used. */
-  public void assumeUncompressed() throws IOException {
-    if (onDiskSizeWithoutHeader != uncompressedSizeWithoutHeader +
-        totalChecksumBytes()) {
+  public void sanityCheckUncompressedSize() throws IOException {
+    if (onDiskSizeWithoutHeader != uncompressedSizeWithoutHeader + totalChecksumBytes()) {
       throw new IOException("Using no compression but "
           + "onDiskSizeWithoutHeader=" + onDiskSizeWithoutHeader + ", "
           + "uncompressedSizeWithoutHeader=" + uncompressedSizeWithoutHeader
@@ -680,11 +661,14 @@ public class HFileBlock implements Cacheable {
     }
   }
 
-  /** @return the offset of this block in the file it was read from */
+  /**
+   * Cannot be {@link #UNSET}. Must be a legitimate value. Used re-making the {@link CacheKey} when
+   * block is returned to the cache.
+   * @return the offset of this block in the file it was read from
+   */
   long getOffset() {
     if (offset < 0) {
-      throw new IllegalStateException(
-          "HFile block offset not initialized properly");
+      throw new IllegalStateException("HFile block offset not initialized properly");
     }
     return offset;
   }
@@ -744,7 +728,6 @@ public class HFileBlock implements Cacheable {
         // We could not read the "extra data", but that is OK.
         break;
       }
-
       if (ret < 0) {
         throw new IOException("Premature EOF from inputStream (read "
             + "returned " + ret + ", was trying to read " + necessaryLen
@@ -799,14 +782,6 @@ public class HFileBlock implements Cacheable {
   }
 
   /**
-   * @return the on-disk size of the next block (including the header size)
-   *         that was read by peeking into the next block's header
-   */
-  public int getNextBlockOnDiskSizeWithHeader() {
-    return nextBlockOnDiskSizeWithHeader;
-  }
-
-  /**
    * Unified version 2 {@link HFile} block writer. The intended usage pattern
    * is as follows:
    * <ol>
@@ -838,8 +813,8 @@ public class HFileBlock implements Cacheable {
     private HFileBlockDefaultEncodingContext defaultBlockEncodingCtx;
 
     /**
-     * The stream we use to accumulate data in uncompressed format for each
-     * block. We reset this stream at the end of each block and reuse it. The
+     * The stream we use to accumulate data into a block in an uncompressed format.
+     * We reset this stream at the end of each block and reuse it. The
      * header is written as the first {@link HConstants#HFILEBLOCK_HEADER_SIZE} bytes into this
      * stream.
      */
@@ -867,7 +842,7 @@ public class HFileBlock implements Cacheable {
      * if compression is turned on. It also includes the checksum data that
      * immediately follows the block data. (header + data + checksums)
      */
-    private byte[] onDiskBytesWithHeader;
+    private byte[] onDiskBlockBytesWithHeader;
 
     /**
      * The size of the checksum data on disk. It is used only if data is
@@ -884,7 +859,7 @@ public class HFileBlock implements Cacheable {
      * {@link org.apache.hadoop.hbase.HConstants#HFILEBLOCK_HEADER_SIZE}.
      * Does not store checksums.
      */
-    private byte[] uncompressedBytesWithHeader;
+    private byte[] uncompressedBlockBytesWithHeader;
 
     /**
      * Current block's start offset in the {@link HFile}. Set in
@@ -992,18 +967,19 @@ public class HFileBlock implements Cacheable {
       Preconditions.checkState(state != State.INIT,
           "Unexpected state: " + state);
 
-      if (state == State.BLOCK_READY)
+      if (state == State.BLOCK_READY) {
         return;
+      }
 
       // This will set state to BLOCK_READY.
       finishBlock();
     }
 
     /**
-     * An internal method that flushes the compressing stream (if using
-     * compression), serializes the header, and takes care of the separate
-     * uncompressed stream for caching on write, if applicable. Sets block
-     * write state to "block ready".
+     * Finish up writing of the block.
+     * Flushes the compressing stream (if using compression), fills out the header,
+     * does any compression/encryption of bytes to flush out to disk, and manages
+     * the cache on write content, if applicable. Sets block write state to "block ready".
      */
     private void finishBlock() throws IOException {
       if (blockType == BlockType.DATA) {
@@ -1012,41 +988,40 @@ public class HFileBlock implements Cacheable {
         blockType = dataBlockEncodingCtx.getBlockType();
       }
       userDataStream.flush();
-      // This does an array copy, so it is safe to cache this byte array.
+      // This does an array copy, so it is safe to cache this byte array when cache-on-write.
       // Header is still the empty, 'dummy' header that is yet to be filled out.
-      uncompressedBytesWithHeader = baosInMemory.toByteArray();
+      uncompressedBlockBytesWithHeader = baosInMemory.toByteArray();
       prevOffset = prevOffsetByType[blockType.getId()];
 
-      // We need to set state before we can package the block up for
-      // cache-on-write. In a way, the block is ready, but not yet encoded or
-      // compressed.
+      // We need to set state before we can package the block up for cache-on-write. In a way, the
+      // block is ready, but not yet encoded or compressed.
       state = State.BLOCK_READY;
       if (blockType == BlockType.DATA || blockType == BlockType.ENCODED_DATA) {
-        onDiskBytesWithHeader = dataBlockEncodingCtx
-            .compressAndEncrypt(uncompressedBytesWithHeader);
+        onDiskBlockBytesWithHeader = dataBlockEncodingCtx.
+            compressAndEncrypt(uncompressedBlockBytesWithHeader);
       } else {
-        onDiskBytesWithHeader = this.defaultBlockEncodingCtx.
-            compressAndEncrypt(uncompressedBytesWithHeader);
+        onDiskBlockBytesWithHeader = defaultBlockEncodingCtx.
+            compressAndEncrypt(uncompressedBlockBytesWithHeader);
       }
       // Calculate how many bytes we need for checksum on the tail of the block.
       int numBytes = (int) ChecksumUtil.numBytes(
-          onDiskBytesWithHeader.length,
+          onDiskBlockBytesWithHeader.length,
           fileContext.getBytesPerChecksum());
 
       // Put the header for the on disk bytes; header currently is unfilled-out
-      putHeader(onDiskBytesWithHeader, 0,
-          onDiskBytesWithHeader.length + numBytes,
-          uncompressedBytesWithHeader.length, onDiskBytesWithHeader.length);
+      putHeader(onDiskBlockBytesWithHeader, 0,
+          onDiskBlockBytesWithHeader.length + numBytes,
+          uncompressedBlockBytesWithHeader.length, onDiskBlockBytesWithHeader.length);
       // Set the header for the uncompressed bytes (for cache-on-write) -- IFF different from
-      // onDiskBytesWithHeader array.
-      if (onDiskBytesWithHeader != uncompressedBytesWithHeader) {
-        putHeader(uncompressedBytesWithHeader, 0,
-          onDiskBytesWithHeader.length + numBytes,
-          uncompressedBytesWithHeader.length, onDiskBytesWithHeader.length);
+      // onDiskBlockBytesWithHeader array.
+      if (onDiskBlockBytesWithHeader != uncompressedBlockBytesWithHeader) {
+        putHeader(uncompressedBlockBytesWithHeader, 0,
+          onDiskBlockBytesWithHeader.length + numBytes,
+          uncompressedBlockBytesWithHeader.length, onDiskBlockBytesWithHeader.length);
       }
       onDiskChecksum = new byte[numBytes];
       ChecksumUtil.generateChecksums(
-          onDiskBytesWithHeader, 0, onDiskBytesWithHeader.length,
+          onDiskBlockBytesWithHeader, 0, onDiskBlockBytesWithHeader.length,
           onDiskChecksum, 0, fileContext.getChecksumType(), fileContext.getBytesPerChecksum());
     }
 
@@ -1101,7 +1076,7 @@ public class HFileBlock implements Cacheable {
     protected void finishBlockAndWriteHeaderAndData(DataOutputStream out)
       throws IOException {
       ensureBlockReady();
-      out.write(onDiskBytesWithHeader);
+      out.write(onDiskBlockBytesWithHeader);
       out.write(onDiskChecksum);
     }
 
@@ -1120,12 +1095,12 @@ public class HFileBlock implements Cacheable {
       // This is not very optimal, because we are doing an extra copy.
       // But this method is used only by unit tests.
       byte[] output =
-          new byte[onDiskBytesWithHeader.length
+          new byte[onDiskBlockBytesWithHeader.length
               + onDiskChecksum.length];
-      System.arraycopy(onDiskBytesWithHeader, 0, output, 0,
-          onDiskBytesWithHeader.length);
+      System.arraycopy(onDiskBlockBytesWithHeader, 0, output, 0,
+          onDiskBlockBytesWithHeader.length);
       System.arraycopy(onDiskChecksum, 0, output,
-          onDiskBytesWithHeader.length, onDiskChecksum.length);
+          onDiskBlockBytesWithHeader.length, onDiskChecksum.length);
       return output;
     }
 
@@ -1153,7 +1128,7 @@ public class HFileBlock implements Cacheable {
      */
     int getOnDiskSizeWithoutHeader() {
       expectState(State.BLOCK_READY);
-      return onDiskBytesWithHeader.length +
+      return onDiskBlockBytesWithHeader.length +
           onDiskChecksum.length - HConstants.HFILEBLOCK_HEADER_SIZE;
     }
 
@@ -1166,7 +1141,7 @@ public class HFileBlock implements Cacheable {
      */
     int getOnDiskSizeWithHeader() {
       expectState(State.BLOCK_READY);
-      return onDiskBytesWithHeader.length + onDiskChecksum.length;
+      return onDiskBlockBytesWithHeader.length + onDiskChecksum.length;
     }
 
     /**
@@ -1174,7 +1149,7 @@ public class HFileBlock implements Cacheable {
      */
     int getUncompressedSizeWithoutHeader() {
       expectState(State.BLOCK_READY);
-      return uncompressedBytesWithHeader.length - HConstants.HFILEBLOCK_HEADER_SIZE;
+      return uncompressedBlockBytesWithHeader.length - HConstants.HFILEBLOCK_HEADER_SIZE;
     }
 
     /**
@@ -1182,7 +1157,7 @@ public class HFileBlock implements Cacheable {
      */
     int getUncompressedSizeWithHeader() {
       expectState(State.BLOCK_READY);
-      return uncompressedBytesWithHeader.length;
+      return uncompressedBlockBytesWithHeader.length;
     }
 
     /** @return true if a block is being written  */
@@ -1212,7 +1187,7 @@ public class HFileBlock implements Cacheable {
      */
     ByteBuffer getUncompressedBufferWithHeader() {
       expectState(State.BLOCK_READY);
-      return ByteBuffer.wrap(uncompressedBytesWithHeader);
+      return ByteBuffer.wrap(uncompressedBlockBytesWithHeader);
     }
 
     /**
@@ -1225,7 +1200,7 @@ public class HFileBlock implements Cacheable {
      */
     ByteBuffer getOnDiskBufferWithHeader() {
       expectState(State.BLOCK_READY);
-      return ByteBuffer.wrap(onDiskBytesWithHeader);
+      return ByteBuffer.wrap(onDiskBlockBytesWithHeader);
     }
 
     private void expectState(State expectedState) {
@@ -1257,6 +1232,10 @@ public class HFileBlock implements Cacheable {
      * block does not have checksum data even though the header minor
      * version is MINOR_VERSION_WITH_CHECKSUM. This is indicated by setting a
      * 0 value in bytesPerChecksum.
+     *
+     * <p>TODO: Should there be an option where a cache can ask that hbase preserve block
+     * checksums for checking after a block comes out of the cache? Otehrwise, cache is responsible
+     * for blocks being wholesome (ECC memory or if file-backed, it does checksumming).
      */
     HFileBlock getBlockForCaching(CacheConfig cacheConf) {
       HFileContext newContext = new HFileContextBuilder()
@@ -1270,13 +1249,13 @@ public class HFileBlock implements Cacheable {
                                 .withIncludesMvcc(fileContext.isIncludesMvcc())
                                 .withIncludesTags(fileContext.isIncludesTags())
                                 .build();
-      return new HFileBlock(blockType, getOnDiskSizeWithoutHeader(),
+       return new HFileBlock(blockType, getOnDiskSizeWithoutHeader(),
           getUncompressedSizeWithoutHeader(), prevOffset,
-          cacheConf.shouldCacheCompressed(blockType.getCategory()) ?
+          cacheConf.shouldCacheCompressed(blockType.getCategory())?
             getOnDiskBufferWithHeader() :
             getUncompressedBufferWithHeader(),
-          FILL_HEADER, startOffset,
-          onDiskBytesWithHeader.length + onDiskChecksum.length, newContext);
+          FILL_HEADER, startOffset, UNSET,
+          onDiskBlockBytesWithHeader.length + onDiskChecksum.length, newContext);
     }
   }
 
@@ -1322,12 +1301,9 @@ public class HFileBlock implements Cacheable {
      * @param offset
      * @param onDiskSize the on-disk size of the entire block, including all
      *          applicable headers, or -1 if unknown
-     * @param uncompressedSize the uncompressed size of the compressed part of
-     *          the block, or -1 if unknown
      * @return the newly read block
      */
-    HFileBlock readBlockData(long offset, long onDiskSize,
-        int uncompressedSize, boolean pread) throws IOException;
+    HFileBlock readBlockData(long offset, long onDiskSize, boolean pread) throws IOException;
 
     /**
      * Creates a block iterator over the given portion of the {@link HFile}.
@@ -1380,6 +1356,11 @@ public class HFileBlock implements Cacheable {
     /** Default context used when BlockType != {@link BlockType#ENCODED_DATA}. */
     private final HFileBlockDefaultDecodingContext defaultDecodingCtx;
 
+    /**
+     * When we read a block, we overread and pull in the next blocks header too. We will save it
+     * here. If moving serially through the file, we will trip over this caching of the next blocks
+     * header so we won't have to do explicit seek to find next blocks lengths, etc.
+     */
     private ThreadLocal<PrefetchedHeader> prefetchedHeaderForThread =
         new ThreadLocal<PrefetchedHeader>() {
       @Override
@@ -1443,7 +1424,7 @@ public class HFileBlock implements Cacheable {
         public HFileBlock nextBlock() throws IOException {
           if (offset >= endOffset)
             return null;
-          HFileBlock b = readBlockData(offset, -1, -1, false);
+          HFileBlock b = readBlockData(offset, -1, false);
           offset += b.getOnDiskSizeWithHeader();
           return b.unpack(fileContext, owner);
         }
@@ -1463,7 +1444,7 @@ public class HFileBlock implements Cacheable {
 
     /**
      * Does a positional read or a seek and read into the given buffer. Returns
-     * the on-disk size of the next block, or -1 if it could not be determined.
+     * the on-disk size of the next block, or -1 if it could not be read/determined; e.g. EOF.
      *
      * @param dest destination buffer
      * @param destOffset offset into the destination buffer at where to put the bytes we read
@@ -1473,7 +1454,8 @@ public class HFileBlock implements Cacheable {
      * @param pread whether we should do a positional read
      * @param istream The input source of data
      * @return the on-disk size of the next block with header size included, or
-     *         -1 if it could not be determined
+     *         -1 if it could not be determined; if not -1, the <code>dest</code> INCLUDES the
+     *         next header
      * @throws IOException
      */
     protected int readAtOffset(FSDataInputStream istream, byte [] dest, int destOffset, int size,
@@ -1505,16 +1487,16 @@ public class HFileBlock implements Cacheable {
           }
 
           // Try to read the next block header.
-          if (!readWithExtra(istream, dest, destOffset, size, hdrSize))
+          if (!readWithExtra(istream, dest, destOffset, size, hdrSize)) {
             return -1;
+          }
         } finally {
           streamLock.unlock();
         }
       } else {
         // Positional read. Better for random reads; or when the streamLock is already locked.
         int extraSize = peekIntoNextBlock ? hdrSize : 0;
-        if (!positionalReadWithExtra(istream, fileOffset, dest, destOffset,
-            size, extraSize)) {
+        if (!positionalReadWithExtra(istream, fileOffset, dest, destOffset, size, extraSize)) {
           return -1;
         }
       }
@@ -1530,16 +1512,12 @@ public class HFileBlock implements Cacheable {
      * @param offset the offset in the stream to read at
      * @param onDiskSizeWithHeaderL the on-disk size of the block, including
      *          the header, or -1 if unknown
-     * @param uncompressedSize the uncompressed size of the the block. Always
-     *          expected to be -1. This parameter is only used in version 1.
      * @param pread whether to use a positional read
      */
     @Override
-    public HFileBlock readBlockData(long offset, long onDiskSizeWithHeaderL,
-        int uncompressedSize, boolean pread)
+    public HFileBlock readBlockData(long offset, long onDiskSizeWithHeaderL, boolean pread)
     throws IOException {
-
-      // get a copy of the current state of whether to validate
+      // Get a copy of the current state of whether to validate
       // hbase checksums or not for this read call. This is not
       // thread-safe but the one constaint is that if we decide
       // to skip hbase checksum verification then we are
@@ -1548,8 +1526,7 @@ public class HFileBlock implements Cacheable {
       FSDataInputStream is = streamWrapper.getStream(doVerificationThruHBaseChecksum);
 
       HFileBlock blk = readBlockDataInternal(is, offset,
-                         onDiskSizeWithHeaderL,
-                         uncompressedSize, pread,
+                         onDiskSizeWithHeaderL, pread,
                          doVerificationThruHBaseChecksum);
       if (blk == null) {
         HFile.LOG.warn("HBase checksum verification failed for file " +
@@ -1576,8 +1553,7 @@ public class HFileBlock implements Cacheable {
         // a few more than precisely this number.
         is = this.streamWrapper.fallbackToFsChecksum(CHECKSUM_VERIFICATION_NUM_IO_THRESHOLD);
         doVerificationThruHBaseChecksum = false;
-        blk = readBlockDataInternal(is, offset, onDiskSizeWithHeaderL,
-                                    uncompressedSize, pread,
+        blk = readBlockDataInternal(is, offset, onDiskSizeWithHeaderL, pread,
                                     doVerificationThruHBaseChecksum);
         if (blk != null) {
           HFile.LOG.warn("HDFS checksum verification suceeded for file " +
@@ -1605,175 +1581,139 @@ public class HFileBlock implements Cacheable {
     }
 
     /**
+     * @return Check <code>onDiskSizeWithHeaderL</code> size is healthy and then return it as an int
+     * @throws IOException
+     */
+    private static int checkAndGetSizeAsInt(final long onDiskSizeWithHeaderL, final int hdrSize)
+    throws IOException {
+      if ((onDiskSizeWithHeaderL < hdrSize && onDiskSizeWithHeaderL != -1)
+          || onDiskSizeWithHeaderL >= Integer.MAX_VALUE) {
+        throw new IOException("Invalid onDisksize=" + onDiskSizeWithHeaderL
+            + ": expected to be at least " + hdrSize
+            + " and at most " + Integer.MAX_VALUE + ", or -1");
+      }
+      return (int)onDiskSizeWithHeaderL;
+    }
+
+    /**
+     * Check threadlocal cache for this block's header; we usually read it on the tail of reading
+     * the previous block to save a seek. Otherwise, we have to do a seek to read the header before
+     * we can pull in the block.
+     * @return The cached block header or null if not found.
+     * @see #cacheNextBlockHeader(long, byte[], int, int)
+     */
+    private ByteBuffer getCachedHeader(final long offset) {
+      PrefetchedHeader prefetchedHeader = prefetchedHeaderForThread.get();
+      // PrefetchedHeader prefetchedHeader = prefetchedHeaderForThread.get();
+      return prefetchedHeader != null && prefetchedHeader.offset == offset?
+          prefetchedHeader.buf: null;
+    }
+
+    /**
+     * Save away the next blocks header in thread local.
+     * @see #getCachedHeader(long)
+     */
+    private void cacheNextBlockHeader(final long nextBlockOffset,
+        final byte [] header, final int headerOffset, final int headerLength) {
+      PrefetchedHeader prefetchedHeader = prefetchedHeaderForThread.get();
+      prefetchedHeader.offset = nextBlockOffset;
+      System.arraycopy(header, headerOffset, prefetchedHeader.header, 0, headerLength);
+    }
+
+    /**
+     * Verify the passed in onDiskSizeWithHeader aligns with what is in the header else something
+     * is not right.
+     * @throws IOException
+     */
+    private void verifyOnDiskSizeMatchesHeader(final int passedIn, final ByteBuffer headerBuf,
+        final long offset)
+    throws IOException {
+      // Assert size provided aligns with what is in the header
+      int fromHeader = getOnDiskSizeWithHeader(headerBuf);
+      if (passedIn != fromHeader) {
+        throw new IOException("Passed in onDiskSizeWithHeader=" + passedIn + " != " + fromHeader +
+            ", offset=" + offset + ", fileContext=" + this.fileContext);
+      }
+    }
+
+    /**
      * Reads a version 2 block.
      *
      * @param offset the offset in the stream to read at
      * @param onDiskSizeWithHeaderL the on-disk size of the block, including
-     *          the header, or -1 if unknown
-     * @param uncompressedSize the uncompressed size of the the block. Always
-     *          expected to be -1. This parameter is only used in version 1.
+     *          the header and checksums if present or -1 if unknown
      * @param pread whether to use a positional read
      * @param verifyChecksum Whether to use HBase checksums.
      *        If HBase checksum is switched off, then use HDFS checksum.
      * @return the HFileBlock or null if there is a HBase checksum mismatch
      */
     private HFileBlock readBlockDataInternal(FSDataInputStream is, long offset,
-        long onDiskSizeWithHeaderL, int uncompressedSize, boolean pread,
-        boolean verifyChecksum)
+        long onDiskSizeWithHeaderL, boolean pread, boolean verifyChecksum)
     throws IOException {
       if (offset < 0) {
         throw new IOException("Invalid offset=" + offset + " trying to read "
-            + "block (onDiskSize=" + onDiskSizeWithHeaderL
-            + ", uncompressedSize=" + uncompressedSize + ")");
-      }
-
-      if (uncompressedSize != -1) {
-        throw new IOException("Version 2 block reader API does not need " +
-            "the uncompressed size parameter");
+            + "block (onDiskSize=" + onDiskSizeWithHeaderL + ")");
       }
-
-      if ((onDiskSizeWithHeaderL < hdrSize && onDiskSizeWithHeaderL != -1)
-          || onDiskSizeWithHeaderL >= Integer.MAX_VALUE) {
-        throw new IOException("Invalid onDisksize=" + onDiskSizeWithHeaderL
-            + ": expected to be at least " + hdrSize
-            + " and at most " + Integer.MAX_VALUE + ", or -1 (offset="
-            + offset + ", uncompressedSize=" + uncompressedSize + ")");
-      }
-
-      int onDiskSizeWithHeader = (int) onDiskSizeWithHeaderL;
-
-      // See if we can avoid reading the header. This is desirable, because
-      // we will not incur a backward seek operation if we have already
-      // read this block's header as part of the previous read's look-ahead.
-      // And we also want to skip reading the header again if it has already
-      // been read.
-      // TODO: How often does this optimization fire? Has to be same thread so the thread local
-      // is pertinent and we have to be reading next block as in a big scan.
-      ByteBuffer headerBuf = null;
-      PrefetchedHeader prefetchedHeader = prefetchedHeaderForThread.get();
-      boolean preReadHeader = false;
-      if (prefetchedHeader != null && prefetchedHeader.offset == offset) {
-        headerBuf = prefetchedHeader.buf;
-        preReadHeader = true;
+      int onDiskSizeWithHeader = checkAndGetSizeAsInt(onDiskSizeWithHeaderL, hdrSize);
+      ByteBuffer headerBuf = getCachedHeader(offset);
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Reading " + this.fileContext.getHFileName() + " at offset=" + offset +
+          ", pread=" + pread + ", verifyChecksum=" + verifyChecksum + ", cachedHeader=" +
+          headerBuf + ", onDiskSizeWithHeader=" + onDiskSizeWithHeader);
       }
-      // Allocate enough space to fit the next block's header too.
-      int nextBlockOnDiskSize = 0;
-      byte[] onDiskBlock = null;
-
-      HFileBlock b = null;
-      boolean fastPath = false;
-      boolean readHdrOnly = false;
-      if (onDiskSizeWithHeader > 0) {
-        fastPath = true;
-        // We know the total on-disk size. Read the entire block into memory,
-        // then parse the header. This code path is used when
-        // doing a random read operation relying on the block index, as well as
-        // when the client knows the on-disk size from peeking into the next
-        // block's header (e.g. this block's header) when reading the previous
-        // block. This is the faster and more preferable case.
-
-        // Size that we have to skip in case we have already read the header.
-        int preReadHeaderSize = headerBuf == null ? 0 : hdrSize;
-        onDiskBlock = new byte[onDiskSizeWithHeader + hdrSize]; // room for this block plus the
-                                                                // next block's header
-        nextBlockOnDiskSize = readAtOffset(is, onDiskBlock,
-            preReadHeaderSize, onDiskSizeWithHeader - preReadHeaderSize,
-            true, offset + preReadHeaderSize, pread);
-        if (headerBuf != null) {
-          // the header has been read when reading the previous block, copy
-          // to this block's header
-          // headerBuf is HBB
-          assert headerBuf.hasArray();
-          System.arraycopy(headerBuf.array(),
-              headerBuf.arrayOffset(), onDiskBlock, 0, hdrSize);
-        } else {
-          headerBuf = ByteBuffer.wrap(onDiskBlock, 0, hdrSize);
-        }
-        // We know the total on-disk size but not the uncompressed size. Parse the header.
-        try {
-          // TODO: FIX!!! Expensive parse just to get a length
-          b = new HFileBlock(headerBuf, fileContext.isUseHBaseChecksum());
-        } catch (IOException ex) {
-          // Seen in load testing. Provide comprehensive debug info.
-          throw new IOException("Failed to read compressed block at "
-              + offset
-              + ", onDiskSizeWithoutHeader="
-              + onDiskSizeWithHeader
-              + ", preReadHeaderSize="
-              + hdrSize
-              + ", header.length="
-              + prefetchedHeader.header.length
-              + ", header bytes: "
-              + Bytes.toStringBinary(prefetchedHeader.header, 0,
-                  hdrSize), ex);
-        }
-        // if the caller specifies a onDiskSizeWithHeader, validate it.
-        int onDiskSizeWithoutHeader = onDiskSizeWithHeader - hdrSize;
-        assert onDiskSizeWithoutHeader >= 0;
-        b.validateOnDiskSizeWithoutHeader(onDiskSizeWithoutHeader);
-      } else {
-        // Check headerBuf to see if we have read this block's header as part of
-        // reading the previous block. This is an optimization of peeking into
-        // the next block's header (e.g.this block's header) when reading the
-        // previous block. This is the faster and more preferable case. If the
-        // header is already there, don't read the header again.
-
-        // Unfortunately, we still have to do a separate read operation to
-        // read the header.
+      if (onDiskSizeWithHeader <= 0) {
+        // We were not passed the block size. Need to get it from the header. If header was not in
+        // cache, need to seek to pull it in. This latter might happen when we are doing the first
+        // read in a series of reads or a random read, and we don't have access to the block index.
+        // This is costly and should happen very rarely.
         if (headerBuf == null) {
-          readHdrOnly = true;
-          // From the header, determine the on-disk size of the given hfile
-          // block, and read the remaining data, thereby incurring two read
-          // operations. This might happen when we are doing the first read
-          // in a series of reads or a random read, and we don't have access
-          // to the block index. This is costly and should happen very rarely.
           headerBuf = ByteBuffer.allocate(hdrSize);
-          // headerBuf is HBB
-          readAtOffset(is, headerBuf.array(), headerBuf.arrayOffset(),
-              hdrSize, false, offset, pread);
+          readAtOffset(is, headerBuf.array(), headerBuf.arrayOffset(), hdrSize, false,
+              offset, pread);
         }
-        // TODO: FIX!!! Expensive parse just to get a length
-        b = new HFileBlock(headerBuf, fileContext.isUseHBaseChecksum());
-        // onDiskBlock is whole block + header + checksums then extra hdrSize to read next header
-        onDiskBlock = new byte[b.getOnDiskSizeWithHeader() + hdrSize];
-        // headerBuf is HBB. Copy hdr into onDiskBlock
-        System.arraycopy(headerBuf.array(), headerBuf.arrayOffset(), onDiskBlock, 0, hdrSize);
-        nextBlockOnDiskSize = readAtOffset(is, onDiskBlock, hdrSize,
-            b.getOnDiskSizeWithHeader() - hdrSize, true, offset + hdrSize, pread);
-        onDiskSizeWithHeader = b.onDiskSizeWithoutHeader + hdrSize;
-      }
-
-      if (!fileContext.isCompressedOrEncrypted()) {
-        b.assumeUncompressed();
+        onDiskSizeWithHeader = getOnDiskSizeWithHeader(headerBuf);
       }
-
-      if (verifyChecksum && !validateBlockChecksum(b, offset, onDiskBlock, hdrSize)) {
-        return null;             // checksum mismatch
+      int preReadHeaderSize = headerBuf == null? 0 : hdrSize;
+      // Allocate enough space to fit the next block's header too; saves a seek next time through.
+      // onDiskBlock is whole block + header + checksums then extra hdrSize to read next header;
+      // onDiskSizeWithHeader is header, body, and any checksums if present.
+      // TODO: Make this ByteBuffer-based. Will make it easier to go to HDFS with BBPool (offheap).
+      byte[] onDiskBlock = new byte[onDiskSizeWithHeader + hdrSize];
+      int nextBlockOnDiskSize = readAtOffset(is, onDiskBlock, preReadHeaderSize,
+          onDiskSizeWithHeader - preReadHeaderSize, true, offset + preReadHeaderSize, pread);
+      if (headerBuf != null) {
+        // The header has been read when reading the previous block OR in a distinct header-only
+        // read. Copy to this block's header.
+        System.arraycopy(headerBuf.array(), headerBuf.arrayOffset(), onDiskBlock, 0, hdrSize);
+      } else {
+        headerBuf = ByteBuffer.wrap(onDiskBlock, 0, hdrSize);
       }
-
+      // Do a few checks before we go instantiate HFileBlock.
+      assert onDiskSizeWithHeader > this.hdrSize;
+      verifyOnDiskSizeMatchesHeader(onDiskSizeWithHeader, headerBuf, offset);
       // The onDiskBlock will become the headerAndDataBuffer for this block.
       // If nextBlockOnDiskSizeWithHeader is not zero, the onDiskBlock already
-      // contains the header of next block, so no need to set next
-      // block's header in it.
-      b = new HFileBlock(ByteBuffer.wrap(onDiskBlock, 0, onDiskSizeWithHeader),
-        this.fileContext.isUseHBaseChecksum());
-
-      b.nextBlockOnDiskSizeWithHeader = nextBlockOnDiskSize;
-
-      // Set prefetched header
-      if (b.hasNextBlockHeader()) {
-        prefetchedHeader.offset = offset + b.getOnDiskSizeWithHeader();
-        System.arraycopy(onDiskBlock, onDiskSizeWithHeader, prefetchedHeader.header, 0, hdrSize);
+      // contains the header of next block, so no need to set next block's header in it.
+      HFileBlock hFileBlock =
+          new HFileBlock(new SingleByteBuff(ByteBuffer.wrap(onDiskBlock, 0, onDiskSizeWithHeader)),
+              this.fileContext.isUseHBaseChecksum(), MemoryType.EXCLUSIVE, offset,
+              nextBlockOnDiskSize, fileContext);
+      // Run check on uncompressed sizings.
+      if (!fileContext.isCompressedOrEncrypted()) {
+        hFileBlock.sanityCheckUncompressed();
+      }
+      if (verifyChecksum && !validateBlockChecksum(hFileBlock, offset, onDiskBlock, hdrSize)) {
+        return null;
       }
-
-      b.offset = offset;
-      b.fileContext.setIncludesTags(this.fileContext.isIncludesTags());
-      b.fileContext.setIncludesMvcc(this.fileContext.isIncludesMvcc());
       if (LOG.isTraceEnabled()) {
-        LOG.trace("Read preReadHeader=" + preReadHeader + ", fastPath=" + fastPath +
-            ", readHdrOnly=" + readHdrOnly + ", " + b);
+        LOG.trace("Read " + hFileBlock);
+      }
+      // Cache next block header if we read it for the next time through here.
+      if (nextBlockOnDiskSize != -1) {
+        cacheNextBlockHeader(offset + hFileBlock.getOnDiskSizeWithHeader(),
+            onDiskBlock, onDiskSizeWithHeader, hdrSize);
       }
-      return b;
+      return hFileBlock;
     }
 
     @Override
@@ -1819,42 +1759,73 @@ public class HFileBlock implements Cacheable {
     }
   }
 
+  /** An additional sanity-check in case no compression or encryption is being used. */
+  void sanityCheckUncompressed() throws IOException {
+    if (onDiskSizeWithoutHeader != uncompressedSizeWithoutHeader +
+        totalChecksumBytes()) {
+      throw new IOException("Using no compression but "
+          + "onDiskSizeWithoutHeader=" + onDiskSizeWithoutHeader + ", "
+          + "uncompressedSizeWithoutHeader=" + uncompressedSizeWithoutHeader
+          + ", numChecksumbytes=" + totalChecksumBytes());
+    }
+  }
+
+  // Cacheable implementation
   @Override
   public int getSerializedLength() {
     if (buf != null) {
-      // include extra bytes for the next header when it's available.
-      int extraSpace = hasNextBlockHeader() ? headerSize() : 0;
-      return this.buf.limit() + extraSpace + HFileBlock.EXTRA_SERIALIZATION_SPACE;
+      // Include extra bytes for block metadata.
+      return this.buf.limit() + BLOCK_METADATA_SPACE;
     }
     return 0;
   }
 
+  // Cacheable implementation
   @Override
   public void serialize(ByteBuffer destination) {
-    this.buf.get(destination, 0, getSerializedLength() - EXTRA_SERIALIZATION_SPACE);
-    serializeExtraInfo(destination);
+    // BE CAREFUL!! There is a custom version of this serialization over in BucketCache#doDrain.
+    // Make sure any changes in here are reflected over there.
+    this.buf.get(destination, 0, getSerializedLength() - BLOCK_METADATA_SPACE);
+    destination = addMetaData(destination);
+
+    // Make it ready for reading. flip sets position to zero and limit to current position which
+    // is what we want if we do not want to serialize the block plus checksums if present plus
+    // metadata.
+    destination.flip();
+  }
+
+  /**
+   * For use by bucketcache. This exposes internals.
+   */
+  public ByteBuffer getMetaData() {
+    ByteBuffer bb = ByteBuffer.allocate(BLOCK_METADATA_SPACE);
+    bb = addMetaData(bb);
+    bb.flip();
+    return bb;
   }
 
   /**
-   * Write out the content of EXTRA_SERIALIZATION_SPACE. Public so can be accessed by BucketCache.
+   * Adds metadata at current position (position is moved forward). Does not flip or reset.
+   * @return The passed <code>destination</code> with metadata added.
    */
-  public void serializeExtraInfo(ByteBuffer destination) {
+  private ByteBuffer addMetaData(final ByteBuffer destination) {
     destination.put(this.fileContext.isUseHBaseChecksum() ? (byte) 1 : (byte) 0);
     destination.putLong(this.offset);
-    destination.putInt(this.nextBlockOnDiskSizeWithHeader);
-    destination.rewind();
+    destination.putInt(this.nextBlockOnDiskSize);
+    return destination;
   }
 
+  // Cacheable implementation
   @Override
   public CacheableDeserializer<Cacheable> getDeserializer() {
-    return HFileBlock.blockDeserializer;
+    return HFileBlock.BLOCK_DESERIALIZER;
   }
 
   @Override
   public int hashCode() {
     int result = 1;
     result = result * 31 + blockType.hashCode();
-    result = result * 31 + nextBlockOnDiskSizeWithHeader;
+    result = result * 31 + nextBlockOnDiskSize;
     result = result * 31 + (int) (offset ^ (offset >>> 32));
     result = result * 31 + onDiskSizeWithoutHeader;
     result = result * 31 + (int) (prevBlockOffset ^ (prevBlockOffset >>> 32));
@@ -1880,9 +1851,10 @@ public class HFileBlock implements Cacheable {
     if (castedComparison.blockType != this.blockType) {
       return false;
     }
-    if (castedComparison.nextBlockOnDiskSizeWithHeader != this.nextBlockOnDiskSizeWithHeader) {
+    if (castedComparison.nextBlockOnDiskSize != this.nextBlockOnDiskSize) {
       return false;
     }
+    // Offset is important. Needed when we have to remake cachekey when block is returned to cache.
     if (castedComparison.offset != this.offset) {
       return false;
     }
@@ -1968,7 +1940,7 @@ public class HFileBlock implements Cacheable {
   }
 
   /**
-   * @return the HFileContext used to create this HFileBlock. Not necessary the
+   * @return This HFileBlocks fileContext which will a derivative of the
    * fileContext for the file from which this block's data was originally read.
    */
   HFileContext getHFileContext() {
@@ -1992,6 +1964,7 @@ public class HFileBlock implements Cacheable {
    * This is mostly helpful for debugging. This assumes that the block
    * has minor version > 0.
    */
+  @VisibleForTesting
   static String toStringHeader(ByteBuff buf) throws IOException {
     byte[] magicBuf = new byte[Math.min(buf.limit() - buf.position(), BlockType.MAGIC_LENGTH)];
     buf.get(magicBuf);

http://git-wip-us.apache.org/repos/asf/hbase/blob/000117ad/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
index 9f29f97..506f08d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
@@ -60,7 +60,7 @@ import org.apache.hadoop.util.StringUtils;
  * Examples of how to use the block index writer can be found in
  * {@link org.apache.hadoop.hbase.io.hfile.CompoundBloomFilterWriter} and
  *  {@link HFileWriterImpl}. Examples of how to use the reader can be
- *  found in {@link HFileWriterImpl} and
+ *  found in {@link HFileReaderImpl} and
  *  {@link org.apache.hadoop.hbase.io.hfile.TestHFileBlockIndex}.
  */
 @InterfaceAudience.Private

http://git-wip-us.apache.org/repos/asf/hbase/blob/000117ad/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
index 8f5040e..d71911f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
@@ -252,18 +252,20 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
           long end = 0;
           try {
             end = getTrailer().getLoadOnOpenDataOffset();
-            HFileBlock prevBlock = null;
             if (LOG.isTraceEnabled()) {
               LOG.trace("File=" + path.toString() + ", offset=" + offset + ", end=" + end);
             }
+            // TODO: Could we use block iterator in here? Would that get stuff into the cache?
+            HFileBlock prevBlock = null;
             while (offset < end) {
               if (Thread.interrupted()) {
                 break;
               }
-              long onDiskSize = -1;
-              if (prevBlock != null) {
-                onDiskSize = prevBlock.getNextBlockOnDiskSizeWithHeader();
-              }
+              // Perhaps we got our block from cache? Unlikely as this may be, if it happens, then
+              // the internal-to-hfileblock thread local which holds the overread that gets the
+              // next header, will not have happened...so, pass in the onDiskSize gotten from the
+              // cached block. This 'optimization' triggers extremely rarely I'd say.
+              long onDiskSize = prevBlock != null? prevBlock.getNextBlockOnDiskSize(): -1;
               HFileBlock block = readBlock(offset, onDiskSize, true, false, false, false,
                 null, null);
               // Need not update the current block. Ideally here the readBlock won't find the
@@ -903,9 +905,8 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
 
         // We are reading the next block without block type validation, because
         // it might turn out to be a non-data block.
-        block = reader.readBlock(block.getOffset()
-            + block.getOnDiskSizeWithHeader(),
-            block.getNextBlockOnDiskSizeWithHeader(), cacheBlocks, pread,
+        block = reader.readBlock(block.getOffset() + block.getOnDiskSizeWithHeader(),
+            block.getNextBlockOnDiskSize(), cacheBlocks, pread,
             isCompaction, true, null, getEffectiveDataBlockEncoding());
         if (block != null && !block.getBlockType().isData()) { // Findbugs: NP_NULL_ON_SOME_PATH
           // Whatever block we read we will be returning it unless
@@ -1439,8 +1440,8 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
         // Cache Miss, please load.
       }
 
-      HFileBlock metaBlock = fsBlockReader.readBlockData(metaBlockOffset,
-          blockSize, -1, true).unpack(hfileContext, fsBlockReader);
+      HFileBlock metaBlock = fsBlockReader.readBlockData(metaBlockOffset, blockSize, true).
+          unpack(hfileContext, fsBlockReader);
 
       // Cache the block
       if (cacheBlock) {
@@ -1526,8 +1527,8 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
           traceScope.getSpan().addTimelineAnnotation("blockCacheMiss");
         }
         // Load block from filesystem.
-        HFileBlock hfileBlock = fsBlockReader.readBlockData(dataBlockOffset, onDiskBlockSize, -1,
-            pread);
+        HFileBlock hfileBlock =
+            fsBlockReader.readBlockData(dataBlockOffset, onDiskBlockSize, pread);
         validateBlockType(hfileBlock, expectedBlockType);
         HFileBlock unpacked = hfileBlock.unpack(hfileContext, fsBlockReader);
         BlockType.BlockCategory category = hfileBlock.getBlockType().getCategory();
@@ -1871,6 +1872,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
    * @return Scanner on this file.
    */
   @Override
+  @VisibleForTesting
   public HFileScanner getScanner(boolean cacheBlocks, final boolean pread) {
     return getScanner(cacheBlocks, pread, false);
   }


[22/50] [abbrv] hbase git commit: HBASE-15300 Upgrade to zookeeper 3.4.8

Posted by sy...@apache.org.
HBASE-15300 Upgrade to zookeeper 3.4.8


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

Branch: refs/heads/hbase-12439
Commit: 95e6d2276b710e386cf010bb9072af7293fdb890
Parents: 7177e4b
Author: tedyu <yu...@gmail.com>
Authored: Wed Mar 23 13:05:17 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Wed Mar 23 13:05:17 2016 -0700

----------------------------------------------------------------------
 pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/95e6d227/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 99bab51..450275c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1214,7 +1214,7 @@
     <protobuf.version>2.5.0</protobuf.version>
     <thrift.path>thrift</thrift.path>
     <thrift.version>0.9.3</thrift.version>
-    <zookeeper.version>3.4.6</zookeeper.version>
+    <zookeeper.version>3.4.8</zookeeper.version>
     <slf4j.version>1.7.7</slf4j.version>
     <clover.version>4.0.3</clover.version>
     <jamon-runtime.version>2.4.1</jamon-runtime.version>


[14/50] [abbrv] hbase git commit: Revert "HBASE-15477 Purge 'next block header' from cached blocks"

Posted by sy...@apache.org.
Revert "HBASE-15477 Purge 'next block header' from cached blocks"

Overcommit. Revert to fix.

This reverts commit 000117ad9fd7eb59074c9bb0da2cf1f9544d4bed.


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

Branch: refs/heads/hbase-12439
Commit: 54a543de229b358b438c920f04f7f2d1ff767cab
Parents: 3f3613a
Author: stack <st...@apache.org>
Authored: Tue Mar 22 18:37:25 2016 -0700
Committer: stack <st...@apache.org>
Committed: Tue Mar 22 18:37:25 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hbase/io/hfile/BlockType.java |   4 -
 .../hbase/io/hfile/HFileContextBuilder.java     |  20 -
 .../org/apache/hadoop/hbase/nio/ByteBuff.java   |   6 -
 .../hbase/io/hfile/MemcachedBlockCache.java     |   2 +-
 .../hadoop/hbase/io/hfile/ChecksumUtil.java     |   5 +-
 .../hadoop/hbase/io/hfile/HFileBlock.java       | 997 ++++++++++---------
 .../hadoop/hbase/io/hfile/HFileBlockIndex.java  |   2 +-
 .../hadoop/hbase/io/hfile/HFileReaderImpl.java  |  26 +-
 .../hadoop/hbase/io/hfile/HFileScanner.java     |  12 -
 .../hbase/io/hfile/bucket/BucketCache.java      |  15 +-
 .../hbase/regionserver/KeyValueScanner.java     |  12 +-
 .../hadoop/hbase/regionserver/StoreFile.java    |   4 +-
 .../hadoop/hbase/io/hfile/CacheTestUtils.java   |  23 +-
 .../hadoop/hbase/io/hfile/TestCacheOnWrite.java |  10 +-
 .../hadoop/hbase/io/hfile/TestChecksum.java     |  27 +-
 .../hadoop/hbase/io/hfile/TestHFileBlock.java   |  27 +-
 .../io/hfile/TestHFileBlockCompatibility.java   | 750 ++++++++++++++
 .../hbase/io/hfile/TestHFileBlockIndex.java     |   3 +-
 .../io/hfile/TestHFileDataBlockEncoder.java     |  10 +-
 .../hbase/io/hfile/TestHFileEncryption.java     |   2 +-
 .../hbase/io/hfile/TestHFileWriterV3.java       |   7 +-
 .../hadoop/hbase/io/hfile/TestPrefetch.java     |   9 +-
 .../regionserver/TestCacheOnWriteInSchema.java  |   8 +-
 23 files changed, 1374 insertions(+), 607 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/54a543de/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java
index 32eb0b2..4228f57 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java
@@ -132,10 +132,6 @@ public enum BlockType {
     out.write(magic);
   }
 
-  public void write(ByteBuffer buf) {
-    buf.put(magic);
-  }
-
   public void write(ByteBuff buf) {
     buf.put(magic);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/54a543de/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContextBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContextBuilder.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContextBuilder.java
index a6645a6..6d3bb13 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContextBuilder.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContextBuilder.java
@@ -55,26 +55,6 @@ public class HFileContextBuilder {
 
   private String hfileName = null;
 
-  public HFileContextBuilder() {}
-
-  /**
-   * Use this constructor if you want to change a few settings only in another context.
-   */
-  public HFileContextBuilder(final HFileContext hfc) {
-    this.usesHBaseChecksum = hfc.isUseHBaseChecksum();
-    this.includesMvcc = hfc.isIncludesMvcc();
-    this.includesTags = hfc.isIncludesTags();
-    this.compression = hfc.getCompression();
-    this.compressTags = hfc.isCompressTags();
-    this.checksumType = hfc.getChecksumType();
-    this.bytesPerChecksum = hfc.getBytesPerChecksum();
-    this.blocksize = hfc.getBlocksize();
-    this.encoding = hfc.getDataBlockEncoding();
-    this.cryptoContext = hfc.getEncryptionContext();
-    this.fileCreateTime = hfc.getFileCreateTime();
-    this.hfileName = hfc.getHFileName();
-  }
-
   public HFileContextBuilder withHBaseCheckSum(boolean useHBaseCheckSum) {
     this.usesHBaseChecksum = useHBaseCheckSum;
     return this;

http://git-wip-us.apache.org/repos/asf/hbase/blob/54a543de/hbase-common/src/main/java/org/apache/hadoop/hbase/nio/ByteBuff.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/nio/ByteBuff.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/nio/ByteBuff.java
index 183a031..1e0e957 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/nio/ByteBuff.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/nio/ByteBuff.java
@@ -496,12 +496,6 @@ public abstract class ByteBuff {
     return -(low + 1); // key not found.
   }
 
-  @Override
-  public String toString() {
-    return this.getClass().getSimpleName() + "[pos=" + position() + ", lim=" + limit() +
-        ", cap= " + capacity() + "]";
-  }
-
   public static String toStringBinary(final ByteBuff b, int off, int len) {
     StringBuilder result = new StringBuilder();
     // Just in case we are passed a 'len' that is > buffer length...

http://git-wip-us.apache.org/repos/asf/hbase/blob/54a543de/hbase-external-blockcache/src/main/java/org/apache/hadoop/hbase/io/hfile/MemcachedBlockCache.java
----------------------------------------------------------------------
diff --git a/hbase-external-blockcache/src/main/java/org/apache/hadoop/hbase/io/hfile/MemcachedBlockCache.java b/hbase-external-blockcache/src/main/java/org/apache/hadoop/hbase/io/hfile/MemcachedBlockCache.java
index ae871c4..536872e 100644
--- a/hbase-external-blockcache/src/main/java/org/apache/hadoop/hbase/io/hfile/MemcachedBlockCache.java
+++ b/hbase-external-blockcache/src/main/java/org/apache/hadoop/hbase/io/hfile/MemcachedBlockCache.java
@@ -260,7 +260,7 @@ public class MemcachedBlockCache implements BlockCache {
     public HFileBlock decode(CachedData d) {
       try {
         ByteBuff buf = new SingleByteBuff(ByteBuffer.wrap(d.getData()));
-        return (HFileBlock) HFileBlock.BLOCK_DESERIALIZER.deserialize(buf, true,
+        return (HFileBlock) HFileBlock.blockDeserializer.deserialize(buf, true,
           MemoryType.EXCLUSIVE);
       } catch (IOException e) {
         LOG.warn("Error deserializing data from memcached",e);

http://git-wip-us.apache.org/repos/asf/hbase/blob/54a543de/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ChecksumUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ChecksumUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ChecksumUtil.java
index b0b1714..69f4330 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ChecksumUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ChecksumUtil.java
@@ -91,7 +91,7 @@ public class ChecksumUtil {
 
     // If this is an older version of the block that does not have
     // checksums, then return false indicating that checksum verification
-    // did not succeed. Actually, this method should never be called
+    // did not succeed. Actually, this methiod should never be called
     // when the minorVersion is 0, thus this is a defensive check for a
     // cannot-happen case. Since this is a cannot-happen case, it is
     // better to return false to indicate a checksum validation failure.
@@ -141,7 +141,8 @@ public class ChecksumUtil {
    * @return The number of bytes needed to store the checksum values
    */
   static long numBytes(long datasize, int bytesPerChecksum) {
-    return numChunks(datasize, bytesPerChecksum) * HFileBlock.CHECKSUM_SIZE;
+    return numChunks(datasize, bytesPerChecksum) *
+                     HFileBlock.CHECKSUM_SIZE;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/54a543de/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
index f3402da..6268f2e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
@@ -56,131 +56,50 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 
 /**
- * Reads {@link HFile} version 2 blocks to HFiles and via {@link Cacheable} Interface to caches.
- * Version 2 was introduced in hbase-0.92.0. No longer has support for version 1 blocks since
- * hbase-1.3.0.
- *
- * <p>Version 1 was the original file block. Version 2 was introduced when we changed the hbase file
- * format to support multi-level block indexes and compound bloom filters (HBASE-3857).
+ * Reads {@link HFile} version 1 and version 2 blocks but writes version 2 blocks only.
+ * Version 2 was introduced in hbase-0.92.0. Does read and write out to the filesystem but also
+ * the read and write to Cache.
  *
+ * <h3>HFileBlock: Version 1</h3>
+ * As of this writing, there should be no more version 1 blocks found out in the wild. Version 2
+ * as introduced in hbase-0.92.0.
+ * In version 1 all blocks are always compressed or uncompressed, as
+ * specified by the {@link HFile}'s compression algorithm, with a type-specific
+ * magic record stored in the beginning of the compressed data (i.e. one needs
+ * to uncompress the compressed block to determine the block type). There is
+ * only a single compression algorithm setting for all blocks. Offset and size
+ * information from the block index are required to read a block.
  * <h3>HFileBlock: Version 2</h3>
  * In version 2, a block is structured as follows:
  * <ul>
- * <li><b>Header:</b> See Writer#putHeader() for where header is written; header total size is
- * HFILEBLOCK_HEADER_SIZE
+ * <li><b>Header:</b> See Writer#putHeader(); header total size is HFILEBLOCK_HEADER_SIZE)
  * <ul>
- * <li>0. blockType: Magic record identifying the {@link BlockType} (8 bytes):
- * e.g. <code>DATABLK*</code>
- * <li>1. onDiskSizeWithoutHeader: Compressed -- a.k.a 'on disk' -- block size, excluding header,
- * but including tailing checksum bytes (4 bytes)
- * <li>2. uncompressedSizeWithoutHeader: Uncompressed block size, excluding header, and excluding
- * checksum bytes (4 bytes)
- * <li>3. prevBlockOffset: The offset of the previous block of the same type (8 bytes). This is
+ * <li>Magic record identifying the {@link BlockType} (8 bytes): e.g. <code>DATABLK*</code>
+ * <li>Compressed -- a.k.a 'on disk' -- block size, excluding header, but including
+ *     tailing checksum bytes (4 bytes)
+ * <li>Uncompressed block size, excluding header, and excluding checksum bytes (4 bytes)
+ * <li>The offset of the previous block of the same type (8 bytes). This is
  * used to navigate to the previous block without having to go to the block index
- * <li>4: For minorVersions &gt;=1, the ordinal describing checksum type (1 byte)
- * <li>5: For minorVersions &gt;=1, the number of data bytes/checksum chunk (4 bytes)
- * <li>6: onDiskDataSizeWithHeader: For minorVersions &gt;=1, the size of data 'on disk', including
- * header, excluding checksums (4 bytes)
+ * <li>For minorVersions &gt;=1, the ordinal describing checksum type (1 byte)
+ * <li>For minorVersions &gt;=1, the number of data bytes/checksum chunk (4 bytes)
+ * <li>For minorVersions &gt;=1, the size of data 'on disk', including header,
+ * excluding checksums (4 bytes)
  * </ul>
  * </li>
- * <li><b>Raw/Compressed/Encrypted/Encoded data:</b> The compression
- * algorithm is the same for all the blocks in an {@link HFile}. If compression is NONE, this is
- * just raw, serialized Cells.
+ * <li><b>Raw/Compressed/Encrypted/Encoded data:</b> The compression algorithm is the
+ * same for all the blocks in the {@link HFile}, similarly to what was done in
+ * version 1. If compression is NONE, this is just raw, serialized Cells.
  * <li><b>Tail:</b> For minorVersions &gt;=1, a series of 4 byte checksums, one each for
  * the number of bytes specified by bytesPerChecksum.
  * </ul>
- *
- * <h3>Caching</h3>
- * Caches cache whole blocks with trailing checksums if any. We then tag on some metadata, the
- * content of BLOCK_METADATA_SPACE which will be flag on if we are doing 'hbase'
- * checksums and then the offset into the file which is needed when we re-make a cache key
- * when we return the block to the cache as 'done'. See {@link Cacheable#serialize(ByteBuffer)} and
- * {@link Cacheable#getDeserializer()}.
- *
- * <p>TODO: Should we cache the checksums? Down in Writer#getBlockForCaching(CacheConfig) where
- * we make a block to cache-on-write, there is an attempt at turning off checksums. This is not the
- * only place we get blocks to cache. We also will cache the raw return from an hdfs read. In this
- * case, the checksums may be present. If the cache is backed by something that doesn't do ECC,
- * say an SSD, we might want to preserve checksums. For now this is open question.
- * <p>TODO: Over in BucketCache, we save a block allocation by doing a custom serialization.
- * Be sure to change it if serialization changes in here. Could we add a method here that takes an
- * IOEngine and that then serializes to it rather than expose our internals over in BucketCache?
- * IOEngine is in the bucket subpackage. Pull it up? Then this class knows about bucketcache. Ugh.
+ * <p>Be aware that when we read from HDFS, we overread pulling in the next blocks' header too.
+ * We do this to save having to do two seeks to read an HFileBlock; a seek to read the header
+ * to figure lengths, etc., and then another seek to pull in the data.
  */
 @InterfaceAudience.Private
 public class HFileBlock implements Cacheable {
   private static final Log LOG = LogFactory.getLog(HFileBlock.class);
 
-  /** Type of block. Header field 0. */
-  private BlockType blockType;
-
-  /**
-   * Size on disk excluding header, including checksum. Header field 1.
-   * @see Writer#putHeader(byte[], int, int, int, int)
-   */
-  private int onDiskSizeWithoutHeader;
-
-  /**
-   * Size of pure data. Does not include header or checksums. Header field 2.
-   * @see Writer#putHeader(byte[], int, int, int, int)
-   */
-  private int uncompressedSizeWithoutHeader;
-
-  /**
-   * The offset of the previous block on disk. Header field 3.
-   * @see Writer#putHeader(byte[], int, int, int, int)
-   */
-  private long prevBlockOffset;
-
-  /**
-   * Size on disk of header + data. Excludes checksum. Header field 6,
-   * OR calculated from {@link #onDiskSizeWithoutHeader} when using HDFS checksum.
-   * @see Writer#putHeader(byte[], int, int, int, int)
-   */
-  private int onDiskDataSizeWithHeader;
-
-
-  /**
-   * The in-memory representation of the hfile block. Can be on or offheap. Can be backed by
-   * a single ByteBuffer or by many. Make no assumptions.
-   *
-   * <p>Be careful reading from this <code>buf</code>. Duplicate and work on the duplicate or if
-   * not, be sure to reset position and limit else trouble down the road.
-   *
-   * <p>TODO: Make this read-only once made.
-   *
-   * <p>We are using the ByteBuff type. ByteBuffer is not extensible yet we need to be able to have
-   * a ByteBuffer-like API across multiple ByteBuffers reading from a cache such as BucketCache.
-   * So, we have this ByteBuff type. Unfortunately, it is spread all about HFileBlock. Would be
-   * good if could be confined to cache-use only but hard-to-do.
-   */
-  private ByteBuff buf;
-
-  /** Meta data that holds meta information on the hfileblock.
-   */
-  private HFileContext fileContext;
-
-  /**
-   * The offset of this block in the file. Populated by the reader for
-   * convenience of access. This offset is not part of the block header.
-   */
-  private long offset = UNSET;
-
-  private MemoryType memType = MemoryType.EXCLUSIVE;
-
-  /**
-   * The on-disk size of the next block, including the header and checksums if present, obtained by
-   * peeking into the first {@link HConstants#HFILEBLOCK_HEADER_SIZE} bytes of the next block's
-   * header, or UNSET if unknown.
-   *
-   * Blocks try to carry the size of the next block to read in this data member. They will even have
-   * this value when served from cache. Could save a seek in the case where we are iterating through
-   * a file and some of the blocks come from cache. If from cache, then having this info to hand
-   * will save us doing a seek to read the header so we can read the body of a block.
-   * TODO: see how effective this is at saving seeks.
-   */
-  private int nextBlockOnDiskSize = UNSET;
-
   /**
    * On a checksum failure, do these many succeeding read requests using hdfs checksums before
    * auto-reenabling hbase checksum verification.
@@ -196,18 +115,14 @@ public class HFileBlock implements Cacheable {
       (int)ClassSize.estimateBase(MultiByteBuff.class, false);
 
   /**
-   * Space for metadata on a block that gets stored along with the block when we cache it.
-   * There are a few bytes stuck on the end of the HFileBlock that we pull in from HDFS (note,
+   * See #blockDeserializer method for more info.
+   * 13 bytes of extra stuff stuck on the end of the HFileBlock that we pull in from HDFS (note,
    * when we read from HDFS, we pull in an HFileBlock AND the header of the next block if one).
-   * 8 bytes are offset of this block (long) in the file. Offset is important because
-   * used when we remake the CacheKey when we return the block to cache when done. There is also
-   * a flag on whether checksumming is being done by hbase or not. See class comment for note on
-   * uncertain state of checksumming of blocks that come out of cache (should we or should we not?).
-   * Finally there 4 bytes to hold the length of the next block which can save a seek on occasion.
-   * <p>This EXTRA came in with original commit of the bucketcache, HBASE-7404. Was formerly
-   * known as EXTRA_SERIALIZATION_SPACE.
+   * The 13 bytes are: usesHBaseChecksum (1 byte) + offset of this block (long) +
+   * nextBlockOnDiskSizeWithHeader (int).
    */
-  static final int BLOCK_METADATA_SPACE = Bytes.SIZEOF_BYTE + Bytes.SIZEOF_LONG + Bytes.SIZEOF_INT;
+  public static final int EXTRA_SERIALIZATION_SPACE =
+      Bytes.SIZEOF_BYTE + Bytes.SIZEOF_INT + Bytes.SIZEOF_LONG;
 
   /**
    * Each checksum value is an integer that can be stored in 4 bytes.
@@ -220,47 +135,57 @@ public class HFileBlock implements Cacheable {
   /**
    * Used deserializing blocks from Cache.
    *
-   * <code>
+   * Serializing to cache is a little hard to follow. See Writer#finishBlock for where it is done.
+   * When we start to append to a new HFileBlock,
+   * we skip over where the header should go before we start adding Cells. When the block is
+   * done, we'll then go back and fill in the header and the checksum tail. Be aware that what
+   * gets serialized into the blockcache is a byte array that contains an HFileBlock followed by
+   * its checksums and then the header of the next HFileBlock (needed to help navigate), followed
+   * again by an extra 13 bytes of meta info needed when time to recreate the HFileBlock from cache.
+   *
    * ++++++++++++++
    * + HFileBlock +
    * ++++++++++++++
-   * + Checksums  + <= Optional
+   * + Checksums  +
+   * ++++++++++++++
+   * + NextHeader +
    * ++++++++++++++
-   * + Metadata!  +
+   * + ExtraMeta! +
    * ++++++++++++++
-   * </code>
-   * @see #serialize(ByteBuffer)
+   *
+   * TODO: Fix it so we do NOT put the NextHeader into blockcache. It is not necessary.
    */
-  static final CacheableDeserializer<Cacheable> BLOCK_DESERIALIZER =
+  static final CacheableDeserializer<Cacheable> blockDeserializer =
       new CacheableDeserializer<Cacheable>() {
         public HFileBlock deserialize(ByteBuff buf, boolean reuse, MemoryType memType)
         throws IOException {
-          // The buf has the file block followed by block metadata.
-          // Set limit to just before the BLOCK_METADATA_SPACE then rewind.
-          buf.limit(buf.limit() - BLOCK_METADATA_SPACE).rewind();
-          // Get a new buffer to pass the HFileBlock for it to 'own'.
-          ByteBuff newByteBuff;
+          // Rewind to just before the EXTRA_SERIALIZATION_SPACE.
+          buf.limit(buf.limit() - HFileBlock.EXTRA_SERIALIZATION_SPACE).rewind();
+          // Get a new buffer to pass the deserialized HFileBlock for it to 'own'.
+          ByteBuff newByteBuffer;
           if (reuse) {
-            newByteBuff = buf.slice();
+            newByteBuffer = buf.slice();
           } else {
             int len = buf.limit();
-            newByteBuff = new SingleByteBuff(ByteBuffer.allocate(len));
-            newByteBuff.put(0, buf, buf.position(), len);
+            newByteBuffer = new SingleByteBuff(ByteBuffer.allocate(len));
+            newByteBuffer.put(0, buf, buf.position(), len);
           }
-          // Read out the BLOCK_METADATA_SPACE content and shove into our HFileBlock.
+          // Read out the EXTRA_SERIALIZATION_SPACE content and shove into our HFileBlock.
           buf.position(buf.limit());
-          buf.limit(buf.limit() + HFileBlock.BLOCK_METADATA_SPACE);
+          buf.limit(buf.limit() + HFileBlock.EXTRA_SERIALIZATION_SPACE);
           boolean usesChecksum = buf.get() == (byte)1;
-          long offset = buf.getLong();
-          int nextBlockOnDiskSize = buf.getInt();
-          HFileBlock hFileBlock =
-              new HFileBlock(newByteBuff, usesChecksum, memType, offset, nextBlockOnDiskSize, null);
+          HFileBlock hFileBlock = new HFileBlock(newByteBuffer, usesChecksum, memType);
+          hFileBlock.offset = buf.getLong();
+          hFileBlock.nextBlockOnDiskSizeWithHeader = buf.getInt();
+          if (hFileBlock.hasNextBlockHeader()) {
+            hFileBlock.buf.limit(hFileBlock.buf.limit() - hFileBlock.headerSize());
+          }
           return hFileBlock;
         }
 
         @Override
         public int getDeserialiserIdentifier() {
-          return DESERIALIZER_IDENTIFIER;
+          return deserializerIdentifier;
         }
 
         @Override
@@ -270,36 +195,65 @@ public class HFileBlock implements Cacheable {
         }
       };
 
-  private static final int DESERIALIZER_IDENTIFIER;
+  private static final int deserializerIdentifier;
   static {
-    DESERIALIZER_IDENTIFIER =
-        CacheableDeserializerIdManager.registerDeserializer(BLOCK_DESERIALIZER);
+    deserializerIdentifier = CacheableDeserializerIdManager
+        .registerDeserializer(blockDeserializer);
   }
 
+  /** Type of block. Header field 0. */
+  private BlockType blockType;
+
   /**
-   * Copy constructor. Creates a shallow copy of {@code that}'s buffer.
+   * Size on disk excluding header, including checksum. Header field 1.
+   * @see Writer#putHeader(byte[], int, int, int, int)
    */
-  private HFileBlock(HFileBlock that) {
-    this.blockType = that.blockType;
-    this.onDiskSizeWithoutHeader = that.onDiskSizeWithoutHeader;
-    this.uncompressedSizeWithoutHeader = that.uncompressedSizeWithoutHeader;
-    this.prevBlockOffset = that.prevBlockOffset;
-    this.buf = that.buf.duplicate();
-    this.offset = that.offset;
-    this.onDiskDataSizeWithHeader = that.onDiskDataSizeWithHeader;
-    this.fileContext = that.fileContext;
-    this.nextBlockOnDiskSize = that.nextBlockOnDiskSize;
-  }
+  private int onDiskSizeWithoutHeader;
+
+  /**
+   * Size of pure data. Does not include header or checksums. Header field 2.
+   * @see Writer#putHeader(byte[], int, int, int, int)
+   */
+  private final int uncompressedSizeWithoutHeader;
+
+  /**
+   * The offset of the previous block on disk. Header field 3.
+   * @see Writer#putHeader(byte[], int, int, int, int)
+   */
+  private final long prevBlockOffset;
+
+  /**
+   * Size on disk of header + data. Excludes checksum. Header field 6,
+   * OR calculated from {@link #onDiskSizeWithoutHeader} when using HDFS checksum.
+   * @see Writer#putHeader(byte[], int, int, int, int)
+   */
+  private final int onDiskDataSizeWithHeader;
+
+  /** The in-memory representation of the hfile block */
+  private ByteBuff buf;
+
+  /** Meta data that holds meta information on the hfileblock */
+  private HFileContext fileContext;
+
+  /**
+   * The offset of this block in the file. Populated by the reader for
+   * convenience of access. This offset is not part of the block header.
+   */
+  private long offset = UNSET;
+
+  /**
+   * The on-disk size of the next block, including the header, obtained by
+   * peeking into the first {@link HConstants#HFILEBLOCK_HEADER_SIZE} bytes of the next block's
+   * header, or -1 if unknown.
+   */
+  private int nextBlockOnDiskSizeWithHeader = UNSET;
+
+  private MemoryType memType = MemoryType.EXCLUSIVE;
 
   /**
    * Creates a new {@link HFile} block from the given fields. This constructor
    * is used when the block data has already been read and uncompressed,
-   * and is sitting in a byte buffer and we want to stuff the block into cache.
-   * See {@link Writer#getBlockForCaching(CacheConfig)}.
-   *
-   * <p>TODO: The caller presumes no checksumming
-   * required of this block instance since going into cache; checksum already verified on
-   * underlying block data pulled in from filesystem. Is that correct? What if cache is SSD?
+   * and is sitting in a byte buffer.
    *
    * @param blockType the type of this block, see {@link BlockType}
    * @param onDiskSizeWithoutHeader see {@link #onDiskSizeWithoutHeader}
@@ -313,94 +267,86 @@ public class HFileBlock implements Cacheable {
    * @param fileContext HFile meta data
    */
   HFileBlock(BlockType blockType, int onDiskSizeWithoutHeader, int uncompressedSizeWithoutHeader,
-      long prevBlockOffset, ByteBuffer b, boolean fillHeader, long offset,
-      final int nextBlockOnDiskSize, int onDiskDataSizeWithHeader, HFileContext fileContext) {
-    init(blockType, onDiskSizeWithoutHeader, uncompressedSizeWithoutHeader,
-        prevBlockOffset, offset, onDiskDataSizeWithHeader, nextBlockOnDiskSize, fileContext);
-    this.buf = new SingleByteBuff(b);
+      long prevBlockOffset, ByteBuff buf, boolean fillHeader, long offset,
+      int onDiskDataSizeWithHeader, HFileContext fileContext) {
+    this.blockType = blockType;
+    this.onDiskSizeWithoutHeader = onDiskSizeWithoutHeader;
+    this.uncompressedSizeWithoutHeader = uncompressedSizeWithoutHeader;
+    this.prevBlockOffset = prevBlockOffset;
+    this.buf = buf;
+    this.offset = offset;
+    this.onDiskDataSizeWithHeader = onDiskDataSizeWithHeader;
+    this.fileContext = fileContext;
     if (fillHeader) {
       overwriteHeader();
     }
     this.buf.rewind();
   }
 
-  /**
-   * Creates a block from an existing buffer starting with a header. Rewinds
-   * and takes ownership of the buffer. By definition of rewind, ignores the
-   * buffer position, but if you slice the buffer beforehand, it will rewind
-   * to that point.
-   * @param buf Has header, content, and trailing checksums if present.
-   */
-  HFileBlock(ByteBuff buf, boolean usesHBaseChecksum, MemoryType memType, final long offset,
-      final int nextBlockOnDiskSize, HFileContext fileContext) throws IOException {
-    buf.rewind();
-    final BlockType blockType = BlockType.read(buf);
-    final int onDiskSizeWithoutHeader = buf.getInt();
-    final int uncompressedSizeWithoutHeader = buf.getInt();
-    final long prevBlockOffset = buf.getLong();
-    byte checksumType = buf.get();
-    int bytesPerChecksum = buf.getInt();
-    int onDiskDataSizeWithHeader = buf.getInt();
-    // This constructor is called when we deserialize a block from cache and when we read a block in
-    // from the fs. fileCache is null when deserialized from cache so need to make up one.
-    HFileContextBuilder fileContextBuilder = fileContext != null?
-        new HFileContextBuilder(fileContext): new HFileContextBuilder();
-    fileContextBuilder.withHBaseCheckSum(usesHBaseChecksum);
-    if (usesHBaseChecksum) {
-      // Use the checksum type and bytes per checksum from header, not from filecontext.
-      fileContextBuilder.withChecksumType(ChecksumType.codeToType(checksumType));
-      fileContextBuilder.withBytesPerCheckSum(bytesPerChecksum);
-    } else {
-      fileContextBuilder.withChecksumType(ChecksumType.NULL);
-      fileContextBuilder.withBytesPerCheckSum(0);
-      // Need to fix onDiskDataSizeWithHeader; there are not checksums after-block-data
-      onDiskDataSizeWithHeader = onDiskSizeWithoutHeader + headerSize(usesHBaseChecksum);
-    }
-    fileContext = fileContextBuilder.build();
-    assert usesHBaseChecksum == fileContext.isUseHBaseChecksum();
-    init(blockType, onDiskSizeWithoutHeader, uncompressedSizeWithoutHeader,
-        prevBlockOffset, offset, onDiskDataSizeWithHeader, nextBlockOnDiskSize, fileContext);
-    this.memType = memType;
-    this.offset = offset;
-    this.buf = buf;
-    this.buf.rewind();
+  HFileBlock(BlockType blockType, int onDiskSizeWithoutHeader, int uncompressedSizeWithoutHeader,
+      long prevBlockOffset, ByteBuffer buf, boolean fillHeader, long offset,
+      int onDiskDataSizeWithHeader, HFileContext fileContext) {
+    this(blockType, onDiskSizeWithoutHeader, uncompressedSizeWithoutHeader, prevBlockOffset,
+        new SingleByteBuff(buf), fillHeader, offset, onDiskDataSizeWithHeader, fileContext);
   }
 
   /**
-   * Called from constructors.
+   * Copy constructor. Creates a shallow copy of {@code that}'s buffer.
    */
-  private void init(BlockType blockType, int onDiskSizeWithoutHeader,
-      int uncompressedSizeWithoutHeader, long prevBlockOffset,
-      long offset, int onDiskDataSizeWithHeader, final int nextBlockOnDiskSize,
-      HFileContext fileContext) {
-    this.blockType = blockType;
-    this.onDiskSizeWithoutHeader = onDiskSizeWithoutHeader;
-    this.uncompressedSizeWithoutHeader = uncompressedSizeWithoutHeader;
-    this.prevBlockOffset = prevBlockOffset;
-    this.offset = offset;
-    this.onDiskDataSizeWithHeader = onDiskDataSizeWithHeader;
-    this.nextBlockOnDiskSize = nextBlockOnDiskSize;
-    this.fileContext = fileContext;
+  HFileBlock(HFileBlock that) {
+    this.blockType = that.blockType;
+    this.onDiskSizeWithoutHeader = that.onDiskSizeWithoutHeader;
+    this.uncompressedSizeWithoutHeader = that.uncompressedSizeWithoutHeader;
+    this.prevBlockOffset = that.prevBlockOffset;
+    this.buf = that.buf.duplicate();
+    this.offset = that.offset;
+    this.onDiskDataSizeWithHeader = that.onDiskDataSizeWithHeader;
+    this.fileContext = that.fileContext;
+    this.nextBlockOnDiskSizeWithHeader = that.nextBlockOnDiskSizeWithHeader;
+  }
+
+  HFileBlock(ByteBuffer b, boolean usesHBaseChecksum) throws IOException {
+    this(new SingleByteBuff(b), usesHBaseChecksum);
   }
 
   /**
-   * Parse total ondisk size including header and checksum. Its second field in header after
-   * the magic bytes.
-   * @param headerBuf Header ByteBuffer. Presumed exact size of header.
-   * @return Size of the block with header included.
+   * Creates a block from an existing buffer starting with a header. Rewinds
+   * and takes ownership of the buffer. By definition of rewind, ignores the
+   * buffer position, but if you slice the buffer beforehand, it will rewind
+   * to that point.
    */
-  private static int getOnDiskSizeWithHeader(final ByteBuffer headerBuf) {
-    // Set hbase checksum to true always calling headerSize.
-    return headerBuf.getInt(BlockType.MAGIC_LENGTH) + headerSize(true);
+  HFileBlock(ByteBuff b, boolean usesHBaseChecksum) throws IOException {
+    this(b, usesHBaseChecksum, MemoryType.EXCLUSIVE);
   }
 
   /**
-   * @return the on-disk size of the next block (including the header size and any checksums if
-   * present) read by peeking into the next block's header; use as a hint when doing
-   * a read of the next block when scanning or running over a file.
+   * Creates a block from an existing buffer starting with a header. Rewinds
+   * and takes ownership of the buffer. By definition of rewind, ignores the
+   * buffer position, but if you slice the buffer beforehand, it will rewind
+   * to that point.
    */
-  public int getNextBlockOnDiskSize() {
-    return nextBlockOnDiskSize;
+  HFileBlock(ByteBuff b, boolean usesHBaseChecksum, MemoryType memType) throws IOException {
+    b.rewind();
+    blockType = BlockType.read(b);
+    onDiskSizeWithoutHeader = b.getInt();
+    uncompressedSizeWithoutHeader = b.getInt();
+    prevBlockOffset = b.getLong();
+    HFileContextBuilder contextBuilder = new HFileContextBuilder();
+    contextBuilder.withHBaseCheckSum(usesHBaseChecksum);
+    if (usesHBaseChecksum) {
+      contextBuilder.withChecksumType(ChecksumType.codeToType(b.get()));
+      contextBuilder.withBytesPerCheckSum(b.getInt());
+      this.onDiskDataSizeWithHeader = b.getInt();
+    } else {
+      contextBuilder.withChecksumType(ChecksumType.NULL);
+      contextBuilder.withBytesPerCheckSum(0);
+      this.onDiskDataSizeWithHeader =
+          onDiskSizeWithoutHeader + HConstants.HFILEBLOCK_HEADER_SIZE_NO_CHECKSUM;
+    }
+    this.fileContext = contextBuilder.build();
+    this.memType = memType;
+    buf = b;
+    buf.rewind();
   }
 
   public BlockType getBlockType() {
@@ -468,26 +414,49 @@ public class HFileBlock implements Cacheable {
    * @return the buffer with header skipped and checksum omitted.
    */
   public ByteBuff getBufferWithoutHeader() {
-    ByteBuff dup = getBufferReadOnly();
-    // Now set it up so Buffer spans content only -- no header or no checksums.
-    return dup.position(headerSize()).limit(buf.limit() - totalChecksumBytes()).slice();
+    ByteBuff dup = this.buf.duplicate();
+    dup.position(headerSize());
+    dup.limit(buf.limit() - totalChecksumBytes());
+    return dup.slice();
   }
 
   /**
-   * Returns a read-only duplicate of the buffer this block stores internally ready to be read.
-   * Clients must not modify the buffer object though they may set position and limit on the
-   * returned buffer since we pass back a duplicate. This method has to be public because it is used
+   * Returns the buffer this block stores internally. The clients must not
+   * modify the buffer object. This method has to be public because it is used
    * in {@link CompoundBloomFilter} to avoid object creation on every Bloom
-   * filter lookup, but has to be used with caution. Buffer holds header, block content,
-   * and any follow-on checksums if present.
+   * filter lookup, but has to be used with caution. Checksum data is not
+   * included in the returned buffer but header data is.
    *
    * @return the buffer of this block for read-only operations
    */
-  public ByteBuff getBufferReadOnly() {
-    // TODO: ByteBuf does not support asReadOnlyBuffer(). Fix.
+  ByteBuff getBufferReadOnly() {
+    ByteBuff dup = this.buf.duplicate();
+    dup.limit(buf.limit() - totalChecksumBytes());
+    return dup.slice();
+  }
+
+  /**
+   * Returns the buffer of this block, including header data. The clients must
+   * not modify the buffer object. This method has to be public because it is
+   * used in {@link org.apache.hadoop.hbase.io.hfile.bucket.BucketCache} to avoid buffer copy.
+   *
+   * @return the buffer with header and checksum included for read-only operations
+   */
+  public ByteBuff getBufferReadOnlyWithHeader() {
     ByteBuff dup = this.buf.duplicate();
-    assert dup.position() == 0;
-    return dup;
+    return dup.slice();
+  }
+
+  /**
+   * Returns a byte buffer of this block, including header data and checksum, positioned at
+   * the beginning of header. The underlying data array is not copied.
+   *
+   * @return the byte buffer with header and checksum included
+   */
+  ByteBuff getBufferWithHeader() {
+    ByteBuff dupBuf = buf.duplicate();
+    dupBuf.rewind();
+    return dupBuf;
   }
 
   private void sanityCheckAssertion(long valueFromBuf, long valueFromField,
@@ -512,38 +481,39 @@ public class HFileBlock implements Cacheable {
    * valid header consistent with the fields. Assumes a packed block structure.
    * This function is primary for testing and debugging, and is not
    * thread-safe, because it alters the internal buffer pointer.
-   * Used by tests only.
    */
-  @VisibleForTesting
   void sanityCheck() throws IOException {
-    // Duplicate so no side-effects
-    ByteBuff dup = this.buf.duplicate().rewind();
-    sanityCheckAssertion(BlockType.read(dup), blockType);
+    buf.rewind();
+
+    sanityCheckAssertion(BlockType.read(buf), blockType);
 
-    sanityCheckAssertion(dup.getInt(), onDiskSizeWithoutHeader, "onDiskSizeWithoutHeader");
+    sanityCheckAssertion(buf.getInt(), onDiskSizeWithoutHeader,
+        "onDiskSizeWithoutHeader");
 
-    sanityCheckAssertion(dup.getInt(), uncompressedSizeWithoutHeader,
+    sanityCheckAssertion(buf.getInt(), uncompressedSizeWithoutHeader,
         "uncompressedSizeWithoutHeader");
 
-    sanityCheckAssertion(dup.getLong(), prevBlockOffset, "prevBlockOffset");
+    sanityCheckAssertion(buf.getLong(), prevBlockOffset, "prevBlocKOffset");
     if (this.fileContext.isUseHBaseChecksum()) {
-      sanityCheckAssertion(dup.get(), this.fileContext.getChecksumType().getCode(), "checksumType");
-      sanityCheckAssertion(dup.getInt(), this.fileContext.getBytesPerChecksum(),
+      sanityCheckAssertion(buf.get(), this.fileContext.getChecksumType().getCode(), "checksumType");
+      sanityCheckAssertion(buf.getInt(), this.fileContext.getBytesPerChecksum(),
           "bytesPerChecksum");
-      sanityCheckAssertion(dup.getInt(), onDiskDataSizeWithHeader, "onDiskDataSizeWithHeader");
+      sanityCheckAssertion(buf.getInt(), onDiskDataSizeWithHeader, "onDiskDataSizeWithHeader");
     }
 
     int cksumBytes = totalChecksumBytes();
     int expectedBufLimit = onDiskDataSizeWithHeader + cksumBytes;
-    if (dup.limit() != expectedBufLimit) {
-      throw new AssertionError("Expected limit " + expectedBufLimit + ", got " + dup.limit());
+    if (buf.limit() != expectedBufLimit) {
+      throw new AssertionError("Expected buffer limit " + expectedBufLimit
+          + ", got " + buf.limit());
     }
 
     // We might optionally allocate HFILEBLOCK_HEADER_SIZE more bytes to read the next
     // block's header, so there are two sensible values for buffer capacity.
     int hdrSize = headerSize();
-    if (dup.capacity() != expectedBufLimit && dup.capacity() != expectedBufLimit + hdrSize) {
-      throw new AssertionError("Invalid buffer capacity: " + dup.capacity() +
+    if (buf.capacity() != expectedBufLimit &&
+        buf.capacity() != expectedBufLimit + hdrSize) {
+      throw new AssertionError("Invalid buffer capacity: " + buf.capacity() +
           ", expected " + expectedBufLimit + " or " + (expectedBufLimit + hdrSize));
     }
   }
@@ -590,6 +560,30 @@ public class HFileBlock implements Cacheable {
   }
 
   /**
+   * Called after reading a block with provided onDiskSizeWithHeader.
+   */
+  private void validateOnDiskSizeWithoutHeader(int expectedOnDiskSizeWithoutHeader)
+  throws IOException {
+    if (onDiskSizeWithoutHeader != expectedOnDiskSizeWithoutHeader) {
+      String dataBegin = null;
+      if (buf.hasArray()) {
+        dataBegin = Bytes.toStringBinary(buf.array(), buf.arrayOffset(), Math.min(32, buf.limit()));
+      } else {
+        ByteBuff bufDup = getBufferReadOnly();
+        byte[] dataBeginBytes = new byte[Math.min(32, bufDup.limit() - bufDup.position())];
+        bufDup.get(dataBeginBytes);
+        dataBegin = Bytes.toStringBinary(dataBeginBytes);
+      }
+      String blockInfoMsg =
+        "Block offset: " + offset + ", data starts with: " + dataBegin;
+      throw new IOException("On-disk size without header provided is "
+          + expectedOnDiskSizeWithoutHeader + ", but block "
+          + "header contains " + onDiskSizeWithoutHeader + ". " +
+          blockInfoMsg);
+    }
+  }
+
+  /**
    * Retrieves the decompressed/decrypted view of this block. An encoded block remains in its
    * encoded structure. Internal structures are shared between instances where applicable.
    */
@@ -613,10 +607,33 @@ public class HFileBlock implements Cacheable {
     ctx.prepareDecoding(unpacked.getOnDiskSizeWithoutHeader(),
       unpacked.getUncompressedSizeWithoutHeader(), unpacked.getBufferWithoutHeader(),
       dup);
+
+    // Preserve the next block's header bytes in the new block if we have them.
+    if (unpacked.hasNextBlockHeader()) {
+      // Both the buffers are limited till checksum bytes and avoid the next block's header.
+      // Below call to copyFromBufferToBuffer() will try positional read/write from/to buffers when
+      // any of the buffer is DBB. So we change the limit on a dup buffer. No copying just create
+      // new BB objects
+      ByteBuff inDup = this.buf.duplicate();
+      inDup.limit(inDup.limit() + headerSize());
+      ByteBuff outDup = unpacked.buf.duplicate();
+      outDup.limit(outDup.limit() + unpacked.headerSize());
+      outDup.put(
+          unpacked.headerSize() + unpacked.uncompressedSizeWithoutHeader
+              + unpacked.totalChecksumBytes(), inDup, this.onDiskDataSizeWithHeader,
+          unpacked.headerSize());
+    }
     return unpacked;
   }
 
   /**
+   * Return true when this buffer includes next block's header.
+   */
+  private boolean hasNextBlockHeader() {
+    return nextBlockOnDiskSizeWithHeader > 0;
+  }
+
+  /**
    * Always allocates a new buffer of the correct size. Copies header bytes
    * from the existing buffer. Does not change header fields.
    * Reserve room to keep checksum bytes too.
@@ -624,7 +641,8 @@ public class HFileBlock implements Cacheable {
   private void allocateBuffer() {
     int cksumBytes = totalChecksumBytes();
     int headerSize = headerSize();
-    int capacityNeeded = headerSize + uncompressedSizeWithoutHeader + cksumBytes;
+    int capacityNeeded = headerSize + uncompressedSizeWithoutHeader +
+        cksumBytes + (hasNextBlockHeader() ? headerSize : 0);
 
     // TODO we need consider allocating offheap here?
     ByteBuffer newBuf = ByteBuffer.allocate(capacityNeeded);
@@ -652,8 +670,9 @@ public class HFileBlock implements Cacheable {
   }
 
   /** An additional sanity-check in case no compression or encryption is being used. */
-  public void sanityCheckUncompressedSize() throws IOException {
-    if (onDiskSizeWithoutHeader != uncompressedSizeWithoutHeader + totalChecksumBytes()) {
+  public void assumeUncompressed() throws IOException {
+    if (onDiskSizeWithoutHeader != uncompressedSizeWithoutHeader +
+        totalChecksumBytes()) {
       throw new IOException("Using no compression but "
           + "onDiskSizeWithoutHeader=" + onDiskSizeWithoutHeader + ", "
           + "uncompressedSizeWithoutHeader=" + uncompressedSizeWithoutHeader
@@ -661,14 +680,11 @@ public class HFileBlock implements Cacheable {
     }
   }
 
-  /**
-   * Cannot be {@link #UNSET}. Must be a legitimate value. Used re-making the {@link CacheKey} when
-   * block is returned to the cache.
-   * @return the offset of this block in the file it was read from
-   */
+  /** @return the offset of this block in the file it was read from */
   long getOffset() {
     if (offset < 0) {
-      throw new IllegalStateException("HFile block offset not initialized properly");
+      throw new IllegalStateException(
+          "HFile block offset not initialized properly");
     }
     return offset;
   }
@@ -728,6 +744,7 @@ public class HFileBlock implements Cacheable {
         // We could not read the "extra data", but that is OK.
         break;
       }
+
       if (ret < 0) {
         throw new IOException("Premature EOF from inputStream (read "
             + "returned " + ret + ", was trying to read " + necessaryLen
@@ -782,6 +799,14 @@ public class HFileBlock implements Cacheable {
   }
 
   /**
+   * @return the on-disk size of the next block (including the header size)
+   *         that was read by peeking into the next block's header
+   */
+  public int getNextBlockOnDiskSizeWithHeader() {
+    return nextBlockOnDiskSizeWithHeader;
+  }
+
+  /**
    * Unified version 2 {@link HFile} block writer. The intended usage pattern
    * is as follows:
    * <ol>
@@ -813,8 +838,8 @@ public class HFileBlock implements Cacheable {
     private HFileBlockDefaultEncodingContext defaultBlockEncodingCtx;
 
     /**
-     * The stream we use to accumulate data into a block in an uncompressed format.
-     * We reset this stream at the end of each block and reuse it. The
+     * The stream we use to accumulate data in uncompressed format for each
+     * block. We reset this stream at the end of each block and reuse it. The
      * header is written as the first {@link HConstants#HFILEBLOCK_HEADER_SIZE} bytes into this
      * stream.
      */
@@ -842,7 +867,7 @@ public class HFileBlock implements Cacheable {
      * if compression is turned on. It also includes the checksum data that
      * immediately follows the block data. (header + data + checksums)
      */
-    private byte[] onDiskBlockBytesWithHeader;
+    private byte[] onDiskBytesWithHeader;
 
     /**
      * The size of the checksum data on disk. It is used only if data is
@@ -859,7 +884,7 @@ public class HFileBlock implements Cacheable {
      * {@link org.apache.hadoop.hbase.HConstants#HFILEBLOCK_HEADER_SIZE}.
      * Does not store checksums.
      */
-    private byte[] uncompressedBlockBytesWithHeader;
+    private byte[] uncompressedBytesWithHeader;
 
     /**
      * Current block's start offset in the {@link HFile}. Set in
@@ -967,19 +992,18 @@ public class HFileBlock implements Cacheable {
       Preconditions.checkState(state != State.INIT,
           "Unexpected state: " + state);
 
-      if (state == State.BLOCK_READY) {
+      if (state == State.BLOCK_READY)
         return;
-      }
 
       // This will set state to BLOCK_READY.
       finishBlock();
     }
 
     /**
-     * Finish up writing of the block.
-     * Flushes the compressing stream (if using compression), fills out the header,
-     * does any compression/encryption of bytes to flush out to disk, and manages
-     * the cache on write content, if applicable. Sets block write state to "block ready".
+     * An internal method that flushes the compressing stream (if using
+     * compression), serializes the header, and takes care of the separate
+     * uncompressed stream for caching on write, if applicable. Sets block
+     * write state to "block ready".
      */
     private void finishBlock() throws IOException {
       if (blockType == BlockType.DATA) {
@@ -988,40 +1012,41 @@ public class HFileBlock implements Cacheable {
         blockType = dataBlockEncodingCtx.getBlockType();
       }
       userDataStream.flush();
-      // This does an array copy, so it is safe to cache this byte array when cache-on-write.
+      // This does an array copy, so it is safe to cache this byte array.
       // Header is still the empty, 'dummy' header that is yet to be filled out.
-      uncompressedBlockBytesWithHeader = baosInMemory.toByteArray();
+      uncompressedBytesWithHeader = baosInMemory.toByteArray();
       prevOffset = prevOffsetByType[blockType.getId()];
 
-      // We need to set state before we can package the block up for cache-on-write. In a way, the
-      // block is ready, but not yet encoded or compressed.
+      // We need to set state before we can package the block up for
+      // cache-on-write. In a way, the block is ready, but not yet encoded or
+      // compressed.
       state = State.BLOCK_READY;
       if (blockType == BlockType.DATA || blockType == BlockType.ENCODED_DATA) {
-        onDiskBlockBytesWithHeader = dataBlockEncodingCtx.
-            compressAndEncrypt(uncompressedBlockBytesWithHeader);
+        onDiskBytesWithHeader = dataBlockEncodingCtx
+            .compressAndEncrypt(uncompressedBytesWithHeader);
       } else {
-        onDiskBlockBytesWithHeader = defaultBlockEncodingCtx.
-            compressAndEncrypt(uncompressedBlockBytesWithHeader);
+        onDiskBytesWithHeader = this.defaultBlockEncodingCtx.
+            compressAndEncrypt(uncompressedBytesWithHeader);
       }
       // Calculate how many bytes we need for checksum on the tail of the block.
       int numBytes = (int) ChecksumUtil.numBytes(
-          onDiskBlockBytesWithHeader.length,
+          onDiskBytesWithHeader.length,
           fileContext.getBytesPerChecksum());
 
       // Put the header for the on disk bytes; header currently is unfilled-out
-      putHeader(onDiskBlockBytesWithHeader, 0,
-          onDiskBlockBytesWithHeader.length + numBytes,
-          uncompressedBlockBytesWithHeader.length, onDiskBlockBytesWithHeader.length);
+      putHeader(onDiskBytesWithHeader, 0,
+          onDiskBytesWithHeader.length + numBytes,
+          uncompressedBytesWithHeader.length, onDiskBytesWithHeader.length);
       // Set the header for the uncompressed bytes (for cache-on-write) -- IFF different from
-      // onDiskBlockBytesWithHeader array.
-      if (onDiskBlockBytesWithHeader != uncompressedBlockBytesWithHeader) {
-        putHeader(uncompressedBlockBytesWithHeader, 0,
-          onDiskBlockBytesWithHeader.length + numBytes,
-          uncompressedBlockBytesWithHeader.length, onDiskBlockBytesWithHeader.length);
+      // onDiskBytesWithHeader array.
+      if (onDiskBytesWithHeader != uncompressedBytesWithHeader) {
+        putHeader(uncompressedBytesWithHeader, 0,
+          onDiskBytesWithHeader.length + numBytes,
+          uncompressedBytesWithHeader.length, onDiskBytesWithHeader.length);
       }
       onDiskChecksum = new byte[numBytes];
       ChecksumUtil.generateChecksums(
-          onDiskBlockBytesWithHeader, 0, onDiskBlockBytesWithHeader.length,
+          onDiskBytesWithHeader, 0, onDiskBytesWithHeader.length,
           onDiskChecksum, 0, fileContext.getChecksumType(), fileContext.getBytesPerChecksum());
     }
 
@@ -1076,7 +1101,7 @@ public class HFileBlock implements Cacheable {
     protected void finishBlockAndWriteHeaderAndData(DataOutputStream out)
       throws IOException {
       ensureBlockReady();
-      out.write(onDiskBlockBytesWithHeader);
+      out.write(onDiskBytesWithHeader);
       out.write(onDiskChecksum);
     }
 
@@ -1095,12 +1120,12 @@ public class HFileBlock implements Cacheable {
       // This is not very optimal, because we are doing an extra copy.
       // But this method is used only by unit tests.
       byte[] output =
-          new byte[onDiskBlockBytesWithHeader.length
+          new byte[onDiskBytesWithHeader.length
               + onDiskChecksum.length];
-      System.arraycopy(onDiskBlockBytesWithHeader, 0, output, 0,
-          onDiskBlockBytesWithHeader.length);
+      System.arraycopy(onDiskBytesWithHeader, 0, output, 0,
+          onDiskBytesWithHeader.length);
       System.arraycopy(onDiskChecksum, 0, output,
-          onDiskBlockBytesWithHeader.length, onDiskChecksum.length);
+          onDiskBytesWithHeader.length, onDiskChecksum.length);
       return output;
     }
 
@@ -1128,7 +1153,7 @@ public class HFileBlock implements Cacheable {
      */
     int getOnDiskSizeWithoutHeader() {
       expectState(State.BLOCK_READY);
-      return onDiskBlockBytesWithHeader.length +
+      return onDiskBytesWithHeader.length +
           onDiskChecksum.length - HConstants.HFILEBLOCK_HEADER_SIZE;
     }
 
@@ -1141,7 +1166,7 @@ public class HFileBlock implements Cacheable {
      */
     int getOnDiskSizeWithHeader() {
       expectState(State.BLOCK_READY);
-      return onDiskBlockBytesWithHeader.length + onDiskChecksum.length;
+      return onDiskBytesWithHeader.length + onDiskChecksum.length;
     }
 
     /**
@@ -1149,7 +1174,7 @@ public class HFileBlock implements Cacheable {
      */
     int getUncompressedSizeWithoutHeader() {
       expectState(State.BLOCK_READY);
-      return uncompressedBlockBytesWithHeader.length - HConstants.HFILEBLOCK_HEADER_SIZE;
+      return uncompressedBytesWithHeader.length - HConstants.HFILEBLOCK_HEADER_SIZE;
     }
 
     /**
@@ -1157,7 +1182,7 @@ public class HFileBlock implements Cacheable {
      */
     int getUncompressedSizeWithHeader() {
       expectState(State.BLOCK_READY);
-      return uncompressedBlockBytesWithHeader.length;
+      return uncompressedBytesWithHeader.length;
     }
 
     /** @return true if a block is being written  */
@@ -1187,7 +1212,7 @@ public class HFileBlock implements Cacheable {
      */
     ByteBuffer getUncompressedBufferWithHeader() {
       expectState(State.BLOCK_READY);
-      return ByteBuffer.wrap(uncompressedBlockBytesWithHeader);
+      return ByteBuffer.wrap(uncompressedBytesWithHeader);
     }
 
     /**
@@ -1200,7 +1225,7 @@ public class HFileBlock implements Cacheable {
      */
     ByteBuffer getOnDiskBufferWithHeader() {
       expectState(State.BLOCK_READY);
-      return ByteBuffer.wrap(onDiskBlockBytesWithHeader);
+      return ByteBuffer.wrap(onDiskBytesWithHeader);
     }
 
     private void expectState(State expectedState) {
@@ -1232,10 +1257,6 @@ public class HFileBlock implements Cacheable {
      * block does not have checksum data even though the header minor
      * version is MINOR_VERSION_WITH_CHECKSUM. This is indicated by setting a
      * 0 value in bytesPerChecksum.
-     *
-     * <p>TODO: Should there be an option where a cache can ask that hbase preserve block
-     * checksums for checking after a block comes out of the cache? Otehrwise, cache is responsible
-     * for blocks being wholesome (ECC memory or if file-backed, it does checksumming).
      */
     HFileBlock getBlockForCaching(CacheConfig cacheConf) {
       HFileContext newContext = new HFileContextBuilder()
@@ -1249,13 +1270,13 @@ public class HFileBlock implements Cacheable {
                                 .withIncludesMvcc(fileContext.isIncludesMvcc())
                                 .withIncludesTags(fileContext.isIncludesTags())
                                 .build();
-       return new HFileBlock(blockType, getOnDiskSizeWithoutHeader(),
+      return new HFileBlock(blockType, getOnDiskSizeWithoutHeader(),
           getUncompressedSizeWithoutHeader(), prevOffset,
-          cacheConf.shouldCacheCompressed(blockType.getCategory())?
+          cacheConf.shouldCacheCompressed(blockType.getCategory()) ?
             getOnDiskBufferWithHeader() :
             getUncompressedBufferWithHeader(),
-          FILL_HEADER, startOffset, UNSET,
-          onDiskBlockBytesWithHeader.length + onDiskChecksum.length, newContext);
+          FILL_HEADER, startOffset,
+          onDiskBytesWithHeader.length + onDiskChecksum.length, newContext);
     }
   }
 
@@ -1301,9 +1322,12 @@ public class HFileBlock implements Cacheable {
      * @param offset
      * @param onDiskSize the on-disk size of the entire block, including all
      *          applicable headers, or -1 if unknown
+     * @param uncompressedSize the uncompressed size of the compressed part of
+     *          the block, or -1 if unknown
      * @return the newly read block
      */
-    HFileBlock readBlockData(long offset, long onDiskSize, boolean pread) throws IOException;
+    HFileBlock readBlockData(long offset, long onDiskSize,
+        int uncompressedSize, boolean pread) throws IOException;
 
     /**
      * Creates a block iterator over the given portion of the {@link HFile}.
@@ -1356,11 +1380,6 @@ public class HFileBlock implements Cacheable {
     /** Default context used when BlockType != {@link BlockType#ENCODED_DATA}. */
     private final HFileBlockDefaultDecodingContext defaultDecodingCtx;
 
-    /**
-     * When we read a block, we overread and pull in the next blocks header too. We will save it
-     * here. If moving serially through the file, we will trip over this caching of the next blocks
-     * header so we won't have to do explicit seek to find next blocks lengths, etc.
-     */
     private ThreadLocal<PrefetchedHeader> prefetchedHeaderForThread =
         new ThreadLocal<PrefetchedHeader>() {
       @Override
@@ -1424,7 +1443,7 @@ public class HFileBlock implements Cacheable {
         public HFileBlock nextBlock() throws IOException {
           if (offset >= endOffset)
             return null;
-          HFileBlock b = readBlockData(offset, -1, false);
+          HFileBlock b = readBlockData(offset, -1, -1, false);
           offset += b.getOnDiskSizeWithHeader();
           return b.unpack(fileContext, owner);
         }
@@ -1444,7 +1463,7 @@ public class HFileBlock implements Cacheable {
 
     /**
      * Does a positional read or a seek and read into the given buffer. Returns
-     * the on-disk size of the next block, or -1 if it could not be read/determined; e.g. EOF.
+     * the on-disk size of the next block, or -1 if it could not be determined.
      *
      * @param dest destination buffer
      * @param destOffset offset into the destination buffer at where to put the bytes we read
@@ -1454,8 +1473,7 @@ public class HFileBlock implements Cacheable {
      * @param pread whether we should do a positional read
      * @param istream The input source of data
      * @return the on-disk size of the next block with header size included, or
-     *         -1 if it could not be determined; if not -1, the <code>dest</code> INCLUDES the
-     *         next header
+     *         -1 if it could not be determined
      * @throws IOException
      */
     protected int readAtOffset(FSDataInputStream istream, byte [] dest, int destOffset, int size,
@@ -1487,16 +1505,16 @@ public class HFileBlock implements Cacheable {
           }
 
           // Try to read the next block header.
-          if (!readWithExtra(istream, dest, destOffset, size, hdrSize)) {
+          if (!readWithExtra(istream, dest, destOffset, size, hdrSize))
             return -1;
-          }
         } finally {
           streamLock.unlock();
         }
       } else {
         // Positional read. Better for random reads; or when the streamLock is already locked.
         int extraSize = peekIntoNextBlock ? hdrSize : 0;
-        if (!positionalReadWithExtra(istream, fileOffset, dest, destOffset, size, extraSize)) {
+        if (!positionalReadWithExtra(istream, fileOffset, dest, destOffset,
+            size, extraSize)) {
           return -1;
         }
       }
@@ -1512,12 +1530,16 @@ public class HFileBlock implements Cacheable {
      * @param offset the offset in the stream to read at
      * @param onDiskSizeWithHeaderL the on-disk size of the block, including
      *          the header, or -1 if unknown
+     * @param uncompressedSize the uncompressed size of the the block. Always
+     *          expected to be -1. This parameter is only used in version 1.
      * @param pread whether to use a positional read
      */
     @Override
-    public HFileBlock readBlockData(long offset, long onDiskSizeWithHeaderL, boolean pread)
+    public HFileBlock readBlockData(long offset, long onDiskSizeWithHeaderL,
+        int uncompressedSize, boolean pread)
     throws IOException {
-      // Get a copy of the current state of whether to validate
+
+      // get a copy of the current state of whether to validate
       // hbase checksums or not for this read call. This is not
       // thread-safe but the one constaint is that if we decide
       // to skip hbase checksum verification then we are
@@ -1526,7 +1548,8 @@ public class HFileBlock implements Cacheable {
       FSDataInputStream is = streamWrapper.getStream(doVerificationThruHBaseChecksum);
 
       HFileBlock blk = readBlockDataInternal(is, offset,
-                         onDiskSizeWithHeaderL, pread,
+                         onDiskSizeWithHeaderL,
+                         uncompressedSize, pread,
                          doVerificationThruHBaseChecksum);
       if (blk == null) {
         HFile.LOG.warn("HBase checksum verification failed for file " +
@@ -1553,7 +1576,8 @@ public class HFileBlock implements Cacheable {
         // a few more than precisely this number.
         is = this.streamWrapper.fallbackToFsChecksum(CHECKSUM_VERIFICATION_NUM_IO_THRESHOLD);
         doVerificationThruHBaseChecksum = false;
-        blk = readBlockDataInternal(is, offset, onDiskSizeWithHeaderL, pread,
+        blk = readBlockDataInternal(is, offset, onDiskSizeWithHeaderL,
+                                    uncompressedSize, pread,
                                     doVerificationThruHBaseChecksum);
         if (blk != null) {
           HFile.LOG.warn("HDFS checksum verification suceeded for file " +
@@ -1581,139 +1605,175 @@ public class HFileBlock implements Cacheable {
     }
 
     /**
-     * @return Check <code>onDiskSizeWithHeaderL</code> size is healthy and then return it as an int
-     * @throws IOException
-     */
-    private static int checkAndGetSizeAsInt(final long onDiskSizeWithHeaderL, final int hdrSize)
-    throws IOException {
-      if ((onDiskSizeWithHeaderL < hdrSize && onDiskSizeWithHeaderL != -1)
-          || onDiskSizeWithHeaderL >= Integer.MAX_VALUE) {
-        throw new IOException("Invalid onDisksize=" + onDiskSizeWithHeaderL
-            + ": expected to be at least " + hdrSize
-            + " and at most " + Integer.MAX_VALUE + ", or -1");
-      }
-      return (int)onDiskSizeWithHeaderL;
-    }
-
-    /**
-     * Check threadlocal cache for this block's header; we usually read it on the tail of reading
-     * the previous block to save a seek. Otherwise, we have to do a seek to read the header before
-     * we can pull in the block.
-     * @return The cached block header or null if not found.
-     * @see #cacheNextBlockHeader(long, byte[], int, int)
-     */
-    private ByteBuffer getCachedHeader(final long offset) {
-      PrefetchedHeader prefetchedHeader = prefetchedHeaderForThread.get();
-      // PrefetchedHeader prefetchedHeader = prefetchedHeaderForThread.get();
-      return prefetchedHeader != null && prefetchedHeader.offset == offset?
-          prefetchedHeader.buf: null;
-    }
-
-    /**
-     * Save away the next blocks header in thread local.
-     * @see #getCachedHeader(long)
-     */
-    private void cacheNextBlockHeader(final long nextBlockOffset,
-        final byte [] header, final int headerOffset, final int headerLength) {
-      PrefetchedHeader prefetchedHeader = prefetchedHeaderForThread.get();
-      prefetchedHeader.offset = nextBlockOffset;
-      System.arraycopy(header, headerOffset, prefetchedHeader.header, 0, headerLength);
-    }
-
-    /**
-     * Verify the passed in onDiskSizeWithHeader aligns with what is in the header else something
-     * is not right.
-     * @throws IOException
-     */
-    private void verifyOnDiskSizeMatchesHeader(final int passedIn, final ByteBuffer headerBuf,
-        final long offset)
-    throws IOException {
-      // Assert size provided aligns with what is in the header
-      int fromHeader = getOnDiskSizeWithHeader(headerBuf);
-      if (passedIn != fromHeader) {
-        throw new IOException("Passed in onDiskSizeWithHeader=" + passedIn + " != " + fromHeader +
-            ", offset=" + offset + ", fileContext=" + this.fileContext);
-      }
-    }
-
-    /**
      * Reads a version 2 block.
      *
      * @param offset the offset in the stream to read at
      * @param onDiskSizeWithHeaderL the on-disk size of the block, including
-     *          the header and checksums if present or -1 if unknown
+     *          the header, or -1 if unknown
+     * @param uncompressedSize the uncompressed size of the the block. Always
+     *          expected to be -1. This parameter is only used in version 1.
      * @param pread whether to use a positional read
      * @param verifyChecksum Whether to use HBase checksums.
      *        If HBase checksum is switched off, then use HDFS checksum.
      * @return the HFileBlock or null if there is a HBase checksum mismatch
      */
     private HFileBlock readBlockDataInternal(FSDataInputStream is, long offset,
-        long onDiskSizeWithHeaderL, boolean pread, boolean verifyChecksum)
+        long onDiskSizeWithHeaderL, int uncompressedSize, boolean pread,
+        boolean verifyChecksum)
     throws IOException {
       if (offset < 0) {
         throw new IOException("Invalid offset=" + offset + " trying to read "
-            + "block (onDiskSize=" + onDiskSizeWithHeaderL + ")");
+            + "block (onDiskSize=" + onDiskSizeWithHeaderL
+            + ", uncompressedSize=" + uncompressedSize + ")");
       }
-      int onDiskSizeWithHeader = checkAndGetSizeAsInt(onDiskSizeWithHeaderL, hdrSize);
-      ByteBuffer headerBuf = getCachedHeader(offset);
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("Reading " + this.fileContext.getHFileName() + " at offset=" + offset +
-          ", pread=" + pread + ", verifyChecksum=" + verifyChecksum + ", cachedHeader=" +
-          headerBuf + ", onDiskSizeWithHeader=" + onDiskSizeWithHeader);
+
+      if (uncompressedSize != -1) {
+        throw new IOException("Version 2 block reader API does not need " +
+            "the uncompressed size parameter");
+      }
+
+      if ((onDiskSizeWithHeaderL < hdrSize && onDiskSizeWithHeaderL != -1)
+          || onDiskSizeWithHeaderL >= Integer.MAX_VALUE) {
+        throw new IOException("Invalid onDisksize=" + onDiskSizeWithHeaderL
+            + ": expected to be at least " + hdrSize
+            + " and at most " + Integer.MAX_VALUE + ", or -1 (offset="
+            + offset + ", uncompressedSize=" + uncompressedSize + ")");
+      }
+
+      int onDiskSizeWithHeader = (int) onDiskSizeWithHeaderL;
+
+      // See if we can avoid reading the header. This is desirable, because
+      // we will not incur a backward seek operation if we have already
+      // read this block's header as part of the previous read's look-ahead.
+      // And we also want to skip reading the header again if it has already
+      // been read.
+      // TODO: How often does this optimization fire? Has to be same thread so the thread local
+      // is pertinent and we have to be reading next block as in a big scan.
+      ByteBuffer headerBuf = null;
+      PrefetchedHeader prefetchedHeader = prefetchedHeaderForThread.get();
+      boolean preReadHeader = false;
+      if (prefetchedHeader != null && prefetchedHeader.offset == offset) {
+        headerBuf = prefetchedHeader.buf;
+        preReadHeader = true;
       }
-      if (onDiskSizeWithHeader <= 0) {
-        // We were not passed the block size. Need to get it from the header. If header was not in
-        // cache, need to seek to pull it in. This latter might happen when we are doing the first
-        // read in a series of reads or a random read, and we don't have access to the block index.
-        // This is costly and should happen very rarely.
+      // Allocate enough space to fit the next block's header too.
+      int nextBlockOnDiskSize = 0;
+      byte[] onDiskBlock = null;
+
+      HFileBlock b = null;
+      boolean fastPath = false;
+      boolean readHdrOnly = false;
+      if (onDiskSizeWithHeader > 0) {
+        fastPath = true;
+        // We know the total on-disk size. Read the entire block into memory,
+        // then parse the header. This code path is used when
+        // doing a random read operation relying on the block index, as well as
+        // when the client knows the on-disk size from peeking into the next
+        // block's header (e.g. this block's header) when reading the previous
+        // block. This is the faster and more preferable case.
+
+        // Size that we have to skip in case we have already read the header.
+        int preReadHeaderSize = headerBuf == null ? 0 : hdrSize;
+        onDiskBlock = new byte[onDiskSizeWithHeader + hdrSize]; // room for this block plus the
+                                                                // next block's header
+        nextBlockOnDiskSize = readAtOffset(is, onDiskBlock,
+            preReadHeaderSize, onDiskSizeWithHeader - preReadHeaderSize,
+            true, offset + preReadHeaderSize, pread);
+        if (headerBuf != null) {
+          // the header has been read when reading the previous block, copy
+          // to this block's header
+          // headerBuf is HBB
+          assert headerBuf.hasArray();
+          System.arraycopy(headerBuf.array(),
+              headerBuf.arrayOffset(), onDiskBlock, 0, hdrSize);
+        } else {
+          headerBuf = ByteBuffer.wrap(onDiskBlock, 0, hdrSize);
+        }
+        // We know the total on-disk size but not the uncompressed size. Parse the header.
+        try {
+          // TODO: FIX!!! Expensive parse just to get a length
+          b = new HFileBlock(headerBuf, fileContext.isUseHBaseChecksum());
+        } catch (IOException ex) {
+          // Seen in load testing. Provide comprehensive debug info.
+          throw new IOException("Failed to read compressed block at "
+              + offset
+              + ", onDiskSizeWithoutHeader="
+              + onDiskSizeWithHeader
+              + ", preReadHeaderSize="
+              + hdrSize
+              + ", header.length="
+              + prefetchedHeader.header.length
+              + ", header bytes: "
+              + Bytes.toStringBinary(prefetchedHeader.header, 0,
+                  hdrSize), ex);
+        }
+        // if the caller specifies a onDiskSizeWithHeader, validate it.
+        int onDiskSizeWithoutHeader = onDiskSizeWithHeader - hdrSize;
+        assert onDiskSizeWithoutHeader >= 0;
+        b.validateOnDiskSizeWithoutHeader(onDiskSizeWithoutHeader);
+      } else {
+        // Check headerBuf to see if we have read this block's header as part of
+        // reading the previous block. This is an optimization of peeking into
+        // the next block's header (e.g.this block's header) when reading the
+        // previous block. This is the faster and more preferable case. If the
+        // header is already there, don't read the header again.
+
+        // Unfortunately, we still have to do a separate read operation to
+        // read the header.
         if (headerBuf == null) {
+          readHdrOnly = true;
+          // From the header, determine the on-disk size of the given hfile
+          // block, and read the remaining data, thereby incurring two read
+          // operations. This might happen when we are doing the first read
+          // in a series of reads or a random read, and we don't have access
+          // to the block index. This is costly and should happen very rarely.
           headerBuf = ByteBuffer.allocate(hdrSize);
-          readAtOffset(is, headerBuf.array(), headerBuf.arrayOffset(), hdrSize, false,
-              offset, pread);
+          // headerBuf is HBB
+          readAtOffset(is, headerBuf.array(), headerBuf.arrayOffset(),
+              hdrSize, false, offset, pread);
         }
-        onDiskSizeWithHeader = getOnDiskSizeWithHeader(headerBuf);
-      }
-      int preReadHeaderSize = headerBuf == null? 0 : hdrSize;
-      // Allocate enough space to fit the next block's header too; saves a seek next time through.
-      // onDiskBlock is whole block + header + checksums then extra hdrSize to read next header;
-      // onDiskSizeWithHeader is header, body, and any checksums if present.
-      // TODO: Make this ByteBuffer-based. Will make it easier to go to HDFS with BBPool (offheap).
-      byte[] onDiskBlock = new byte[onDiskSizeWithHeader + hdrSize];
-      int nextBlockOnDiskSize = readAtOffset(is, onDiskBlock, preReadHeaderSize,
-          onDiskSizeWithHeader - preReadHeaderSize, true, offset + preReadHeaderSize, pread);
-      if (headerBuf != null) {
-        // The header has been read when reading the previous block OR in a distinct header-only
-        // read. Copy to this block's header.
+        // TODO: FIX!!! Expensive parse just to get a length
+        b = new HFileBlock(headerBuf, fileContext.isUseHBaseChecksum());
+        // onDiskBlock is whole block + header + checksums then extra hdrSize to read next header
+        onDiskBlock = new byte[b.getOnDiskSizeWithHeader() + hdrSize];
+        // headerBuf is HBB. Copy hdr into onDiskBlock
         System.arraycopy(headerBuf.array(), headerBuf.arrayOffset(), onDiskBlock, 0, hdrSize);
-      } else {
-        headerBuf = ByteBuffer.wrap(onDiskBlock, 0, hdrSize);
+        nextBlockOnDiskSize = readAtOffset(is, onDiskBlock, hdrSize,
+            b.getOnDiskSizeWithHeader() - hdrSize, true, offset + hdrSize, pread);
+        onDiskSizeWithHeader = b.onDiskSizeWithoutHeader + hdrSize;
       }
-      // Do a few checks before we go instantiate HFileBlock.
-      assert onDiskSizeWithHeader > this.hdrSize;
-      verifyOnDiskSizeMatchesHeader(onDiskSizeWithHeader, headerBuf, offset);
-      // The onDiskBlock will become the headerAndDataBuffer for this block.
-      // If nextBlockOnDiskSizeWithHeader is not zero, the onDiskBlock already
-      // contains the header of next block, so no need to set next block's header in it.
-      HFileBlock hFileBlock =
-          new HFileBlock(new SingleByteBuff(ByteBuffer.wrap(onDiskBlock, 0, onDiskSizeWithHeader)),
-              this.fileContext.isUseHBaseChecksum(), MemoryType.EXCLUSIVE, offset,
-              nextBlockOnDiskSize, fileContext);
-      // Run check on uncompressed sizings.
+
       if (!fileContext.isCompressedOrEncrypted()) {
-        hFileBlock.sanityCheckUncompressed();
+        b.assumeUncompressed();
       }
-      if (verifyChecksum && !validateBlockChecksum(hFileBlock, offset, onDiskBlock, hdrSize)) {
-        return null;
+
+      if (verifyChecksum && !validateBlockChecksum(b, offset, onDiskBlock, hdrSize)) {
+        return null;             // checksum mismatch
       }
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("Read " + hFileBlock);
+
+      // The onDiskBlock will become the headerAndDataBuffer for this block.
+      // If nextBlockOnDiskSizeWithHeader is not zero, the onDiskBlock already
+      // contains the header of next block, so no need to set next
+      // block's header in it.
+      b = new HFileBlock(ByteBuffer.wrap(onDiskBlock, 0, onDiskSizeWithHeader),
+        this.fileContext.isUseHBaseChecksum());
+
+      b.nextBlockOnDiskSizeWithHeader = nextBlockOnDiskSize;
+
+      // Set prefetched header
+      if (b.hasNextBlockHeader()) {
+        prefetchedHeader.offset = offset + b.getOnDiskSizeWithHeader();
+        System.arraycopy(onDiskBlock, onDiskSizeWithHeader, prefetchedHeader.header, 0, hdrSize);
       }
-      // Cache next block header if we read it for the next time through here.
-      if (nextBlockOnDiskSize != -1) {
-        cacheNextBlockHeader(offset + hFileBlock.getOnDiskSizeWithHeader(),
-            onDiskBlock, onDiskSizeWithHeader, hdrSize);
+
+      b.offset = offset;
+      b.fileContext.setIncludesTags(this.fileContext.isIncludesTags());
+      b.fileContext.setIncludesMvcc(this.fileContext.isIncludesMvcc());
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Read preReadHeader=" + preReadHeader + ", fastPath=" + fastPath +
+            ", readHdrOnly=" + readHdrOnly + ", " + b);
       }
-      return hFileBlock;
+      return b;
     }
 
     @Override
@@ -1759,73 +1819,42 @@ public class HFileBlock implements Cacheable {
     }
   }
 
-  /** An additional sanity-check in case no compression or encryption is being used. */
-  void sanityCheckUncompressed() throws IOException {
-    if (onDiskSizeWithoutHeader != uncompressedSizeWithoutHeader +
-        totalChecksumBytes()) {
-      throw new IOException("Using no compression but "
-          + "onDiskSizeWithoutHeader=" + onDiskSizeWithoutHeader + ", "
-          + "uncompressedSizeWithoutHeader=" + uncompressedSizeWithoutHeader
-          + ", numChecksumbytes=" + totalChecksumBytes());
-    }
-  }
-
-  // Cacheable implementation
   @Override
   public int getSerializedLength() {
     if (buf != null) {
-      // Include extra bytes for block metadata.
-      return this.buf.limit() + BLOCK_METADATA_SPACE;
+      // include extra bytes for the next header when it's available.
+      int extraSpace = hasNextBlockHeader() ? headerSize() : 0;
+      return this.buf.limit() + extraSpace + HFileBlock.EXTRA_SERIALIZATION_SPACE;
     }
     return 0;
   }
 
-  // Cacheable implementation
   @Override
   public void serialize(ByteBuffer destination) {
-    // BE CAREFUL!! There is a custom version of this serialization over in BucketCache#doDrain.
-    // Make sure any changes in here are reflected over there.
-    this.buf.get(destination, 0, getSerializedLength() - BLOCK_METADATA_SPACE);
-    destination = addMetaData(destination);
-
-    // Make it ready for reading. flip sets position to zero and limit to current position which
-    // is what we want if we do not want to serialize the block plus checksums if present plus
-    // metadata.
-    destination.flip();
-  }
-
-  /**
-   * For use by bucketcache. This exposes internals.
-   */
-  public ByteBuffer getMetaData() {
-    ByteBuffer bb = ByteBuffer.allocate(BLOCK_METADATA_SPACE);
-    bb = addMetaData(bb);
-    bb.flip();
-    return bb;
+    this.buf.get(destination, 0, getSerializedLength() - EXTRA_SERIALIZATION_SPACE);
+    serializeExtraInfo(destination);
   }
 
   /**
-   * Adds metadata at current position (position is moved forward). Does not flip or reset.
-   * @return The passed <code>destination</code> with metadata added.
+   * Write out the content of EXTRA_SERIALIZATION_SPACE. Public so can be accessed by BucketCache.
    */
-  private ByteBuffer addMetaData(final ByteBuffer destination) {
+  public void serializeExtraInfo(ByteBuffer destination) {
     destination.put(this.fileContext.isUseHBaseChecksum() ? (byte) 1 : (byte) 0);
     destination.putLong(this.offset);
-    destination.putInt(this.nextBlockOnDiskSize);
-    return destination;
+    destination.putInt(this.nextBlockOnDiskSizeWithHeader);
+    destination.rewind();
   }
 
-  // Cacheable implementation
   @Override
   public CacheableDeserializer<Cacheable> getDeserializer() {
-    return HFileBlock.BLOCK_DESERIALIZER;
+    return HFileBlock.blockDeserializer;
   }
 
   @Override
   public int hashCode() {
     int result = 1;
     result = result * 31 + blockType.hashCode();
-    result = result * 31 + nextBlockOnDiskSize;
+    result = result * 31 + nextBlockOnDiskSizeWithHeader;
     result = result * 31 + (int) (offset ^ (offset >>> 32));
     result = result * 31 + onDiskSizeWithoutHeader;
     result = result * 31 + (int) (prevBlockOffset ^ (prevBlockOffset >>> 32));
@@ -1851,10 +1880,9 @@ public class HFileBlock implements Cacheable {
     if (castedComparison.blockType != this.blockType) {
       return false;
     }
-    if (castedComparison.nextBlockOnDiskSize != this.nextBlockOnDiskSize) {
+    if (castedComparison.nextBlockOnDiskSizeWithHeader != this.nextBlockOnDiskSizeWithHeader) {
       return false;
     }
-    // Offset is important. Needed when we have to remake cachekey when block is returned to cache.
     if (castedComparison.offset != this.offset) {
       return false;
     }
@@ -1940,7 +1968,7 @@ public class HFileBlock implements Cacheable {
   }
 
   /**
-   * @return This HFileBlocks fileContext which will a derivative of the
+   * @return the HFileContext used to create this HFileBlock. Not necessary the
    * fileContext for the file from which this block's data was originally read.
    */
   HFileContext getHFileContext() {
@@ -1964,7 +1992,6 @@ public class HFileBlock implements Cacheable {
    * This is mostly helpful for debugging. This assumes that the block
    * has minor version > 0.
    */
-  @VisibleForTesting
   static String toStringHeader(ByteBuff buf) throws IOException {
     byte[] magicBuf = new byte[Math.min(buf.limit() - buf.position(), BlockType.MAGIC_LENGTH)];
     buf.get(magicBuf);

http://git-wip-us.apache.org/repos/asf/hbase/blob/54a543de/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
index 506f08d..9f29f97 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
@@ -60,7 +60,7 @@ import org.apache.hadoop.util.StringUtils;
  * Examples of how to use the block index writer can be found in
  * {@link org.apache.hadoop.hbase.io.hfile.CompoundBloomFilterWriter} and
  *  {@link HFileWriterImpl}. Examples of how to use the reader can be
- *  found in {@link HFileReaderImpl} and
+ *  found in {@link HFileWriterImpl} and
  *  {@link org.apache.hadoop.hbase.io.hfile.TestHFileBlockIndex}.
  */
 @InterfaceAudience.Private

http://git-wip-us.apache.org/repos/asf/hbase/blob/54a543de/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
index d71911f..8f5040e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
@@ -252,20 +252,18 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
           long end = 0;
           try {
             end = getTrailer().getLoadOnOpenDataOffset();
+            HFileBlock prevBlock = null;
             if (LOG.isTraceEnabled()) {
               LOG.trace("File=" + path.toString() + ", offset=" + offset + ", end=" + end);
             }
-            // TODO: Could we use block iterator in here? Would that get stuff into the cache?
-            HFileBlock prevBlock = null;
             while (offset < end) {
               if (Thread.interrupted()) {
                 break;
               }
-              // Perhaps we got our block from cache? Unlikely as this may be, if it happens, then
-              // the internal-to-hfileblock thread local which holds the overread that gets the
-              // next header, will not have happened...so, pass in the onDiskSize gotten from the
-              // cached block. This 'optimization' triggers extremely rarely I'd say.
-              long onDiskSize = prevBlock != null? prevBlock.getNextBlockOnDiskSize(): -1;
+              long onDiskSize = -1;
+              if (prevBlock != null) {
+                onDiskSize = prevBlock.getNextBlockOnDiskSizeWithHeader();
+              }
               HFileBlock block = readBlock(offset, onDiskSize, true, false, false, false,
                 null, null);
               // Need not update the current block. Ideally here the readBlock won't find the
@@ -905,8 +903,9 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
 
         // We are reading the next block without block type validation, because
         // it might turn out to be a non-data block.
-        block = reader.readBlock(block.getOffset() + block.getOnDiskSizeWithHeader(),
-            block.getNextBlockOnDiskSize(), cacheBlocks, pread,
+        block = reader.readBlock(block.getOffset()
+            + block.getOnDiskSizeWithHeader(),
+            block.getNextBlockOnDiskSizeWithHeader(), cacheBlocks, pread,
             isCompaction, true, null, getEffectiveDataBlockEncoding());
         if (block != null && !block.getBlockType().isData()) { // Findbugs: NP_NULL_ON_SOME_PATH
           // Whatever block we read we will be returning it unless
@@ -1440,8 +1439,8 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
         // Cache Miss, please load.
       }
 
-      HFileBlock metaBlock = fsBlockReader.readBlockData(metaBlockOffset, blockSize, true).
-          unpack(hfileContext, fsBlockReader);
+      HFileBlock metaBlock = fsBlockReader.readBlockData(metaBlockOffset,
+          blockSize, -1, true).unpack(hfileContext, fsBlockReader);
 
       // Cache the block
       if (cacheBlock) {
@@ -1527,8 +1526,8 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
           traceScope.getSpan().addTimelineAnnotation("blockCacheMiss");
         }
         // Load block from filesystem.
-        HFileBlock hfileBlock =
-            fsBlockReader.readBlockData(dataBlockOffset, onDiskBlockSize, pread);
+        HFileBlock hfileBlock = fsBlockReader.readBlockData(dataBlockOffset, onDiskBlockSize, -1,
+            pread);
         validateBlockType(hfileBlock, expectedBlockType);
         HFileBlock unpacked = hfileBlock.unpack(hfileContext, fsBlockReader);
         BlockType.BlockCategory category = hfileBlock.getBlockType().getCategory();
@@ -1872,7 +1871,6 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
    * @return Scanner on this file.
    */
   @Override
-  @VisibleForTesting
   public HFileScanner getScanner(boolean cacheBlocks, final boolean pread) {
     return getScanner(cacheBlocks, pread, false);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/54a543de/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileScanner.java
index e0f3d74..c67bdd4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileScanner.java
@@ -99,21 +99,18 @@ public interface HFileScanner extends Shipper, Closeable {
    * @throws IOException
    */
   boolean seekTo() throws IOException;
-
   /**
    * Scans to the next entry in the file.
    * @return Returns false if you are at the end otherwise true if more in file.
    * @throws IOException
    */
   boolean next() throws IOException;
-
   /**
    * Gets the current key in the form of a cell. You must call
    * {@link #seekTo(Cell)} before this method.
    * @return gets the current key as a Cell.
    */
   Cell getKey();
-
   /**
    * Gets a buffer view to the current value.  You must call
    * {@link #seekTo(Cell)} before this method.
@@ -122,35 +119,26 @@ public interface HFileScanner extends Shipper, Closeable {
    * the position is 0, the start of the buffer view.
    */
   ByteBuffer getValue();
-
   /**
    * @return Instance of {@link org.apache.hadoop.hbase.Cell}.
    */
   Cell getCell();
-
   /**
    * Convenience method to get a copy of the key as a string - interpreting the
    * bytes as UTF8. You must call {@link #seekTo(Cell)} before this method.
    * @return key as a string
-   * @deprecated Since hbase-2.0.0
    */
-  @Deprecated
   String getKeyString();
-
   /**
    * Convenience method to get a copy of the value as a string - interpreting
    * the bytes as UTF8. You must call {@link #seekTo(Cell)} before this method.
    * @return value as a string
-   * @deprecated Since hbase-2.0.0
    */
-  @Deprecated
   String getValueString();
-
   /**
    * @return Reader that underlies this Scanner instance.
    */
   HFile.Reader getReader();
-
   /**
    * @return True is scanner has had one of the seek calls invoked; i.e.
    * {@link #seekBefore(Cell)} or {@link #seekTo()} or {@link #seekTo(Cell)}.


[40/50] [abbrv] hbase git commit: HBASE-15265 Implement an asynchronous FSHLog

Posted by sy...@apache.org.
HBASE-15265 Implement an asynchronous FSHLog


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

Branch: refs/heads/hbase-12439
Commit: c96b642f15ddc24ad1f52616a4fb74feb51483c4
Parents: 1a9837a
Author: zhangduo <zh...@apache.org>
Authored: Sat Mar 26 15:40:45 2016 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sat Mar 26 15:41:27 2016 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/LogRoller.java    |    3 +
 .../hbase/regionserver/wal/AbstractFSWAL.java   |  910 +++++++++++
 .../wal/AbstractProtobufLogWriter.java          |  170 ++
 .../hbase/regionserver/wal/AsyncFSWAL.java      |  732 +++++++++
 .../wal/AsyncProtobufLogWriter.java             |  203 +++
 .../hadoop/hbase/regionserver/wal/FSHLog.java   | 1518 ++++--------------
 .../hbase/regionserver/wal/FSWALEntry.java      |   24 +-
 .../regionserver/wal/ProtobufLogWriter.java     |  148 +-
 .../hbase/regionserver/wal/SyncFuture.java      |  137 +-
 .../hbase/regionserver/wal/WriterBase.java      |   62 -
 .../FanOutOneBlockAsyncDFSOutputHelper.java     |   40 +-
 .../hadoop/hbase/wal/AbstractFSWALProvider.java |  361 +++++
 .../hadoop/hbase/wal/AsyncFSWALProvider.java    |   73 +
 .../hadoop/hbase/wal/DefaultWALProvider.java    |  348 +---
 .../hadoop/hbase/wal/DisabledWALProvider.java   |    4 +
 .../java/org/apache/hadoop/hbase/wal/WAL.java   |    9 +-
 .../org/apache/hadoop/hbase/wal/WALFactory.java |    6 +-
 .../apache/hadoop/hbase/wal/WALProvider.java    |   10 +-
 .../wal/AbstractTestLogRolling.java             |  332 ++++
 .../wal/AbstractTestProtobufLog.java            |  209 +++
 .../regionserver/wal/SequenceFileLogWriter.java |   22 +-
 .../wal/TestAsyncLogRollPeriod.java             |   36 +
 .../regionserver/wal/TestAsyncLogRolling.java   |   65 +
 .../regionserver/wal/TestAsyncProtobufLog.java  |   82 +
 .../regionserver/wal/TestAsyncWALReplay.java    |   36 +
 .../wal/TestAsyncWALReplayCompressed.java       |   38 +
 .../hbase/regionserver/wal/TestDurability.java  |   29 +-
 .../regionserver/wal/TestLogRollAbort.java      |    6 +-
 .../regionserver/wal/TestLogRollPeriod.java     |   12 +-
 .../hbase/regionserver/wal/TestLogRolling.java  |  404 +----
 .../hbase/regionserver/wal/TestProtobufLog.java |  187 +--
 .../wal/TestWALReplayCompressed.java            |    1 -
 32 files changed, 3918 insertions(+), 2299 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java
index 40edc05..fa217ee 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java
@@ -167,6 +167,9 @@ public class LogRoller extends HasThread {
         }
       }
     }
+    for (WAL wal : walNeedsRoll.keySet()) {
+      wal.logRollerExited();
+    }
     LOG.info("LogRoller exiting.");
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
new file mode 100644
index 0000000..f189ff1
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
@@ -0,0 +1,910 @@
+/**
+ * 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.wal;
+
+import static org.apache.hadoop.hbase.wal.AbstractFSWALProvider.WAL_FILE_NAME_DELIMITER;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.lang.management.ManagementFactory;
+import java.lang.management.MemoryUsage;
+import java.net.URLEncoder;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ConcurrentNavigableMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.ReentrantLock;
+
+import com.google.common.annotations.VisibleForTesting;
+
+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.fs.PathFilter;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.io.util.HeapMemorySizeUtil;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.DrainBarrier;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.htrace.NullScope;
+import org.apache.htrace.Span;
+import org.apache.htrace.Trace;
+import org.apache.htrace.TraceScope;
+
+/**
+ * Implementation of {@link WAL} to go against {@link FileSystem}; i.e. keep WALs in HDFS. Only one
+ * WAL is ever being written at a time. When a WAL hits a configured maximum size, it is rolled.
+ * This is done internal to the implementation.
+ * <p>
+ * As data is flushed from the MemStore to other on-disk structures (files sorted by key, hfiles), a
+ * WAL becomes obsolete. We can let go of all the log edits/entries for a given HRegion-sequence id.
+ * A bunch of work in the below is done keeping account of these region sequence ids -- what is
+ * flushed out to hfiles, and what is yet in WAL and in memory only.
+ * <p>
+ * It is only practical to delete entire files. Thus, we delete an entire on-disk file
+ * <code>F</code> when all of the edits in <code>F</code> have a log-sequence-id that's older
+ * (smaller) than the most-recent flush.
+ * <p>
+ * To read an WAL, call
+ * {@link WALFactory#createReader(org.apache.hadoop.fs.FileSystem, org.apache.hadoop.fs.Path)}. *
+ * <h2>Failure Semantic</h2> If an exception on append or sync, roll the WAL because the current WAL
+ * is now a lame duck; any more appends or syncs will fail also with the same original exception. If
+ * we have made successful appends to the WAL and we then are unable to sync them, our current
+ * semantic is to return error to the client that the appends failed but also to abort the current
+ * context, usually the hosting server. We need to replay the WALs. <br>
+ * TODO: Change this semantic. A roll of WAL may be sufficient as long as we have flagged client
+ * that the append failed. <br>
+ * TODO: replication may pick up these last edits though they have been marked as failed append
+ * (Need to keep our own file lengths, not rely on HDFS).
+ */
+@InterfaceAudience.Private
+public abstract class AbstractFSWAL<W> implements WAL {
+
+  private static final Log LOG = LogFactory.getLog(AbstractFSWAL.class);
+
+  protected static final int DEFAULT_SLOW_SYNC_TIME_MS = 100; // in ms
+
+  /**
+   * file system instance
+   */
+  protected final FileSystem fs;
+
+  /**
+   * WAL directory, where all WAL files would be placed.
+   */
+  protected final Path walDir;
+
+  /**
+   * dir path where old logs are kept.
+   */
+  protected final Path walArchiveDir;
+
+  /**
+   * Matches just those wal files that belong to this wal instance.
+   */
+  protected final PathFilter ourFiles;
+
+  /**
+   * Prefix of a WAL file, usually the region server name it is hosted on.
+   */
+  protected final String walFilePrefix;
+
+  /**
+   * Suffix included on generated wal file names
+   */
+  protected final String walFileSuffix;
+
+  /**
+   * Prefix used when checking for wal membership.
+   */
+  protected final String prefixPathStr;
+
+  protected final WALCoprocessorHost coprocessorHost;
+
+  /**
+   * conf object
+   */
+  protected final Configuration conf;
+
+  /** Listeners that are called on WAL events. */
+  protected final List<WALActionsListener> listeners =
+      new CopyOnWriteArrayList<WALActionsListener>();
+
+  /**
+   * Class that does accounting of sequenceids in WAL subsystem. Holds oldest outstanding sequence
+   * id as yet not flushed as well as the most recent edit sequence id appended to the WAL. Has
+   * facility for answering questions such as "Is it safe to GC a WAL?".
+   */
+  protected final SequenceIdAccounting sequenceIdAccounting = new SequenceIdAccounting();
+
+  /** The barrier used to ensure that close() waits for all log rolls and flushes to finish. */
+  protected final DrainBarrier closeBarrier = new DrainBarrier();
+
+  protected final int slowSyncNs;
+
+  // If > than this size, roll the log.
+  protected final long logrollsize;
+
+  /*
+   * If more than this many logs, force flush of oldest region to oldest edit goes to disk. If too
+   * many and we crash, then will take forever replaying. Keep the number of logs tidy.
+   */
+  protected final int maxLogs;
+
+  /**
+   * This lock makes sure only one log roll runs at a time. Should not be taken while any other lock
+   * is held. We don't just use synchronized because that results in bogus and tedious findbugs
+   * warning when it thinks synchronized controls writer thread safety. It is held when we are
+   * actually rolling the log. It is checked when we are looking to see if we should roll the log or
+   * not.
+   */
+  protected final ReentrantLock rollWriterLock = new ReentrantLock(true);
+
+  // The timestamp (in ms) when the log file was created.
+  protected final AtomicLong filenum = new AtomicLong(-1);
+
+  // Number of transactions in the current Wal.
+  protected final AtomicInteger numEntries = new AtomicInteger(0);
+
+  /**
+   * The highest known outstanding unsync'd WALEdit transaction id. Usually, we use a queue to pass
+   * WALEdit to background consumer thread, and the transaction id is the sequence number of the
+   * corresponding entry in queue.
+   */
+  protected volatile long highestUnsyncedTxid = -1;
+
+  /**
+   * Updated to the transaction id of the last successful sync call. This can be less than
+   * {@link #highestUnsyncedTxid} for case where we have an append where a sync has not yet come in
+   * for it.
+   */
+  protected final AtomicLong highestSyncedTxid = new AtomicLong(0);
+
+  /**
+   * The total size of wal
+   */
+  protected final AtomicLong totalLogSize = new AtomicLong(0);
+  /**
+   * Current log file.
+   */
+  volatile W writer;
+
+  protected volatile boolean closed = false;
+
+  protected final AtomicBoolean shutdown = new AtomicBoolean(false);
+  /**
+   * WAL Comparator; it compares the timestamp (log filenum), present in the log file name. Throws
+   * an IllegalArgumentException if used to compare paths from different wals.
+   */
+  final Comparator<Path> LOG_NAME_COMPARATOR = new Comparator<Path>() {
+    @Override
+    public int compare(Path o1, Path o2) {
+      return Long.compare(getFileNumFromFileName(o1), getFileNumFromFileName(o2));
+    }
+  };
+
+  /**
+   * Map of WAL log file to the latest sequence ids of all regions it has entries of. The map is
+   * sorted by the log file creation timestamp (contained in the log file name).
+   */
+  protected ConcurrentNavigableMap<Path, Map<byte[], Long>> byWalRegionSequenceIds =
+      new ConcurrentSkipListMap<Path, Map<byte[], Long>>(LOG_NAME_COMPARATOR);
+
+  /**
+   * Map of {@link SyncFuture}s keyed by Handler objects. Used so we reuse SyncFutures.
+   * <p>
+   * TODO: Reus FSWALEntry's rather than create them anew each time as we do SyncFutures here.
+   * <p>
+   * TODO: Add a FSWalEntry and SyncFuture as thread locals on handlers rather than have them get
+   * them from this Map?
+   */
+  private final ConcurrentMap<Thread, SyncFuture> syncFuturesByHandler;
+
+  /**
+   * A log file has a creation timestamp (in ms) in its file name ({@link #filenum}. This helper
+   * method returns the creation timestamp from a given log file. It extracts the timestamp assuming
+   * the filename is created with the {@link #computeFilename(long filenum)} method.
+   * @return timestamp, as in the log file name.
+   */
+  protected long getFileNumFromFileName(Path fileName) {
+    if (fileName == null) {
+      throw new IllegalArgumentException("file name can't be null");
+    }
+    if (!ourFiles.accept(fileName)) {
+      throw new IllegalArgumentException("The log file " + fileName
+          + " doesn't belong to this WAL. (" + toString() + ")");
+    }
+    final String fileNameString = fileName.toString();
+    String chompedPath =
+        fileNameString.substring(prefixPathStr.length(),
+          (fileNameString.length() - walFileSuffix.length()));
+    return Long.parseLong(chompedPath);
+  }
+
+  private int calculateMaxLogFiles(float memstoreSizeRatio, long logRollSize) {
+    MemoryUsage mu = ManagementFactory.getMemoryMXBean().getHeapMemoryUsage();
+    return Math.round(mu.getMax() * memstoreSizeRatio * 2 / logRollSize);
+  }
+
+  protected AbstractFSWAL(final FileSystem fs, final Path rootDir, final String logDir,
+      final String archiveDir, final Configuration conf, final List<WALActionsListener> listeners,
+      final boolean failIfWALExists, final String prefix, final String suffix)
+      throws FailedLogCloseException, IOException {
+    this.fs = fs;
+    this.walDir = new Path(rootDir, logDir);
+    this.walArchiveDir = new Path(rootDir, archiveDir);
+    this.conf = conf;
+
+    if (!fs.exists(walDir) && !fs.mkdirs(walDir)) {
+      throw new IOException("Unable to mkdir " + walDir);
+    }
+
+    if (!fs.exists(this.walArchiveDir)) {
+      if (!fs.mkdirs(this.walArchiveDir)) {
+        throw new IOException("Unable to mkdir " + this.walArchiveDir);
+      }
+    }
+
+    // If prefix is null||empty then just name it wal
+    this.walFilePrefix =
+        prefix == null || prefix.isEmpty() ? "wal" : URLEncoder.encode(prefix, "UTF8");
+    // we only correctly differentiate suffices when numeric ones start with '.'
+    if (suffix != null && !(suffix.isEmpty()) && !(suffix.startsWith(WAL_FILE_NAME_DELIMITER))) {
+      throw new IllegalArgumentException("WAL suffix must start with '" + WAL_FILE_NAME_DELIMITER
+          + "' but instead was '" + suffix + "'");
+    }
+    // Now that it exists, set the storage policy for the entire directory of wal files related to
+    // this FSHLog instance
+    FSUtils.setStoragePolicy(fs, conf, this.walDir, HConstants.WAL_STORAGE_POLICY,
+      HConstants.DEFAULT_WAL_STORAGE_POLICY);
+    this.walFileSuffix = (suffix == null) ? "" : URLEncoder.encode(suffix, "UTF8");
+    this.prefixPathStr = new Path(walDir, walFilePrefix + WAL_FILE_NAME_DELIMITER).toString();
+
+    this.ourFiles = new PathFilter() {
+      @Override
+      public boolean accept(final Path fileName) {
+        // The path should start with dir/<prefix> and end with our suffix
+        final String fileNameString = fileName.toString();
+        if (!fileNameString.startsWith(prefixPathStr)) {
+          return false;
+        }
+        if (walFileSuffix.isEmpty()) {
+          // in the case of the null suffix, we need to ensure the filename ends with a timestamp.
+          return org.apache.commons.lang.StringUtils.isNumeric(fileNameString
+              .substring(prefixPathStr.length()));
+        } else if (!fileNameString.endsWith(walFileSuffix)) {
+          return false;
+        }
+        return true;
+      }
+    };
+
+    if (failIfWALExists) {
+      final FileStatus[] walFiles = FSUtils.listStatus(fs, walDir, ourFiles);
+      if (null != walFiles && 0 != walFiles.length) {
+        throw new IOException("Target WAL already exists within directory " + walDir);
+      }
+    }
+
+    // Register listeners. TODO: Should this exist anymore? We have CPs?
+    if (listeners != null) {
+      for (WALActionsListener i : listeners) {
+        registerWALActionsListener(i);
+      }
+    }
+    this.coprocessorHost = new WALCoprocessorHost(this, conf);
+
+    // Get size to roll log at. Roll at 95% of HDFS block size so we avoid crossing HDFS blocks
+    // (it costs a little x'ing bocks)
+    final long blocksize =
+        this.conf.getLong("hbase.regionserver.hlog.blocksize",
+          FSUtils.getDefaultBlockSize(this.fs, this.walDir));
+    this.logrollsize =
+        (long) (blocksize * conf.getFloat("hbase.regionserver.logroll.multiplier", 0.95f));
+
+    float memstoreRatio =
+        conf.getFloat(HeapMemorySizeUtil.MEMSTORE_SIZE_KEY, conf.getFloat(
+          HeapMemorySizeUtil.MEMSTORE_SIZE_OLD_KEY, HeapMemorySizeUtil.DEFAULT_MEMSTORE_SIZE));
+    boolean maxLogsDefined = conf.get("hbase.regionserver.maxlogs") != null;
+    if (maxLogsDefined) {
+      LOG.warn("'hbase.regionserver.maxlogs' was deprecated.");
+    }
+    this.maxLogs =
+        conf.getInt("hbase.regionserver.maxlogs",
+          Math.max(32, calculateMaxLogFiles(memstoreRatio, logrollsize)));
+
+    LOG.info("WAL configuration: blocksize=" + StringUtils.byteDesc(blocksize) + ", rollsize="
+        + StringUtils.byteDesc(this.logrollsize) + ", prefix=" + this.walFilePrefix + ", suffix="
+        + walFileSuffix + ", logDir=" + this.walDir + ", archiveDir=" + this.walArchiveDir);
+    this.slowSyncNs =
+        1000000 * conf.getInt("hbase.regionserver.hlog.slowsync.ms", DEFAULT_SLOW_SYNC_TIME_MS);
+    int maxHandlersCount = conf.getInt(HConstants.REGION_SERVER_HANDLER_COUNT, 200);
+    // Presize our map of SyncFutures by handler objects.
+    this.syncFuturesByHandler = new ConcurrentHashMap<Thread, SyncFuture>(maxHandlersCount);
+  }
+
+  @Override
+  public void registerWALActionsListener(WALActionsListener listener) {
+    this.listeners.add(listener);
+  }
+
+  @Override
+  public boolean unregisterWALActionsListener(WALActionsListener listener) {
+    return this.listeners.remove(listener);
+  }
+
+  @Override
+  public WALCoprocessorHost getCoprocessorHost() {
+    return coprocessorHost;
+  }
+
+  @Override
+  public Long startCacheFlush(byte[] encodedRegionName, Set<byte[]> families) {
+    if (!closeBarrier.beginOp()) {
+      LOG.info("Flush not started for " + Bytes.toString(encodedRegionName) + "; server closing.");
+      return null;
+    }
+    return this.sequenceIdAccounting.startCacheFlush(encodedRegionName, families);
+  }
+
+  @Override
+  public void completeCacheFlush(byte[] encodedRegionName) {
+    this.sequenceIdAccounting.completeCacheFlush(encodedRegionName);
+    closeBarrier.endOp();
+  }
+
+  @Override
+  public void abortCacheFlush(byte[] encodedRegionName) {
+    this.sequenceIdAccounting.abortCacheFlush(encodedRegionName);
+    closeBarrier.endOp();
+  }
+
+  @Override
+  public long getEarliestMemstoreSeqNum(byte[] encodedRegionName) {
+    // Used by tests. Deprecated as too subtle for general usage.
+    return this.sequenceIdAccounting.getLowestSequenceId(encodedRegionName);
+  }
+
+  @Override
+  public long getEarliestMemstoreSeqNum(byte[] encodedRegionName, byte[] familyName) {
+    // This method is used by tests and for figuring if we should flush or not because our
+    // sequenceids are too old. It is also used reporting the master our oldest sequenceid for use
+    // figuring what edits can be skipped during log recovery. getEarliestMemStoreSequenceId
+    // from this.sequenceIdAccounting is looking first in flushingOldestStoreSequenceIds, the
+    // currently flushing sequence ids, and if anything found there, it is returning these. This is
+    // the right thing to do for the reporting oldest sequenceids to master; we won't skip edits if
+    // we crash during the flush. For figuring what to flush, we might get requeued if our sequence
+    // id is old even though we are currently flushing. This may mean we do too much flushing.
+    return this.sequenceIdAccounting.getLowestSequenceId(encodedRegionName, familyName);
+  }
+
+  @Override
+  public byte[][] rollWriter() throws FailedLogCloseException, IOException {
+    return rollWriter(false);
+  }
+
+  /**
+   * This is a convenience method that computes a new filename with a given file-number.
+   * @param filenum to use
+   * @return Path
+   */
+  protected Path computeFilename(final long filenum) {
+    if (filenum < 0) {
+      throw new RuntimeException("WAL file number can't be < 0");
+    }
+    String child = walFilePrefix + WAL_FILE_NAME_DELIMITER + filenum + walFileSuffix;
+    return new Path(walDir, child);
+  }
+
+  /**
+   * This is a convenience method that computes a new filename with a given using the current WAL
+   * file-number
+   * @return Path
+   */
+  public Path getCurrentFileName() {
+    return computeFilename(this.filenum.get());
+  }
+
+  /**
+   * retrieve the next path to use for writing. Increments the internal filenum.
+   */
+  private Path getNewPath() throws IOException {
+    this.filenum.set(System.currentTimeMillis());
+    Path newPath = getCurrentFileName();
+    while (fs.exists(newPath)) {
+      this.filenum.incrementAndGet();
+      newPath = getCurrentFileName();
+    }
+    return newPath;
+  }
+
+  @VisibleForTesting
+  Path getOldPath() {
+    long currentFilenum = this.filenum.get();
+    Path oldPath = null;
+    if (currentFilenum > 0) {
+      // ComputeFilename will take care of meta wal filename
+      oldPath = computeFilename(currentFilenum);
+    } // I presume if currentFilenum is <= 0, this is first file and null for oldPath if fine?
+    return oldPath;
+  }
+
+  /**
+   * Tell listeners about pre log roll.
+   */
+  private void tellListenersAboutPreLogRoll(final Path oldPath, final Path newPath)
+      throws IOException {
+    if (!this.listeners.isEmpty()) {
+      for (WALActionsListener i : this.listeners) {
+        i.preLogRoll(oldPath, newPath);
+      }
+    }
+  }
+
+  /**
+   * Tell listeners about post log roll.
+   */
+  private void tellListenersAboutPostLogRoll(final Path oldPath, final Path newPath)
+      throws IOException {
+    if (!this.listeners.isEmpty()) {
+      for (WALActionsListener i : this.listeners) {
+        i.postLogRoll(oldPath, newPath);
+      }
+    }
+  }
+
+  // public only until class moves to o.a.h.h.wal
+  /** @return the number of rolled log files */
+  public int getNumRolledLogFiles() {
+    return byWalRegionSequenceIds.size();
+  }
+
+  // public only until class moves to o.a.h.h.wal
+  /** @return the number of log files in use */
+  public int getNumLogFiles() {
+    // +1 for current use log
+    return getNumRolledLogFiles() + 1;
+  }
+
+  /**
+   * If the number of un-archived WAL files is greater than maximum allowed, check the first
+   * (oldest) WAL file, and returns those regions which should be flushed so that it can be
+   * archived.
+   * @return regions (encodedRegionNames) to flush in order to archive oldest WAL file.
+   */
+  byte[][] findRegionsToForceFlush() throws IOException {
+    byte[][] regions = null;
+    int logCount = getNumRolledLogFiles();
+    if (logCount > this.maxLogs && logCount > 0) {
+      Map.Entry<Path, Map<byte[], Long>> firstWALEntry = this.byWalRegionSequenceIds.firstEntry();
+      regions = this.sequenceIdAccounting.findLower(firstWALEntry.getValue());
+    }
+    if (regions != null) {
+      StringBuilder sb = new StringBuilder();
+      for (int i = 0; i < regions.length; i++) {
+        if (i > 0) {
+          sb.append(", ");
+        }
+        sb.append(Bytes.toStringBinary(regions[i]));
+      }
+      LOG.info("Too many WALs; count=" + logCount + ", max=" + this.maxLogs + "; forcing flush of "
+          + regions.length + " regions(s): " + sb.toString());
+    }
+    return regions;
+  }
+
+  /**
+   * Archive old logs. A WAL is eligible for archiving if all its WALEdits have been flushed.
+   */
+  private void cleanOldLogs() throws IOException {
+    List<Path> logsToArchive = null;
+    // For each log file, look at its Map of regions to highest sequence id; if all sequence ids
+    // are older than what is currently in memory, the WAL can be GC'd.
+    for (Map.Entry<Path, Map<byte[], Long>> e : this.byWalRegionSequenceIds.entrySet()) {
+      Path log = e.getKey();
+      Map<byte[], Long> sequenceNums = e.getValue();
+      if (this.sequenceIdAccounting.areAllLower(sequenceNums)) {
+        if (logsToArchive == null) {
+          logsToArchive = new ArrayList<Path>();
+        }
+        logsToArchive.add(log);
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("WAL file ready for archiving " + log);
+        }
+      }
+    }
+    if (logsToArchive != null) {
+      for (Path p : logsToArchive) {
+        this.totalLogSize.addAndGet(-this.fs.getFileStatus(p).getLen());
+        archiveLogFile(p);
+        this.byWalRegionSequenceIds.remove(p);
+      }
+    }
+  }
+
+  /*
+   * only public so WALSplitter can use.
+   * @return archived location of a WAL file with the given path p
+   */
+  public static Path getWALArchivePath(Path archiveDir, Path p) {
+    return new Path(archiveDir, p.getName());
+  }
+
+  private void archiveLogFile(final Path p) throws IOException {
+    Path newPath = getWALArchivePath(this.walArchiveDir, p);
+    // Tell our listeners that a log is going to be archived.
+    if (!this.listeners.isEmpty()) {
+      for (WALActionsListener i : this.listeners) {
+        i.preLogArchive(p, newPath);
+      }
+    }
+    LOG.info("Archiving " + p + " to " + newPath);
+    if (!FSUtils.renameAndSetModifyTime(this.fs, p, newPath)) {
+      throw new IOException("Unable to rename " + p + " to " + newPath);
+    }
+    // Tell our listeners that a log has been archived.
+    if (!this.listeners.isEmpty()) {
+      for (WALActionsListener i : this.listeners) {
+        i.postLogArchive(p, newPath);
+      }
+    }
+  }
+
+  /**
+   * Cleans up current writer closing it and then puts in place the passed in
+   * <code>nextWriter</code>.
+   * <p>
+   * <ul>
+   * <li>In the case of creating a new WAL, oldPath will be null.</li>
+   * <li>In the case of rolling over from one file to the next, none of the parameters will be null.
+   * </li>
+   * <li>In the case of closing out this FSHLog with no further use newPath and nextWriter will be
+   * null.</li>
+   * </ul>
+   * @param oldPath may be null
+   * @param newPath may be null
+   * @param nextWriter may be null
+   * @return the passed in <code>newPath</code>
+   * @throws IOException if there is a problem flushing or closing the underlying FS
+   */
+  Path replaceWriter(Path oldPath, Path newPath, W nextWriter) throws IOException {
+    TraceScope scope = Trace.startSpan("FSHFile.replaceWriter");
+    try {
+      long oldFileLen = 0L;
+      doReplaceWriter(oldPath, newPath, nextWriter);
+      int oldNumEntries = this.numEntries.get();
+      final String newPathString = (null == newPath ? null : FSUtils.getPath(newPath));
+      if (oldPath != null) {
+        this.byWalRegionSequenceIds.put(oldPath, this.sequenceIdAccounting.resetHighest());
+        this.totalLogSize.addAndGet(oldFileLen);
+        LOG.info("Rolled WAL " + FSUtils.getPath(oldPath) + " with entries=" + oldNumEntries
+            + ", filesize=" + StringUtils.byteDesc(oldFileLen) + "; new WAL " + newPathString);
+      } else {
+        LOG.info("New WAL " + newPathString);
+      }
+      return newPath;
+    } finally {
+      scope.close();
+    }
+  }
+
+  protected Span blockOnSync(final SyncFuture syncFuture) throws IOException {
+    // Now we have published the ringbuffer, halt the current thread until we get an answer back.
+    try {
+      syncFuture.get();
+      return syncFuture.getSpan();
+    } catch (InterruptedException ie) {
+      LOG.warn("Interrupted", ie);
+      throw convertInterruptedExceptionToIOException(ie);
+    } catch (ExecutionException e) {
+      throw ensureIOException(e.getCause());
+    }
+  }
+
+  private static IOException ensureIOException(final Throwable t) {
+    return (t instanceof IOException) ? (IOException) t : new IOException(t);
+  }
+
+  private IOException convertInterruptedExceptionToIOException(final InterruptedException ie) {
+    Thread.currentThread().interrupt();
+    IOException ioe = new InterruptedIOException();
+    ioe.initCause(ie);
+    return ioe;
+  }
+
+  @Override
+  public byte[][] rollWriter(boolean force) throws FailedLogCloseException, IOException {
+    rollWriterLock.lock();
+    try {
+      // Return if nothing to flush.
+      if (!force && this.writer != null && this.numEntries.get() <= 0) {
+        return null;
+      }
+      byte[][] regionsToFlush = null;
+      if (this.closed) {
+        LOG.debug("WAL closed. Skipping rolling of writer");
+        return regionsToFlush;
+      }
+      if (!closeBarrier.beginOp()) {
+        LOG.debug("WAL closing. Skipping rolling of writer");
+        return regionsToFlush;
+      }
+      TraceScope scope = Trace.startSpan("FSHLog.rollWriter");
+      try {
+        Path oldPath = getOldPath();
+        Path newPath = getNewPath();
+        // Any exception from here on is catastrophic, non-recoverable so we currently abort.
+        W nextWriter = this.createWriterInstance(newPath);
+        tellListenersAboutPreLogRoll(oldPath, newPath);
+        // NewPath could be equal to oldPath if replaceWriter fails.
+        newPath = replaceWriter(oldPath, newPath, nextWriter);
+        tellListenersAboutPostLogRoll(oldPath, newPath);
+        // Can we delete any of the old log files?
+        if (getNumRolledLogFiles() > 0) {
+          cleanOldLogs();
+          regionsToFlush = findRegionsToForceFlush();
+        }
+      } finally {
+        closeBarrier.endOp();
+        assert scope == NullScope.INSTANCE || !scope.isDetached();
+        scope.close();
+      }
+      return regionsToFlush;
+    } finally {
+      rollWriterLock.unlock();
+    }
+  }
+
+  // public only until class moves to o.a.h.h.wal
+  /** @return the size of log files in use */
+  public long getLogFileSize() {
+    return this.totalLogSize.get();
+  }
+
+  // public only until class moves to o.a.h.h.wal
+  public void requestLogRoll() {
+    requestLogRoll(false);
+  }
+
+  /**
+   * Get the backing files associated with this WAL.
+   * @return may be null if there are no files.
+   */
+  protected FileStatus[] getFiles() throws IOException {
+    return FSUtils.listStatus(fs, walDir, ourFiles);
+  }
+
+  @Override
+  public void shutdown() throws IOException {
+    if (!shutdown.compareAndSet(false, true)) {
+      return;
+    }
+    closed = true;
+    try {
+      // Prevent all further flushing and rolling.
+      closeBarrier.stopAndDrainOps();
+    } catch (InterruptedException e) {
+      LOG.error("Exception while waiting for cache flushes and log rolls", e);
+      Thread.currentThread().interrupt();
+    }
+    // Tell our listeners that the log is closing
+    if (!this.listeners.isEmpty()) {
+      for (WALActionsListener i : this.listeners) {
+        i.logCloseRequested();
+      }
+    }
+    doShutdown();
+  }
+
+  @Override
+  public void close() throws IOException {
+    shutdown();
+    final FileStatus[] files = getFiles();
+    if (null != files && 0 != files.length) {
+      for (FileStatus file : files) {
+        Path p = getWALArchivePath(this.walArchiveDir, file.getPath());
+        // Tell our listeners that a log is going to be archived.
+        if (!this.listeners.isEmpty()) {
+          for (WALActionsListener i : this.listeners) {
+            i.preLogArchive(file.getPath(), p);
+          }
+        }
+
+        if (!FSUtils.renameAndSetModifyTime(fs, file.getPath(), p)) {
+          throw new IOException("Unable to rename " + file.getPath() + " to " + p);
+        }
+        // Tell our listeners that a log was archived.
+        if (!this.listeners.isEmpty()) {
+          for (WALActionsListener i : this.listeners) {
+            i.postLogArchive(file.getPath(), p);
+          }
+        }
+      }
+      LOG.debug("Moved " + files.length + " WAL file(s) to " + FSUtils.getPath(this.walArchiveDir));
+    }
+    LOG.info("Closed WAL: " + toString());
+  }
+
+  protected SyncFuture getSyncFuture(final long sequence, Span span) {
+    SyncFuture syncFuture = this.syncFuturesByHandler.get(Thread.currentThread());
+    if (syncFuture == null) {
+      syncFuture = new SyncFuture();
+      this.syncFuturesByHandler.put(Thread.currentThread(), syncFuture);
+    }
+    return syncFuture.reset(sequence, span);
+  }
+
+  protected void requestLogRoll(boolean tooFewReplicas) {
+    if (!this.listeners.isEmpty()) {
+      for (WALActionsListener i : this.listeners) {
+        i.logRollRequested(tooFewReplicas);
+      }
+    }
+  }
+
+  long getUnflushedEntriesCount() {
+    long highestSynced = this.highestSyncedTxid.get();
+    long highestUnsynced = this.highestUnsyncedTxid;
+    return highestSynced >= highestUnsynced ? 0 : highestUnsynced - highestSynced;
+  }
+
+  boolean isUnflushedEntries() {
+    return getUnflushedEntriesCount() > 0;
+  }
+
+  /**
+   * Exposed for testing only. Use to tricks like halt the ring buffer appending.
+   */
+  @VisibleForTesting
+  void atHeadOfRingBufferEventHandlerAppend() {
+    // Noop
+  }
+
+  protected boolean append(W writer, FSWALEntry entry) throws IOException {
+    // TODO: WORK ON MAKING THIS APPEND FASTER. DOING WAY TOO MUCH WORK WITH CPs, PBing, etc.
+    atHeadOfRingBufferEventHandlerAppend();
+    long start = EnvironmentEdgeManager.currentTime();
+    byte[] encodedRegionName = entry.getKey().getEncodedRegionName();
+    long regionSequenceId = WALKey.NO_SEQUENCE_ID;
+    // We are about to append this edit; update the region-scoped sequence number. Do it
+    // here inside this single appending/writing thread. Events are ordered on the ringbuffer
+    // so region sequenceids will also be in order.
+    regionSequenceId = entry.stampRegionSequenceId();
+    // Edits are empty, there is nothing to append. Maybe empty when we are looking for a
+    // region sequence id only, a region edit/sequence id that is not associated with an actual
+    // edit. It has to go through all the rigmarole to be sure we have the right ordering.
+    if (entry.getEdit().isEmpty()) {
+      return false;
+    }
+
+    // Coprocessor hook.
+    if (!coprocessorHost.preWALWrite(entry.getHRegionInfo(), entry.getKey(), entry.getEdit())) {
+      if (entry.getEdit().isReplay()) {
+        // Set replication scope null so that this won't be replicated
+        entry.getKey().serializeReplicationScope(false);
+      }
+    }
+    if (!listeners.isEmpty()) {
+      for (WALActionsListener i : listeners) {
+        i.visitLogEntryBeforeWrite(entry.getKey(), entry.getEdit());
+      }
+    }
+    doAppend(writer, entry);
+    assert highestUnsyncedTxid < entry.getTxid();
+    highestUnsyncedTxid = entry.getTxid();
+    sequenceIdAccounting.update(encodedRegionName, entry.getFamilyNames(), regionSequenceId,
+      entry.isInMemstore());
+    coprocessorHost.postWALWrite(entry.getHRegionInfo(), entry.getKey(), entry.getEdit());
+    // Update metrics.
+    postAppend(entry, EnvironmentEdgeManager.currentTime() - start);
+    numEntries.incrementAndGet();
+    return true;
+  }
+
+  private long postAppend(final Entry e, final long elapsedTime) {
+    long len = 0;
+    if (!listeners.isEmpty()) {
+      for (Cell cell : e.getEdit().getCells()) {
+        len += CellUtil.estimatedSerializedSizeOf(cell);
+      }
+      for (WALActionsListener listener : listeners) {
+        listener.postAppend(len, elapsedTime);
+      }
+    }
+    return len;
+  }
+
+  protected void postSync(final long timeInNanos, final int handlerSyncs) {
+    if (timeInNanos > this.slowSyncNs) {
+      String msg =
+          new StringBuilder().append("Slow sync cost: ").append(timeInNanos / 1000000)
+              .append(" ms, current pipeline: ").append(Arrays.toString(getPipeline())).toString();
+      Trace.addTimelineAnnotation(msg);
+      LOG.info(msg);
+    }
+    if (!listeners.isEmpty()) {
+      for (WALActionsListener listener : listeners) {
+        listener.postSync(timeInNanos, handlerSyncs);
+      }
+    }
+  }
+
+  /**
+   * NOTE: This append, at a time that is usually after this call returns, starts an mvcc
+   * transaction by calling 'begin' wherein which we assign this update a sequenceid. At assignment
+   * time, we stamp all the passed in Cells inside WALEdit with their sequenceId. You must
+   * 'complete' the transaction this mvcc transaction by calling
+   * MultiVersionConcurrencyControl#complete(...) or a variant otherwise mvcc will get stuck. Do it
+   * in the finally of a try/finally block within which this append lives and any subsequent
+   * operations like sync or update of memstore, etc. Get the WriteEntry to pass mvcc out of the
+   * passed in WALKey <code>walKey</code> parameter. Be warned that the WriteEntry is not
+   * immediately available on return from this method. It WILL be available subsequent to a sync of
+   * this append; otherwise, you will just have to wait on the WriteEntry to get filled in.
+   */
+  @Override
+  public abstract long append(HRegionInfo info, WALKey key, WALEdit edits, boolean inMemstore)
+      throws IOException;
+
+  protected abstract void doAppend(W writer, FSWALEntry entry) throws IOException;
+
+  protected abstract W createWriterInstance(Path path) throws IOException;
+
+  /**
+   * @return old wal file size
+   */
+  protected abstract long doReplaceWriter(Path oldPath, Path newPath, W nextWriter)
+      throws IOException;
+
+  protected abstract void doShutdown() throws IOException;
+
+  /**
+   * This method gets the pipeline for the current WAL.
+   */
+  @VisibleForTesting
+  abstract DatanodeInfo[] getPipeline();
+
+  /**
+   * This method gets the datanode replication count for the current WAL.
+   */
+  @VisibleForTesting
+  abstract int getLogReplication();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractProtobufLogWriter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractProtobufLogWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractProtobufLogWriter.java
new file mode 100644
index 0000000..66f1f54
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractProtobufLogWriter.java
@@ -0,0 +1,170 @@
+/**
+ * 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.wal;
+
+import static org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader.DEFAULT_WAL_TRAILER_WARN_SIZE;
+import static org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader.WAL_TRAILER_WARN_SIZE;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.codec.Codec;
+import org.apache.hadoop.hbase.io.util.LRUDictionary;
+import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALHeader;
+import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALTrailer;
+import org.apache.hadoop.hbase.util.FSUtils;
+
+/**
+ * Base class for Protobuf log writer.
+ */
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
+public abstract class AbstractProtobufLogWriter {
+
+  private static final Log LOG = LogFactory.getLog(AbstractProtobufLogWriter.class);
+
+  protected CompressionContext compressionContext;
+  protected Configuration conf;
+  protected Codec.Encoder cellEncoder;
+  protected WALCellCodec.ByteStringCompressor compressor;
+  protected boolean trailerWritten;
+  protected WALTrailer trailer;
+  // maximum size of the wal Trailer in bytes. If a user writes/reads a trailer with size larger
+  // than this size, it is written/read respectively, with a WARN message in the log.
+  protected int trailerWarnSize;
+
+  protected AtomicLong length = new AtomicLong();
+
+  private WALCellCodec getCodec(Configuration conf, CompressionContext compressionContext)
+      throws IOException {
+    return WALCellCodec.create(conf, null, compressionContext);
+  }
+
+  protected WALHeader buildWALHeader(Configuration conf, WALHeader.Builder builder)
+      throws IOException {
+    if (!builder.hasWriterClsName()) {
+      builder.setWriterClsName(ProtobufLogWriter.class.getSimpleName());
+    }
+    if (!builder.hasCellCodecClsName()) {
+      builder.setCellCodecClsName(WALCellCodec.getWALCellCodecClass(conf));
+    }
+    return builder.build();
+  }
+
+  private boolean initializeCompressionContext(Configuration conf, Path path) throws IOException {
+    boolean doCompress = conf.getBoolean(HConstants.ENABLE_WAL_COMPRESSION, false);
+    if (doCompress) {
+      try {
+        this.compressionContext = new CompressionContext(LRUDictionary.class,
+            FSUtils.isRecoveredEdits(path),
+            conf.getBoolean(CompressionContext.ENABLE_WAL_TAGS_COMPRESSION, true));
+      } catch (Exception e) {
+        throw new IOException("Failed to initiate CompressionContext", e);
+      }
+    }
+    return doCompress;
+  }
+
+  public void init(FileSystem fs, Path path, Configuration conf, boolean overwritable)
+      throws IOException {
+    this.conf = conf;
+    boolean doCompress = initializeCompressionContext(conf, path);
+    this.trailerWarnSize = conf.getInt(WAL_TRAILER_WARN_SIZE, DEFAULT_WAL_TRAILER_WARN_SIZE);
+    int bufferSize = FSUtils.getDefaultBufferSize(fs);
+    short replication = (short) conf.getInt("hbase.regionserver.hlog.replication",
+      FSUtils.getDefaultReplication(fs, path));
+    long blockSize = conf.getLong("hbase.regionserver.hlog.blocksize",
+      FSUtils.getDefaultBlockSize(fs, path));
+
+    initOutput(fs, path, overwritable, bufferSize, replication, blockSize);
+
+    boolean doTagCompress = doCompress
+        && conf.getBoolean(CompressionContext.ENABLE_WAL_TAGS_COMPRESSION, true);
+    length.set(writeMagicAndWALHeader(ProtobufLogReader.PB_WAL_MAGIC, buildWALHeader(conf,
+      WALHeader.newBuilder().setHasCompression(doCompress).setHasTagCompression(doTagCompress))));
+
+    initAfterHeader(doCompress);
+
+    // instantiate trailer to default value.
+    trailer = WALTrailer.newBuilder().build();
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Initialized protobuf WAL=" + path + ", compression=" + doCompress);
+    }
+  }
+
+  protected void initAfterHeader(boolean doCompress) throws IOException {
+    WALCellCodec codec = getCodec(conf, this.compressionContext);
+    this.cellEncoder = codec.getEncoder(getOutputStreamForCellEncoder());
+    if (doCompress) {
+      this.compressor = codec.getByteStringCompressor();
+    }
+  }
+
+  void setWALTrailer(WALTrailer walTrailer) {
+    this.trailer = walTrailer;
+  }
+
+  public long getLength() {
+    return length.get();
+  }
+
+  private WALTrailer buildWALTrailer(WALTrailer.Builder builder) {
+    return builder.build();
+  }
+
+  protected void writeWALTrailer() {
+    try {
+      int trailerSize = 0;
+      if (this.trailer == null) {
+        // use default trailer.
+        LOG.warn("WALTrailer is null. Continuing with default.");
+        this.trailer = buildWALTrailer(WALTrailer.newBuilder());
+        trailerSize = this.trailer.getSerializedSize();
+      } else if ((trailerSize = this.trailer.getSerializedSize()) > this.trailerWarnSize) {
+        // continue writing after warning the user.
+        LOG.warn("Please investigate WALTrailer usage. Trailer size > maximum size : " + trailerSize
+            + " > " + this.trailerWarnSize);
+      }
+      length.set(writeWALTrailerAndMagic(trailer, ProtobufLogReader.PB_WAL_COMPLETE_MAGIC));
+      this.trailerWritten = true;
+    } catch (IOException ioe) {
+      LOG.warn("Failed to write trailer, non-fatal, continuing...", ioe);
+    }
+  }
+
+  protected abstract void initOutput(FileSystem fs, Path path, boolean overwritable, int bufferSize,
+      short replication, long blockSize) throws IOException;
+
+  /**
+   * return the file length after written.
+   */
+  protected abstract long writeMagicAndWALHeader(byte[] magic, WALHeader header) throws IOException;
+
+  protected abstract long writeWALTrailerAndMagic(WALTrailer trailer, byte[] magic)
+      throws IOException;
+
+  protected abstract OutputStream getOutputStreamForCellEncoder();
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
new file mode 100644
index 0000000..b80f2c9
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
@@ -0,0 +1,732 @@
+/**
+ * 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.wal;
+
+import static org.apache.hadoop.hbase.HConstants.REGION_SERVER_HANDLER_COUNT;
+import static org.apache.hadoop.hbase.util.FanOutOneBlockAsyncDFSOutputHelper.shouldRetryCreate;
+import io.netty.channel.EventLoop;
+import io.netty.util.concurrent.Future;
+import io.netty.util.concurrent.Promise;
+import io.netty.util.concurrent.ScheduledFuture;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.nio.channels.CompletionHandler;
+import java.util.ArrayDeque;
+import java.util.Comparator;
+import java.util.Deque;
+import java.util.Iterator;
+import java.util.List;
+import java.util.PriorityQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+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.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.ConnectionUtils;
+import org.apache.hadoop.hbase.util.FanOutOneBlockAsyncDFSOutput;
+import org.apache.hadoop.hbase.util.FanOutOneBlockAsyncDFSOutputHelper.NameNodeException;
+import org.apache.hadoop.hbase.wal.AsyncFSWALProvider;
+import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.hbase.wal.WALProvider.AsyncWriter;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.htrace.NullScope;
+import org.apache.htrace.Trace;
+import org.apache.htrace.TraceScope;
+
+/**
+ * An asynchronous implementation of FSWAL.
+ * <p>
+ * Here 'waitingConsumePayloads' acts as the RingBuffer in FSHLog. We do not use RingBuffer here
+ * because RingBuffer need an exclusive thread to consume the entries in it, and here we want to run
+ * the append and sync operation inside EventLoop. We can not use EventLoop as the RingBuffer's
+ * executor otherwise the EventLoop can not process any other events such as socket read and write.
+ * <p>
+ * For append, we process it as follow:
+ * <ol>
+ * <li>In the caller thread(typically, in the rpc handler thread):
+ * <ol>
+ * <li>Lock 'waitingConsumePayloads', bump nextTxid, and insert the entry to
+ * 'waitingConsumePayloads'.</li>
+ * <li>Schedule the consumer task if needed. See {@link #shouldScheduleConsumer()} for more details.
+ * </li>
+ * </ol>
+ * </li>
+ * <li>In the consumer task(in the EventLoop thread)
+ * <ol>
+ * <li>Poll the entry from 'waitingConsumePayloads' and insert it into 'waitingAppendEntries'</li>
+ * <li>Poll the entry from 'waitingAppendEntries', append it to the AsyncWriter, and insert it into
+ * 'unackedEntries'</li>
+ * <li>If the buffered size reaches {@link #batchSize}, or there is a sync request, then we call
+ * sync on the AsyncWriter.</li>
+ * <li>In the callback methods(CompletionHandler):
+ * <ul>
+ * <li>If succeeded, poll the entry from 'unackedEntries' and drop it.</li>
+ * <li>If failed, add all the entries in 'unackedEntries' back to 'waitingAppendEntries' and wait
+ * for writing them again.</li>
+ * </ul>
+ * </li>
+ * </ol>
+ * </li>
+ * </ol>
+ * For sync, the processing stages are almost same except that if it is not assigned with a new
+ * 'txid', we just assign the previous 'txid' to it without bumping the 'nextTxid'. And different
+ * from FSHLog, we will open a new writer and rewrite unacked entries to the new writer and sync
+ * again if we hit a sync error.
+ * <p>
+ * Here we only describe the logic of doReplaceWriter. The main logic of rollWriter is same with
+ * FSHLog.<br>
+ * For a normal roll request(for example, we have reached the log roll size):
+ * <ol>
+ * <li>In the log roller thread, we add a roll payload to 'waitingConsumePayloads', and then wait on
+ * the rollPromise(see {@link #waitForSafePoint()}).</li>
+ * <li>In the consumer thread, we will stop polling entries from 'waitingConsumePayloads' if we hit
+ * a Payload which contains a roll request.</li>
+ * <li>Append all entries to current writer, issue a sync request if possible.</li>
+ * <li>If sync succeeded, check if we could finish a roll request. There 3 conditions:
+ * <ul>
+ * <li>'rollPromise' is not null which means we have a pending roll request.</li>
+ * <li>'waitingAppendEntries' is empty.</li>
+ * <li>'unackedEntries' is empty.</li>
+ * </ul>
+ * </li>
+ * <li>Back to the log roller thread, now we can confirm that there are no out-going entries, i.e.,
+ * we reach a safe point. So it is safe to replace old writer with new writer now.</li>
+ * <li>Acquire 'waitingConsumePayloads' lock, set 'writerBroken' and 'waitingRoll' to false, cancel
+ * log roller exit checker if any(see the comments in the 'failed' method of the sync
+ * CompletionHandler to see why we need a checker here).</li>
+ * <li>Schedule the consumer task if needed.</li>
+ * <li>Schedule a background task to close the old writer.</li>
+ * </ol>
+ * For a broken writer roll request, the only difference is that we can bypass the wait for safe
+ * point stage. See the comments in the 'failed' method of the sync CompletionHandler for more
+ * details.
+ */
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
+public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
+
+  private static final Log LOG = LogFactory.getLog(AsyncFSWAL.class);
+
+  public static final String WAL_BATCH_SIZE = "hbase.wal.batch.size";
+  public static final long DEFAULT_WAL_BATCH_SIZE = 64L * 1024;
+
+  public static final String ASYNC_WAL_CREATE_MAX_RETRIES = "hbase.wal.async.create.retries";
+  public static final int DEFAULT_ASYNC_WAL_CREATE_MAX_RETRIES = 10;
+
+  public static final String ASYNC_WAL_LOG_ROLLER_EXITED_CHECK_INTERVAL_MS =
+      "hbase.wal.async.logroller.exited.check.interval.ms";
+  public static final long DEFAULT_ASYNC_WAL_LOG_ROLLER_EXITED_CHECK_INTERVAL_MS = 1000;
+
+  /**
+   * Carry things that we want to pass to the consume task in event loop. Only one field can be
+   * non-null.
+   * <p>
+   * TODO: need to unify this and {@link RingBufferTruck}. There are mostly the same thing.
+   */
+  private static final class Payload {
+
+    // a wal entry which need to be appended
+    public final FSWALEntry entry;
+
+    // indicate that we need to sync our wal writer.
+    public final SyncFuture sync;
+
+    // incidate that we want to roll the writer.
+    public final Promise<Void> roll;
+
+    public Payload(FSWALEntry entry) {
+      this.entry = entry;
+      this.sync = null;
+      this.roll = null;
+    }
+
+    public Payload(SyncFuture sync) {
+      this.entry = null;
+      this.sync = sync;
+      this.roll = null;
+    }
+
+    public Payload(Promise<Void> roll) {
+      this.entry = null;
+      this.sync = null;
+      this.roll = roll;
+    }
+
+    @Override
+    public String toString() {
+      return "Payload [entry=" + entry + ", sync=" + sync + ", roll=" + roll + "]";
+    }
+  }
+
+  private final EventLoop eventLoop;
+
+  private final Deque<Payload> waitingConsumePayloads;
+
+  // like the ringbuffer sequence. Every FSWALEntry and SyncFuture will be assigned a txid and
+  // then added to waitingConsumePayloads.
+  private long nextTxid = 1L;
+
+  private boolean consumerScheduled;
+
+  // new writer is created and we are waiting for old writer to be closed.
+  private boolean waitingRoll;
+
+  // writer is broken and rollWriter is needed.
+  private boolean writerBroken;
+
+  private final long batchSize;
+
+  private final int createMaxRetries;
+
+  private final long logRollerExitedCheckIntervalMs;
+
+  private final ExecutorService closeExecutor = Executors
+      .newCachedThreadPool(new ThreadFactoryBuilder().setDaemon(true)
+          .setNameFormat("Close-WAL-Writer-%d").build());
+
+  private volatile FanOutOneBlockAsyncDFSOutput hdfsOut;
+
+  private final Deque<FSWALEntry> waitingAppendEntries = new ArrayDeque<FSWALEntry>();
+
+  private final Deque<FSWALEntry> unackedEntries = new ArrayDeque<FSWALEntry>();
+
+  private final PriorityQueue<SyncFuture> syncFutures = new PriorityQueue<SyncFuture>(11,
+      SEQ_COMPARATOR);
+
+  private Promise<Void> rollPromise;
+
+  // the highest txid of WAL entries being processed
+  private long highestProcessedTxid;
+
+  // file length when we issue last sync request on the writer
+  private long fileLengthAtLastSync;
+
+  private volatile boolean logRollerExited;
+
+  private final class LogRollerExitedChecker implements Runnable {
+
+    private boolean cancelled;
+
+    private ScheduledFuture<?> future;
+
+    public synchronized void setFuture(ScheduledFuture<?> future) {
+      this.future = future;
+    }
+
+    @Override
+    public void run() {
+      if (!logRollerExited) {
+        return;
+      }
+      // rollWriter is called in the log roller thread, and logRollerExited will be set just before
+      // the log rolled exit. So here we can confirm that no one could cancel us if the 'canceled'
+      // check passed. So it is safe to release the lock after checking 'canceled' flag.
+      synchronized (this) {
+        if (cancelled) {
+          return;
+        }
+      }
+      unackedEntries.clear();
+      waitingAppendEntries.clear();
+      IOException error = new IOException("sync failed but log roller exited");
+      for (SyncFuture future; (future = syncFutures.peek()) != null;) {
+        future.done(highestProcessedTxid, error);
+        syncFutures.remove();
+      }
+      synchronized (waitingConsumePayloads) {
+        for (Payload p : waitingConsumePayloads) {
+          if (p.entry != null) {
+            try {
+              p.entry.stampRegionSequenceId();
+            } catch (IOException e) {
+              throw new AssertionError("should not happen", e);
+            }
+          } else if (p.sync != null) {
+            p.sync.done(nextTxid, error);
+          }
+        }
+        waitingConsumePayloads.clear();
+      }
+    }
+
+    public synchronized void cancel() {
+      future.cancel(false);
+      cancelled = true;
+    }
+  }
+
+  private LogRollerExitedChecker logRollerExitedChecker;
+
+  public AsyncFSWAL(FileSystem fs, Path rootDir, String logDir, String archiveDir,
+      Configuration conf, List<WALActionsListener> listeners, boolean failIfWALExists,
+      String prefix, String suffix, EventLoop eventLoop) throws FailedLogCloseException,
+      IOException {
+    super(fs, rootDir, logDir, archiveDir, conf, listeners, failIfWALExists, prefix, suffix);
+    this.eventLoop = eventLoop;
+    int maxHandlersCount = conf.getInt(REGION_SERVER_HANDLER_COUNT, 200);
+    waitingConsumePayloads = new ArrayDeque<Payload>(maxHandlersCount * 3);
+    batchSize = conf.getLong(WAL_BATCH_SIZE, DEFAULT_WAL_BATCH_SIZE);
+    createMaxRetries =
+        conf.getInt(ASYNC_WAL_CREATE_MAX_RETRIES, DEFAULT_ASYNC_WAL_CREATE_MAX_RETRIES);
+    logRollerExitedCheckIntervalMs =
+        conf.getLong(ASYNC_WAL_LOG_ROLLER_EXITED_CHECK_INTERVAL_MS,
+          DEFAULT_ASYNC_WAL_LOG_ROLLER_EXITED_CHECK_INTERVAL_MS);
+    rollWriter();
+  }
+
+  private void tryFinishRoll() {
+    // 1. a roll is requested
+    // 2. we have written out all entries before the roll point.
+    // 3. all entries have been acked.
+    if (rollPromise != null && waitingAppendEntries.isEmpty() && unackedEntries.isEmpty()) {
+      rollPromise.trySuccess(null);
+      rollPromise = null;
+    }
+  }
+
+  private void sync(final AsyncWriter writer, final long processedTxid) {
+    fileLengthAtLastSync = writer.getLength();
+    final long startTimeNs = System.nanoTime();
+    writer.sync(new CompletionHandler<Long, Void>() {
+
+      @Override
+      public void completed(Long result, Void attachment) {
+        highestSyncedTxid.set(processedTxid);
+        int syncCount = finishSync(true);
+        for (Iterator<FSWALEntry> iter = unackedEntries.iterator(); iter.hasNext();) {
+          if (iter.next().getTxid() <= processedTxid) {
+            iter.remove();
+          } else {
+            break;
+          }
+        }
+        postSync(System.nanoTime() - startTimeNs, syncCount);
+        tryFinishRoll();
+        if (!rollWriterLock.tryLock()) {
+          return;
+        }
+        try {
+          if (writer.getLength() >= logrollsize) {
+            requestLogRoll();
+          }
+        } finally {
+          rollWriterLock.unlock();
+        }
+      }
+
+      @Override
+      public void failed(Throwable exc, Void attachment) {
+        LOG.warn("sync failed", exc);
+        // Here we depends on the implementation of FanOutOneBlockAsyncDFSOutput and netty.
+        // When error occur, FanOutOneBlockAsyncDFSOutput will fail all pending flush requests. It
+        // is execute inside EventLoop. And in DefaultPromise in netty, it will notifyListener
+        // directly if it is already in the EventLoop thread. And in the listener method, it will
+        // call us. So here we know that all failed flush request will call us continuously, and
+        // before the last one finish, no other task can be executed in EventLoop. So here we are
+        // safe to use writerBroken as a guard.
+        // Do not forget to revisit this if we change the implementation of
+        // FanOutOneBlockAsyncDFSOutput!
+        synchronized (waitingConsumePayloads) {
+          if (writerBroken) {
+            return;
+          }
+          // schedule a periodical task to check if log roller is exited. Otherwise the the sync
+          // request maybe blocked forever since we are still waiting for a new writer to write the
+          // pending data and sync it...
+          logRollerExitedChecker = new LogRollerExitedChecker();
+          // we are currently in the EventLoop thread, so it is safe to set the future after
+          // schedule it since the task can not be executed before we release the thread.
+          logRollerExitedChecker.setFuture(eventLoop.scheduleAtFixedRate(logRollerExitedChecker,
+            logRollerExitedCheckIntervalMs, logRollerExitedCheckIntervalMs, TimeUnit.MILLISECONDS));
+          writerBroken = true;
+        }
+        for (Iterator<FSWALEntry> iter = unackedEntries.descendingIterator(); iter.hasNext();) {
+          waitingAppendEntries.addFirst(iter.next());
+        }
+        highestUnsyncedTxid = highestSyncedTxid.get();
+        if (rollPromise != null) {
+          rollPromise.trySuccess(null);
+          rollPromise = null;
+          return;
+        }
+        // request a roll.
+        if (!rollWriterLock.tryLock()) {
+          return;
+        }
+        try {
+          requestLogRoll();
+        } finally {
+          rollWriterLock.unlock();
+        }
+      }
+    }, null);
+  }
+
+  private void addTimeAnnotation(SyncFuture future, String annotation) {
+    TraceScope scope = Trace.continueSpan(future.getSpan());
+    Trace.addTimelineAnnotation(annotation);
+    future.setSpan(scope.detach());
+  }
+
+  private int finishSync(boolean addSyncTrace) {
+    long doneTxid = highestSyncedTxid.get();
+    int finished = 0;
+    for (SyncFuture future; (future = syncFutures.peek()) != null;) {
+      if (future.getTxid() <= doneTxid) {
+        future.done(doneTxid, null);
+        syncFutures.remove();
+        finished++;
+        addTimeAnnotation(future, "writer synced");
+      } else {
+        break;
+      }
+    }
+    return finished;
+  }
+
+  private void consume() {
+    final AsyncWriter writer = this.writer;
+    // maybe a sync request is not queued when we issue a sync, so check here to see if we could
+    // finish some.
+    finishSync(false);
+    long newHighestProcessedTxid = -1L;
+    for (Iterator<FSWALEntry> iter = waitingAppendEntries.iterator(); iter.hasNext();) {
+      FSWALEntry entry = iter.next();
+      boolean appended;
+      try {
+        appended = append(writer, entry);
+      } catch (IOException e) {
+        throw new AssertionError("should not happen", e);
+      }
+      newHighestProcessedTxid = entry.getTxid();
+      iter.remove();
+      if (appended) {
+        unackedEntries.addLast(entry);
+        if (writer.getLength() - fileLengthAtLastSync >= batchSize) {
+          break;
+        }
+      }
+    }
+    // if we have a newer transaction id, update it.
+    // otherwise, use the previous transaction id.
+    if (newHighestProcessedTxid > 0) {
+      highestProcessedTxid = newHighestProcessedTxid;
+    } else {
+      newHighestProcessedTxid = highestProcessedTxid;
+    }
+    if (writer.getLength() - fileLengthAtLastSync >= batchSize) {
+      // sync because buffer size limit.
+      sync(writer, newHighestProcessedTxid);
+    } else if ((!syncFutures.isEmpty() || rollPromise != null)
+        && writer.getLength() > fileLengthAtLastSync) {
+      // first we should have at least one sync request or a roll request
+      // second we should have some unsynced data.
+      sync(writer, newHighestProcessedTxid);
+    } else if (writer.getLength() == fileLengthAtLastSync) {
+      // we haven't written anything out, just advance the highestSyncedSequence since we may only
+      // stamped some region sequence id.
+      highestSyncedTxid.set(newHighestProcessedTxid);
+      finishSync(false);
+      tryFinishRoll();
+    }
+  }
+
+  private static final Comparator<SyncFuture> SEQ_COMPARATOR = new Comparator<SyncFuture>() {
+
+    @Override
+    public int compare(SyncFuture o1, SyncFuture o2) {
+      int c = Long.compare(o1.getTxid(), o2.getTxid());
+      return c != 0 ? c : Integer.compare(System.identityHashCode(o1), System.identityHashCode(o2));
+    }
+  };
+
+  private final Runnable consumer = new Runnable() {
+
+    @Override
+    public void run() {
+      synchronized (waitingConsumePayloads) {
+        assert consumerScheduled;
+        if (writerBroken) {
+          // waiting for reschedule after rollWriter.
+          consumerScheduled = false;
+          return;
+        }
+        if (waitingRoll) {
+          // we may have toWriteEntries if the consume method does not write all pending entries
+          // out, this is usually happen if we have too many toWriteEntries that exceeded the
+          // batchSize limit.
+          if (waitingAppendEntries.isEmpty()) {
+            consumerScheduled = false;
+            return;
+          }
+        } else {
+          for (Payload p; (p = waitingConsumePayloads.pollFirst()) != null;) {
+            if (p.entry != null) {
+              waitingAppendEntries.addLast(p.entry);
+            } else if (p.sync != null) {
+              syncFutures.add(p.sync);
+            } else {
+              rollPromise = p.roll;
+              waitingRoll = true;
+              break;
+            }
+          }
+        }
+      }
+      consume();
+      synchronized (waitingConsumePayloads) {
+        if (waitingRoll) {
+          if (waitingAppendEntries.isEmpty()) {
+            consumerScheduled = false;
+            return;
+          }
+        } else {
+          if (waitingConsumePayloads.isEmpty() && waitingAppendEntries.isEmpty()) {
+            consumerScheduled = false;
+            return;
+          }
+        }
+      }
+      // reschedule if we still have something to write.
+      eventLoop.execute(this);
+    }
+  };
+
+  private boolean shouldScheduleConsumer() {
+    if (writerBroken || waitingRoll) {
+      return false;
+    }
+    if (consumerScheduled) {
+      return false;
+    }
+    consumerScheduled = true;
+    return true;
+  }
+
+  @Override
+  public long append(HRegionInfo hri, WALKey key, WALEdit edits, boolean inMemstore)
+      throws IOException {
+    boolean scheduleTask;
+    long txid;
+    synchronized (waitingConsumePayloads) {
+      if (this.closed) {
+        throw new IOException("Cannot append; log is closed");
+      }
+      txid = nextTxid++;
+      FSWALEntry entry = new FSWALEntry(txid, key, edits, hri, inMemstore);
+      scheduleTask = shouldScheduleConsumer();
+      waitingConsumePayloads.add(new Payload(entry));
+    }
+    if (scheduleTask) {
+      eventLoop.execute(consumer);
+    }
+    return txid;
+  }
+
+  @Override
+  public void sync() throws IOException {
+    TraceScope scope = Trace.startSpan("AsyncFSWAL.sync");
+    try {
+      SyncFuture future;
+      boolean scheduleTask;
+      synchronized (waitingConsumePayloads) {
+        scheduleTask = shouldScheduleConsumer();
+        future = getSyncFuture(nextTxid - 1, scope.detach());
+        waitingConsumePayloads.addLast(new Payload(future));
+      }
+      if (scheduleTask) {
+        eventLoop.execute(consumer);
+      }
+      scope = Trace.continueSpan(blockOnSync(future));
+    } finally {
+      assert scope == NullScope.INSTANCE || !scope.isDetached();
+      scope.close();
+    }
+  }
+
+  @Override
+  public void sync(long txid) throws IOException {
+    if (highestSyncedTxid.get() >= txid) {
+      return;
+    }
+    TraceScope scope = Trace.startSpan("AsyncFSWAL.sync");
+    try {
+      SyncFuture future = getSyncFuture(txid, scope.detach());
+      boolean scheduleTask;
+      synchronized (waitingConsumePayloads) {
+        scheduleTask = shouldScheduleConsumer();
+        waitingConsumePayloads.addLast(new Payload(future));
+      }
+      if (scheduleTask) {
+        eventLoop.execute(consumer);
+      }
+      scope = Trace.continueSpan(blockOnSync(future));
+    } finally {
+      assert scope == NullScope.INSTANCE || !scope.isDetached();
+      scope.close();
+    }
+  }
+
+  @Override
+  public void logRollerExited() {
+    logRollerExited = true;
+  }
+
+  @Override
+  protected AsyncWriter createWriterInstance(Path path) throws IOException {
+    boolean overwrite = false;
+    for (int retry = 0;; retry++) {
+      try {
+        return AsyncFSWALProvider.createAsyncWriter(conf, fs, path, overwrite, eventLoop);
+      } catch (RemoteException e) {
+        LOG.warn("create wal log writer " + path + " failed, retry = " + retry, e);
+        if (shouldRetryCreate(e)) {
+          if (retry >= createMaxRetries) {
+            break;
+          }
+        } else {
+          throw e.unwrapRemoteException();
+        }
+      } catch (NameNodeException e) {
+        throw e;
+      } catch (IOException e) {
+        LOG.warn("create wal log writer " + path + " failed, retry = " + retry, e);
+        if (retry >= createMaxRetries) {
+          break;
+        }
+        // overwrite the old broken file.
+        overwrite = true;
+        try {
+          Thread.sleep(ConnectionUtils.getPauseTime(100, retry));
+        } catch (InterruptedException ie) {
+          throw new InterruptedIOException();
+        }
+      }
+    }
+    throw new IOException("Failed to create wal log writer " + path + " after retrying "
+        + createMaxRetries + " time(s)");
+  }
+
+  private void waitForSafePoint() {
+    Future<Void> roll;
+    boolean scheduleTask;
+    synchronized (waitingConsumePayloads) {
+      if (!writerBroken && this.writer != null) {
+        Promise<Void> promise = eventLoop.newPromise();
+        if (consumerScheduled) {
+          scheduleTask = false;
+        } else {
+          scheduleTask = consumerScheduled = true;
+        }
+        waitingConsumePayloads.addLast(new Payload(promise));
+        roll = promise;
+      } else {
+        roll = eventLoop.newSucceededFuture(null);
+        scheduleTask = false;
+      }
+    }
+    if (scheduleTask) {
+      eventLoop.execute(consumer);
+    }
+    roll.awaitUninterruptibly();
+  }
+
+  @Override
+  protected long doReplaceWriter(Path oldPath, Path newPath, AsyncWriter nextWriter)
+      throws IOException {
+    waitForSafePoint();
+    final AsyncWriter oldWriter = this.writer;
+    this.writer = nextWriter;
+    if (nextWriter != null && nextWriter instanceof AsyncProtobufLogWriter) {
+      this.hdfsOut = ((AsyncProtobufLogWriter) nextWriter).getOutput();
+    }
+    this.fileLengthAtLastSync = 0L;
+    boolean scheduleTask;
+    synchronized (waitingConsumePayloads) {
+      writerBroken = waitingRoll = false;
+      if (logRollerExitedChecker != null) {
+        logRollerExitedChecker.cancel();
+        logRollerExitedChecker = null;
+      }
+      if (consumerScheduled) {
+        scheduleTask = false;
+      } else {
+        if (waitingConsumePayloads.isEmpty() && waitingAppendEntries.isEmpty()) {
+          scheduleTask = false;
+        } else {
+          scheduleTask = consumerScheduled = true;
+        }
+      }
+    }
+    if (scheduleTask) {
+      eventLoop.execute(consumer);
+    }
+    long oldFileLen;
+    if (oldWriter != null) {
+      oldFileLen = oldWriter.getLength();
+      closeExecutor.execute(new Runnable() {
+
+        @Override
+        public void run() {
+          try {
+            oldWriter.close();
+          } catch (IOException e) {
+            LOG.warn("close old writer failed", e);
+          }
+        }
+      });
+    } else {
+      oldFileLen = 0L;
+    }
+    return oldFileLen;
+  }
+
+  @Override
+  protected void doShutdown() throws IOException {
+    waitForSafePoint();
+    this.writer.close();
+    this.writer = null;
+    closeExecutor.shutdown();
+  }
+
+  @Override
+  protected void doAppend(AsyncWriter writer, FSWALEntry entry) {
+    writer.append(entry);
+  }
+
+  @Override
+  DatanodeInfo[] getPipeline() {
+    FanOutOneBlockAsyncDFSOutput output = this.hdfsOut;
+    return output != null ? output.getPipeline() : new DatanodeInfo[0];
+  }
+
+  @Override
+  int getLogReplication() {
+    return getPipeline().length;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncProtobufLogWriter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncProtobufLogWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncProtobufLogWriter.java
new file mode 100644
index 0000000..894f3dd
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncProtobufLogWriter.java
@@ -0,0 +1,203 @@
+/**
+ * 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.wal;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.io.OutputStream;
+import java.nio.channels.CompletionHandler;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.io.ByteArrayOutputStream;
+import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALHeader;
+import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALTrailer;
+import org.apache.hadoop.hbase.util.FanOutOneBlockAsyncDFSOutput;
+import org.apache.hadoop.hbase.util.FanOutOneBlockAsyncDFSOutputHelper;
+import org.apache.hadoop.hbase.wal.AsyncFSWALProvider;
+import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+
+import com.google.common.base.Throwables;
+import com.google.common.primitives.Ints;
+
+import io.netty.channel.EventLoop;
+
+/**
+ * AsyncWriter for protobuf-based WAL.
+ */
+@InterfaceAudience.Private
+public class AsyncProtobufLogWriter extends AbstractProtobufLogWriter implements
+    AsyncFSWALProvider.AsyncWriter {
+
+  private static final Log LOG = LogFactory.getLog(AsyncProtobufLogWriter.class);
+
+  private static final class BlockingCompletionHandler implements CompletionHandler<Long, Void> {
+
+    private long size;
+
+    private Throwable error;
+
+    private boolean finished;
+
+    @Override
+    public void completed(Long result, Void attachment) {
+      synchronized (this) {
+        size = result.longValue();
+        finished = true;
+        notifyAll();
+      }
+    }
+
+    @Override
+    public void failed(Throwable exc, Void attachment) {
+      synchronized (this) {
+        error = exc;
+        finished = true;
+        notifyAll();
+      }
+    }
+
+    public long get() throws IOException {
+      synchronized (this) {
+        while (!finished) {
+          try {
+            wait();
+          } catch (InterruptedException e) {
+            throw new InterruptedIOException();
+          }
+        }
+        if (error != null) {
+          Throwables.propagateIfPossible(error, IOException.class);
+          throw new RuntimeException(error);
+        }
+        return size;
+      }
+    }
+  }
+
+  private final EventLoop eventLoop;
+
+  private FanOutOneBlockAsyncDFSOutput output;
+
+  private ByteArrayOutputStream buf;
+
+  public AsyncProtobufLogWriter(EventLoop eventLoop) {
+    this.eventLoop = eventLoop;
+  }
+
+  @Override
+  public void append(Entry entry) {
+    buf.reset();
+    entry.setCompressionContext(compressionContext);
+    try {
+      entry.getKey().getBuilder(compressor).setFollowingKvCount(entry.getEdit().size()).build()
+          .writeDelimitedTo(buf);
+    } catch (IOException e) {
+      throw new AssertionError("should not happen", e);
+    }
+    length.addAndGet(buf.size());
+    output.write(buf.getBuffer(), 0, buf.size());
+    try {
+      for (Cell cell : entry.getEdit().getCells()) {
+        buf.reset();
+        cellEncoder.write(cell);
+        length.addAndGet(buf.size());
+        output.write(buf.getBuffer(), 0, buf.size());
+      }
+    } catch (IOException e) {
+      throw new AssertionError("should not happen", e);
+    }
+  }
+
+  @Override
+  public <A> void sync(CompletionHandler<Long, A> handler, A attachment) {
+    output.flush(attachment, handler, false);
+  }
+
+  @Override
+  public synchronized void close() throws IOException {
+    if (this.output == null) {
+      return;
+    }
+    try {
+      writeWALTrailer();
+      output.close();
+    } catch (Exception e) {
+      LOG.warn("normal close failed, try recover", e);
+      output.recoverAndClose(null);
+    }
+    this.output = null;
+  }
+
+  public FanOutOneBlockAsyncDFSOutput getOutput() {
+    return this.output;
+  }
+
+  @Override
+  protected void initOutput(FileSystem fs, Path path, boolean overwritable, int bufferSize,
+      short replication, long blockSize) throws IOException {
+    this.output =
+        FanOutOneBlockAsyncDFSOutputHelper.createOutput((DistributedFileSystem) fs, path,
+          overwritable, false, replication, blockSize, eventLoop);
+    this.buf = new ByteArrayOutputStream();
+  }
+
+  @Override
+  protected long writeMagicAndWALHeader(byte[] magic, WALHeader header) throws IOException {
+    buf.reset();
+    header.writeDelimitedTo(buf);
+    final BlockingCompletionHandler handler = new BlockingCompletionHandler();
+    eventLoop.execute(new Runnable() {
+
+      @Override
+      public void run() {
+        output.write(ProtobufLogReader.PB_WAL_MAGIC);
+        output.write(buf.getBuffer(), 0, buf.size());
+        output.flush(null, handler, false);
+      }
+    });
+    return handler.get();
+  }
+
+  @Override
+  protected long writeWALTrailerAndMagic(WALTrailer trailer, final byte[] magic)
+      throws IOException {
+    buf.reset();
+    trailer.writeTo(buf);
+    final BlockingCompletionHandler handler = new BlockingCompletionHandler();
+    eventLoop.execute(new Runnable() {
+      public void run() {
+        output.write(buf.getBuffer(), 0, buf.size());
+        output.write(Ints.toByteArray(buf.size()));
+        output.write(magic);
+        output.flush(null, handler, false);
+      }
+    });
+    return handler.get();
+  }
+
+  @Override
+  protected OutputStream getOutputStreamForCellEncoder() {
+    return buf;
+  }
+}


[12/50] [abbrv] hbase git commit: HBASE-15212 RRCServer should enforce max request size

Posted by sy...@apache.org.
HBASE-15212 RRCServer should enforce max request size


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

Branch: refs/heads/hbase-12439
Commit: 3f3613a23485121ae72959206f4db09a0f557cd1
Parents: 8af9ed7
Author: Enis Soztutar <en...@apache.org>
Authored: Tue Mar 22 16:23:15 2016 -0700
Committer: Enis Soztutar <en...@apache.org>
Committed: Tue Mar 22 16:23:15 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/ipc/RpcServer.java  | 15 ++++++-
 .../hadoop/hbase/ipc/AbstractTestIPC.java       | 44 ++++++++++++++++++--
 2 files changed, 53 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/3f3613a2/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 ef6c198..2c6084a 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
@@ -260,15 +260,18 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
 
   protected HBaseRPCErrorHandler errorHandler = null;
 
+  static final String MAX_REQUEST_SIZE = "hbase.ipc.max.request.size";
   private static final String WARN_RESPONSE_TIME = "hbase.ipc.warn.response.time";
   private static final String WARN_RESPONSE_SIZE = "hbase.ipc.warn.response.size";
 
   /** Default value for above params */
+  private static final int DEFAULT_MAX_REQUEST_SIZE = DEFAULT_MAX_CALLQUEUE_SIZE / 4; // 256M
   private static final int DEFAULT_WARN_RESPONSE_TIME = 10000; // milliseconds
   private static final int DEFAULT_WARN_RESPONSE_SIZE = 100 * 1024 * 1024;
 
   private static final ObjectMapper MAPPER = new ObjectMapper();
 
+  private final int maxRequestSize;
   private final int warnResponseTime;
   private final int warnResponseSize;
   private final Server server;
@@ -1239,6 +1242,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
     protected String hostAddress;
     protected int remotePort;
     ConnectionHeader connectionHeader;
+
     /**
      * Codec the client asked use.
      */
@@ -1623,11 +1627,16 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
           }
         }
         if (dataLength < 0) { // A data length of zero is legal.
-          throw new IllegalArgumentException("Unexpected data length "
+          throw new DoNotRetryIOException("Unexpected data length "
               + dataLength + "!! from " + getHostAddress());
         }
 
-       // TODO: check dataLength against some limit so that the client cannot OOM the server
+        if (dataLength > maxRequestSize) {
+          throw new DoNotRetryIOException("RPC data length of " + dataLength + " received from "
+              + getHostAddress() + " is greater than max allowed " + maxRequestSize + ". Set \""
+              + MAX_REQUEST_SIZE + "\" on server to override this limit (not recommended)");
+        }
+
         data = ByteBuffer.allocate(dataLength);
 
         // Increment the rpc count. This counter will be decreased when we write
@@ -2071,6 +2080,8 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
     this.warnResponseTime = conf.getInt(WARN_RESPONSE_TIME, DEFAULT_WARN_RESPONSE_TIME);
     this.warnResponseSize = conf.getInt(WARN_RESPONSE_SIZE, DEFAULT_WARN_RESPONSE_SIZE);
 
+    this.maxRequestSize = conf.getInt(MAX_REQUEST_SIZE, DEFAULT_MAX_REQUEST_SIZE);
+
     // Start the listener here and let it bind to the port
     listener = new Listener(name);
     this.port = listener.getAddress().getPort();

http://git-wip-us.apache.org/repos/asf/hbase/blob/3f3613a2/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
index ffe4d40..e8da9ee 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
@@ -57,6 +57,7 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.io.compress.GzipCodec;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.http.ConnectionClosedException;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -137,13 +138,17 @@ public abstract class AbstractTestIPC {
   static class TestRpcServer extends RpcServer {
 
     TestRpcServer() throws IOException {
-      this(new FifoRpcScheduler(CONF, 1));
+      this(new FifoRpcScheduler(CONF, 1), CONF);
+    }
+
+    TestRpcServer(Configuration conf) throws IOException {
+      this(new FifoRpcScheduler(conf, 1), conf);
     }
 
-    TestRpcServer(RpcScheduler scheduler) throws IOException {
+    TestRpcServer(RpcScheduler scheduler, Configuration conf) throws IOException {
       super(null, "testRpcServer", Lists
           .newArrayList(new BlockingServiceAndInterface(SERVICE, null)), new InetSocketAddress(
-          "localhost", 0), CONF, scheduler);
+          "localhost", 0), conf, scheduler);
     }
 
     @Override
@@ -267,7 +272,7 @@ public abstract class AbstractTestIPC {
   @Test
   public void testRpcScheduler() throws IOException, InterruptedException {
     RpcScheduler scheduler = spy(new FifoRpcScheduler(CONF, 1));
-    RpcServer rpcServer = new TestRpcServer(scheduler);
+    RpcServer rpcServer = new TestRpcServer(scheduler, CONF);
     verify(scheduler).init((RpcScheduler.Context) anyObject());
     AbstractRpcClient client = createRpcClient(CONF);
     try {
@@ -292,6 +297,37 @@ public abstract class AbstractTestIPC {
     }
   }
 
+  /** Tests that the rpc scheduler is called when requests arrive. */
+  @Test
+  public void testRpcMaxRequestSize() throws IOException, InterruptedException {
+    Configuration conf = new Configuration(CONF);
+    conf.setInt(RpcServer.MAX_REQUEST_SIZE, 100);
+    RpcServer rpcServer = new TestRpcServer(conf);
+    AbstractRpcClient client = createRpcClient(conf);
+    try {
+      rpcServer.start();
+      MethodDescriptor md = SERVICE.getDescriptorForType().findMethodByName("echo");
+      // set total RPC size bigger than 100 bytes
+      EchoRequestProto param = EchoRequestProto.newBuilder().setMessage("hello.hello.hello.hello."
+          + "hello.hello.hello.hello.hello.hello.hello.hello.hello.hello.hello.hello").build();
+      InetSocketAddress address = rpcServer.getListenerAddress();
+      if (address == null) {
+        throw new IOException("Listener channel is closed");
+      }
+      try {
+        client.call(new PayloadCarryingRpcController(
+          CellUtil.createCellScanner(ImmutableList.<Cell> of(CELL))), md, param,
+          md.getOutputType().toProto(), User.getCurrent(), address,
+          new MetricsConnection.CallStats());
+        fail("RPC should have failed because it exceeds max request size");
+      } catch(ConnectionClosingException | ConnectionClosedException ex) {
+        // pass
+      }
+    } finally {
+      rpcServer.stop();
+    }
+  }
+
   /**
    * Instance of RpcServer that echoes client hostAddress back to client
    */


[23/50] [abbrv] hbase git commit: HBASE-15392 Single Cell Get reads two HFileBlocks

Posted by sy...@apache.org.
HBASE-15392 Single Cell Get reads two HFileBlocks

M hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java
moreRowsMayExistAfterCell Exploit the fact a Scan is a Get Scan. Also save compares
if no non-default stopRow.

M hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
optimize Add doc on what is being optimized. Also, if a Get Scan, do not
optimize else we'll keep going after our row is DONE.
Another place to make use of the Get Scan fact is when we are DONE.. if
Get Scan, we can close out the scan.

M hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java
Add tests for Get Scans and optimize around block loading.


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

Branch: refs/heads/hbase-12439
Commit: d7a4499dfc8b3936a0eca867589fc2b23b597866
Parents: 95e6d22
Author: stack <st...@apache.org>
Authored: Fri Mar 11 15:41:26 2016 -0800
Committer: stack <st...@apache.org>
Committed: Wed Mar 23 14:37:14 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/CellComparator.java |   2 +-
 .../hbase/io/hfile/CombinedBlockCache.java      |  13 +-
 .../hadoop/hbase/io/hfile/HFileWriterImpl.java  |   2 +-
 .../hbase/regionserver/KeyValueScanner.java     |  16 +-
 .../hbase/regionserver/ScanQueryMatcher.java    |  60 +++--
 .../hadoop/hbase/regionserver/StoreScanner.java |  85 ++++++-
 .../hbase/util/CollectionBackedScanner.java     |   3 +-
 .../hbase/regionserver/KeyValueScanFixture.java |   8 +-
 .../regionserver/TestKeyValueScanFixture.java   |   3 +-
 .../hbase/regionserver/TestStoreScanner.java    | 244 ++++++++++++++++++-
 10 files changed, 372 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d7a4499d/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 a5e26cf..4a5c0b7 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
@@ -433,7 +433,7 @@ public class CellComparator implements Comparator<Cell>, Serializable {
   /**
    * Used to compare two cells based on the column hint provided. This is specifically
    * used when we need to optimize the seeks based on the next indexed key. This is an
-   * advance usage API specifically needed for some optimizations.
+   * advanced usage API specifically needed for some optimizations.
    * @param nextIndexedCell the next indexed cell 
    * @param currentCell the cell to be compared
    * @param foff the family offset of the currentCell

http://git-wip-us.apache.org/repos/asf/hbase/blob/d7a4499d/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java
index 22bffee..666b357 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java
@@ -63,8 +63,8 @@ public class CombinedBlockCache implements ResizableBlockCache, HeapSize {
   @Override
   public void cacheBlock(BlockCacheKey cacheKey, Cacheable buf, boolean inMemory,
       final boolean cacheDataInL1) {
-    boolean isMetaBlock = buf.getBlockType().getCategory() != BlockCategory.DATA;
-    if (isMetaBlock || cacheDataInL1) {
+    boolean metaBlock = buf.getBlockType().getCategory() != BlockCategory.DATA;
+    if (metaBlock || cacheDataInL1) {
       lruCache.cacheBlock(cacheKey, buf, inMemory, cacheDataInL1);
     } else {
       l2Cache.cacheBlock(cacheKey, buf, inMemory, false);
@@ -81,12 +81,9 @@ public class CombinedBlockCache implements ResizableBlockCache, HeapSize {
       boolean repeat, boolean updateCacheMetrics) {
     // TODO: is there a hole here, or just awkwardness since in the lruCache getBlock
     // we end up calling l2Cache.getBlock.
-    if (lruCache.containsBlock(cacheKey)) {
-      return lruCache.getBlock(cacheKey, caching, repeat, updateCacheMetrics);
-    }
-    Cacheable result = l2Cache.getBlock(cacheKey, caching, repeat, updateCacheMetrics);
-
-    return result;
+    return lruCache.containsBlock(cacheKey)?
+        lruCache.getBlock(cacheKey, caching, repeat, updateCacheMetrics):
+        l2Cache.getBlock(cacheKey, caching, repeat, updateCacheMetrics);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/d7a4499d/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
index d310d13..9ab46cf 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
@@ -416,7 +416,7 @@ public class HFileWriterImpl implements HFile.Writer {
     // No opportunity for optimization. Just return right key.
     return right;
   }
-  
+
   /**
    * @param leftArray
    * @param leftOffset

http://git-wip-us.apache.org/repos/asf/hbase/blob/d7a4499d/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java
index 3df284c..ed86a83 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java
@@ -30,6 +30,9 @@ import org.apache.hadoop.hbase.client.Scan;
  * Scanner that returns the next KeyValue.
  */
 @InterfaceAudience.Private
+// TODO: Change name from KeyValueScanner to CellScanner only we already have a simple CellScanner
+// so this should be something else altogether, a decoration on our base CellScanner. TODO.
+// This class shows in CPs so do it all in one swell swoop. HBase-2.0.0.
 public interface KeyValueScanner extends Shipper, Closeable {
   /**
    * The byte array represents for NO_NEXT_INDEXED_KEY;
@@ -136,11 +139,11 @@ public interface KeyValueScanner extends Shipper, Closeable {
    * peek KeyValue of scanner has the same row with specified Cell,
    * otherwise seek the scanner at the first Cell of the row which is the
    * previous row of specified KeyValue
-   * 
+   *
    * @param key seek KeyValue
    * @return true if the scanner is at the valid KeyValue, false if such
    *         KeyValue does not exist
-   * 
+   *
    */
   public boolean backwardSeek(Cell key) throws IOException;
 
@@ -155,7 +158,7 @@ public interface KeyValueScanner extends Shipper, Closeable {
 
   /**
    * Seek the scanner at the first KeyValue of last row
-   * 
+   *
    * @return true if scanner has values left, false if the underlying data is
    *         empty
    * @throws IOException
@@ -163,8 +166,9 @@ public interface KeyValueScanner extends Shipper, Closeable {
   public boolean seekToLastRow() throws IOException;
 
   /**
-   * @return the next key in the index (the key to seek to the next block)
-   * if known, or null otherwise
+   * @return the next key in the index, usually the first key of next block OR a key that falls
+   * between last key of current block and first key of next block..
+   * see HFileWriterImpl#getMidpoint, or null if not known.
    */
   public Cell getNextIndexedKey();
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d7a4499d/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java
index c220b5c..706fc5b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java
@@ -93,7 +93,7 @@ public class ScanQueryMatcher {
   /* row is not private for tests */
   /** Row the query is on */
   Cell curCell;
-  
+
   /**
    * Oldest put in any of the involved store files
    * Used to decide whether it is ok to delete
@@ -119,7 +119,7 @@ public class ScanQueryMatcher {
    * first column.
    * */
   private boolean hasNullColumn = true;
-  
+
   private RegionCoprocessorHost regionCoprocessorHost= null;
 
   // By default, when hbase.hstore.time.to.purge.deletes is 0ms, a delete
@@ -140,22 +140,22 @@ public class ScanQueryMatcher {
   // currently influencing. This is because Puts, that this delete can
   // influence.  may appear out of order.
   private final long timeToPurgeDeletes;
-  
+
   private final boolean isUserScan;
 
   private final boolean isReversed;
 
   /**
+   * True if we are doing a 'Get' Scan. Every Get is actually a one-row Scan.
+   */
+  private final boolean get;
+
+  /**
    * Construct a QueryMatcher for a scan
-   * @param scan
    * @param scanInfo The store's immutable scan info
-   * @param columns
    * @param scanType Type of the scan
    * @param earliestPutTs Earliest put seen in any of the store files.
-   * @param oldestUnexpiredTS the oldest timestamp we are interested in,
-   *  based on TTL
-   * @param regionCoprocessorHost 
-   * @throws IOException 
+   * @param oldestUnexpiredTS the oldest timestamp we are interested in, based on TTL
    */
   public ScanQueryMatcher(Scan scan, ScanInfo scanInfo, NavigableSet<byte[]> columns,
       ScanType scanType, long readPointToUse, long earliestPutTs, long oldestUnexpiredTS,
@@ -166,6 +166,7 @@ public class ScanQueryMatcher {
     } else {
       this.tr = timeRange;
     }
+    this.get = scan.isGetScan();
     this.rowComparator = scanInfo.getComparator();
     this.regionCoprocessorHost = regionCoprocessorHost;
     this.deletes =  instantiateDeleteTracker();
@@ -234,8 +235,8 @@ public class ScanQueryMatcher {
    * @param now the current server time
    * @param dropDeletesFromRow The inclusive left bound of the range; can be EMPTY_START_ROW.
    * @param dropDeletesToRow The exclusive right bound of the range; can be EMPTY_END_ROW.
-   * @param regionCoprocessorHost 
-   * @throws IOException 
+   * @param regionCoprocessorHost
+   * @throws IOException
    */
   public ScanQueryMatcher(Scan scan, ScanInfo scanInfo, NavigableSet<byte[]> columns,
       long readPointToUse, long earliestPutTs, long oldestUnexpiredTS, long now,
@@ -280,7 +281,7 @@ public class ScanQueryMatcher {
    *      caused by a data corruption.
    */
   public MatchCode match(Cell cell) throws IOException {
-      if (filter != null && filter.filterAllRemaining()) {
+    if (filter != null && filter.filterAllRemaining()) {
       return MatchCode.DONE_SCAN;
     }
     if (curCell != null) {
@@ -324,7 +325,7 @@ public class ScanQueryMatcher {
     // check if the cell is expired by cell TTL
     if (HStore.isCellTTLExpired(cell, this.oldestUnexpiredTS, this.now)) {
       return MatchCode.SKIP;
-    }    
+    }
 
     /*
      * The delete logic is pretty complicated now.
@@ -359,10 +360,10 @@ public class ScanQueryMatcher {
         }
         // Can't early out now, because DelFam come before any other keys
       }
-     
+
       if ((!isUserScan)
           && timeToPurgeDeletes > 0
-          && (EnvironmentEdgeManager.currentTime() - timestamp) 
+          && (EnvironmentEdgeManager.currentTime() - timestamp)
             <= timeToPurgeDeletes) {
         return MatchCode.INCLUDE;
       } else if (retainDeletesInOutput || mvccVersion > maxReadPointToTrackVersions) {
@@ -503,22 +504,27 @@ public class ScanQueryMatcher {
     }
   }
 
+  /**
+   * @return Returns false if we know there are no more rows to be scanned (We've reached the
+   * <code>stopRow</code> or we are scanning on row only because this Scan is for a Get, etc.
+   */
   public boolean moreRowsMayExistAfter(Cell kv) {
-    if (this.isReversed) {
-      if (rowComparator.compareRows(kv, stopRow, 0, stopRow.length) <= 0) {
-        return false;
-      } else {
-        return true;
-      }
-    }
-    if (!Bytes.equals(stopRow , HConstants.EMPTY_END_ROW) &&
-        rowComparator.compareRows(kv, stopRow, 0, stopRow.length) >= 0) {
-      // KV >= STOPROW
-      // then NO there is nothing left.
+    // If a 'get' Scan -- we are doing a Get (every Get is a single-row Scan in implementation) --
+    // then we are looking at one row only, the one specified in the Get coordinate..so we know
+    // for sure that there are no more rows on this Scan
+    if (this.get) {
       return false;
-    } else {
+    }
+    // If no stopRow, return that there may be more rows. The tests that follow depend on a
+    // non-empty, non-default stopRow so this little test below short-circuits out doing the
+    // following compares.
+    if (this.stopRow == null || this.stopRow == HConstants.EMPTY_BYTE_ARRAY) {
       return true;
     }
+    return this.isReversed?
+      rowComparator.compareRows(kv, stopRow, 0, stopRow.length) > 0:
+      Bytes.equals(stopRow, HConstants.EMPTY_END_ROW) ||
+        rowComparator.compareRows(kv, stopRow, 0, stopRow.length) < 0;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/d7a4499d/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
index 2f0d284..8dd3d7d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
@@ -133,7 +133,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
   protected List<KeyValueScanner> currentScanners = new ArrayList<KeyValueScanner>();
   // flush update lock
   private ReentrantLock flushLock = new ReentrantLock();
-  
+
   protected final long readPt;
 
   // used by the injection framework to test race between StoreScanner construction and compaction
@@ -600,6 +600,12 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
         continue;
 
       case DONE:
+        // Optimization for Gets! If DONE, no more to get on this row, early exit!
+        if (this.scan.isGetScan()) {
+          // Then no more to this row... exit.
+          close(false);// Do all cleanup except heap.close()
+          return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
+        }
         matcher.curCell = null;
         return scannerContext.setScannerState(NextState.MORE_VALUES).hasMoreValues();
 
@@ -649,11 +655,60 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
     return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
   }
 
-  /*
-   * See if we should actually SEEK or rather just SKIP to the next Cell.
-   * (see HBASE-13109)
+  /**
+   * See if we should actually SEEK or rather just SKIP to the next Cell (see HBASE-13109).
+   * This method works together with ColumnTrackers and Filters. ColumnTrackers may issue SEEK
+   * hints, such as seek to next column, next row, or seek to an arbitrary seek key.
+   * This method intercepts these qcodes and decides whether a seek is the most efficient _actual_
+   * way to get us to the requested cell (SEEKs are more expensive than SKIP, SKIP, SKIP inside the
+   * current, loaded block).
+   * It does this by looking at the next indexed key of the current HFile. This key
+   * is then compared with the _SEEK_ key, where a SEEK key is an artificial 'last possible key
+   * on the row' (only in here, we avoid actually creating a SEEK key; in the compare we work with
+   * the current Cell but compare as though it were a seek key; see down in
+   * matcher.compareKeyForNextRow, etc). If the compare gets us onto the
+   * next block we *_SEEK, otherwise we just INCLUDE or SKIP, and let the ColumnTrackers or Filters
+   * go through the next Cell, and so on)
+   *
+   * <p>The ColumnTrackers and Filters must behave correctly in all cases, i.e. if they are past the
+   * Cells they care about they must issues a SKIP or SEEK.
+   *
+   * <p>Other notes:
+   * <ul>
+   * <li>Rows can straddle block boundaries</li>
+   * <li>Versions of columns can straddle block boundaries (i.e. column C1 at T1 might be in a
+   * different block than column C1 at T2)</li>
+   * <li>We want to SKIP and INCLUDE if the chance is high that we'll find the desired Cell after a
+   * few SKIPs...</li>
+   * <li>We want to INCLUDE_AND_SEEK and SEEK when the chance is high that we'll be able to seek
+   * past many Cells, especially if we know we need to go to the next block.</li>
+   * </ul>
+   * <p>A good proxy (best effort) to determine whether INCLUDE/SKIP is better than SEEK is whether
+   * we'll likely end up seeking to the next block (or past the next block) to get our next column.
+   * Example:
+   * <pre>
+   * |    BLOCK 1              |     BLOCK 2                   |
+   * |  r1/c1, r1/c2, r1/c3    |    r1/c4, r1/c5, r2/c1        |
+   *                                   ^         ^
+   *                                   |         |
+   *                           Next Index Key   SEEK_NEXT_ROW (before r2/c1)
+   *
+   *
+   * |    BLOCK 1                       |     BLOCK 2                      |
+   * |  r1/c1/t5, r1/c1/t4, r1/c1/t3    |    r1/c1/t2, r1/c1/T1, r1/c2/T3  |
+   *                                            ^              ^
+   *                                            |              |
+   *                                    Next Index Key        SEEK_NEXT_COL
+   * </pre>
+   * Now imagine we want columns c1 and c3 (see first diagram above), the 'Next Index Key' of r1/c4
+   * is > r1/c3 so we should seek to get to the c1 on the next row, r2. In second case, say we only
+   * want one version of c1, after we have it, a SEEK_COL will be issued to get to c2. Looking at
+   * the 'Next Index Key', it would land us in the next block, so we should SEEK. In other scenarios
+   * where the SEEK will not land us in the next block, it is very likely better to issues a series
+   * of SKIPs.
    */
-  private ScanQueryMatcher.MatchCode optimize(ScanQueryMatcher.MatchCode qcode, Cell cell) {
+  @VisibleForTesting
+  protected ScanQueryMatcher.MatchCode optimize(ScanQueryMatcher.MatchCode qcode, Cell cell) {
     switch(qcode) {
     case INCLUDE_AND_SEEK_NEXT_COL:
     case SEEK_NEXT_COL:
@@ -668,10 +723,16 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
     case INCLUDE_AND_SEEK_NEXT_ROW:
     case SEEK_NEXT_ROW:
     {
-      Cell nextIndexedKey = getNextIndexedKey();
-      if (nextIndexedKey != null && nextIndexedKey != KeyValueScanner.NO_NEXT_INDEXED_KEY
-          && matcher.compareKeyForNextRow(nextIndexedKey, cell) >= 0) {
-        return qcode == MatchCode.SEEK_NEXT_ROW ? MatchCode.SKIP : MatchCode.INCLUDE;
+      // If it is a Get Scan, then we know that we are done with this row; there are no more
+      // rows beyond the current one: don't try to optimize. We are DONE. Return the *_NEXT_ROW
+      // qcode as is. When the caller gets these flags on a Get Scan, it knows it can shut down the
+      // Scan.
+      if (!this.scan.isGetScan()) {
+        Cell nextIndexedKey = getNextIndexedKey();
+        if (nextIndexedKey != null && nextIndexedKey != KeyValueScanner.NO_NEXT_INDEXED_KEY
+            && matcher.compareKeyForNextRow(nextIndexedKey, cell) > 0) {
+          return qcode == MatchCode.SEEK_NEXT_ROW ? MatchCode.SKIP : MatchCode.INCLUDE;
+        }
       }
       break;
     }
@@ -809,10 +870,10 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
     // check the var without any lock. Suppose even if we see the old
     // value here still it is ok to continue because we will not be resetting
     // the heap but will continue with the referenced memstore's snapshot. For compactions
-    // any way we don't need the updateReaders at all to happen as we still continue with 
+    // any way we don't need the updateReaders at all to happen as we still continue with
     // the older files
     if (flushed) {
-      // If there is a flush and the current scan is notified on the flush ensure that the 
+      // If there is a flush and the current scan is notified on the flush ensure that the
       // scan's heap gets reset and we do a seek on the newly flushed file.
       if(!this.closing) {
         this.lastTop = this.peek();
@@ -842,7 +903,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
     if (scanners.isEmpty()) return;
     int storeFileScannerCount = scanners.size();
     CountDownLatch latch = new CountDownLatch(storeFileScannerCount);
-    List<ParallelSeekHandler> handlers = 
+    List<ParallelSeekHandler> handlers =
         new ArrayList<ParallelSeekHandler>(storeFileScannerCount);
     for (KeyValueScanner scanner : scanners) {
       if (scanner instanceof StoreFileScanner) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/d7a4499d/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CollectionBackedScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CollectionBackedScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CollectionBackedScanner.java
index 9fc068f..4720880 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CollectionBackedScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CollectionBackedScanner.java
@@ -30,8 +30,7 @@ import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.regionserver.NonReversedNonLazyKeyValueScanner;
 
 /**
- * Utility scanner that wraps a sortable collection and serves
- * as a KeyValueScanner.
+ * Utility scanner that wraps a sortable collection and serves as a KeyValueScanner.
  */
 @InterfaceAudience.Private
 public class CollectionBackedScanner extends NonReversedNonLazyKeyValueScanner {

http://git-wip-us.apache.org/repos/asf/hbase/blob/d7a4499d/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/KeyValueScanFixture.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/KeyValueScanFixture.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/KeyValueScanFixture.java
index 3f87a00..a4e7f9b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/KeyValueScanFixture.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/KeyValueScanFixture.java
@@ -22,6 +22,7 @@ package org.apache.hadoop.hbase.regionserver;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.util.CollectionBackedScanner;
@@ -33,9 +34,8 @@ import org.apache.hadoop.hbase.util.CollectionBackedScanner;
  * to be a store file scanner.
  */
 public class KeyValueScanFixture extends CollectionBackedScanner {
-  public KeyValueScanFixture(CellComparator comparator,
-                             KeyValue... incData) {
-    super(comparator, incData);
+  public KeyValueScanFixture(CellComparator comparator, Cell... cells) {
+    super(comparator, cells);
   }
 
   public static List<KeyValueScanner> scanFixture(KeyValue[] ... kvArrays) {
@@ -45,4 +45,4 @@ public class KeyValueScanFixture extends CollectionBackedScanner {
     }
     return scanners;
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/d7a4499d/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeyValueScanFixture.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeyValueScanFixture.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeyValueScanFixture.java
index a8c2c65..0e96682 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeyValueScanFixture.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeyValueScanFixture.java
@@ -46,8 +46,7 @@ public class TestKeyValueScanFixture extends TestCase {
         KeyValueTestUtil.create("RowB", "family", "qf1",
             10, KeyValue.Type.Put, "value-10")
     };
-    KeyValueScanner scan = new KeyValueScanFixture(
-        CellComparator.COMPARATOR, kvs);
+    KeyValueScanner scan = new KeyValueScanFixture(CellComparator.COMPARATOR, kvs);
 
     KeyValue kv = KeyValueUtil.createFirstOnRow(Bytes.toBytes("RowA"));
     // should seek to this:

http://git-wip-us.apache.org/repos/asf/hbase/blob/d7a4499d/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java
index 92c85aa..4c594b0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java
@@ -20,6 +20,7 @@
 package org.apache.hadoop.hbase.regionserver;
 
 import static org.apache.hadoop.hbase.regionserver.KeyValueScanFixture.scanFixture;
+import static org.junit.Assert.*;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -27,16 +28,21 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.NavigableSet;
 import java.util.TreeSet;
+import java.util.concurrent.atomic.AtomicInteger;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CategoryBasedTimeout;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeepDeletedCells;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueTestUtil;
+import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
@@ -53,16 +59,113 @@ import org.junit.rules.TestRule;
 // Can't be small as it plays with EnvironmentEdgeManager
 @Category({RegionServerTests.class, MediumTests.class})
 public class TestStoreScanner {
+  private static final Log LOG = LogFactory.getLog(TestStoreScanner.class);
   @Rule public TestName name = new TestName();
   @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()).
       withLookingForStuckThread(true).build();
   private static final String CF_STR = "cf";
-  final byte [] CF = Bytes.toBytes(CF_STR);
+  private static final byte [] CF = Bytes.toBytes(CF_STR);
   static Configuration CONF = HBaseConfiguration.create();
   private ScanInfo scanInfo = new ScanInfo(CONF, CF, 0, Integer.MAX_VALUE,
       Long.MAX_VALUE, KeepDeletedCells.FALSE, 0, CellComparator.COMPARATOR);
   private ScanType scanType = ScanType.USER_SCAN;
 
+  /**
+   * From here on down, we have a bunch of defines and specific CELL_GRID of Cells. The
+   * CELL_GRID then has a Scanner that can fake out 'block' transitions. All this elaborate
+   * setup is for tests that ensure we don't overread, and that the
+   * {@link StoreScanner#optimize(org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.MatchCode,
+   * Cell)} is not overly enthusiastic.
+   */
+  private static final byte [] ZERO = new byte [] {'0'};
+  private static final byte [] ZERO_POINT_ZERO = new byte [] {'0', '.', '0'};
+  private static final byte [] ONE = new byte [] {'1'};
+  private static final byte [] TWO = new byte [] {'2'};
+  private static final byte [] TWO_POINT_TWO = new byte [] {'2', '.', '2'};
+  private static final byte [] THREE = new byte [] {'3'};
+  private static final byte [] FOUR = new byte [] {'4'};
+  private static final byte [] FIVE = new byte [] {'5'};
+  private static final byte [] VALUE = new byte [] {'v'};
+  private static final int CELL_GRID_BLOCK2_BOUNDARY = 4;
+  private static final int CELL_GRID_BLOCK3_BOUNDARY = 11;
+  private static final int CELL_GRID_BLOCK4_BOUNDARY = 15;
+  private static final int CELL_GRID_BLOCK5_BOUNDARY = 19;
+
+  /**
+   * Five rows by four columns distinguished by column qualifier (column qualifier is one of the
+   * four rows... ONE, TWO, etc.). Exceptions are a weird row after TWO; it is TWO_POINT_TWO.
+   * And then row FOUR has five columns finishing w/ row FIVE having a single column.
+   * We will use this to test scan does the right thing as it
+   * we do Gets, StoreScanner#optimize, and what we do on (faked) block boundaries.
+   */
+  private static final Cell [] CELL_GRID = new Cell [] {
+    CellUtil.createCell(ONE, CF, ONE, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(ONE, CF, TWO, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(ONE, CF, THREE, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(ONE, CF, FOUR, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    // Offset 4 CELL_GRID_BLOCK2_BOUNDARY
+    CellUtil.createCell(TWO, CF, ONE, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(TWO, CF, TWO, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(TWO, CF, THREE, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(TWO, CF, FOUR, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(TWO_POINT_TWO, CF, ZERO, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(TWO_POINT_TWO, CF, ZERO_POINT_ZERO, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(TWO_POINT_TWO, CF, FIVE, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    // Offset 11! CELL_GRID_BLOCK3_BOUNDARY
+    CellUtil.createCell(THREE, CF, ONE, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(THREE, CF, TWO, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(THREE, CF, THREE, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(THREE, CF, FOUR, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    // Offset 15 CELL_GRID_BLOCK4_BOUNDARY
+    CellUtil.createCell(FOUR, CF, ONE, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(FOUR, CF, TWO, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(FOUR, CF, THREE, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(FOUR, CF, FOUR, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    // Offset 19 CELL_GRID_BLOCK5_BOUNDARY
+    CellUtil.createCell(FOUR, CF, FIVE, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(FIVE, CF, ZERO, 1L, KeyValue.Type.Put.getCode(), VALUE),
+  };
+
+  /**
+   * A StoreScanner for our CELL_GRID above. Fakes the block transitions. Does counts of
+   * calls to optimize and counts of when optimize actually did an optimize.
+   */
+  private static class CellGridStoreScanner extends StoreScanner {
+    // Count of how often optimize is called and of how often it does an optimize.
+    final AtomicInteger count = new AtomicInteger(0);
+    final AtomicInteger optimization = new AtomicInteger(0);
+
+    CellGridStoreScanner(final Scan scan, ScanInfo scanInfo, ScanType scanType)
+    throws IOException {
+      super(scan, scanInfo, scanType, scan.getFamilyMap().get(CF),
+        Arrays.<KeyValueScanner>asList(
+          new KeyValueScanner[] {new KeyValueScanFixture(CellComparator.COMPARATOR, CELL_GRID)}));
+    }
+
+    protected ScanQueryMatcher.MatchCode optimize(ScanQueryMatcher.MatchCode qcode, Cell cell) {
+      count.incrementAndGet();
+      ScanQueryMatcher.MatchCode after = super.optimize(qcode, cell);
+      LOG.info("Cell=" + cell + ", nextIndex=" + CellUtil.toString(getNextIndexedKey(), false) +
+          ", before=" + qcode + ", after=" + after);
+      if (qcode != after) {
+        optimization.incrementAndGet();
+      }
+      return after;
+    };
+
+    @Override
+    public Cell getNextIndexedKey() {
+      // Fake block boundaries by having index of next block change as we go through scan.
+      return count.get() > CELL_GRID_BLOCK4_BOUNDARY?
+          CellUtil.createFirstOnRow(CELL_GRID[CELL_GRID_BLOCK5_BOUNDARY]):
+            count.get() > CELL_GRID_BLOCK3_BOUNDARY?
+                CellUtil.createFirstOnRow(CELL_GRID[CELL_GRID_BLOCK4_BOUNDARY]):
+                  count.get() > CELL_GRID_BLOCK2_BOUNDARY?
+                      CellUtil.createFirstOnRow(CELL_GRID[CELL_GRID_BLOCK3_BOUNDARY]):
+                        CellUtil.createFirstOnRow(CELL_GRID[CELL_GRID_BLOCK2_BOUNDARY]);
+    }
+  };
+
   /*
    * Test utility for building a NavigableSet for scanners.
    * @param strCols
@@ -78,6 +181,145 @@ public class TestStoreScanner {
   }
 
   @Test
+  public void testFullRowGetDoesNotOverreadWhenRowInsideOneBlock() throws IOException {
+    // Do a Get against row two. Row two is inside a block that starts with row TWO but ends with
+    // row TWO_POINT_TWO. We should read one block only.
+    Get get = new Get(TWO);
+    Scan scan = new Scan(get);
+    CellGridStoreScanner scanner = new CellGridStoreScanner(scan, this.scanInfo, this.scanType);
+    try {
+      List<Cell> results = new ArrayList<Cell>();
+      while (scanner.next(results)) {
+        continue;
+      }
+      // Should be four results of column 1 (though there are 5 rows in the CELL_GRID -- the
+      // TWO_POINT_TWO row does not have a a column ONE.
+      Assert.assertEquals(4, results.size());
+      // We should have gone the optimize route 5 times totally... an INCLUDE for the four cells
+      // in the row plus the DONE on the end.
+      Assert.assertEquals(5, scanner.count.get());
+      // For a full row Get, there should be no opportunity for scanner optimization.
+      Assert.assertEquals(0, scanner.optimization.get());
+    } finally {
+      scanner.close();
+    }
+  }
+
+  @Test
+  public void testFullRowSpansBlocks() throws IOException {
+    // Do a Get against row FOUR. It spans two blocks.
+    Get get = new Get(FOUR);
+    Scan scan = new Scan(get);
+    CellGridStoreScanner scanner = new CellGridStoreScanner(scan, this.scanInfo, this.scanType);
+    try {
+      List<Cell> results = new ArrayList<Cell>();
+      while (scanner.next(results)) {
+        continue;
+      }
+      // Should be four results of column 1 (though there are 5 rows in the CELL_GRID -- the
+      // TWO_POINT_TWO row does not have a a column ONE.
+      Assert.assertEquals(5, results.size());
+      // We should have gone the optimize route 6 times totally... an INCLUDE for the five cells
+      // in the row plus the DONE on the end.
+      Assert.assertEquals(6, scanner.count.get());
+      // For a full row Get, there should be no opportunity for scanner optimization.
+      Assert.assertEquals(0, scanner.optimization.get());
+    } finally {
+      scanner.close();
+    }
+  }
+
+  /**
+   * Test optimize in StoreScanner. Test that we skip to the next 'block' when we it makes sense
+   * reading the block 'index'.
+   * @throws IOException
+   */
+  @Test
+  public void testOptimize() throws IOException {
+    Scan scan = new Scan();
+    // A scan that just gets the first qualifier on each row of the CELL_GRID
+    scan.addColumn(CF, ONE);
+    CellGridStoreScanner scanner = new CellGridStoreScanner(scan, this.scanInfo, this.scanType);
+    try {
+      List<Cell> results = new ArrayList<Cell>();
+      while (scanner.next(results)) {
+        continue;
+      }
+      // Should be four results of column 1 (though there are 5 rows in the CELL_GRID -- the
+      // TWO_POINT_TWO row does not have a a column ONE.
+      Assert.assertEquals(4, results.size());
+      for (Cell cell: results) {
+        assertTrue(Bytes.equals(ONE, 0, ONE.length,
+            cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength()));
+      }
+      Assert.assertTrue("Optimize should do some optimizations", scanner.optimization.get() > 0);
+    } finally {
+      scanner.close();
+    }
+  }
+
+  /**
+   * Ensure the optimize Scan method in StoreScanner does not get in the way of a Get doing minimum
+   * work... seeking to start of block and then SKIPPING until we find the wanted Cell.
+   * This 'simple' scenario mimics case of all Cells fitting inside a single HFileBlock.
+   * See HBASE-15392. This test is a little cryptic. Takes a bit of staring to figure what it up to.
+   * @throws IOException
+   */
+  @Test
+  public void testOptimizeAndGet() throws IOException {
+    // First test a Get of two columns in the row R2. Every Get is a Scan. Get columns named
+    // R2 and R3.
+    Get get = new Get(TWO);
+    get.addColumn(CF, TWO);
+    get.addColumn(CF, THREE);
+    Scan scan = new Scan(get);
+    CellGridStoreScanner scanner = new CellGridStoreScanner(scan, this.scanInfo, this.scanType);
+    try {
+      List<Cell> results = new ArrayList<Cell>();
+      // For a Get there should be no more next's after the first call.
+      Assert.assertEquals(false, scanner.next(results));
+      // Should be one result only.
+      Assert.assertEquals(2, results.size());
+      // And we should have gone through optimize twice only.
+      Assert.assertEquals("First qcode is SEEK_NEXT_COL and second INCLUDE_AND_SEEK_NEXT_ROW",
+        3, scanner.count.get());
+    } finally {
+      scanner.close();
+    }
+  }
+
+  /**
+   * Ensure that optimize does not cause the Get to do more seeking than required. Optimize
+   * (see HBASE-15392) was causing us to seek all Cells in a block when a Get Scan if the next block
+   * index/start key was a different row to the current one. A bug. We'd call next too often
+   * because we had to exhaust all Cells in the current row making us load the next block just to
+   * discard what we read there. This test is a little cryptic. Takes a bit of staring to figure
+   * what it up to.
+   * @throws IOException
+   */
+  @Test
+  public void testOptimizeAndGetWithFakedNextBlockIndexStart() throws IOException {
+    // First test a Get of second column in the row R2. Every Get is a Scan. Second column has a
+    // qualifier of R2.
+    Get get = new Get(THREE);
+    get.addColumn(CF, TWO);
+    Scan scan = new Scan(get);
+    CellGridStoreScanner scanner = new CellGridStoreScanner(scan, this.scanInfo, this.scanType);
+    try {
+      List<Cell> results = new ArrayList<Cell>();
+      // For a Get there should be no more next's after the first call.
+      Assert.assertEquals(false, scanner.next(results));
+      // Should be one result only.
+      Assert.assertEquals(1, results.size());
+      // And we should have gone through optimize twice only.
+      Assert.assertEquals("First qcode is SEEK_NEXT_COL and second INCLUDE_AND_SEEK_NEXT_ROW",
+        2, scanner.count.get());
+    } finally {
+      scanner.close();
+    }
+  }
+
+  @Test
   public void testScanTimeRange() throws IOException {
     String r1 = "R1";
     // returns only 1 of these 2 even though same timestamp


[19/50] [abbrv] hbase git commit: HBASE-15495 Connection leak in FanOutOneBlockAsyncDFSOutputHelper

Posted by sy...@apache.org.
HBASE-15495 Connection leak in FanOutOneBlockAsyncDFSOutputHelper


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

Branch: refs/heads/hbase-12439
Commit: 5fcadb86ab9981b69fba92e590cffc028e2b78b3
Parents: 12f66e3
Author: zhangduo <zh...@apache.org>
Authored: Tue Mar 22 10:33:03 2016 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Wed Mar 23 09:56:40 2016 +0800

----------------------------------------------------------------------
 .../FanOutOneBlockAsyncDFSOutputHelper.java     |  23 +++-
 .../util/TestFanOutOneBlockAsyncDFSOutput.java  | 137 +++++++++++++------
 2 files changed, 114 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/5fcadb86/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FanOutOneBlockAsyncDFSOutputHelper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FanOutOneBlockAsyncDFSOutputHelper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FanOutOneBlockAsyncDFSOutputHelper.java
index d34bbb0..ea71701 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FanOutOneBlockAsyncDFSOutputHelper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FanOutOneBlockAsyncDFSOutputHelper.java
@@ -99,6 +99,7 @@ import io.netty.handler.timeout.IdleState;
 import io.netty.handler.timeout.IdleStateEvent;
 import io.netty.handler.timeout.IdleStateHandler;
 import io.netty.util.concurrent.Future;
+import io.netty.util.concurrent.FutureListener;
 import io.netty.util.concurrent.Promise;
 
 /**
@@ -594,13 +595,15 @@ public class FanOutOneBlockAsyncDFSOutputHelper {
     beginFileLease(client, src, stat.getFileId());
     boolean succ = false;
     LocatedBlock locatedBlock = null;
-    List<Channel> datanodeList = new ArrayList<>();
+    List<Future<Channel>> futureList = null;
     try {
       DataChecksum summer = createChecksum(client);
       locatedBlock = namenode.addBlock(src, client.getClientName(), null, null, stat.getFileId(),
         null);
-      for (Future<Channel> future : connectToDataNodes(conf, clientName, locatedBlock, 0L, 0L,
-        PIPELINE_SETUP_CREATE, summer, eventLoop)) {
+      List<Channel> datanodeList = new ArrayList<>();
+      futureList = connectToDataNodes(conf, clientName, locatedBlock, 0L, 0L, PIPELINE_SETUP_CREATE,
+        summer, eventLoop);
+      for (Future<Channel> future : futureList) {
         // fail the creation if there are connection failures since we are fail-fast. The upper
         // layer should retry itself if needed.
         datanodeList.add(future.syncUninterruptibly().getNow());
@@ -610,8 +613,18 @@ public class FanOutOneBlockAsyncDFSOutputHelper {
           stat.getFileId(), locatedBlock, eventLoop, datanodeList, summer, ALLOC);
     } finally {
       if (!succ) {
-        for (Channel c : datanodeList) {
-          c.close();
+        if (futureList != null) {
+          for (Future<Channel> f : futureList) {
+            f.addListener(new FutureListener<Channel>() {
+
+              @Override
+              public void operationComplete(Future<Channel> future) throws Exception {
+                if (future.isSuccess()) {
+                  future.getNow().close();
+                }
+              }
+            });
+          }
         }
         endFileLease(client, src, stat.getFileId());
         fsUtils.recoverFileLease(dfs, new Path(src), conf, new CancelOnClose(client));

http://git-wip-us.apache.org/repos/asf/hbase/blob/5fcadb86/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFanOutOneBlockAsyncDFSOutput.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFanOutOneBlockAsyncDFSOutput.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFanOutOneBlockAsyncDFSOutput.java
index 0e9f42e..09cd61e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFanOutOneBlockAsyncDFSOutput.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFanOutOneBlockAsyncDFSOutput.java
@@ -22,24 +22,26 @@ import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
-import io.netty.channel.EventLoop;
-import io.netty.channel.EventLoopGroup;
-import io.netty.channel.nio.NioEventLoopGroup;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.lang.reflect.Field;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ThreadLocalRandom;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.ipc.RemoteException;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
+import org.apache.hadoop.util.Daemon;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Rule;
@@ -47,9 +49,15 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 
+import io.netty.channel.EventLoop;
+import io.netty.channel.EventLoopGroup;
+import io.netty.channel.nio.NioEventLoopGroup;
+
 @Category({ MiscTests.class, MediumTests.class })
 public class TestFanOutOneBlockAsyncDFSOutput {
 
+  private static final Log LOG = LogFactory.getLog(TestFanOutOneBlockAsyncDFSOutput.class);
+
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 
   private static DistributedFileSystem FS;
@@ -63,8 +71,6 @@ public class TestFanOutOneBlockAsyncDFSOutput {
 
   @BeforeClass
   public static void setUp() throws Exception {
-    Logger.getLogger("org.apache.hadoop.hdfs.StateChange").setLevel(Level.DEBUG);
-    Logger.getLogger("BlockStateChange").setLevel(Level.DEBUG);
     TEST_UTIL.getConfiguration().setInt(DFS_CLIENT_SOCKET_TIMEOUT_KEY, READ_TIMEOUT_MS);
     TEST_UTIL.startMiniDFSCluster(3);
     FS = TEST_UTIL.getDFSCluster().getFileSystem();
@@ -79,12 +85,28 @@ public class TestFanOutOneBlockAsyncDFSOutput {
     TEST_UTIL.shutdownMiniDFSCluster();
   }
 
+  private void ensureAllDatanodeAlive() throws InterruptedException {
+    // FanOutOneBlockAsyncDFSOutputHelper.createOutput is fail-fast, so we need to make sure that we
+    // can create a FanOutOneBlockAsyncDFSOutput after a datanode restarting, otherwise some tests
+    // will fail.
+    for (;;) {
+      try {
+        FanOutOneBlockAsyncDFSOutput out = FanOutOneBlockAsyncDFSOutputHelper.createOutput(FS,
+          new Path("/ensureDatanodeAlive"), true, true, (short) 3, FS.getDefaultBlockSize(),
+          EVENT_LOOP_GROUP.next());
+        out.close();
+        break;
+      } catch (IOException e) {
+        Thread.sleep(100);
+      }
+    }
+  }
+
   private void writeAndVerify(EventLoop eventLoop, Path f, final FanOutOneBlockAsyncDFSOutput out)
       throws IOException, InterruptedException, ExecutionException {
     final byte[] b = new byte[10];
     ThreadLocalRandom.current().nextBytes(b);
-    final FanOutOneBlockAsyncDFSOutputFlushHandler handler =
-        new FanOutOneBlockAsyncDFSOutputFlushHandler();
+    final FanOutOneBlockAsyncDFSOutputFlushHandler handler = new FanOutOneBlockAsyncDFSOutputFlushHandler();
     eventLoop.execute(new Runnable() {
 
       @Override
@@ -107,9 +129,8 @@ public class TestFanOutOneBlockAsyncDFSOutput {
   public void test() throws IOException, InterruptedException, ExecutionException {
     Path f = new Path("/" + name.getMethodName());
     EventLoop eventLoop = EVENT_LOOP_GROUP.next();
-    final FanOutOneBlockAsyncDFSOutput out =
-        FanOutOneBlockAsyncDFSOutputHelper.createOutput(FS, f, true, false, (short) 3,
-          FS.getDefaultBlockSize(), eventLoop);
+    final FanOutOneBlockAsyncDFSOutput out = FanOutOneBlockAsyncDFSOutputHelper.createOutput(FS, f,
+      true, false, (short) 3, FS.getDefaultBlockSize(), eventLoop);
     writeAndVerify(eventLoop, f, out);
   }
 
@@ -117,13 +138,11 @@ public class TestFanOutOneBlockAsyncDFSOutput {
   public void testRecover() throws IOException, InterruptedException, ExecutionException {
     Path f = new Path("/" + name.getMethodName());
     EventLoop eventLoop = EVENT_LOOP_GROUP.next();
-    final FanOutOneBlockAsyncDFSOutput out =
-        FanOutOneBlockAsyncDFSOutputHelper.createOutput(FS, f, true, false, (short) 3,
-          FS.getDefaultBlockSize(), eventLoop);
+    final FanOutOneBlockAsyncDFSOutput out = FanOutOneBlockAsyncDFSOutputHelper.createOutput(FS, f,
+      true, false, (short) 3, FS.getDefaultBlockSize(), eventLoop);
     final byte[] b = new byte[10];
     ThreadLocalRandom.current().nextBytes(b);
-    final FanOutOneBlockAsyncDFSOutputFlushHandler handler =
-        new FanOutOneBlockAsyncDFSOutputFlushHandler();
+    final FanOutOneBlockAsyncDFSOutputFlushHandler handler = new FanOutOneBlockAsyncDFSOutputFlushHandler();
     eventLoop.execute(new Runnable() {
 
       @Override
@@ -135,38 +154,41 @@ public class TestFanOutOneBlockAsyncDFSOutput {
     handler.get();
     // restart one datanode which causes one connection broken
     TEST_UTIL.getDFSCluster().restartDataNode(0);
-    handler.reset();
-    eventLoop.execute(new Runnable() {
-
-      @Override
-      public void run() {
-        out.write(b, 0, b.length);
-        out.flush(null, handler, false);
-      }
-    });
     try {
-      handler.get();
-      fail("flush should fail");
-    } catch (ExecutionException e) {
-      // we restarted one datanode so the flush should fail
-      e.printStackTrace();
-    }
-    out.recoverAndClose(null);
-    assertEquals(b.length, FS.getFileStatus(f).getLen());
-    byte[] actual = new byte[b.length];
-    try (FSDataInputStream in = FS.open(f)) {
-      in.readFully(actual);
+      handler.reset();
+      eventLoop.execute(new Runnable() {
+
+        @Override
+        public void run() {
+          out.write(b, 0, b.length);
+          out.flush(null, handler, false);
+        }
+      });
+      try {
+        handler.get();
+        fail("flush should fail");
+      } catch (ExecutionException e) {
+        // we restarted one datanode so the flush should fail
+        LOG.info("expected exception caught", e);
+      }
+      out.recoverAndClose(null);
+      assertEquals(b.length, FS.getFileStatus(f).getLen());
+      byte[] actual = new byte[b.length];
+      try (FSDataInputStream in = FS.open(f)) {
+        in.readFully(actual);
+      }
+      assertArrayEquals(b, actual);
+    } finally {
+      ensureAllDatanodeAlive();
     }
-    assertArrayEquals(b, actual);
   }
 
   @Test
   public void testHeartbeat() throws IOException, InterruptedException, ExecutionException {
     Path f = new Path("/" + name.getMethodName());
     EventLoop eventLoop = EVENT_LOOP_GROUP.next();
-    final FanOutOneBlockAsyncDFSOutput out =
-        FanOutOneBlockAsyncDFSOutputHelper.createOutput(FS, f, true, false, (short) 3,
-          FS.getDefaultBlockSize(), eventLoop);
+    final FanOutOneBlockAsyncDFSOutput out = FanOutOneBlockAsyncDFSOutputHelper.createOutput(FS, f,
+      true, false, (short) 3, FS.getDefaultBlockSize(), eventLoop);
     Thread.sleep(READ_TIMEOUT_MS * 2);
     // the connection to datanode should still alive.
     writeAndVerify(eventLoop, f, out);
@@ -184,7 +206,40 @@ public class TestFanOutOneBlockAsyncDFSOutput {
         FS.getDefaultBlockSize(), eventLoop);
       fail("should fail with parent does not exist");
     } catch (RemoteException e) {
+      LOG.info("expected exception caught", e);
       assertTrue(e.unwrapRemoteException() instanceof FileNotFoundException);
     }
   }
+
+  @Test
+  public void testConnectToDatanodeFailed()
+      throws IOException, ClassNotFoundException, NoSuchMethodException, IllegalAccessException,
+      InvocationTargetException, InterruptedException, NoSuchFieldException {
+    Field xceiverServerDaemonField = DataNode.class.getDeclaredField("dataXceiverServer");
+    xceiverServerDaemonField.setAccessible(true);
+    Class<?> xceiverServerClass = Class
+        .forName("org.apache.hadoop.hdfs.server.datanode.DataXceiverServer");
+    Method numPeersMethod = xceiverServerClass.getDeclaredMethod("getNumPeers");
+    numPeersMethod.setAccessible(true);
+    // make one datanode broken
+    TEST_UTIL.getDFSCluster().getDataNodes().get(0).shutdownDatanode(true);
+    try {
+      Path f = new Path("/test");
+      EventLoop eventLoop = EVENT_LOOP_GROUP.next();
+      try {
+        FanOutOneBlockAsyncDFSOutputHelper.createOutput(FS, f, true, false, (short) 3,
+          FS.getDefaultBlockSize(), eventLoop);
+        fail("should fail with connection error");
+      } catch (IOException e) {
+        LOG.info("expected exception caught", e);
+      }
+      for (DataNode dn : TEST_UTIL.getDFSCluster().getDataNodes()) {
+        Daemon daemon = (Daemon) xceiverServerDaemonField.get(dn);
+        assertEquals(0, numPeersMethod.invoke(daemon.getRunnable()));
+      }
+    } finally {
+      TEST_UTIL.getDFSCluster().restartDataNode(0);
+      ensureAllDatanodeAlive();
+    }
+  }
 }


[06/50] [abbrv] hbase git commit: HBASE-15477 Purge 'next block header' from cached blocks

Posted by sy...@apache.org.
HBASE-15477 Purge 'next block header' from cached blocks

When we read from HDFS, we overread to pick up the next blocks header.
Doing this saves a seek as we move through the hfile; we save having to
do an explicit seek just to read the block header every time we need to
read the body.  We used to read in the next header as part of the
current blocks buffer. This buffer was then what got persisted to
blockcache; so we were over-persisting: our block plus the next blocks'
header (33 bytes).

This patch undoes this over-persisting.

Removes support for version 1 blocks (0.2 was added in hbase-0.92.0).
Not needed any more.

There is an open question on whether checksums should be persisted
when caching. The code seems to say no but if cache is SSD backed or
backed by anything that does not do error correction, we'll want
checksums.

Adds loads of documentation.

M hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java
  (write) Add writing from a ByteBuff.

M hbase-common/src/main/java/org/apache/hadoop/hbase/nio/ByteBuff.java
  (toString) Add one so ByteBuff looks like ByteBuffer when you click on
  it in IDE

M hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
  Remove support for version 1 blocks.

  Cleaned up handling of metadata added when we serialize a block to
  caches. Metadata is smaller now.

  When we serialize (used when caching), do not persist the next blocks
  header if present.

  Removed a bunch of methods, a few of which had overlapping
  functionality and others that exposed too much of our internals.
  Also removed a bunch of constructors and unified the constructors we
  had left over making them share a common init method.
  Shutdown access to defines that should only be used internally here.

  Renamed all to do w/ 'EXTRA' and 'extraSerialization' to instead talk
  about metadata saved to caches; was unclear previously what EXTRA was
  about.

  Renamed static final declarations as all uppercase.

  (readBlockDataInternal): Redid. Couldn't make sense of it previously.
  Undid heavy-duty parse of header by constructing HFileBlock. Other
  cleanups. Its 1/3rd the length it used to be. More to do in here.


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

Branch: refs/heads/hbase-12439
Commit: ef94b552399d75c0c16aca0109a39b66f15af892
Parents: 891569b
Author: stack <st...@apache.org>
Authored: Thu Mar 17 11:18:06 2016 -0700
Committer: stack <st...@apache.org>
Committed: Tue Mar 22 13:34:51 2016 -0700

----------------------------------------------------------------------
 .../hbase/regionserver/KeyValueScanner.java.rej    | 17 +++++++++++++++++
 1 file changed, 17 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ef94b552/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java.rej
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java.rej b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java.rej
new file mode 100644
index 0000000..4e3ebd4
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java.rej
@@ -0,0 +1,17 @@
+diff a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java	(rejected hunks)
+@@ -29,7 +30,7 @@ import org.apache.hadoop.hbase.client.Scan;
+  * Scanner that returns the next KeyValue.
+  */
+ @InterfaceAudience.Private
+-public interface KeyValueScanner extends Shipper {
++public interface KeyValueScanner extends Shipper, Closeable {
+   /**
+    * The byte array represents for NO_NEXT_INDEXED_KEY;
+    * The actual value is irrelevant because this is always compared by reference.
+@@ -165,4 +167,4 @@ public interface KeyValueScanner extends Shipper {
+    * if known, or null otherwise
+    */
+   public Cell getNextIndexedKey();
+-}
++}
+\ No newline at end of file


[33/50] [abbrv] hbase git commit: HBASE-15389 Write out multiple files when compaction

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/11d11d3f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactor.java
new file mode 100644
index 0000000..6ec4cd4
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactor.java
@@ -0,0 +1,212 @@
+/**
+ * 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.compactions;
+
+import static org.apache.hadoop.hbase.regionserver.StripeStoreFileManager.STRIPE_END_KEY;
+import static org.apache.hadoop.hbase.regionserver.StripeStoreFileManager.STRIPE_START_KEY;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyBoolean;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.TreeMap;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.regionserver.BloomType;
+import org.apache.hadoop.hbase.regionserver.InternalScanner;
+import org.apache.hadoop.hbase.regionserver.ScannerContext;
+import org.apache.hadoop.hbase.regionserver.StoreFile;
+import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
+import org.apache.hadoop.hbase.regionserver.StripeMultiFileWriter;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+public class TestCompactor {
+
+  public static StoreFile createDummyStoreFile(long maxSequenceId) throws Exception {
+    // "Files" are totally unused, it's Scanner class below that gives compactor fake KVs.
+    // But compaction depends on everything under the sun, so stub everything with dummies.
+    StoreFile sf = mock(StoreFile.class);
+    StoreFile.Reader r = mock(StoreFile.Reader.class);
+    when(r.length()).thenReturn(1L);
+    when(r.getBloomFilterType()).thenReturn(BloomType.NONE);
+    when(r.getHFileReader()).thenReturn(mock(HFile.Reader.class));
+    when(r.getStoreFileScanner(anyBoolean(), anyBoolean(), anyBoolean(), anyLong()))
+        .thenReturn(mock(StoreFileScanner.class));
+    when(sf.getReader()).thenReturn(r);
+    when(sf.createReader()).thenReturn(r);
+    when(sf.createReader(anyBoolean())).thenReturn(r);
+    when(sf.cloneForReader()).thenReturn(sf);
+    when(sf.getMaxSequenceId()).thenReturn(maxSequenceId);
+    return sf;
+  }
+
+  public static CompactionRequest createDummyRequest() throws Exception {
+    return new CompactionRequest(Arrays.asList(createDummyStoreFile(1L)));
+  }
+
+  // StoreFile.Writer has private ctor and is unwieldy, so this has to be convoluted.
+  public static class StoreFileWritersCapture
+      implements Answer<StoreFile.Writer>, StripeMultiFileWriter.WriterFactory {
+    public static class Writer {
+      public ArrayList<KeyValue> kvs = new ArrayList<KeyValue>();
+      public TreeMap<byte[], byte[]> data = new TreeMap<byte[], byte[]>(Bytes.BYTES_COMPARATOR);
+      public boolean hasMetadata;
+    }
+
+    private List<Writer> writers = new ArrayList<Writer>();
+
+    @Override
+    public StoreFile.Writer createWriter() throws IOException {
+      final Writer realWriter = new Writer();
+      writers.add(realWriter);
+      StoreFile.Writer writer = mock(StoreFile.Writer.class);
+      doAnswer(new Answer<Object>() {
+        public Object answer(InvocationOnMock invocation) {
+          return realWriter.kvs.add((KeyValue) invocation.getArguments()[0]);
+        }
+      }).when(writer).append(any(KeyValue.class));
+      doAnswer(new Answer<Object>() {
+        public Object answer(InvocationOnMock invocation) {
+          Object[] args = invocation.getArguments();
+          return realWriter.data.put((byte[]) args[0], (byte[]) args[1]);
+        }
+      }).when(writer).appendFileInfo(any(byte[].class), any(byte[].class));
+      doAnswer(new Answer<Void>() {
+        @Override
+        public Void answer(InvocationOnMock invocation) throws Throwable {
+          realWriter.hasMetadata = true;
+          return null;
+        }
+      }).when(writer).appendMetadata(any(long.class), any(boolean.class));
+      doAnswer(new Answer<Path>() {
+        @Override
+        public Path answer(InvocationOnMock invocation) throws Throwable {
+          return new Path("foo");
+        }
+      }).when(writer).getPath();
+      return writer;
+    }
+
+    @Override
+    public StoreFile.Writer answer(InvocationOnMock invocation) throws Throwable {
+      return createWriter();
+    }
+
+    public void verifyKvs(KeyValue[][] kvss, boolean allFiles, boolean requireMetadata) {
+      if (allFiles) {
+        assertEquals(kvss.length, writers.size());
+      }
+      int skippedWriters = 0;
+      for (int i = 0; i < kvss.length; ++i) {
+        KeyValue[] kvs = kvss[i];
+        if (kvs != null) {
+          Writer w = writers.get(i - skippedWriters);
+          if (requireMetadata) {
+            assertNotNull(w.data.get(STRIPE_START_KEY));
+            assertNotNull(w.data.get(STRIPE_END_KEY));
+          } else {
+            assertNull(w.data.get(STRIPE_START_KEY));
+            assertNull(w.data.get(STRIPE_END_KEY));
+          }
+          assertEquals(kvs.length, w.kvs.size());
+          for (int j = 0; j < kvs.length; ++j) {
+            assertEquals(kvs[j], w.kvs.get(j));
+          }
+        } else {
+          assertFalse(allFiles);
+          ++skippedWriters;
+        }
+      }
+    }
+
+    public void verifyBoundaries(byte[][] boundaries) {
+      assertEquals(boundaries.length - 1, writers.size());
+      for (int i = 0; i < writers.size(); ++i) {
+        assertArrayEquals("i = " + i, boundaries[i], writers.get(i).data.get(STRIPE_START_KEY));
+        assertArrayEquals("i = " + i, boundaries[i + 1], writers.get(i).data.get(STRIPE_END_KEY));
+      }
+    }
+
+    public void verifyKvs(KeyValue[][] kvss, boolean allFiles, List<Long> boundaries) {
+      if (allFiles) {
+        assertEquals(kvss.length, writers.size());
+      }
+      int skippedWriters = 0;
+      for (int i = 0; i < kvss.length; ++i) {
+        KeyValue[] kvs = kvss[i];
+        if (kvs != null) {
+          Writer w = writers.get(i - skippedWriters);
+          assertEquals(kvs.length, w.kvs.size());
+          for (int j = 0; j < kvs.length; ++j) {
+            assertTrue(kvs[j].getTimestamp() >= boundaries.get(i));
+            assertTrue(kvs[j].getTimestamp() < boundaries.get(i + 1));
+            assertEquals(kvs[j], w.kvs.get(j));
+          }
+        } else {
+          assertFalse(allFiles);
+          ++skippedWriters;
+        }
+      }
+    }
+
+    public List<Writer> getWriters() {
+      return writers;
+    }
+  }
+
+  public static class Scanner implements InternalScanner {
+    private final ArrayList<KeyValue> kvs;
+
+    public Scanner(KeyValue... kvs) {
+      this.kvs = new ArrayList<KeyValue>(Arrays.asList(kvs));
+    }
+
+    @Override
+    public boolean next(List<Cell> results) throws IOException {
+      if (kvs.isEmpty()) return false;
+      results.add(kvs.remove(0));
+      return !kvs.isEmpty();
+    }
+
+    @Override
+    public boolean next(List<Cell> result, ScannerContext scannerContext) throws IOException {
+      return next(result);
+    }
+
+    @Override
+    public void close() throws IOException {
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/11d11d3f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestDateTieredCompactor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestDateTieredCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestDateTieredCompactor.java
new file mode 100644
index 0000000..38d9f99
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestDateTieredCompactor.java
@@ -0,0 +1,169 @@
+/**
+ * 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.compactions;
+
+import static org.apache.hadoop.hbase.regionserver.compactions.TestCompactor.createDummyRequest;
+import static org.apache.hadoop.hbase.regionserver.compactions.TestCompactor.createDummyStoreFile;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyBoolean;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.io.compress.Compression;
+import org.apache.hadoop.hbase.regionserver.InternalScanner;
+import org.apache.hadoop.hbase.regionserver.ScanInfo;
+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.StoreFileScanner;
+import org.apache.hadoop.hbase.regionserver.compactions.TestCompactor.Scanner;
+import org.apache.hadoop.hbase.regionserver.compactions.TestCompactor.StoreFileWritersCapture;
+import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameter;
+import org.junit.runners.Parameterized.Parameters;
+
+@RunWith(Parameterized.class)
+@Category({ RegionServerTests.class, SmallTests.class })
+public class TestDateTieredCompactor {
+
+  private static final byte[] NAME_OF_THINGS = Bytes.toBytes("foo");
+
+  private static final TableName TABLE_NAME = TableName.valueOf(NAME_OF_THINGS, NAME_OF_THINGS);
+
+  private static final KeyValue KV_A = new KeyValue(Bytes.toBytes("aaa"), 100L);
+
+  private static final KeyValue KV_B = new KeyValue(Bytes.toBytes("bbb"), 200L);
+
+  private static final KeyValue KV_C = new KeyValue(Bytes.toBytes("ccc"), 300L);
+
+  private static final KeyValue KV_D = new KeyValue(Bytes.toBytes("ddd"), 400L);
+
+  @Parameters(name = "{index}: usePrivateReaders={0}")
+  public static Iterable<Object[]> data() {
+    return Arrays.asList(new Object[] { true }, new Object[] { false });
+  }
+
+  @Parameter
+  public boolean usePrivateReaders;
+
+  private DateTieredCompactor createCompactor(StoreFileWritersCapture writers,
+      final KeyValue[] input, List<StoreFile> storefiles) throws Exception {
+    Configuration conf = HBaseConfiguration.create();
+    conf.setBoolean("hbase.regionserver.compaction.private.readers", usePrivateReaders);
+    final Scanner scanner = new Scanner(input);
+    // Create store mock that is satisfactory for compactor.
+    HColumnDescriptor col = new HColumnDescriptor(NAME_OF_THINGS);
+    ScanInfo si = new ScanInfo(conf, col, Long.MAX_VALUE, 0, CellComparator.COMPARATOR);
+    final Store store = mock(Store.class);
+    when(store.getStorefiles()).thenReturn(storefiles);
+    when(store.getFamily()).thenReturn(col);
+    when(store.getScanInfo()).thenReturn(si);
+    when(store.areWritesEnabled()).thenReturn(true);
+    when(store.getFileSystem()).thenReturn(mock(FileSystem.class));
+    when(store.getRegionInfo()).thenReturn(new HRegionInfo(TABLE_NAME));
+    when(store.createWriterInTmp(anyLong(), any(Compression.Algorithm.class), anyBoolean(),
+      anyBoolean(), anyBoolean(), anyBoolean())).thenAnswer(writers);
+    when(store.getComparator()).thenReturn(CellComparator.COMPARATOR);
+    long maxSequenceId = StoreFile.getMaxSequenceIdInList(storefiles);
+    when(store.getMaxSequenceId()).thenReturn(maxSequenceId);
+
+    return new DateTieredCompactor(conf, store) {
+      @Override
+      protected InternalScanner createScanner(Store store, List<StoreFileScanner> scanners,
+          long smallestReadPoint, long earliestPutTs, byte[] dropDeletesFromRow,
+          byte[] dropDeletesToRow) throws IOException {
+        return scanner;
+      }
+
+      @Override
+      protected InternalScanner createScanner(Store store, List<StoreFileScanner> scanners,
+          ScanType scanType, long smallestReadPoint, long earliestPutTs) throws IOException {
+        return scanner;
+      }
+    };
+  }
+
+  private void verify(KeyValue[] input, List<Long> boundaries, KeyValue[][] output,
+      boolean allFiles) throws Exception {
+    StoreFileWritersCapture writers = new StoreFileWritersCapture();
+    StoreFile sf1 = createDummyStoreFile(1L);
+    StoreFile sf2 = createDummyStoreFile(2L);
+    DateTieredCompactor dtc = createCompactor(writers, input, Arrays.asList(sf1, sf2));
+    List<Path> paths = dtc.compact(new CompactionRequest(Arrays.asList(sf1)),
+      boundaries.subList(0, boundaries.size() - 1), NoLimitThroughputController.INSTANCE, null);
+    writers.verifyKvs(output, allFiles, boundaries);
+    if (allFiles) {
+      assertEquals(output.length, paths.size());
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  private static <T> T[] a(T... a) {
+    return a;
+  }
+
+  @Test
+  public void test() throws Exception {
+    verify(a(KV_A, KV_B, KV_C, KV_D), Arrays.asList(100L, 200L, 300L, 400L, 500L),
+      a(a(KV_A), a(KV_B), a(KV_C), a(KV_D)), true);
+    verify(a(KV_A, KV_B, KV_C, KV_D), Arrays.asList(Long.MIN_VALUE, 200L, Long.MAX_VALUE),
+      a(a(KV_A), a(KV_B, KV_C, KV_D)), false);
+    verify(a(KV_A, KV_B, KV_C, KV_D), Arrays.asList(Long.MIN_VALUE, Long.MAX_VALUE),
+      new KeyValue[][] { a(KV_A, KV_B, KV_C, KV_D) }, false);
+  }
+
+  @Test
+  public void testEmptyOutputFile() throws Exception {
+    StoreFileWritersCapture writers = new StoreFileWritersCapture();
+    CompactionRequest request = createDummyRequest();
+    DateTieredCompactor dtc = createCompactor(writers, new KeyValue[0],
+      new ArrayList<StoreFile>(request.getFiles()));
+    List<Path> paths = dtc.compact(request, Arrays.asList(Long.MIN_VALUE, Long.MAX_VALUE),
+      NoLimitThroughputController.INSTANCE, null);
+    assertEquals(1, paths.size());
+    List<StoreFileWritersCapture.Writer> dummyWriters = writers.getWriters();
+    assertEquals(1, dummyWriters.size());
+    StoreFileWritersCapture.Writer dummyWriter = dummyWriters.get(0);
+    assertTrue(dummyWriter.kvs.isEmpty());
+    assertTrue(dummyWriter.hasMetadata);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/11d11d3f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactionPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactionPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactionPolicy.java
index c440a57..146882b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactionPolicy.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactionPolicy.java
@@ -65,8 +65,8 @@ import org.apache.hadoop.hbase.regionserver.StripeMultiFileWriter;
 import org.apache.hadoop.hbase.regionserver.StripeStoreConfig;
 import org.apache.hadoop.hbase.regionserver.StripeStoreFileManager;
 import org.apache.hadoop.hbase.regionserver.StripeStoreFlusher;
-import org.apache.hadoop.hbase.regionserver.TestStripeCompactor.StoreFileWritersCapture;
 import org.apache.hadoop.hbase.regionserver.compactions.StripeCompactionPolicy.StripeInformationProvider;
+import org.apache.hadoop.hbase.regionserver.compactions.TestCompactor.StoreFileWritersCapture;
 import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
@@ -77,11 +77,16 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.ManualEnvironmentEdge;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameter;
+import org.junit.runners.Parameterized.Parameters;
 import org.mockito.ArgumentMatcher;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
+@RunWith(Parameterized.class)
 @Category({RegionServerTests.class, SmallTests.class})
 public class TestStripeCompactionPolicy {
   private static final byte[] KEY_A = Bytes.toBytes("aaa");
@@ -101,6 +106,13 @@ public class TestStripeCompactionPolicy {
   private final static int defaultInitialCount = 1;
   private static long defaultTtl = 1000 * 1000;
 
+  @Parameters(name = "{index}: usePrivateReaders={0}")
+  public static Iterable<Object[]> data() {
+    return Arrays.asList(new Object[] { true }, new Object[] { false });
+  }
+
+  @Parameter
+  public boolean usePrivateReaders;
   @Test
   public void testNoStripesFromFlush() throws Exception {
     Configuration conf = HBaseConfiguration.create();
@@ -398,6 +410,7 @@ public class TestStripeCompactionPolicy {
     }
   }
 
+  @SuppressWarnings("unchecked")
   private static StripeCompactionPolicy.StripeInformationProvider createStripesWithFiles(
       List<StoreFile>... stripeFiles) throws Exception {
     return createStripesWithFiles(createBoundaries(stripeFiles.length),
@@ -576,9 +589,10 @@ public class TestStripeCompactionPolicy {
   protected void verifyFlush(StripeCompactionPolicy policy, StripeInformationProvider si,
       KeyValue[] input, KeyValue[][] expected, byte[][] boundaries) throws IOException {
     StoreFileWritersCapture writers = new StoreFileWritersCapture();
-    StripeStoreFlusher.StripeFlushRequest req = policy.selectFlush(si, input.length);
+    StripeStoreFlusher.StripeFlushRequest req = policy.selectFlush(CellComparator.COMPARATOR, si,
+      input.length);
     StripeMultiFileWriter mw = req.createWriter();
-    mw.init(null, writers, CellComparator.COMPARATOR);
+    mw.init(null, writers);
     for (KeyValue kv : input) {
       mw.append(kv);
     }
@@ -740,6 +754,7 @@ public class TestStripeCompactionPolicy {
     when(sf.getReader()).thenReturn(r);
     when(sf.createReader(anyBoolean())).thenReturn(r);
     when(sf.createReader()).thenReturn(r);
+    when(sf.cloneForReader()).thenReturn(sf);
     return sf;
   }
 
@@ -752,7 +767,7 @@ public class TestStripeCompactionPolicy {
     when(sf.getMetadataValue(StripeStoreFileManager.STRIPE_END_KEY)).thenReturn(endKey);
   }
 
-  private static StripeCompactor createCompactor() throws Exception {
+  private StripeCompactor createCompactor() throws Exception {
     HColumnDescriptor col = new HColumnDescriptor(Bytes.toBytes("foo"));
     StoreFileWritersCapture writers = new StoreFileWritersCapture();
     Store store = mock(Store.class);
@@ -765,6 +780,7 @@ public class TestStripeCompactionPolicy {
         anyBoolean(), anyBoolean(), anyBoolean())).thenAnswer(writers);
 
     Configuration conf = HBaseConfiguration.create();
+    conf.setBoolean("hbase.regionserver.compaction.private.readers", usePrivateReaders);
     final Scanner scanner = new Scanner();
     return new StripeCompactor(conf, store) {
       @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/11d11d3f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactor.java
new file mode 100644
index 0000000..97331e8
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactor.java
@@ -0,0 +1,223 @@
+/**
+ * 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.compactions;
+
+import static org.apache.hadoop.hbase.regionserver.StripeStoreFileManager.OPEN_KEY;
+import static org.apache.hadoop.hbase.regionserver.compactions.TestCompactor.createDummyRequest;
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyBoolean;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.io.compress.Compression;
+import org.apache.hadoop.hbase.regionserver.InternalScanner;
+import org.apache.hadoop.hbase.regionserver.ScanInfo;
+import org.apache.hadoop.hbase.regionserver.ScanType;
+import org.apache.hadoop.hbase.regionserver.Store;
+import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
+import org.apache.hadoop.hbase.regionserver.compactions.TestCompactor.Scanner;
+import org.apache.hadoop.hbase.regionserver.compactions.TestCompactor.StoreFileWritersCapture;
+import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameter;
+import org.junit.runners.Parameterized.Parameters;
+
+@RunWith(Parameterized.class)
+@Category({ RegionServerTests.class, SmallTests.class })
+public class TestStripeCompactor {
+  private static final byte[] NAME_OF_THINGS = Bytes.toBytes("foo");
+  private static final TableName TABLE_NAME = TableName.valueOf(NAME_OF_THINGS, NAME_OF_THINGS);
+
+  private static final byte[] KEY_B = Bytes.toBytes("bbb");
+  private static final byte[] KEY_C = Bytes.toBytes("ccc");
+  private static final byte[] KEY_D = Bytes.toBytes("ddd");
+
+  private static final KeyValue KV_A = kvAfter(Bytes.toBytes("aaa"));
+  private static final KeyValue KV_B = kvAfter(KEY_B);
+  private static final KeyValue KV_C = kvAfter(KEY_C);
+  private static final KeyValue KV_D = kvAfter(KEY_D);
+
+  @Parameters(name = "{index}: usePrivateReaders={0}")
+  public static Iterable<Object[]> data() {
+    return Arrays.asList(new Object[] { true }, new Object[] { false });
+  }
+
+  @Parameter
+  public boolean usePrivateReaders;
+
+  private static KeyValue kvAfter(byte[] key) {
+    return new KeyValue(Arrays.copyOf(key, key.length + 1), 0L);
+  }
+
+  @SuppressWarnings("unchecked")
+  private static <T> T[] a(T... a) {
+    return a;
+  }
+
+  private static KeyValue[] e() {
+    return TestStripeCompactor.<KeyValue> a();
+  }
+
+  @Test
+  public void testBoundaryCompactions() throws Exception {
+    // General verification
+    verifyBoundaryCompaction(a(KV_A, KV_A, KV_B, KV_B, KV_C, KV_D),
+      a(OPEN_KEY, KEY_B, KEY_D, OPEN_KEY), a(a(KV_A, KV_A), a(KV_B, KV_B, KV_C), a(KV_D)));
+    verifyBoundaryCompaction(a(KV_B, KV_C), a(KEY_B, KEY_C, KEY_D), a(a(KV_B), a(KV_C)));
+    verifyBoundaryCompaction(a(KV_B, KV_C), a(KEY_B, KEY_D), new KeyValue[][] { a(KV_B, KV_C) });
+  }
+
+  @Test
+  public void testBoundaryCompactionEmptyFiles() throws Exception {
+    // No empty file if there're already files.
+    verifyBoundaryCompaction(a(KV_B), a(KEY_B, KEY_C, KEY_D, OPEN_KEY), a(a(KV_B), null, null),
+      null, null, false);
+    verifyBoundaryCompaction(a(KV_A, KV_C), a(OPEN_KEY, KEY_B, KEY_C, KEY_D),
+      a(a(KV_A), null, a(KV_C)), null, null, false);
+    // But should be created if there are no file.
+    verifyBoundaryCompaction(e(), a(OPEN_KEY, KEY_B, KEY_C, OPEN_KEY), a(null, null, e()), null,
+      null, false);
+    // In major range if there's major range.
+    verifyBoundaryCompaction(e(), a(OPEN_KEY, KEY_B, KEY_C, OPEN_KEY), a(null, e(), null), KEY_B,
+      KEY_C, false);
+    verifyBoundaryCompaction(e(), a(OPEN_KEY, KEY_B, KEY_C, OPEN_KEY), a(e(), e(), null), OPEN_KEY,
+      KEY_C, false);
+    // Major range should have files regardless of KVs.
+    verifyBoundaryCompaction(a(KV_A), a(OPEN_KEY, KEY_B, KEY_C, KEY_D, OPEN_KEY),
+      a(a(KV_A), e(), e(), null), KEY_B, KEY_D, false);
+    verifyBoundaryCompaction(a(KV_C), a(OPEN_KEY, KEY_B, KEY_C, KEY_D, OPEN_KEY),
+      a(null, null, a(KV_C), e()), KEY_C, OPEN_KEY, false);
+
+  }
+
+  private void verifyBoundaryCompaction(KeyValue[] input, byte[][] boundaries, KeyValue[][] output)
+      throws Exception {
+    verifyBoundaryCompaction(input, boundaries, output, null, null, true);
+  }
+
+  private void verifyBoundaryCompaction(KeyValue[] input, byte[][] boundaries, KeyValue[][] output,
+      byte[] majorFrom, byte[] majorTo, boolean allFiles) throws Exception {
+    StoreFileWritersCapture writers = new StoreFileWritersCapture();
+    StripeCompactor sc = createCompactor(writers, input);
+    List<Path> paths = sc.compact(createDummyRequest(), Arrays.asList(boundaries), majorFrom,
+      majorTo, NoLimitThroughputController.INSTANCE, null);
+    writers.verifyKvs(output, allFiles, true);
+    if (allFiles) {
+      assertEquals(output.length, paths.size());
+      writers.verifyBoundaries(boundaries);
+    }
+  }
+
+  @Test
+  public void testSizeCompactions() throws Exception {
+    // General verification with different sizes.
+    verifySizeCompaction(a(KV_A, KV_A, KV_B, KV_C, KV_D), 3, 2, OPEN_KEY, OPEN_KEY,
+      a(a(KV_A, KV_A), a(KV_B, KV_C), a(KV_D)));
+    verifySizeCompaction(a(KV_A, KV_B, KV_C, KV_D), 4, 1, OPEN_KEY, OPEN_KEY,
+      a(a(KV_A), a(KV_B), a(KV_C), a(KV_D)));
+    verifySizeCompaction(a(KV_B, KV_C), 2, 1, KEY_B, KEY_D, a(a(KV_B), a(KV_C)));
+    // Verify row boundaries are preserved.
+    verifySizeCompaction(a(KV_A, KV_A, KV_A, KV_C, KV_D), 3, 2, OPEN_KEY, OPEN_KEY,
+      a(a(KV_A, KV_A, KV_A), a(KV_C, KV_D)));
+    verifySizeCompaction(a(KV_A, KV_B, KV_B, KV_C), 3, 1, OPEN_KEY, OPEN_KEY,
+      a(a(KV_A), a(KV_B, KV_B), a(KV_C)));
+    // Too much data, count limits the number of files.
+    verifySizeCompaction(a(KV_A, KV_B, KV_C, KV_D), 2, 1, OPEN_KEY, OPEN_KEY,
+      a(a(KV_A), a(KV_B, KV_C, KV_D)));
+    verifySizeCompaction(a(KV_A, KV_B, KV_C), 1, Long.MAX_VALUE, OPEN_KEY, KEY_D,
+      new KeyValue[][] { a(KV_A, KV_B, KV_C) });
+    // Too little data/large count, no extra files.
+    verifySizeCompaction(a(KV_A, KV_B, KV_C, KV_D), Integer.MAX_VALUE, 2, OPEN_KEY, OPEN_KEY,
+      a(a(KV_A, KV_B), a(KV_C, KV_D)));
+  }
+
+  private void verifySizeCompaction(KeyValue[] input, int targetCount, long targetSize, byte[] left,
+      byte[] right, KeyValue[][] output) throws Exception {
+    StoreFileWritersCapture writers = new StoreFileWritersCapture();
+    StripeCompactor sc = createCompactor(writers, input);
+    List<Path> paths = sc.compact(createDummyRequest(), targetCount, targetSize, left, right, null,
+      null, NoLimitThroughputController.INSTANCE, null);
+    assertEquals(output.length, paths.size());
+    writers.verifyKvs(output, true, true);
+    List<byte[]> boundaries = new ArrayList<byte[]>();
+    boundaries.add(left);
+    for (int i = 1; i < output.length; ++i) {
+      boundaries.add(CellUtil.cloneRow(output[i][0]));
+    }
+    boundaries.add(right);
+    writers.verifyBoundaries(boundaries.toArray(new byte[][] {}));
+  }
+
+  private StripeCompactor createCompactor(StoreFileWritersCapture writers, KeyValue[] input)
+      throws Exception {
+    Configuration conf = HBaseConfiguration.create();
+    conf.setBoolean("hbase.regionserver.compaction.private.readers", usePrivateReaders);
+    final Scanner scanner = new Scanner(input);
+
+    // Create store mock that is satisfactory for compactor.
+    HColumnDescriptor col = new HColumnDescriptor(NAME_OF_THINGS);
+    ScanInfo si = new ScanInfo(conf, col, Long.MAX_VALUE, 0, CellComparator.COMPARATOR);
+    Store store = mock(Store.class);
+    when(store.getFamily()).thenReturn(col);
+    when(store.getScanInfo()).thenReturn(si);
+    when(store.areWritesEnabled()).thenReturn(true);
+    when(store.getFileSystem()).thenReturn(mock(FileSystem.class));
+    when(store.getRegionInfo()).thenReturn(new HRegionInfo(TABLE_NAME));
+    when(store.createWriterInTmp(anyLong(), any(Compression.Algorithm.class), anyBoolean(),
+      anyBoolean(), anyBoolean(), anyBoolean())).thenAnswer(writers);
+    when(store.getComparator()).thenReturn(CellComparator.COMPARATOR);
+
+    return new StripeCompactor(conf, store) {
+      @Override
+      protected InternalScanner createScanner(Store store, List<StoreFileScanner> scanners,
+          long smallestReadPoint, long earliestPutTs, byte[] dropDeletesFromRow,
+          byte[] dropDeletesToRow) throws IOException {
+        return scanner;
+      }
+
+      @Override
+      protected InternalScanner createScanner(Store store, List<StoreFileScanner> scanners,
+          ScanType scanType, long smallestReadPoint, long earliestPutTs) throws IOException {
+        return scanner;
+      }
+    };
+  }
+}


[04/50] [abbrv] hbase git commit: HBASE-15392 Single Cell Get reads two HFileBlocks M hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java moreRowsMayExistAfterCell Exploit the fact a Scan is a Get Scan. Also save compare

Posted by sy...@apache.org.
HBASE-15392 Single Cell Get reads two HFileBlocks M hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java moreRowsMayExistAfterCell Exploit the fact a Scan is a Get Scan. Also save compares if no non-default stopRow.

M hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
optimize Add doc on what is being optimized. Also, if a Get Scan, do not
optimize else we'll keep going after our row is DONE.
Another place to make use of the Get Scan fact is when we are DONE.. if
Get Scan, we can close out the scan.

M hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java
Add tests for Get Scans and optimize around block loading.


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

Branch: refs/heads/hbase-12439
Commit: 7073f69993e446c2a63308f7827639d675a2da58
Parents: b6e1f63
Author: stack <st...@apache.org>
Authored: Fri Mar 11 15:41:26 2016 -0800
Committer: stack <st...@apache.org>
Committed: Tue Mar 22 13:00:41 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/CellComparator.java |   2 +-
 .../hbase/io/hfile/CombinedBlockCache.java      |  13 +-
 .../hadoop/hbase/io/hfile/HFileWriterImpl.java  |   2 +-
 .../hbase/regionserver/KeyValueScanner.java     |  10 +-
 .../hbase/regionserver/ScanQueryMatcher.java    |  60 +++--
 .../hadoop/hbase/regionserver/StoreScanner.java |  87 ++++++-
 .../hbase/util/CollectionBackedScanner.java     |   3 +-
 .../hbase/regionserver/KeyValueScanFixture.java |   8 +-
 .../regionserver/TestKeyValueScanFixture.java   |   3 +-
 .../hbase/regionserver/TestStoreScanner.java    | 244 ++++++++++++++++++-
 10 files changed, 370 insertions(+), 62 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/7073f699/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 a5e26cf..4a5c0b7 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
@@ -433,7 +433,7 @@ public class CellComparator implements Comparator<Cell>, Serializable {
   /**
    * Used to compare two cells based on the column hint provided. This is specifically
    * used when we need to optimize the seeks based on the next indexed key. This is an
-   * advance usage API specifically needed for some optimizations.
+   * advanced usage API specifically needed for some optimizations.
    * @param nextIndexedCell the next indexed cell 
    * @param currentCell the cell to be compared
    * @param foff the family offset of the currentCell

http://git-wip-us.apache.org/repos/asf/hbase/blob/7073f699/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java
index 22bffee..666b357 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java
@@ -63,8 +63,8 @@ public class CombinedBlockCache implements ResizableBlockCache, HeapSize {
   @Override
   public void cacheBlock(BlockCacheKey cacheKey, Cacheable buf, boolean inMemory,
       final boolean cacheDataInL1) {
-    boolean isMetaBlock = buf.getBlockType().getCategory() != BlockCategory.DATA;
-    if (isMetaBlock || cacheDataInL1) {
+    boolean metaBlock = buf.getBlockType().getCategory() != BlockCategory.DATA;
+    if (metaBlock || cacheDataInL1) {
       lruCache.cacheBlock(cacheKey, buf, inMemory, cacheDataInL1);
     } else {
       l2Cache.cacheBlock(cacheKey, buf, inMemory, false);
@@ -81,12 +81,9 @@ public class CombinedBlockCache implements ResizableBlockCache, HeapSize {
       boolean repeat, boolean updateCacheMetrics) {
     // TODO: is there a hole here, or just awkwardness since in the lruCache getBlock
     // we end up calling l2Cache.getBlock.
-    if (lruCache.containsBlock(cacheKey)) {
-      return lruCache.getBlock(cacheKey, caching, repeat, updateCacheMetrics);
-    }
-    Cacheable result = l2Cache.getBlock(cacheKey, caching, repeat, updateCacheMetrics);
-
-    return result;
+    return lruCache.containsBlock(cacheKey)?
+        lruCache.getBlock(cacheKey, caching, repeat, updateCacheMetrics):
+        l2Cache.getBlock(cacheKey, caching, repeat, updateCacheMetrics);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/7073f699/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
index d310d13..9ab46cf 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
@@ -416,7 +416,7 @@ public class HFileWriterImpl implements HFile.Writer {
     // No opportunity for optimization. Just return right key.
     return right;
   }
-  
+
   /**
    * @param leftArray
    * @param leftOffset

http://git-wip-us.apache.org/repos/asf/hbase/blob/7073f699/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java
index eae713f..e26022e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java
@@ -29,6 +29,9 @@ import org.apache.hadoop.hbase.client.Scan;
  * Scanner that returns the next KeyValue.
  */
 @InterfaceAudience.Private
+// TODO: Change name from KeyValueScanner to CellScanner only we already have a simple CellScanner
+// so this should be something else altogether, a decoration on our base CellScanner. TODO.
+// This class shows in CPs so do it all in one swell swoop. HBase-2.0.0.
 public interface KeyValueScanner extends Shipper {
   /**
    * The byte array represents for NO_NEXT_INDEXED_KEY;
@@ -161,8 +164,9 @@ public interface KeyValueScanner extends Shipper {
   public boolean seekToLastRow() throws IOException;
 
   /**
-   * @return the next key in the index (the key to seek to the next block)
-   * if known, or null otherwise
+   * @return the next key in the index, usually the first key of next block OR a key that falls
+   * between last key of current block and first key of next block..
+   * see HFileWriterImpl#getMidpoint, or null if not known.
    */
   public Cell getNextIndexedKey();
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/7073f699/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java
index c220b5c..706fc5b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java
@@ -93,7 +93,7 @@ public class ScanQueryMatcher {
   /* row is not private for tests */
   /** Row the query is on */
   Cell curCell;
-  
+
   /**
    * Oldest put in any of the involved store files
    * Used to decide whether it is ok to delete
@@ -119,7 +119,7 @@ public class ScanQueryMatcher {
    * first column.
    * */
   private boolean hasNullColumn = true;
-  
+
   private RegionCoprocessorHost regionCoprocessorHost= null;
 
   // By default, when hbase.hstore.time.to.purge.deletes is 0ms, a delete
@@ -140,22 +140,22 @@ public class ScanQueryMatcher {
   // currently influencing. This is because Puts, that this delete can
   // influence.  may appear out of order.
   private final long timeToPurgeDeletes;
-  
+
   private final boolean isUserScan;
 
   private final boolean isReversed;
 
   /**
+   * True if we are doing a 'Get' Scan. Every Get is actually a one-row Scan.
+   */
+  private final boolean get;
+
+  /**
    * Construct a QueryMatcher for a scan
-   * @param scan
    * @param scanInfo The store's immutable scan info
-   * @param columns
    * @param scanType Type of the scan
    * @param earliestPutTs Earliest put seen in any of the store files.
-   * @param oldestUnexpiredTS the oldest timestamp we are interested in,
-   *  based on TTL
-   * @param regionCoprocessorHost 
-   * @throws IOException 
+   * @param oldestUnexpiredTS the oldest timestamp we are interested in, based on TTL
    */
   public ScanQueryMatcher(Scan scan, ScanInfo scanInfo, NavigableSet<byte[]> columns,
       ScanType scanType, long readPointToUse, long earliestPutTs, long oldestUnexpiredTS,
@@ -166,6 +166,7 @@ public class ScanQueryMatcher {
     } else {
       this.tr = timeRange;
     }
+    this.get = scan.isGetScan();
     this.rowComparator = scanInfo.getComparator();
     this.regionCoprocessorHost = regionCoprocessorHost;
     this.deletes =  instantiateDeleteTracker();
@@ -234,8 +235,8 @@ public class ScanQueryMatcher {
    * @param now the current server time
    * @param dropDeletesFromRow The inclusive left bound of the range; can be EMPTY_START_ROW.
    * @param dropDeletesToRow The exclusive right bound of the range; can be EMPTY_END_ROW.
-   * @param regionCoprocessorHost 
-   * @throws IOException 
+   * @param regionCoprocessorHost
+   * @throws IOException
    */
   public ScanQueryMatcher(Scan scan, ScanInfo scanInfo, NavigableSet<byte[]> columns,
       long readPointToUse, long earliestPutTs, long oldestUnexpiredTS, long now,
@@ -280,7 +281,7 @@ public class ScanQueryMatcher {
    *      caused by a data corruption.
    */
   public MatchCode match(Cell cell) throws IOException {
-      if (filter != null && filter.filterAllRemaining()) {
+    if (filter != null && filter.filterAllRemaining()) {
       return MatchCode.DONE_SCAN;
     }
     if (curCell != null) {
@@ -324,7 +325,7 @@ public class ScanQueryMatcher {
     // check if the cell is expired by cell TTL
     if (HStore.isCellTTLExpired(cell, this.oldestUnexpiredTS, this.now)) {
       return MatchCode.SKIP;
-    }    
+    }
 
     /*
      * The delete logic is pretty complicated now.
@@ -359,10 +360,10 @@ public class ScanQueryMatcher {
         }
         // Can't early out now, because DelFam come before any other keys
       }
-     
+
       if ((!isUserScan)
           && timeToPurgeDeletes > 0
-          && (EnvironmentEdgeManager.currentTime() - timestamp) 
+          && (EnvironmentEdgeManager.currentTime() - timestamp)
             <= timeToPurgeDeletes) {
         return MatchCode.INCLUDE;
       } else if (retainDeletesInOutput || mvccVersion > maxReadPointToTrackVersions) {
@@ -503,22 +504,27 @@ public class ScanQueryMatcher {
     }
   }
 
+  /**
+   * @return Returns false if we know there are no more rows to be scanned (We've reached the
+   * <code>stopRow</code> or we are scanning on row only because this Scan is for a Get, etc.
+   */
   public boolean moreRowsMayExistAfter(Cell kv) {
-    if (this.isReversed) {
-      if (rowComparator.compareRows(kv, stopRow, 0, stopRow.length) <= 0) {
-        return false;
-      } else {
-        return true;
-      }
-    }
-    if (!Bytes.equals(stopRow , HConstants.EMPTY_END_ROW) &&
-        rowComparator.compareRows(kv, stopRow, 0, stopRow.length) >= 0) {
-      // KV >= STOPROW
-      // then NO there is nothing left.
+    // If a 'get' Scan -- we are doing a Get (every Get is a single-row Scan in implementation) --
+    // then we are looking at one row only, the one specified in the Get coordinate..so we know
+    // for sure that there are no more rows on this Scan
+    if (this.get) {
       return false;
-    } else {
+    }
+    // If no stopRow, return that there may be more rows. The tests that follow depend on a
+    // non-empty, non-default stopRow so this little test below short-circuits out doing the
+    // following compares.
+    if (this.stopRow == null || this.stopRow == HConstants.EMPTY_BYTE_ARRAY) {
       return true;
     }
+    return this.isReversed?
+      rowComparator.compareRows(kv, stopRow, 0, stopRow.length) > 0:
+      Bytes.equals(stopRow, HConstants.EMPTY_END_ROW) ||
+        rowComparator.compareRows(kv, stopRow, 0, stopRow.length) < 0;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/7073f699/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
index 2f0d284..66f846a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
@@ -133,7 +133,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
   protected List<KeyValueScanner> currentScanners = new ArrayList<KeyValueScanner>();
   // flush update lock
   private ReentrantLock flushLock = new ReentrantLock();
-  
+
   protected final long readPt;
 
   // used by the injection framework to test race between StoreScanner construction and compaction
@@ -600,6 +600,12 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
         continue;
 
       case DONE:
+        // Optimization for Gets! If DONE, no more to get on this row, early exit!
+        if (this.scan.isGetScan()) {
+          // Then no more to this row... exit.
+          close(false);// Do all cleanup except heap.close()
+          return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
+        }
         matcher.curCell = null;
         return scannerContext.setScannerState(NextState.MORE_VALUES).hasMoreValues();
 
@@ -649,18 +655,67 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
     return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
   }
 
-  /*
-   * See if we should actually SEEK or rather just SKIP to the next Cell.
-   * (see HBASE-13109)
+  /**
+   * See if we should actually SEEK or rather just SKIP to the next Cell (see HBASE-13109).
+   * This method works together with ColumnTrackers and Filters. ColumnTrackers may issue SEEK
+   * hints, such as seek to next column, next row, or seek to an arbitrary seek key.
+   * This method intercepts these qcodes and decides whether a seek is the most efficient _actual_
+   * way to get us to the requested cell (SEEKs are more expensive than SKIP, SKIP, SKIP inside the
+   * current, loaded block).
+   * It does this by looking at the next indexed key of the current HFile. This key
+   * is then compared with the _SEEK_ key, where a SEEK key is an artificial 'last possible key
+   * on the row' (only in here, we avoid actually creating a SEEK key; in the compare we work with
+   * the current Cell but compare as though it were a seek key; see down in
+   * matcher.compareKeyForNextRow, etc). If the compare gets us onto the
+   * next block we *_SEEK, otherwise we just INCLUDE or SKIP, and let the ColumnTrackers or Filters
+   * go through the next Cell, and so on)
+   *
+   * <p>The ColumnTrackers and Filters must behave correctly in all cases, i.e. if they are past the
+   * Cells they care about they must issues a SKIP or SEEK.
+   *
+   * <p>Other notes:
+   * <ul>
+   * <li>Rows can straddle block boundaries</li>
+   * <li>Versions of columns can straddle block boundaries (i.e. column C1 at T1 might be in a
+   * different block than column C1 at T2)</li>
+   * <li>We want to SKIP and INCLUDE if the chance is high that we'll find the desired Cell after a
+   * few SKIPs...</li>
+   * <li>We want to INCLUDE_AND_SEEK and SEEK when the chance is high that we'll be able to seek
+   * past many Cells, especially if we know we need to go to the next block.</li>
+   * </ul>
+   * <p>A good proxy (best effort) to determine whether INCLUDE/SKIP is better than SEEK is whether
+   * we'll likely end up seeking to the next block (or past the next block) to get our next column.
+   * Example:
+   * <pre>
+   * |    BLOCK 1              |     BLOCK 2                   |
+   * |  r1/c1, r1/c2, r1/c3    |    r1/c4, r1/c5, r2/c1        |
+   *                                   ^         ^
+   *                                   |         |
+   *                           Next Index Key   SEEK_NEXT_ROW (before r2/c1)
+   *
+   *
+   * |    BLOCK 1                       |     BLOCK 2                      |
+   * |  r1/c1/t5, r1/c1/t4, r1/c1/t3    |    r1/c1/t2, r1/c1/T1, r1/c2/T3  |
+   *                                            ^              ^
+   *                                            |              |
+   *                                    Next Index Key        SEEK_NEXT_COL
+   * </pre>
+   * Now imagine we want columns c1 and c3 (see first diagram above), the 'Next Index Key' of r1/c4
+   * is > r1/c3 so we should seek to get to the c1 on the next row, r2. In second case, say we only
+   * want one version of c1, after we have it, a SEEK_COL will be issued to get to c2. Looking at
+   * the 'Next Index Key', it would land us in the next block, so we should SEEK. In other scenarios
+   * where the SEEK will not land us in the next block, it is very likely better to issues a series
+   * of SKIPs.
    */
-  private ScanQueryMatcher.MatchCode optimize(ScanQueryMatcher.MatchCode qcode, Cell cell) {
+  @VisibleForTesting
+  protected ScanQueryMatcher.MatchCode optimize(ScanQueryMatcher.MatchCode qcode, Cell cell) {
     switch(qcode) {
     case INCLUDE_AND_SEEK_NEXT_COL:
     case SEEK_NEXT_COL:
     {
       Cell nextIndexedKey = getNextIndexedKey();
       if (nextIndexedKey != null && nextIndexedKey != KeyValueScanner.NO_NEXT_INDEXED_KEY
-          && matcher.compareKeyForNextColumn(nextIndexedKey, cell) >= 0) {
+          && matcher.compareKeyForNextColumn(nextIndexedKey, cell) > 0) {
         return qcode == MatchCode.SEEK_NEXT_COL ? MatchCode.SKIP : MatchCode.INCLUDE;
       }
       break;
@@ -668,10 +723,16 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
     case INCLUDE_AND_SEEK_NEXT_ROW:
     case SEEK_NEXT_ROW:
     {
-      Cell nextIndexedKey = getNextIndexedKey();
-      if (nextIndexedKey != null && nextIndexedKey != KeyValueScanner.NO_NEXT_INDEXED_KEY
-          && matcher.compareKeyForNextRow(nextIndexedKey, cell) >= 0) {
-        return qcode == MatchCode.SEEK_NEXT_ROW ? MatchCode.SKIP : MatchCode.INCLUDE;
+      // If it is a Get Scan, then we know that we are done with this row; there are no more
+      // rows beyond the current one: don't try to optimize. We are DONE. Return the *_NEXT_ROW
+      // qcode as is. When the caller gets these flags on a Get Scan, it knows it can shut down the
+      // Scan.
+      if (!this.scan.isGetScan()) {
+        Cell nextIndexedKey = getNextIndexedKey();
+        if (nextIndexedKey != null && nextIndexedKey != KeyValueScanner.NO_NEXT_INDEXED_KEY
+            && matcher.compareKeyForNextRow(nextIndexedKey, cell) > 0) {
+          return qcode == MatchCode.SEEK_NEXT_ROW ? MatchCode.SKIP : MatchCode.INCLUDE;
+        }
       }
       break;
     }
@@ -809,10 +870,10 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
     // check the var without any lock. Suppose even if we see the old
     // value here still it is ok to continue because we will not be resetting
     // the heap but will continue with the referenced memstore's snapshot. For compactions
-    // any way we don't need the updateReaders at all to happen as we still continue with 
+    // any way we don't need the updateReaders at all to happen as we still continue with
     // the older files
     if (flushed) {
-      // If there is a flush and the current scan is notified on the flush ensure that the 
+      // If there is a flush and the current scan is notified on the flush ensure that the
       // scan's heap gets reset and we do a seek on the newly flushed file.
       if(!this.closing) {
         this.lastTop = this.peek();
@@ -842,7 +903,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
     if (scanners.isEmpty()) return;
     int storeFileScannerCount = scanners.size();
     CountDownLatch latch = new CountDownLatch(storeFileScannerCount);
-    List<ParallelSeekHandler> handlers = 
+    List<ParallelSeekHandler> handlers =
         new ArrayList<ParallelSeekHandler>(storeFileScannerCount);
     for (KeyValueScanner scanner : scanners) {
       if (scanner instanceof StoreFileScanner) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/7073f699/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CollectionBackedScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CollectionBackedScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CollectionBackedScanner.java
index 9fc068f..4720880 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CollectionBackedScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CollectionBackedScanner.java
@@ -30,8 +30,7 @@ import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.regionserver.NonReversedNonLazyKeyValueScanner;
 
 /**
- * Utility scanner that wraps a sortable collection and serves
- * as a KeyValueScanner.
+ * Utility scanner that wraps a sortable collection and serves as a KeyValueScanner.
  */
 @InterfaceAudience.Private
 public class CollectionBackedScanner extends NonReversedNonLazyKeyValueScanner {

http://git-wip-us.apache.org/repos/asf/hbase/blob/7073f699/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/KeyValueScanFixture.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/KeyValueScanFixture.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/KeyValueScanFixture.java
index 3f87a00..a4e7f9b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/KeyValueScanFixture.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/KeyValueScanFixture.java
@@ -22,6 +22,7 @@ package org.apache.hadoop.hbase.regionserver;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.util.CollectionBackedScanner;
@@ -33,9 +34,8 @@ import org.apache.hadoop.hbase.util.CollectionBackedScanner;
  * to be a store file scanner.
  */
 public class KeyValueScanFixture extends CollectionBackedScanner {
-  public KeyValueScanFixture(CellComparator comparator,
-                             KeyValue... incData) {
-    super(comparator, incData);
+  public KeyValueScanFixture(CellComparator comparator, Cell... cells) {
+    super(comparator, cells);
   }
 
   public static List<KeyValueScanner> scanFixture(KeyValue[] ... kvArrays) {
@@ -45,4 +45,4 @@ public class KeyValueScanFixture extends CollectionBackedScanner {
     }
     return scanners;
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/7073f699/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeyValueScanFixture.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeyValueScanFixture.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeyValueScanFixture.java
index a8c2c65..0e96682 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeyValueScanFixture.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeyValueScanFixture.java
@@ -46,8 +46,7 @@ public class TestKeyValueScanFixture extends TestCase {
         KeyValueTestUtil.create("RowB", "family", "qf1",
             10, KeyValue.Type.Put, "value-10")
     };
-    KeyValueScanner scan = new KeyValueScanFixture(
-        CellComparator.COMPARATOR, kvs);
+    KeyValueScanner scan = new KeyValueScanFixture(CellComparator.COMPARATOR, kvs);
 
     KeyValue kv = KeyValueUtil.createFirstOnRow(Bytes.toBytes("RowA"));
     // should seek to this:

http://git-wip-us.apache.org/repos/asf/hbase/blob/7073f699/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java
index 92c85aa..4c594b0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java
@@ -20,6 +20,7 @@
 package org.apache.hadoop.hbase.regionserver;
 
 import static org.apache.hadoop.hbase.regionserver.KeyValueScanFixture.scanFixture;
+import static org.junit.Assert.*;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -27,16 +28,21 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.NavigableSet;
 import java.util.TreeSet;
+import java.util.concurrent.atomic.AtomicInteger;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CategoryBasedTimeout;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeepDeletedCells;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueTestUtil;
+import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
@@ -53,16 +59,113 @@ import org.junit.rules.TestRule;
 // Can't be small as it plays with EnvironmentEdgeManager
 @Category({RegionServerTests.class, MediumTests.class})
 public class TestStoreScanner {
+  private static final Log LOG = LogFactory.getLog(TestStoreScanner.class);
   @Rule public TestName name = new TestName();
   @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()).
       withLookingForStuckThread(true).build();
   private static final String CF_STR = "cf";
-  final byte [] CF = Bytes.toBytes(CF_STR);
+  private static final byte [] CF = Bytes.toBytes(CF_STR);
   static Configuration CONF = HBaseConfiguration.create();
   private ScanInfo scanInfo = new ScanInfo(CONF, CF, 0, Integer.MAX_VALUE,
       Long.MAX_VALUE, KeepDeletedCells.FALSE, 0, CellComparator.COMPARATOR);
   private ScanType scanType = ScanType.USER_SCAN;
 
+  /**
+   * From here on down, we have a bunch of defines and specific CELL_GRID of Cells. The
+   * CELL_GRID then has a Scanner that can fake out 'block' transitions. All this elaborate
+   * setup is for tests that ensure we don't overread, and that the
+   * {@link StoreScanner#optimize(org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.MatchCode,
+   * Cell)} is not overly enthusiastic.
+   */
+  private static final byte [] ZERO = new byte [] {'0'};
+  private static final byte [] ZERO_POINT_ZERO = new byte [] {'0', '.', '0'};
+  private static final byte [] ONE = new byte [] {'1'};
+  private static final byte [] TWO = new byte [] {'2'};
+  private static final byte [] TWO_POINT_TWO = new byte [] {'2', '.', '2'};
+  private static final byte [] THREE = new byte [] {'3'};
+  private static final byte [] FOUR = new byte [] {'4'};
+  private static final byte [] FIVE = new byte [] {'5'};
+  private static final byte [] VALUE = new byte [] {'v'};
+  private static final int CELL_GRID_BLOCK2_BOUNDARY = 4;
+  private static final int CELL_GRID_BLOCK3_BOUNDARY = 11;
+  private static final int CELL_GRID_BLOCK4_BOUNDARY = 15;
+  private static final int CELL_GRID_BLOCK5_BOUNDARY = 19;
+
+  /**
+   * Five rows by four columns distinguished by column qualifier (column qualifier is one of the
+   * four rows... ONE, TWO, etc.). Exceptions are a weird row after TWO; it is TWO_POINT_TWO.
+   * And then row FOUR has five columns finishing w/ row FIVE having a single column.
+   * We will use this to test scan does the right thing as it
+   * we do Gets, StoreScanner#optimize, and what we do on (faked) block boundaries.
+   */
+  private static final Cell [] CELL_GRID = new Cell [] {
+    CellUtil.createCell(ONE, CF, ONE, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(ONE, CF, TWO, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(ONE, CF, THREE, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(ONE, CF, FOUR, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    // Offset 4 CELL_GRID_BLOCK2_BOUNDARY
+    CellUtil.createCell(TWO, CF, ONE, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(TWO, CF, TWO, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(TWO, CF, THREE, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(TWO, CF, FOUR, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(TWO_POINT_TWO, CF, ZERO, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(TWO_POINT_TWO, CF, ZERO_POINT_ZERO, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(TWO_POINT_TWO, CF, FIVE, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    // Offset 11! CELL_GRID_BLOCK3_BOUNDARY
+    CellUtil.createCell(THREE, CF, ONE, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(THREE, CF, TWO, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(THREE, CF, THREE, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(THREE, CF, FOUR, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    // Offset 15 CELL_GRID_BLOCK4_BOUNDARY
+    CellUtil.createCell(FOUR, CF, ONE, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(FOUR, CF, TWO, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(FOUR, CF, THREE, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(FOUR, CF, FOUR, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    // Offset 19 CELL_GRID_BLOCK5_BOUNDARY
+    CellUtil.createCell(FOUR, CF, FIVE, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(FIVE, CF, ZERO, 1L, KeyValue.Type.Put.getCode(), VALUE),
+  };
+
+  /**
+   * A StoreScanner for our CELL_GRID above. Fakes the block transitions. Does counts of
+   * calls to optimize and counts of when optimize actually did an optimize.
+   */
+  private static class CellGridStoreScanner extends StoreScanner {
+    // Count of how often optimize is called and of how often it does an optimize.
+    final AtomicInteger count = new AtomicInteger(0);
+    final AtomicInteger optimization = new AtomicInteger(0);
+
+    CellGridStoreScanner(final Scan scan, ScanInfo scanInfo, ScanType scanType)
+    throws IOException {
+      super(scan, scanInfo, scanType, scan.getFamilyMap().get(CF),
+        Arrays.<KeyValueScanner>asList(
+          new KeyValueScanner[] {new KeyValueScanFixture(CellComparator.COMPARATOR, CELL_GRID)}));
+    }
+
+    protected ScanQueryMatcher.MatchCode optimize(ScanQueryMatcher.MatchCode qcode, Cell cell) {
+      count.incrementAndGet();
+      ScanQueryMatcher.MatchCode after = super.optimize(qcode, cell);
+      LOG.info("Cell=" + cell + ", nextIndex=" + CellUtil.toString(getNextIndexedKey(), false) +
+          ", before=" + qcode + ", after=" + after);
+      if (qcode != after) {
+        optimization.incrementAndGet();
+      }
+      return after;
+    };
+
+    @Override
+    public Cell getNextIndexedKey() {
+      // Fake block boundaries by having index of next block change as we go through scan.
+      return count.get() > CELL_GRID_BLOCK4_BOUNDARY?
+          CellUtil.createFirstOnRow(CELL_GRID[CELL_GRID_BLOCK5_BOUNDARY]):
+            count.get() > CELL_GRID_BLOCK3_BOUNDARY?
+                CellUtil.createFirstOnRow(CELL_GRID[CELL_GRID_BLOCK4_BOUNDARY]):
+                  count.get() > CELL_GRID_BLOCK2_BOUNDARY?
+                      CellUtil.createFirstOnRow(CELL_GRID[CELL_GRID_BLOCK3_BOUNDARY]):
+                        CellUtil.createFirstOnRow(CELL_GRID[CELL_GRID_BLOCK2_BOUNDARY]);
+    }
+  };
+
   /*
    * Test utility for building a NavigableSet for scanners.
    * @param strCols
@@ -78,6 +181,145 @@ public class TestStoreScanner {
   }
 
   @Test
+  public void testFullRowGetDoesNotOverreadWhenRowInsideOneBlock() throws IOException {
+    // Do a Get against row two. Row two is inside a block that starts with row TWO but ends with
+    // row TWO_POINT_TWO. We should read one block only.
+    Get get = new Get(TWO);
+    Scan scan = new Scan(get);
+    CellGridStoreScanner scanner = new CellGridStoreScanner(scan, this.scanInfo, this.scanType);
+    try {
+      List<Cell> results = new ArrayList<Cell>();
+      while (scanner.next(results)) {
+        continue;
+      }
+      // Should be four results of column 1 (though there are 5 rows in the CELL_GRID -- the
+      // TWO_POINT_TWO row does not have a a column ONE.
+      Assert.assertEquals(4, results.size());
+      // We should have gone the optimize route 5 times totally... an INCLUDE for the four cells
+      // in the row plus the DONE on the end.
+      Assert.assertEquals(5, scanner.count.get());
+      // For a full row Get, there should be no opportunity for scanner optimization.
+      Assert.assertEquals(0, scanner.optimization.get());
+    } finally {
+      scanner.close();
+    }
+  }
+
+  @Test
+  public void testFullRowSpansBlocks() throws IOException {
+    // Do a Get against row FOUR. It spans two blocks.
+    Get get = new Get(FOUR);
+    Scan scan = new Scan(get);
+    CellGridStoreScanner scanner = new CellGridStoreScanner(scan, this.scanInfo, this.scanType);
+    try {
+      List<Cell> results = new ArrayList<Cell>();
+      while (scanner.next(results)) {
+        continue;
+      }
+      // Should be four results of column 1 (though there are 5 rows in the CELL_GRID -- the
+      // TWO_POINT_TWO row does not have a a column ONE.
+      Assert.assertEquals(5, results.size());
+      // We should have gone the optimize route 6 times totally... an INCLUDE for the five cells
+      // in the row plus the DONE on the end.
+      Assert.assertEquals(6, scanner.count.get());
+      // For a full row Get, there should be no opportunity for scanner optimization.
+      Assert.assertEquals(0, scanner.optimization.get());
+    } finally {
+      scanner.close();
+    }
+  }
+
+  /**
+   * Test optimize in StoreScanner. Test that we skip to the next 'block' when we it makes sense
+   * reading the block 'index'.
+   * @throws IOException
+   */
+  @Test
+  public void testOptimize() throws IOException {
+    Scan scan = new Scan();
+    // A scan that just gets the first qualifier on each row of the CELL_GRID
+    scan.addColumn(CF, ONE);
+    CellGridStoreScanner scanner = new CellGridStoreScanner(scan, this.scanInfo, this.scanType);
+    try {
+      List<Cell> results = new ArrayList<Cell>();
+      while (scanner.next(results)) {
+        continue;
+      }
+      // Should be four results of column 1 (though there are 5 rows in the CELL_GRID -- the
+      // TWO_POINT_TWO row does not have a a column ONE.
+      Assert.assertEquals(4, results.size());
+      for (Cell cell: results) {
+        assertTrue(Bytes.equals(ONE, 0, ONE.length,
+            cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength()));
+      }
+      Assert.assertTrue("Optimize should do some optimizations", scanner.optimization.get() > 0);
+    } finally {
+      scanner.close();
+    }
+  }
+
+  /**
+   * Ensure the optimize Scan method in StoreScanner does not get in the way of a Get doing minimum
+   * work... seeking to start of block and then SKIPPING until we find the wanted Cell.
+   * This 'simple' scenario mimics case of all Cells fitting inside a single HFileBlock.
+   * See HBASE-15392. This test is a little cryptic. Takes a bit of staring to figure what it up to.
+   * @throws IOException
+   */
+  @Test
+  public void testOptimizeAndGet() throws IOException {
+    // First test a Get of two columns in the row R2. Every Get is a Scan. Get columns named
+    // R2 and R3.
+    Get get = new Get(TWO);
+    get.addColumn(CF, TWO);
+    get.addColumn(CF, THREE);
+    Scan scan = new Scan(get);
+    CellGridStoreScanner scanner = new CellGridStoreScanner(scan, this.scanInfo, this.scanType);
+    try {
+      List<Cell> results = new ArrayList<Cell>();
+      // For a Get there should be no more next's after the first call.
+      Assert.assertEquals(false, scanner.next(results));
+      // Should be one result only.
+      Assert.assertEquals(2, results.size());
+      // And we should have gone through optimize twice only.
+      Assert.assertEquals("First qcode is SEEK_NEXT_COL and second INCLUDE_AND_SEEK_NEXT_ROW",
+        3, scanner.count.get());
+    } finally {
+      scanner.close();
+    }
+  }
+
+  /**
+   * Ensure that optimize does not cause the Get to do more seeking than required. Optimize
+   * (see HBASE-15392) was causing us to seek all Cells in a block when a Get Scan if the next block
+   * index/start key was a different row to the current one. A bug. We'd call next too often
+   * because we had to exhaust all Cells in the current row making us load the next block just to
+   * discard what we read there. This test is a little cryptic. Takes a bit of staring to figure
+   * what it up to.
+   * @throws IOException
+   */
+  @Test
+  public void testOptimizeAndGetWithFakedNextBlockIndexStart() throws IOException {
+    // First test a Get of second column in the row R2. Every Get is a Scan. Second column has a
+    // qualifier of R2.
+    Get get = new Get(THREE);
+    get.addColumn(CF, TWO);
+    Scan scan = new Scan(get);
+    CellGridStoreScanner scanner = new CellGridStoreScanner(scan, this.scanInfo, this.scanType);
+    try {
+      List<Cell> results = new ArrayList<Cell>();
+      // For a Get there should be no more next's after the first call.
+      Assert.assertEquals(false, scanner.next(results));
+      // Should be one result only.
+      Assert.assertEquals(1, results.size());
+      // And we should have gone through optimize twice only.
+      Assert.assertEquals("First qcode is SEEK_NEXT_COL and second INCLUDE_AND_SEEK_NEXT_ROW",
+        2, scanner.count.get());
+    } finally {
+      scanner.close();
+    }
+  }
+
+  @Test
   public void testScanTimeRange() throws IOException {
     String r1 = "R1";
     // returns only 1 of these 2 even though same timestamp


[02/50] [abbrv] hbase git commit: HBASE-15488 Add ACL for setting split merge switch

Posted by sy...@apache.org.
HBASE-15488 Add ACL for setting split merge switch


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

Branch: refs/heads/hbase-12439
Commit: 47471c35e3a98ba023ef9095b90b4698fd15da3b
Parents: 797562e
Author: tedyu <yu...@gmail.com>
Authored: Tue Mar 22 10:38:13 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Tue Mar 22 10:38:13 2016 -0700

----------------------------------------------------------------------
 .../hbase/rsgroup/RSGroupAdminEndpoint.java     | 12 ++++++++++
 .../hbase/coprocessor/BaseMasterObserver.java   | 12 ++++++++++
 .../hbase/coprocessor/MasterObserver.java       | 19 ++++++++++++++++
 .../hbase/master/MasterCoprocessorHost.java     | 23 ++++++++++++++++++++
 .../hadoop/hbase/master/MasterRpcServices.java  | 11 +++++++++-
 .../hbase/security/access/AccessController.java | 13 +++++++++++
 .../visibility/VisibilityController.java        | 12 ++++++++++
 .../hbase/coprocessor/TestMasterObserver.java   | 11 ++++++++++
 .../security/access/TestAccessController.java   | 16 ++++++++++++++
 9 files changed, 128 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/47471c35/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
index e51dcc2..22bad72 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
@@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.constraint.ConstraintException;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
 import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
@@ -825,6 +826,17 @@ public class RSGroupAdminEndpoint extends RSGroupAdminService
   }
 
   @Override
+  public boolean preSetSplitOrMergeEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      final boolean newValue, final Admin.MasterSwitchType switchType) throws IOException {
+    return false;
+  }
+
+  @Override
+  public void postSetSplitOrMergeEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      final boolean newValue, final Admin.MasterSwitchType switchType) throws IOException {
+  }
+
+  @Override
   public void preSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName,
                               Quotas quotas) throws IOException {
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/47471c35/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java
index 0adb179..18c6a0a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.ServerName;
 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.client.Admin;
 import org.apache.hadoop.hbase.master.RegionPlan;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
@@ -440,6 +441,17 @@ public class BaseMasterObserver implements MasterObserver {
   }
 
   @Override
+  public boolean preSetSplitOrMergeEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      final boolean newValue, final Admin.MasterSwitchType switchType) throws IOException {
+    return false;
+  }
+
+  @Override
+  public void postSetSplitOrMergeEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      final boolean newValue, final Admin.MasterSwitchType switchType) throws IOException {
+  }
+
+  @Override
   public void preBalance(ObserverContext<MasterCoprocessorEnvironment> ctx)
       throws IOException {
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/47471c35/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
index 82b6ffe..81f97aa 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.ServerName;
 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.client.Admin;
 import org.apache.hadoop.hbase.master.RegionPlan;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
@@ -800,6 +801,24 @@ public interface MasterObserver extends Coprocessor {
       throws IOException;
 
   /**
+   * Called prior to setting split / merge switch
+   * @param ctx the coprocessor instance's environment
+   * @param newValue the new value submitted in the call
+   * @param switchType type of switch
+   */
+  boolean preSetSplitOrMergeEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      final boolean newValue, final Admin.MasterSwitchType switchType) throws IOException;
+
+  /**
+   * Called after setting split / merge switch
+   * @param ctx the coprocessor instance's environment
+   * @param newValue the new value submitted in the call
+   * @param switchType type of switch
+   */
+  void postSetSplitOrMergeEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      final boolean newValue, final Admin.MasterSwitchType switchType) throws IOException;
+
+  /**
    * Called prior to modifying the flag used to enable/disable region balancing.
    * @param ctx the coprocessor instance's environment
    * @param newValue the new flag value submitted in the call

http://git-wip-us.apache.org/repos/asf/hbase/blob/47471c35/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
index b16c232..313b1ad 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
 import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
@@ -777,6 +778,28 @@ public class MasterCoprocessorHost
     });
   }
 
+  public boolean preSetSplitOrMergeEnabled(final boolean newValue,
+      final Admin.MasterSwitchType switchType) throws IOException {
+    return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+      @Override
+      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
+          throws IOException {
+        oserver.preSetSplitOrMergeEnabled(ctx, newValue, switchType);
+      }
+    });
+  }
+
+  public void postSetSplitOrMergeEnabled(final boolean newValue,
+      final Admin.MasterSwitchType switchType) throws IOException {
+    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+      @Override
+      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
+          throws IOException {
+        oserver.postSetSplitOrMergeEnabled(ctx, newValue, switchType);
+      }
+    });
+  }
+
   public boolean preBalanceSwitch(final boolean b) throws IOException {
     return execOperationWithResult(b, coprocessors.isEmpty() ? null :
         new CoprocessorOperationWithResult<Boolean>() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/47471c35/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 753ecb6..6a60c2c 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
@@ -1524,8 +1524,17 @@ public class MasterRpcServices extends RSRpcServices
       for (MasterSwitchType masterSwitchType : request.getSwitchTypesList()) {
         Admin.MasterSwitchType switchType = convert(masterSwitchType);
         boolean oldValue = master.isSplitOrMergeEnabled(switchType);
-        master.getSplitOrMergeTracker().setSplitOrMergeEnabled(newValue, switchType);
         response.addPrevValue(oldValue);
+        boolean bypass = false;
+        if (master.cpHost != null) {
+          bypass = master.cpHost.preSetSplitOrMergeEnabled(newValue, switchType);
+        }
+        if (!bypass) {
+          master.getSplitOrMergeTracker().setSplitOrMergeEnabled(newValue, switchType);
+        }
+        if (master.cpHost != null) {
+          master.cpHost.postSetSplitOrMergeEnabled(newValue, switchType);
+        }
       }
     } catch (IOException e) {
       throw new ServiceException(e);

http://git-wip-us.apache.org/repos/asf/hbase/blob/47471c35/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
index 0a88a86..3b36ada 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
@@ -60,6 +60,7 @@ import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.TagRewriteCell;
 import org.apache.hadoop.hbase.TagUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Durability;
@@ -1260,6 +1261,18 @@ public class AccessController extends BaseMasterAndRegionObserver
   }
 
   @Override
+  public boolean preSetSplitOrMergeEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      final boolean newValue, final Admin.MasterSwitchType switchType) throws IOException {
+    requirePermission("setSplitOrMergeEnabled", Action.ADMIN);
+    return false;
+  }
+
+  @Override
+  public void postSetSplitOrMergeEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      final boolean newValue, final Admin.MasterSwitchType switchType) throws IOException {
+  }
+
+  @Override
   public void preBalance(ObserverContext<MasterCoprocessorEnvironment> c)
       throws IOException {
     requirePermission("balance", Action.ADMIN);

http://git-wip-us.apache.org/repos/asf/hbase/blob/47471c35/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
index 347f3da..6e2f8ed 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
@@ -51,6 +51,7 @@ import org.apache.hadoop.hbase.TagRewriteCell;
 import org.apache.hadoop.hbase.TagType;
 import org.apache.hadoop.hbase.TagUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
@@ -308,6 +309,17 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
   }
 
   @Override
+  public boolean preSetSplitOrMergeEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      final boolean newValue, final Admin.MasterSwitchType switchType) throws IOException {
+    return false;
+  }
+
+  @Override
+  public void postSetSplitOrMergeEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      final boolean newValue, final Admin.MasterSwitchType switchType) throws IOException {
+  }
+
+  @Override
   public void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
       MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
     if (c.getEnvironment().getRegion().getRegionInfo().getTable().isSystemTable()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/47471c35/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
index 3810f1e..65fc0c3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
@@ -347,6 +347,17 @@ public class TestMasterObserver {
     }
 
     @Override
+    public boolean preSetSplitOrMergeEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+        final boolean newValue, final Admin.MasterSwitchType switchType) throws IOException {
+      return false;
+    }
+
+    @Override
+    public void postSetSplitOrMergeEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+        final boolean newValue, final Admin.MasterSwitchType switchType) throws IOException {
+    }
+
+    @Override
     public void preModifyTable(ObserverContext<MasterCoprocessorEnvironment> env,
         TableName tableName, HTableDescriptor htd) throws IOException {
       if (bypass) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/47471c35/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 66ea050..7b1454d 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
@@ -705,6 +705,22 @@ public class TestAccessController extends SecureTestUtil {
   }
 
   @Test (timeout=180000)
+  public void testSetSplitOrMergeEnabled() throws Exception {
+    AccessTestAction action = new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preSetSplitOrMergeEnabled(ObserverContext.createAndPrepare(CP_ENV, null),
+          true, Admin.MasterSwitchType.MERGE);
+        return null;
+      }
+    };
+
+    verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
+    verifyDenied(action, USER_CREATE, USER_OWNER, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ,
+      USER_GROUP_WRITE, USER_GROUP_CREATE);
+  }
+
+  @Test (timeout=180000)
   public void testBalance() throws Exception {
     AccessTestAction action = new AccessTestAction() {
       @Override


[18/50] [abbrv] hbase git commit: HBASE-15477 Purge 'next block header' from cached blocks

Posted by sy...@apache.org.
HBASE-15477 Purge 'next block header' from cached blocks

When we read from HDFS, we overread to pick up the next blocks header.
Doing this saves a seek as we move through the hfile; we save having to
do an explicit seek just to read the block header every time we need to
read the body.  We used to read in the next header as part of the
current blocks buffer. This buffer was then what got persisted to
blockcache; so we were over-persisting: our block plus the next blocks'
header (33 bytes).

This patch undoes this over-persisting.

Removes support for version 1 blocks (0.2 was added in hbase-0.92.0).
Not needed any more.

There is an open question on whether checksums should be persisted
when caching. The code seems to say no but if cache is SSD backed or
backed by anything that does not do error correction, we'll want
checksums.

Adds loads of documentation.

M hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java
  (write) Add writing from a ByteBuff.

M hbase-common/src/main/java/org/apache/hadoop/hbase/nio/ByteBuff.java
  (toString) Add one so ByteBuff looks like ByteBuffer when you click on
  it in IDE

M hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
  Remove support for version 1 blocks.

  Cleaned up handling of metadata added when we serialize a block to
  caches. Metadata is smaller now.

  When we serialize (used when caching), do not persist the next blocks
  header if present.

  Removed a bunch of methods, a few of which had overlapping
  functionality and others that exposed too much of our internals.
  Also removed a bunch of constructors and unified the constructors we
  had left over making them share a common init method.
  Shutdown access to defines that should only be used internally here.

  Renamed all to do w/ 'EXTRA' and 'extraSerialization' to instead talk
  about metadata saved to caches; was unclear previously what EXTRA was
  about.

  Renamed static final declarations as all uppercase.

  (readBlockDataInternal): Redid. Couldn't make sense of it previously.
  Undid heavy-duty parse of header by constructing HFileBlock. Other
  cleanups. Its 1/3rd the length it used to be. More to do in here.


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

Branch: refs/heads/hbase-12439
Commit: 12f66e3060339acc569ee425385e2dc255bb3e94
Parents: a13d6e0
Author: stack <st...@apache.org>
Authored: Thu Mar 17 11:18:06 2016 -0700
Committer: stack <st...@apache.org>
Committed: Tue Mar 22 18:45:17 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hbase/io/hfile/BlockType.java |   4 +
 .../hbase/io/hfile/HFileContextBuilder.java     |  20 +
 .../org/apache/hadoop/hbase/nio/ByteBuff.java   |   6 +
 .../hbase/io/hfile/MemcachedBlockCache.java     |   2 +-
 .../hadoop/hbase/io/hfile/ChecksumUtil.java     |   5 +-
 .../hadoop/hbase/io/hfile/HFileBlock.java       | 997 +++++++++----------
 .../hadoop/hbase/io/hfile/HFileBlockIndex.java  |   2 +-
 .../hadoop/hbase/io/hfile/HFileReaderImpl.java  |  26 +-
 .../hadoop/hbase/io/hfile/HFileScanner.java     |  12 +
 .../hbase/io/hfile/bucket/BucketCache.java      |  15 +-
 .../hbase/regionserver/KeyValueScanner.java     |   6 +-
 .../hadoop/hbase/regionserver/StoreFile.java    |   4 +-
 .../hadoop/hbase/io/hfile/CacheTestUtils.java   |  23 +-
 .../hadoop/hbase/io/hfile/TestCacheOnWrite.java |  10 +-
 .../hadoop/hbase/io/hfile/TestChecksum.java     |  27 +-
 .../hadoop/hbase/io/hfile/TestHFileBlock.java   |  27 +-
 .../io/hfile/TestHFileBlockCompatibility.java   | 750 --------------
 .../hbase/io/hfile/TestHFileBlockIndex.java     |   3 +-
 .../io/hfile/TestHFileDataBlockEncoder.java     |  10 +-
 .../hbase/io/hfile/TestHFileEncryption.java     |   2 +-
 .../hbase/io/hfile/TestHFileWriterV3.java       |   7 +-
 .../hadoop/hbase/io/hfile/TestPrefetch.java     |   9 +-
 .../regionserver/TestCacheOnWriteInSchema.java  |   8 +-
 23 files changed, 604 insertions(+), 1371 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/12f66e30/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java
index 4228f57..32eb0b2 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java
@@ -132,6 +132,10 @@ public enum BlockType {
     out.write(magic);
   }
 
+  public void write(ByteBuffer buf) {
+    buf.put(magic);
+  }
+
   public void write(ByteBuff buf) {
     buf.put(magic);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/12f66e30/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContextBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContextBuilder.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContextBuilder.java
index 6d3bb13..a6645a6 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContextBuilder.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContextBuilder.java
@@ -55,6 +55,26 @@ public class HFileContextBuilder {
 
   private String hfileName = null;
 
+  public HFileContextBuilder() {}
+
+  /**
+   * Use this constructor if you want to change a few settings only in another context.
+   */
+  public HFileContextBuilder(final HFileContext hfc) {
+    this.usesHBaseChecksum = hfc.isUseHBaseChecksum();
+    this.includesMvcc = hfc.isIncludesMvcc();
+    this.includesTags = hfc.isIncludesTags();
+    this.compression = hfc.getCompression();
+    this.compressTags = hfc.isCompressTags();
+    this.checksumType = hfc.getChecksumType();
+    this.bytesPerChecksum = hfc.getBytesPerChecksum();
+    this.blocksize = hfc.getBlocksize();
+    this.encoding = hfc.getDataBlockEncoding();
+    this.cryptoContext = hfc.getEncryptionContext();
+    this.fileCreateTime = hfc.getFileCreateTime();
+    this.hfileName = hfc.getHFileName();
+  }
+
   public HFileContextBuilder withHBaseCheckSum(boolean useHBaseCheckSum) {
     this.usesHBaseChecksum = useHBaseCheckSum;
     return this;

http://git-wip-us.apache.org/repos/asf/hbase/blob/12f66e30/hbase-common/src/main/java/org/apache/hadoop/hbase/nio/ByteBuff.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/nio/ByteBuff.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/nio/ByteBuff.java
index 1e0e957..183a031 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/nio/ByteBuff.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/nio/ByteBuff.java
@@ -496,6 +496,12 @@ public abstract class ByteBuff {
     return -(low + 1); // key not found.
   }
 
+  @Override
+  public String toString() {
+    return this.getClass().getSimpleName() + "[pos=" + position() + ", lim=" + limit() +
+        ", cap= " + capacity() + "]";
+  }
+
   public static String toStringBinary(final ByteBuff b, int off, int len) {
     StringBuilder result = new StringBuilder();
     // Just in case we are passed a 'len' that is > buffer length...

http://git-wip-us.apache.org/repos/asf/hbase/blob/12f66e30/hbase-external-blockcache/src/main/java/org/apache/hadoop/hbase/io/hfile/MemcachedBlockCache.java
----------------------------------------------------------------------
diff --git a/hbase-external-blockcache/src/main/java/org/apache/hadoop/hbase/io/hfile/MemcachedBlockCache.java b/hbase-external-blockcache/src/main/java/org/apache/hadoop/hbase/io/hfile/MemcachedBlockCache.java
index 536872e..ae871c4 100644
--- a/hbase-external-blockcache/src/main/java/org/apache/hadoop/hbase/io/hfile/MemcachedBlockCache.java
+++ b/hbase-external-blockcache/src/main/java/org/apache/hadoop/hbase/io/hfile/MemcachedBlockCache.java
@@ -260,7 +260,7 @@ public class MemcachedBlockCache implements BlockCache {
     public HFileBlock decode(CachedData d) {
       try {
         ByteBuff buf = new SingleByteBuff(ByteBuffer.wrap(d.getData()));
-        return (HFileBlock) HFileBlock.blockDeserializer.deserialize(buf, true,
+        return (HFileBlock) HFileBlock.BLOCK_DESERIALIZER.deserialize(buf, true,
           MemoryType.EXCLUSIVE);
       } catch (IOException e) {
         LOG.warn("Error deserializing data from memcached",e);

http://git-wip-us.apache.org/repos/asf/hbase/blob/12f66e30/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ChecksumUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ChecksumUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ChecksumUtil.java
index 69f4330..b0b1714 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ChecksumUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ChecksumUtil.java
@@ -91,7 +91,7 @@ public class ChecksumUtil {
 
     // If this is an older version of the block that does not have
     // checksums, then return false indicating that checksum verification
-    // did not succeed. Actually, this methiod should never be called
+    // did not succeed. Actually, this method should never be called
     // when the minorVersion is 0, thus this is a defensive check for a
     // cannot-happen case. Since this is a cannot-happen case, it is
     // better to return false to indicate a checksum validation failure.
@@ -141,8 +141,7 @@ public class ChecksumUtil {
    * @return The number of bytes needed to store the checksum values
    */
   static long numBytes(long datasize, int bytesPerChecksum) {
-    return numChunks(datasize, bytesPerChecksum) *
-                     HFileBlock.CHECKSUM_SIZE;
+    return numChunks(datasize, bytesPerChecksum) * HFileBlock.CHECKSUM_SIZE;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/12f66e30/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
index 6268f2e..f3402da 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
@@ -56,50 +56,131 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 
 /**
- * Reads {@link HFile} version 1 and version 2 blocks but writes version 2 blocks only.
- * Version 2 was introduced in hbase-0.92.0. Does read and write out to the filesystem but also
- * the read and write to Cache.
+ * Reads {@link HFile} version 2 blocks to HFiles and via {@link Cacheable} Interface to caches.
+ * Version 2 was introduced in hbase-0.92.0. No longer has support for version 1 blocks since
+ * hbase-1.3.0.
+ *
+ * <p>Version 1 was the original file block. Version 2 was introduced when we changed the hbase file
+ * format to support multi-level block indexes and compound bloom filters (HBASE-3857).
  *
- * <h3>HFileBlock: Version 1</h3>
- * As of this writing, there should be no more version 1 blocks found out in the wild. Version 2
- * as introduced in hbase-0.92.0.
- * In version 1 all blocks are always compressed or uncompressed, as
- * specified by the {@link HFile}'s compression algorithm, with a type-specific
- * magic record stored in the beginning of the compressed data (i.e. one needs
- * to uncompress the compressed block to determine the block type). There is
- * only a single compression algorithm setting for all blocks. Offset and size
- * information from the block index are required to read a block.
  * <h3>HFileBlock: Version 2</h3>
  * In version 2, a block is structured as follows:
  * <ul>
- * <li><b>Header:</b> See Writer#putHeader(); header total size is HFILEBLOCK_HEADER_SIZE)
+ * <li><b>Header:</b> See Writer#putHeader() for where header is written; header total size is
+ * HFILEBLOCK_HEADER_SIZE
  * <ul>
- * <li>Magic record identifying the {@link BlockType} (8 bytes): e.g. <code>DATABLK*</code>
- * <li>Compressed -- a.k.a 'on disk' -- block size, excluding header, but including
- *     tailing checksum bytes (4 bytes)
- * <li>Uncompressed block size, excluding header, and excluding checksum bytes (4 bytes)
- * <li>The offset of the previous block of the same type (8 bytes). This is
+ * <li>0. blockType: Magic record identifying the {@link BlockType} (8 bytes):
+ * e.g. <code>DATABLK*</code>
+ * <li>1. onDiskSizeWithoutHeader: Compressed -- a.k.a 'on disk' -- block size, excluding header,
+ * but including tailing checksum bytes (4 bytes)
+ * <li>2. uncompressedSizeWithoutHeader: Uncompressed block size, excluding header, and excluding
+ * checksum bytes (4 bytes)
+ * <li>3. prevBlockOffset: The offset of the previous block of the same type (8 bytes). This is
  * used to navigate to the previous block without having to go to the block index
- * <li>For minorVersions &gt;=1, the ordinal describing checksum type (1 byte)
- * <li>For minorVersions &gt;=1, the number of data bytes/checksum chunk (4 bytes)
- * <li>For minorVersions &gt;=1, the size of data 'on disk', including header,
- * excluding checksums (4 bytes)
+ * <li>4: For minorVersions &gt;=1, the ordinal describing checksum type (1 byte)
+ * <li>5: For minorVersions &gt;=1, the number of data bytes/checksum chunk (4 bytes)
+ * <li>6: onDiskDataSizeWithHeader: For minorVersions &gt;=1, the size of data 'on disk', including
+ * header, excluding checksums (4 bytes)
  * </ul>
  * </li>
- * <li><b>Raw/Compressed/Encrypted/Encoded data:</b> The compression algorithm is the
- * same for all the blocks in the {@link HFile}, similarly to what was done in
- * version 1. If compression is NONE, this is just raw, serialized Cells.
+ * <li><b>Raw/Compressed/Encrypted/Encoded data:</b> The compression
+ * algorithm is the same for all the blocks in an {@link HFile}. If compression is NONE, this is
+ * just raw, serialized Cells.
  * <li><b>Tail:</b> For minorVersions &gt;=1, a series of 4 byte checksums, one each for
  * the number of bytes specified by bytesPerChecksum.
  * </ul>
- * <p>Be aware that when we read from HDFS, we overread pulling in the next blocks' header too.
- * We do this to save having to do two seeks to read an HFileBlock; a seek to read the header
- * to figure lengths, etc., and then another seek to pull in the data.
+ *
+ * <h3>Caching</h3>
+ * Caches cache whole blocks with trailing checksums if any. We then tag on some metadata, the
+ * content of BLOCK_METADATA_SPACE which will be flag on if we are doing 'hbase'
+ * checksums and then the offset into the file which is needed when we re-make a cache key
+ * when we return the block to the cache as 'done'. See {@link Cacheable#serialize(ByteBuffer)} and
+ * {@link Cacheable#getDeserializer()}.
+ *
+ * <p>TODO: Should we cache the checksums? Down in Writer#getBlockForCaching(CacheConfig) where
+ * we make a block to cache-on-write, there is an attempt at turning off checksums. This is not the
+ * only place we get blocks to cache. We also will cache the raw return from an hdfs read. In this
+ * case, the checksums may be present. If the cache is backed by something that doesn't do ECC,
+ * say an SSD, we might want to preserve checksums. For now this is open question.
+ * <p>TODO: Over in BucketCache, we save a block allocation by doing a custom serialization.
+ * Be sure to change it if serialization changes in here. Could we add a method here that takes an
+ * IOEngine and that then serializes to it rather than expose our internals over in BucketCache?
+ * IOEngine is in the bucket subpackage. Pull it up? Then this class knows about bucketcache. Ugh.
  */
 @InterfaceAudience.Private
 public class HFileBlock implements Cacheable {
   private static final Log LOG = LogFactory.getLog(HFileBlock.class);
 
+  /** Type of block. Header field 0. */
+  private BlockType blockType;
+
+  /**
+   * Size on disk excluding header, including checksum. Header field 1.
+   * @see Writer#putHeader(byte[], int, int, int, int)
+   */
+  private int onDiskSizeWithoutHeader;
+
+  /**
+   * Size of pure data. Does not include header or checksums. Header field 2.
+   * @see Writer#putHeader(byte[], int, int, int, int)
+   */
+  private int uncompressedSizeWithoutHeader;
+
+  /**
+   * The offset of the previous block on disk. Header field 3.
+   * @see Writer#putHeader(byte[], int, int, int, int)
+   */
+  private long prevBlockOffset;
+
+  /**
+   * Size on disk of header + data. Excludes checksum. Header field 6,
+   * OR calculated from {@link #onDiskSizeWithoutHeader} when using HDFS checksum.
+   * @see Writer#putHeader(byte[], int, int, int, int)
+   */
+  private int onDiskDataSizeWithHeader;
+
+
+  /**
+   * The in-memory representation of the hfile block. Can be on or offheap. Can be backed by
+   * a single ByteBuffer or by many. Make no assumptions.
+   *
+   * <p>Be careful reading from this <code>buf</code>. Duplicate and work on the duplicate or if
+   * not, be sure to reset position and limit else trouble down the road.
+   *
+   * <p>TODO: Make this read-only once made.
+   *
+   * <p>We are using the ByteBuff type. ByteBuffer is not extensible yet we need to be able to have
+   * a ByteBuffer-like API across multiple ByteBuffers reading from a cache such as BucketCache.
+   * So, we have this ByteBuff type. Unfortunately, it is spread all about HFileBlock. Would be
+   * good if could be confined to cache-use only but hard-to-do.
+   */
+  private ByteBuff buf;
+
+  /** Meta data that holds meta information on the hfileblock.
+   */
+  private HFileContext fileContext;
+
+  /**
+   * The offset of this block in the file. Populated by the reader for
+   * convenience of access. This offset is not part of the block header.
+   */
+  private long offset = UNSET;
+
+  private MemoryType memType = MemoryType.EXCLUSIVE;
+
+  /**
+   * The on-disk size of the next block, including the header and checksums if present, obtained by
+   * peeking into the first {@link HConstants#HFILEBLOCK_HEADER_SIZE} bytes of the next block's
+   * header, or UNSET if unknown.
+   *
+   * Blocks try to carry the size of the next block to read in this data member. They will even have
+   * this value when served from cache. Could save a seek in the case where we are iterating through
+   * a file and some of the blocks come from cache. If from cache, then having this info to hand
+   * will save us doing a seek to read the header so we can read the body of a block.
+   * TODO: see how effective this is at saving seeks.
+   */
+  private int nextBlockOnDiskSize = UNSET;
+
   /**
    * On a checksum failure, do these many succeeding read requests using hdfs checksums before
    * auto-reenabling hbase checksum verification.
@@ -115,14 +196,18 @@ public class HFileBlock implements Cacheable {
       (int)ClassSize.estimateBase(MultiByteBuff.class, false);
 
   /**
-   * See #blockDeserializer method for more info.
-   * 13 bytes of extra stuff stuck on the end of the HFileBlock that we pull in from HDFS (note,
+   * Space for metadata on a block that gets stored along with the block when we cache it.
+   * There are a few bytes stuck on the end of the HFileBlock that we pull in from HDFS (note,
    * when we read from HDFS, we pull in an HFileBlock AND the header of the next block if one).
-   * The 13 bytes are: usesHBaseChecksum (1 byte) + offset of this block (long) +
-   * nextBlockOnDiskSizeWithHeader (int).
+   * 8 bytes are offset of this block (long) in the file. Offset is important because
+   * used when we remake the CacheKey when we return the block to cache when done. There is also
+   * a flag on whether checksumming is being done by hbase or not. See class comment for note on
+   * uncertain state of checksumming of blocks that come out of cache (should we or should we not?).
+   * Finally there 4 bytes to hold the length of the next block which can save a seek on occasion.
+   * <p>This EXTRA came in with original commit of the bucketcache, HBASE-7404. Was formerly
+   * known as EXTRA_SERIALIZATION_SPACE.
    */
-  public static final int EXTRA_SERIALIZATION_SPACE =
-      Bytes.SIZEOF_BYTE + Bytes.SIZEOF_INT + Bytes.SIZEOF_LONG;
+  static final int BLOCK_METADATA_SPACE = Bytes.SIZEOF_BYTE + Bytes.SIZEOF_LONG + Bytes.SIZEOF_INT;
 
   /**
    * Each checksum value is an integer that can be stored in 4 bytes.
@@ -135,57 +220,47 @@ public class HFileBlock implements Cacheable {
   /**
    * Used deserializing blocks from Cache.
    *
-   * Serializing to cache is a little hard to follow. See Writer#finishBlock for where it is done.
-   * When we start to append to a new HFileBlock,
-   * we skip over where the header should go before we start adding Cells. When the block is
-   * done, we'll then go back and fill in the header and the checksum tail. Be aware that what
-   * gets serialized into the blockcache is a byte array that contains an HFileBlock followed by
-   * its checksums and then the header of the next HFileBlock (needed to help navigate), followed
-   * again by an extra 13 bytes of meta info needed when time to recreate the HFileBlock from cache.
-   *
+   * <code>
    * ++++++++++++++
    * + HFileBlock +
    * ++++++++++++++
-   * + Checksums  +
-   * ++++++++++++++
-   * + NextHeader +
+   * + Checksums  + <= Optional
    * ++++++++++++++
-   * + ExtraMeta! +
+   * + Metadata!  +
    * ++++++++++++++
-   *
-   * TODO: Fix it so we do NOT put the NextHeader into blockcache. It is not necessary.
+   * </code>
+   * @see #serialize(ByteBuffer)
    */
-  static final CacheableDeserializer<Cacheable> blockDeserializer =
+  static final CacheableDeserializer<Cacheable> BLOCK_DESERIALIZER =
       new CacheableDeserializer<Cacheable>() {
         public HFileBlock deserialize(ByteBuff buf, boolean reuse, MemoryType memType)
         throws IOException {
-          // Rewind to just before the EXTRA_SERIALIZATION_SPACE.
-          buf.limit(buf.limit() - HFileBlock.EXTRA_SERIALIZATION_SPACE).rewind();
-          // Get a new buffer to pass the deserialized HFileBlock for it to 'own'.
-          ByteBuff newByteBuffer;
+          // The buf has the file block followed by block metadata.
+          // Set limit to just before the BLOCK_METADATA_SPACE then rewind.
+          buf.limit(buf.limit() - BLOCK_METADATA_SPACE).rewind();
+          // Get a new buffer to pass the HFileBlock for it to 'own'.
+          ByteBuff newByteBuff;
           if (reuse) {
-            newByteBuffer = buf.slice();
+            newByteBuff = buf.slice();
           } else {
             int len = buf.limit();
-            newByteBuffer = new SingleByteBuff(ByteBuffer.allocate(len));
-            newByteBuffer.put(0, buf, buf.position(), len);
+            newByteBuff = new SingleByteBuff(ByteBuffer.allocate(len));
+            newByteBuff.put(0, buf, buf.position(), len);
           }
-          // Read out the EXTRA_SERIALIZATION_SPACE content and shove into our HFileBlock.
+          // Read out the BLOCK_METADATA_SPACE content and shove into our HFileBlock.
           buf.position(buf.limit());
-          buf.limit(buf.limit() + HFileBlock.EXTRA_SERIALIZATION_SPACE);
+          buf.limit(buf.limit() + HFileBlock.BLOCK_METADATA_SPACE);
           boolean usesChecksum = buf.get() == (byte)1;
-          HFileBlock hFileBlock = new HFileBlock(newByteBuffer, usesChecksum, memType);
-          hFileBlock.offset = buf.getLong();
-          hFileBlock.nextBlockOnDiskSizeWithHeader = buf.getInt();
-          if (hFileBlock.hasNextBlockHeader()) {
-            hFileBlock.buf.limit(hFileBlock.buf.limit() - hFileBlock.headerSize());
-          }
+          long offset = buf.getLong();
+          int nextBlockOnDiskSize = buf.getInt();
+          HFileBlock hFileBlock =
+              new HFileBlock(newByteBuff, usesChecksum, memType, offset, nextBlockOnDiskSize, null);
           return hFileBlock;
         }
 
         @Override
         public int getDeserialiserIdentifier() {
-          return deserializerIdentifier;
+          return DESERIALIZER_IDENTIFIER;
         }
 
         @Override
@@ -195,65 +270,36 @@ public class HFileBlock implements Cacheable {
         }
       };
 
-  private static final int deserializerIdentifier;
+  private static final int DESERIALIZER_IDENTIFIER;
   static {
-    deserializerIdentifier = CacheableDeserializerIdManager
-        .registerDeserializer(blockDeserializer);
+    DESERIALIZER_IDENTIFIER =
+        CacheableDeserializerIdManager.registerDeserializer(BLOCK_DESERIALIZER);
   }
 
-  /** Type of block. Header field 0. */
-  private BlockType blockType;
-
-  /**
-   * Size on disk excluding header, including checksum. Header field 1.
-   * @see Writer#putHeader(byte[], int, int, int, int)
-   */
-  private int onDiskSizeWithoutHeader;
-
-  /**
-   * Size of pure data. Does not include header or checksums. Header field 2.
-   * @see Writer#putHeader(byte[], int, int, int, int)
-   */
-  private final int uncompressedSizeWithoutHeader;
-
-  /**
-   * The offset of the previous block on disk. Header field 3.
-   * @see Writer#putHeader(byte[], int, int, int, int)
-   */
-  private final long prevBlockOffset;
-
-  /**
-   * Size on disk of header + data. Excludes checksum. Header field 6,
-   * OR calculated from {@link #onDiskSizeWithoutHeader} when using HDFS checksum.
-   * @see Writer#putHeader(byte[], int, int, int, int)
-   */
-  private final int onDiskDataSizeWithHeader;
-
-  /** The in-memory representation of the hfile block */
-  private ByteBuff buf;
-
-  /** Meta data that holds meta information on the hfileblock */
-  private HFileContext fileContext;
-
   /**
-   * The offset of this block in the file. Populated by the reader for
-   * convenience of access. This offset is not part of the block header.
-   */
-  private long offset = UNSET;
-
-  /**
-   * The on-disk size of the next block, including the header, obtained by
-   * peeking into the first {@link HConstants#HFILEBLOCK_HEADER_SIZE} bytes of the next block's
-   * header, or -1 if unknown.
+   * Copy constructor. Creates a shallow copy of {@code that}'s buffer.
    */
-  private int nextBlockOnDiskSizeWithHeader = UNSET;
-
-  private MemoryType memType = MemoryType.EXCLUSIVE;
+  private HFileBlock(HFileBlock that) {
+    this.blockType = that.blockType;
+    this.onDiskSizeWithoutHeader = that.onDiskSizeWithoutHeader;
+    this.uncompressedSizeWithoutHeader = that.uncompressedSizeWithoutHeader;
+    this.prevBlockOffset = that.prevBlockOffset;
+    this.buf = that.buf.duplicate();
+    this.offset = that.offset;
+    this.onDiskDataSizeWithHeader = that.onDiskDataSizeWithHeader;
+    this.fileContext = that.fileContext;
+    this.nextBlockOnDiskSize = that.nextBlockOnDiskSize;
+  }
 
   /**
    * Creates a new {@link HFile} block from the given fields. This constructor
    * is used when the block data has already been read and uncompressed,
-   * and is sitting in a byte buffer.
+   * and is sitting in a byte buffer and we want to stuff the block into cache.
+   * See {@link Writer#getBlockForCaching(CacheConfig)}.
+   *
+   * <p>TODO: The caller presumes no checksumming
+   * required of this block instance since going into cache; checksum already verified on
+   * underlying block data pulled in from filesystem. Is that correct? What if cache is SSD?
    *
    * @param blockType the type of this block, see {@link BlockType}
    * @param onDiskSizeWithoutHeader see {@link #onDiskSizeWithoutHeader}
@@ -267,86 +313,94 @@ public class HFileBlock implements Cacheable {
    * @param fileContext HFile meta data
    */
   HFileBlock(BlockType blockType, int onDiskSizeWithoutHeader, int uncompressedSizeWithoutHeader,
-      long prevBlockOffset, ByteBuff buf, boolean fillHeader, long offset,
-      int onDiskDataSizeWithHeader, HFileContext fileContext) {
-    this.blockType = blockType;
-    this.onDiskSizeWithoutHeader = onDiskSizeWithoutHeader;
-    this.uncompressedSizeWithoutHeader = uncompressedSizeWithoutHeader;
-    this.prevBlockOffset = prevBlockOffset;
-    this.buf = buf;
-    this.offset = offset;
-    this.onDiskDataSizeWithHeader = onDiskDataSizeWithHeader;
-    this.fileContext = fileContext;
+      long prevBlockOffset, ByteBuffer b, boolean fillHeader, long offset,
+      final int nextBlockOnDiskSize, int onDiskDataSizeWithHeader, HFileContext fileContext) {
+    init(blockType, onDiskSizeWithoutHeader, uncompressedSizeWithoutHeader,
+        prevBlockOffset, offset, onDiskDataSizeWithHeader, nextBlockOnDiskSize, fileContext);
+    this.buf = new SingleByteBuff(b);
     if (fillHeader) {
       overwriteHeader();
     }
     this.buf.rewind();
   }
 
-  HFileBlock(BlockType blockType, int onDiskSizeWithoutHeader, int uncompressedSizeWithoutHeader,
-      long prevBlockOffset, ByteBuffer buf, boolean fillHeader, long offset,
-      int onDiskDataSizeWithHeader, HFileContext fileContext) {
-    this(blockType, onDiskSizeWithoutHeader, uncompressedSizeWithoutHeader, prevBlockOffset,
-        new SingleByteBuff(buf), fillHeader, offset, onDiskDataSizeWithHeader, fileContext);
-  }
-
   /**
-   * Copy constructor. Creates a shallow copy of {@code that}'s buffer.
+   * Creates a block from an existing buffer starting with a header. Rewinds
+   * and takes ownership of the buffer. By definition of rewind, ignores the
+   * buffer position, but if you slice the buffer beforehand, it will rewind
+   * to that point.
+   * @param buf Has header, content, and trailing checksums if present.
    */
-  HFileBlock(HFileBlock that) {
-    this.blockType = that.blockType;
-    this.onDiskSizeWithoutHeader = that.onDiskSizeWithoutHeader;
-    this.uncompressedSizeWithoutHeader = that.uncompressedSizeWithoutHeader;
-    this.prevBlockOffset = that.prevBlockOffset;
-    this.buf = that.buf.duplicate();
-    this.offset = that.offset;
-    this.onDiskDataSizeWithHeader = that.onDiskDataSizeWithHeader;
-    this.fileContext = that.fileContext;
-    this.nextBlockOnDiskSizeWithHeader = that.nextBlockOnDiskSizeWithHeader;
+  HFileBlock(ByteBuff buf, boolean usesHBaseChecksum, MemoryType memType, final long offset,
+      final int nextBlockOnDiskSize, HFileContext fileContext) throws IOException {
+    buf.rewind();
+    final BlockType blockType = BlockType.read(buf);
+    final int onDiskSizeWithoutHeader = buf.getInt();
+    final int uncompressedSizeWithoutHeader = buf.getInt();
+    final long prevBlockOffset = buf.getLong();
+    byte checksumType = buf.get();
+    int bytesPerChecksum = buf.getInt();
+    int onDiskDataSizeWithHeader = buf.getInt();
+    // This constructor is called when we deserialize a block from cache and when we read a block in
+    // from the fs. fileCache is null when deserialized from cache so need to make up one.
+    HFileContextBuilder fileContextBuilder = fileContext != null?
+        new HFileContextBuilder(fileContext): new HFileContextBuilder();
+    fileContextBuilder.withHBaseCheckSum(usesHBaseChecksum);
+    if (usesHBaseChecksum) {
+      // Use the checksum type and bytes per checksum from header, not from filecontext.
+      fileContextBuilder.withChecksumType(ChecksumType.codeToType(checksumType));
+      fileContextBuilder.withBytesPerCheckSum(bytesPerChecksum);
+    } else {
+      fileContextBuilder.withChecksumType(ChecksumType.NULL);
+      fileContextBuilder.withBytesPerCheckSum(0);
+      // Need to fix onDiskDataSizeWithHeader; there are not checksums after-block-data
+      onDiskDataSizeWithHeader = onDiskSizeWithoutHeader + headerSize(usesHBaseChecksum);
+    }
+    fileContext = fileContextBuilder.build();
+    assert usesHBaseChecksum == fileContext.isUseHBaseChecksum();
+    init(blockType, onDiskSizeWithoutHeader, uncompressedSizeWithoutHeader,
+        prevBlockOffset, offset, onDiskDataSizeWithHeader, nextBlockOnDiskSize, fileContext);
+    this.memType = memType;
+    this.offset = offset;
+    this.buf = buf;
+    this.buf.rewind();
   }
 
-  HFileBlock(ByteBuffer b, boolean usesHBaseChecksum) throws IOException {
-    this(new SingleByteBuff(b), usesHBaseChecksum);
+  /**
+   * Called from constructors.
+   */
+  private void init(BlockType blockType, int onDiskSizeWithoutHeader,
+      int uncompressedSizeWithoutHeader, long prevBlockOffset,
+      long offset, int onDiskDataSizeWithHeader, final int nextBlockOnDiskSize,
+      HFileContext fileContext) {
+    this.blockType = blockType;
+    this.onDiskSizeWithoutHeader = onDiskSizeWithoutHeader;
+    this.uncompressedSizeWithoutHeader = uncompressedSizeWithoutHeader;
+    this.prevBlockOffset = prevBlockOffset;
+    this.offset = offset;
+    this.onDiskDataSizeWithHeader = onDiskDataSizeWithHeader;
+    this.nextBlockOnDiskSize = nextBlockOnDiskSize;
+    this.fileContext = fileContext;
   }
 
   /**
-   * Creates a block from an existing buffer starting with a header. Rewinds
-   * and takes ownership of the buffer. By definition of rewind, ignores the
-   * buffer position, but if you slice the buffer beforehand, it will rewind
-   * to that point.
+   * Parse total ondisk size including header and checksum. Its second field in header after
+   * the magic bytes.
+   * @param headerBuf Header ByteBuffer. Presumed exact size of header.
+   * @return Size of the block with header included.
    */
-  HFileBlock(ByteBuff b, boolean usesHBaseChecksum) throws IOException {
-    this(b, usesHBaseChecksum, MemoryType.EXCLUSIVE);
+  private static int getOnDiskSizeWithHeader(final ByteBuffer headerBuf) {
+    // Set hbase checksum to true always calling headerSize.
+    return headerBuf.getInt(BlockType.MAGIC_LENGTH) + headerSize(true);
   }
 
   /**
-   * Creates a block from an existing buffer starting with a header. Rewinds
-   * and takes ownership of the buffer. By definition of rewind, ignores the
-   * buffer position, but if you slice the buffer beforehand, it will rewind
-   * to that point.
+   * @return the on-disk size of the next block (including the header size and any checksums if
+   * present) read by peeking into the next block's header; use as a hint when doing
+   * a read of the next block when scanning or running over a file.
    */
-  HFileBlock(ByteBuff b, boolean usesHBaseChecksum, MemoryType memType) throws IOException {
-    b.rewind();
-    blockType = BlockType.read(b);
-    onDiskSizeWithoutHeader = b.getInt();
-    uncompressedSizeWithoutHeader = b.getInt();
-    prevBlockOffset = b.getLong();
-    HFileContextBuilder contextBuilder = new HFileContextBuilder();
-    contextBuilder.withHBaseCheckSum(usesHBaseChecksum);
-    if (usesHBaseChecksum) {
-      contextBuilder.withChecksumType(ChecksumType.codeToType(b.get()));
-      contextBuilder.withBytesPerCheckSum(b.getInt());
-      this.onDiskDataSizeWithHeader = b.getInt();
-    } else {
-      contextBuilder.withChecksumType(ChecksumType.NULL);
-      contextBuilder.withBytesPerCheckSum(0);
-      this.onDiskDataSizeWithHeader =
-          onDiskSizeWithoutHeader + HConstants.HFILEBLOCK_HEADER_SIZE_NO_CHECKSUM;
-    }
-    this.fileContext = contextBuilder.build();
-    this.memType = memType;
-    buf = b;
-    buf.rewind();
+  public int getNextBlockOnDiskSize() {
+    return nextBlockOnDiskSize;
   }
 
   public BlockType getBlockType() {
@@ -414,49 +468,26 @@ public class HFileBlock implements Cacheable {
    * @return the buffer with header skipped and checksum omitted.
    */
   public ByteBuff getBufferWithoutHeader() {
-    ByteBuff dup = this.buf.duplicate();
-    dup.position(headerSize());
-    dup.limit(buf.limit() - totalChecksumBytes());
-    return dup.slice();
+    ByteBuff dup = getBufferReadOnly();
+    // Now set it up so Buffer spans content only -- no header or no checksums.
+    return dup.position(headerSize()).limit(buf.limit() - totalChecksumBytes()).slice();
   }
 
   /**
-   * Returns the buffer this block stores internally. The clients must not
-   * modify the buffer object. This method has to be public because it is used
+   * Returns a read-only duplicate of the buffer this block stores internally ready to be read.
+   * Clients must not modify the buffer object though they may set position and limit on the
+   * returned buffer since we pass back a duplicate. This method has to be public because it is used
    * in {@link CompoundBloomFilter} to avoid object creation on every Bloom
-   * filter lookup, but has to be used with caution. Checksum data is not
-   * included in the returned buffer but header data is.
+   * filter lookup, but has to be used with caution. Buffer holds header, block content,
+   * and any follow-on checksums if present.
    *
    * @return the buffer of this block for read-only operations
    */
-  ByteBuff getBufferReadOnly() {
-    ByteBuff dup = this.buf.duplicate();
-    dup.limit(buf.limit() - totalChecksumBytes());
-    return dup.slice();
-  }
-
-  /**
-   * Returns the buffer of this block, including header data. The clients must
-   * not modify the buffer object. This method has to be public because it is
-   * used in {@link org.apache.hadoop.hbase.io.hfile.bucket.BucketCache} to avoid buffer copy.
-   *
-   * @return the buffer with header and checksum included for read-only operations
-   */
-  public ByteBuff getBufferReadOnlyWithHeader() {
+  public ByteBuff getBufferReadOnly() {
+    // TODO: ByteBuf does not support asReadOnlyBuffer(). Fix.
     ByteBuff dup = this.buf.duplicate();
-    return dup.slice();
-  }
-
-  /**
-   * Returns a byte buffer of this block, including header data and checksum, positioned at
-   * the beginning of header. The underlying data array is not copied.
-   *
-   * @return the byte buffer with header and checksum included
-   */
-  ByteBuff getBufferWithHeader() {
-    ByteBuff dupBuf = buf.duplicate();
-    dupBuf.rewind();
-    return dupBuf;
+    assert dup.position() == 0;
+    return dup;
   }
 
   private void sanityCheckAssertion(long valueFromBuf, long valueFromField,
@@ -481,39 +512,38 @@ public class HFileBlock implements Cacheable {
    * valid header consistent with the fields. Assumes a packed block structure.
    * This function is primary for testing and debugging, and is not
    * thread-safe, because it alters the internal buffer pointer.
+   * Used by tests only.
    */
+  @VisibleForTesting
   void sanityCheck() throws IOException {
-    buf.rewind();
-
-    sanityCheckAssertion(BlockType.read(buf), blockType);
+    // Duplicate so no side-effects
+    ByteBuff dup = this.buf.duplicate().rewind();
+    sanityCheckAssertion(BlockType.read(dup), blockType);
 
-    sanityCheckAssertion(buf.getInt(), onDiskSizeWithoutHeader,
-        "onDiskSizeWithoutHeader");
+    sanityCheckAssertion(dup.getInt(), onDiskSizeWithoutHeader, "onDiskSizeWithoutHeader");
 
-    sanityCheckAssertion(buf.getInt(), uncompressedSizeWithoutHeader,
+    sanityCheckAssertion(dup.getInt(), uncompressedSizeWithoutHeader,
         "uncompressedSizeWithoutHeader");
 
-    sanityCheckAssertion(buf.getLong(), prevBlockOffset, "prevBlocKOffset");
+    sanityCheckAssertion(dup.getLong(), prevBlockOffset, "prevBlockOffset");
     if (this.fileContext.isUseHBaseChecksum()) {
-      sanityCheckAssertion(buf.get(), this.fileContext.getChecksumType().getCode(), "checksumType");
-      sanityCheckAssertion(buf.getInt(), this.fileContext.getBytesPerChecksum(),
+      sanityCheckAssertion(dup.get(), this.fileContext.getChecksumType().getCode(), "checksumType");
+      sanityCheckAssertion(dup.getInt(), this.fileContext.getBytesPerChecksum(),
           "bytesPerChecksum");
-      sanityCheckAssertion(buf.getInt(), onDiskDataSizeWithHeader, "onDiskDataSizeWithHeader");
+      sanityCheckAssertion(dup.getInt(), onDiskDataSizeWithHeader, "onDiskDataSizeWithHeader");
     }
 
     int cksumBytes = totalChecksumBytes();
     int expectedBufLimit = onDiskDataSizeWithHeader + cksumBytes;
-    if (buf.limit() != expectedBufLimit) {
-      throw new AssertionError("Expected buffer limit " + expectedBufLimit
-          + ", got " + buf.limit());
+    if (dup.limit() != expectedBufLimit) {
+      throw new AssertionError("Expected limit " + expectedBufLimit + ", got " + dup.limit());
     }
 
     // We might optionally allocate HFILEBLOCK_HEADER_SIZE more bytes to read the next
     // block's header, so there are two sensible values for buffer capacity.
     int hdrSize = headerSize();
-    if (buf.capacity() != expectedBufLimit &&
-        buf.capacity() != expectedBufLimit + hdrSize) {
-      throw new AssertionError("Invalid buffer capacity: " + buf.capacity() +
+    if (dup.capacity() != expectedBufLimit && dup.capacity() != expectedBufLimit + hdrSize) {
+      throw new AssertionError("Invalid buffer capacity: " + dup.capacity() +
           ", expected " + expectedBufLimit + " or " + (expectedBufLimit + hdrSize));
     }
   }
@@ -560,30 +590,6 @@ public class HFileBlock implements Cacheable {
   }
 
   /**
-   * Called after reading a block with provided onDiskSizeWithHeader.
-   */
-  private void validateOnDiskSizeWithoutHeader(int expectedOnDiskSizeWithoutHeader)
-  throws IOException {
-    if (onDiskSizeWithoutHeader != expectedOnDiskSizeWithoutHeader) {
-      String dataBegin = null;
-      if (buf.hasArray()) {
-        dataBegin = Bytes.toStringBinary(buf.array(), buf.arrayOffset(), Math.min(32, buf.limit()));
-      } else {
-        ByteBuff bufDup = getBufferReadOnly();
-        byte[] dataBeginBytes = new byte[Math.min(32, bufDup.limit() - bufDup.position())];
-        bufDup.get(dataBeginBytes);
-        dataBegin = Bytes.toStringBinary(dataBeginBytes);
-      }
-      String blockInfoMsg =
-        "Block offset: " + offset + ", data starts with: " + dataBegin;
-      throw new IOException("On-disk size without header provided is "
-          + expectedOnDiskSizeWithoutHeader + ", but block "
-          + "header contains " + onDiskSizeWithoutHeader + ". " +
-          blockInfoMsg);
-    }
-  }
-
-  /**
    * Retrieves the decompressed/decrypted view of this block. An encoded block remains in its
    * encoded structure. Internal structures are shared between instances where applicable.
    */
@@ -607,33 +613,10 @@ public class HFileBlock implements Cacheable {
     ctx.prepareDecoding(unpacked.getOnDiskSizeWithoutHeader(),
       unpacked.getUncompressedSizeWithoutHeader(), unpacked.getBufferWithoutHeader(),
       dup);
-
-    // Preserve the next block's header bytes in the new block if we have them.
-    if (unpacked.hasNextBlockHeader()) {
-      // Both the buffers are limited till checksum bytes and avoid the next block's header.
-      // Below call to copyFromBufferToBuffer() will try positional read/write from/to buffers when
-      // any of the buffer is DBB. So we change the limit on a dup buffer. No copying just create
-      // new BB objects
-      ByteBuff inDup = this.buf.duplicate();
-      inDup.limit(inDup.limit() + headerSize());
-      ByteBuff outDup = unpacked.buf.duplicate();
-      outDup.limit(outDup.limit() + unpacked.headerSize());
-      outDup.put(
-          unpacked.headerSize() + unpacked.uncompressedSizeWithoutHeader
-              + unpacked.totalChecksumBytes(), inDup, this.onDiskDataSizeWithHeader,
-          unpacked.headerSize());
-    }
     return unpacked;
   }
 
   /**
-   * Return true when this buffer includes next block's header.
-   */
-  private boolean hasNextBlockHeader() {
-    return nextBlockOnDiskSizeWithHeader > 0;
-  }
-
-  /**
    * Always allocates a new buffer of the correct size. Copies header bytes
    * from the existing buffer. Does not change header fields.
    * Reserve room to keep checksum bytes too.
@@ -641,8 +624,7 @@ public class HFileBlock implements Cacheable {
   private void allocateBuffer() {
     int cksumBytes = totalChecksumBytes();
     int headerSize = headerSize();
-    int capacityNeeded = headerSize + uncompressedSizeWithoutHeader +
-        cksumBytes + (hasNextBlockHeader() ? headerSize : 0);
+    int capacityNeeded = headerSize + uncompressedSizeWithoutHeader + cksumBytes;
 
     // TODO we need consider allocating offheap here?
     ByteBuffer newBuf = ByteBuffer.allocate(capacityNeeded);
@@ -670,9 +652,8 @@ public class HFileBlock implements Cacheable {
   }
 
   /** An additional sanity-check in case no compression or encryption is being used. */
-  public void assumeUncompressed() throws IOException {
-    if (onDiskSizeWithoutHeader != uncompressedSizeWithoutHeader +
-        totalChecksumBytes()) {
+  public void sanityCheckUncompressedSize() throws IOException {
+    if (onDiskSizeWithoutHeader != uncompressedSizeWithoutHeader + totalChecksumBytes()) {
       throw new IOException("Using no compression but "
           + "onDiskSizeWithoutHeader=" + onDiskSizeWithoutHeader + ", "
           + "uncompressedSizeWithoutHeader=" + uncompressedSizeWithoutHeader
@@ -680,11 +661,14 @@ public class HFileBlock implements Cacheable {
     }
   }
 
-  /** @return the offset of this block in the file it was read from */
+  /**
+   * Cannot be {@link #UNSET}. Must be a legitimate value. Used re-making the {@link CacheKey} when
+   * block is returned to the cache.
+   * @return the offset of this block in the file it was read from
+   */
   long getOffset() {
     if (offset < 0) {
-      throw new IllegalStateException(
-          "HFile block offset not initialized properly");
+      throw new IllegalStateException("HFile block offset not initialized properly");
     }
     return offset;
   }
@@ -744,7 +728,6 @@ public class HFileBlock implements Cacheable {
         // We could not read the "extra data", but that is OK.
         break;
       }
-
       if (ret < 0) {
         throw new IOException("Premature EOF from inputStream (read "
             + "returned " + ret + ", was trying to read " + necessaryLen
@@ -799,14 +782,6 @@ public class HFileBlock implements Cacheable {
   }
 
   /**
-   * @return the on-disk size of the next block (including the header size)
-   *         that was read by peeking into the next block's header
-   */
-  public int getNextBlockOnDiskSizeWithHeader() {
-    return nextBlockOnDiskSizeWithHeader;
-  }
-
-  /**
    * Unified version 2 {@link HFile} block writer. The intended usage pattern
    * is as follows:
    * <ol>
@@ -838,8 +813,8 @@ public class HFileBlock implements Cacheable {
     private HFileBlockDefaultEncodingContext defaultBlockEncodingCtx;
 
     /**
-     * The stream we use to accumulate data in uncompressed format for each
-     * block. We reset this stream at the end of each block and reuse it. The
+     * The stream we use to accumulate data into a block in an uncompressed format.
+     * We reset this stream at the end of each block and reuse it. The
      * header is written as the first {@link HConstants#HFILEBLOCK_HEADER_SIZE} bytes into this
      * stream.
      */
@@ -867,7 +842,7 @@ public class HFileBlock implements Cacheable {
      * if compression is turned on. It also includes the checksum data that
      * immediately follows the block data. (header + data + checksums)
      */
-    private byte[] onDiskBytesWithHeader;
+    private byte[] onDiskBlockBytesWithHeader;
 
     /**
      * The size of the checksum data on disk. It is used only if data is
@@ -884,7 +859,7 @@ public class HFileBlock implements Cacheable {
      * {@link org.apache.hadoop.hbase.HConstants#HFILEBLOCK_HEADER_SIZE}.
      * Does not store checksums.
      */
-    private byte[] uncompressedBytesWithHeader;
+    private byte[] uncompressedBlockBytesWithHeader;
 
     /**
      * Current block's start offset in the {@link HFile}. Set in
@@ -992,18 +967,19 @@ public class HFileBlock implements Cacheable {
       Preconditions.checkState(state != State.INIT,
           "Unexpected state: " + state);
 
-      if (state == State.BLOCK_READY)
+      if (state == State.BLOCK_READY) {
         return;
+      }
 
       // This will set state to BLOCK_READY.
       finishBlock();
     }
 
     /**
-     * An internal method that flushes the compressing stream (if using
-     * compression), serializes the header, and takes care of the separate
-     * uncompressed stream for caching on write, if applicable. Sets block
-     * write state to "block ready".
+     * Finish up writing of the block.
+     * Flushes the compressing stream (if using compression), fills out the header,
+     * does any compression/encryption of bytes to flush out to disk, and manages
+     * the cache on write content, if applicable. Sets block write state to "block ready".
      */
     private void finishBlock() throws IOException {
       if (blockType == BlockType.DATA) {
@@ -1012,41 +988,40 @@ public class HFileBlock implements Cacheable {
         blockType = dataBlockEncodingCtx.getBlockType();
       }
       userDataStream.flush();
-      // This does an array copy, so it is safe to cache this byte array.
+      // This does an array copy, so it is safe to cache this byte array when cache-on-write.
       // Header is still the empty, 'dummy' header that is yet to be filled out.
-      uncompressedBytesWithHeader = baosInMemory.toByteArray();
+      uncompressedBlockBytesWithHeader = baosInMemory.toByteArray();
       prevOffset = prevOffsetByType[blockType.getId()];
 
-      // We need to set state before we can package the block up for
-      // cache-on-write. In a way, the block is ready, but not yet encoded or
-      // compressed.
+      // We need to set state before we can package the block up for cache-on-write. In a way, the
+      // block is ready, but not yet encoded or compressed.
       state = State.BLOCK_READY;
       if (blockType == BlockType.DATA || blockType == BlockType.ENCODED_DATA) {
-        onDiskBytesWithHeader = dataBlockEncodingCtx
-            .compressAndEncrypt(uncompressedBytesWithHeader);
+        onDiskBlockBytesWithHeader = dataBlockEncodingCtx.
+            compressAndEncrypt(uncompressedBlockBytesWithHeader);
       } else {
-        onDiskBytesWithHeader = this.defaultBlockEncodingCtx.
-            compressAndEncrypt(uncompressedBytesWithHeader);
+        onDiskBlockBytesWithHeader = defaultBlockEncodingCtx.
+            compressAndEncrypt(uncompressedBlockBytesWithHeader);
       }
       // Calculate how many bytes we need for checksum on the tail of the block.
       int numBytes = (int) ChecksumUtil.numBytes(
-          onDiskBytesWithHeader.length,
+          onDiskBlockBytesWithHeader.length,
           fileContext.getBytesPerChecksum());
 
       // Put the header for the on disk bytes; header currently is unfilled-out
-      putHeader(onDiskBytesWithHeader, 0,
-          onDiskBytesWithHeader.length + numBytes,
-          uncompressedBytesWithHeader.length, onDiskBytesWithHeader.length);
+      putHeader(onDiskBlockBytesWithHeader, 0,
+          onDiskBlockBytesWithHeader.length + numBytes,
+          uncompressedBlockBytesWithHeader.length, onDiskBlockBytesWithHeader.length);
       // Set the header for the uncompressed bytes (for cache-on-write) -- IFF different from
-      // onDiskBytesWithHeader array.
-      if (onDiskBytesWithHeader != uncompressedBytesWithHeader) {
-        putHeader(uncompressedBytesWithHeader, 0,
-          onDiskBytesWithHeader.length + numBytes,
-          uncompressedBytesWithHeader.length, onDiskBytesWithHeader.length);
+      // onDiskBlockBytesWithHeader array.
+      if (onDiskBlockBytesWithHeader != uncompressedBlockBytesWithHeader) {
+        putHeader(uncompressedBlockBytesWithHeader, 0,
+          onDiskBlockBytesWithHeader.length + numBytes,
+          uncompressedBlockBytesWithHeader.length, onDiskBlockBytesWithHeader.length);
       }
       onDiskChecksum = new byte[numBytes];
       ChecksumUtil.generateChecksums(
-          onDiskBytesWithHeader, 0, onDiskBytesWithHeader.length,
+          onDiskBlockBytesWithHeader, 0, onDiskBlockBytesWithHeader.length,
           onDiskChecksum, 0, fileContext.getChecksumType(), fileContext.getBytesPerChecksum());
     }
 
@@ -1101,7 +1076,7 @@ public class HFileBlock implements Cacheable {
     protected void finishBlockAndWriteHeaderAndData(DataOutputStream out)
       throws IOException {
       ensureBlockReady();
-      out.write(onDiskBytesWithHeader);
+      out.write(onDiskBlockBytesWithHeader);
       out.write(onDiskChecksum);
     }
 
@@ -1120,12 +1095,12 @@ public class HFileBlock implements Cacheable {
       // This is not very optimal, because we are doing an extra copy.
       // But this method is used only by unit tests.
       byte[] output =
-          new byte[onDiskBytesWithHeader.length
+          new byte[onDiskBlockBytesWithHeader.length
               + onDiskChecksum.length];
-      System.arraycopy(onDiskBytesWithHeader, 0, output, 0,
-          onDiskBytesWithHeader.length);
+      System.arraycopy(onDiskBlockBytesWithHeader, 0, output, 0,
+          onDiskBlockBytesWithHeader.length);
       System.arraycopy(onDiskChecksum, 0, output,
-          onDiskBytesWithHeader.length, onDiskChecksum.length);
+          onDiskBlockBytesWithHeader.length, onDiskChecksum.length);
       return output;
     }
 
@@ -1153,7 +1128,7 @@ public class HFileBlock implements Cacheable {
      */
     int getOnDiskSizeWithoutHeader() {
       expectState(State.BLOCK_READY);
-      return onDiskBytesWithHeader.length +
+      return onDiskBlockBytesWithHeader.length +
           onDiskChecksum.length - HConstants.HFILEBLOCK_HEADER_SIZE;
     }
 
@@ -1166,7 +1141,7 @@ public class HFileBlock implements Cacheable {
      */
     int getOnDiskSizeWithHeader() {
       expectState(State.BLOCK_READY);
-      return onDiskBytesWithHeader.length + onDiskChecksum.length;
+      return onDiskBlockBytesWithHeader.length + onDiskChecksum.length;
     }
 
     /**
@@ -1174,7 +1149,7 @@ public class HFileBlock implements Cacheable {
      */
     int getUncompressedSizeWithoutHeader() {
       expectState(State.BLOCK_READY);
-      return uncompressedBytesWithHeader.length - HConstants.HFILEBLOCK_HEADER_SIZE;
+      return uncompressedBlockBytesWithHeader.length - HConstants.HFILEBLOCK_HEADER_SIZE;
     }
 
     /**
@@ -1182,7 +1157,7 @@ public class HFileBlock implements Cacheable {
      */
     int getUncompressedSizeWithHeader() {
       expectState(State.BLOCK_READY);
-      return uncompressedBytesWithHeader.length;
+      return uncompressedBlockBytesWithHeader.length;
     }
 
     /** @return true if a block is being written  */
@@ -1212,7 +1187,7 @@ public class HFileBlock implements Cacheable {
      */
     ByteBuffer getUncompressedBufferWithHeader() {
       expectState(State.BLOCK_READY);
-      return ByteBuffer.wrap(uncompressedBytesWithHeader);
+      return ByteBuffer.wrap(uncompressedBlockBytesWithHeader);
     }
 
     /**
@@ -1225,7 +1200,7 @@ public class HFileBlock implements Cacheable {
      */
     ByteBuffer getOnDiskBufferWithHeader() {
       expectState(State.BLOCK_READY);
-      return ByteBuffer.wrap(onDiskBytesWithHeader);
+      return ByteBuffer.wrap(onDiskBlockBytesWithHeader);
     }
 
     private void expectState(State expectedState) {
@@ -1257,6 +1232,10 @@ public class HFileBlock implements Cacheable {
      * block does not have checksum data even though the header minor
      * version is MINOR_VERSION_WITH_CHECKSUM. This is indicated by setting a
      * 0 value in bytesPerChecksum.
+     *
+     * <p>TODO: Should there be an option where a cache can ask that hbase preserve block
+     * checksums for checking after a block comes out of the cache? Otehrwise, cache is responsible
+     * for blocks being wholesome (ECC memory or if file-backed, it does checksumming).
      */
     HFileBlock getBlockForCaching(CacheConfig cacheConf) {
       HFileContext newContext = new HFileContextBuilder()
@@ -1270,13 +1249,13 @@ public class HFileBlock implements Cacheable {
                                 .withIncludesMvcc(fileContext.isIncludesMvcc())
                                 .withIncludesTags(fileContext.isIncludesTags())
                                 .build();
-      return new HFileBlock(blockType, getOnDiskSizeWithoutHeader(),
+       return new HFileBlock(blockType, getOnDiskSizeWithoutHeader(),
           getUncompressedSizeWithoutHeader(), prevOffset,
-          cacheConf.shouldCacheCompressed(blockType.getCategory()) ?
+          cacheConf.shouldCacheCompressed(blockType.getCategory())?
             getOnDiskBufferWithHeader() :
             getUncompressedBufferWithHeader(),
-          FILL_HEADER, startOffset,
-          onDiskBytesWithHeader.length + onDiskChecksum.length, newContext);
+          FILL_HEADER, startOffset, UNSET,
+          onDiskBlockBytesWithHeader.length + onDiskChecksum.length, newContext);
     }
   }
 
@@ -1322,12 +1301,9 @@ public class HFileBlock implements Cacheable {
      * @param offset
      * @param onDiskSize the on-disk size of the entire block, including all
      *          applicable headers, or -1 if unknown
-     * @param uncompressedSize the uncompressed size of the compressed part of
-     *          the block, or -1 if unknown
      * @return the newly read block
      */
-    HFileBlock readBlockData(long offset, long onDiskSize,
-        int uncompressedSize, boolean pread) throws IOException;
+    HFileBlock readBlockData(long offset, long onDiskSize, boolean pread) throws IOException;
 
     /**
      * Creates a block iterator over the given portion of the {@link HFile}.
@@ -1380,6 +1356,11 @@ public class HFileBlock implements Cacheable {
     /** Default context used when BlockType != {@link BlockType#ENCODED_DATA}. */
     private final HFileBlockDefaultDecodingContext defaultDecodingCtx;
 
+    /**
+     * When we read a block, we overread and pull in the next blocks header too. We will save it
+     * here. If moving serially through the file, we will trip over this caching of the next blocks
+     * header so we won't have to do explicit seek to find next blocks lengths, etc.
+     */
     private ThreadLocal<PrefetchedHeader> prefetchedHeaderForThread =
         new ThreadLocal<PrefetchedHeader>() {
       @Override
@@ -1443,7 +1424,7 @@ public class HFileBlock implements Cacheable {
         public HFileBlock nextBlock() throws IOException {
           if (offset >= endOffset)
             return null;
-          HFileBlock b = readBlockData(offset, -1, -1, false);
+          HFileBlock b = readBlockData(offset, -1, false);
           offset += b.getOnDiskSizeWithHeader();
           return b.unpack(fileContext, owner);
         }
@@ -1463,7 +1444,7 @@ public class HFileBlock implements Cacheable {
 
     /**
      * Does a positional read or a seek and read into the given buffer. Returns
-     * the on-disk size of the next block, or -1 if it could not be determined.
+     * the on-disk size of the next block, or -1 if it could not be read/determined; e.g. EOF.
      *
      * @param dest destination buffer
      * @param destOffset offset into the destination buffer at where to put the bytes we read
@@ -1473,7 +1454,8 @@ public class HFileBlock implements Cacheable {
      * @param pread whether we should do a positional read
      * @param istream The input source of data
      * @return the on-disk size of the next block with header size included, or
-     *         -1 if it could not be determined
+     *         -1 if it could not be determined; if not -1, the <code>dest</code> INCLUDES the
+     *         next header
      * @throws IOException
      */
     protected int readAtOffset(FSDataInputStream istream, byte [] dest, int destOffset, int size,
@@ -1505,16 +1487,16 @@ public class HFileBlock implements Cacheable {
           }
 
           // Try to read the next block header.
-          if (!readWithExtra(istream, dest, destOffset, size, hdrSize))
+          if (!readWithExtra(istream, dest, destOffset, size, hdrSize)) {
             return -1;
+          }
         } finally {
           streamLock.unlock();
         }
       } else {
         // Positional read. Better for random reads; or when the streamLock is already locked.
         int extraSize = peekIntoNextBlock ? hdrSize : 0;
-        if (!positionalReadWithExtra(istream, fileOffset, dest, destOffset,
-            size, extraSize)) {
+        if (!positionalReadWithExtra(istream, fileOffset, dest, destOffset, size, extraSize)) {
           return -1;
         }
       }
@@ -1530,16 +1512,12 @@ public class HFileBlock implements Cacheable {
      * @param offset the offset in the stream to read at
      * @param onDiskSizeWithHeaderL the on-disk size of the block, including
      *          the header, or -1 if unknown
-     * @param uncompressedSize the uncompressed size of the the block. Always
-     *          expected to be -1. This parameter is only used in version 1.
      * @param pread whether to use a positional read
      */
     @Override
-    public HFileBlock readBlockData(long offset, long onDiskSizeWithHeaderL,
-        int uncompressedSize, boolean pread)
+    public HFileBlock readBlockData(long offset, long onDiskSizeWithHeaderL, boolean pread)
     throws IOException {
-
-      // get a copy of the current state of whether to validate
+      // Get a copy of the current state of whether to validate
       // hbase checksums or not for this read call. This is not
       // thread-safe but the one constaint is that if we decide
       // to skip hbase checksum verification then we are
@@ -1548,8 +1526,7 @@ public class HFileBlock implements Cacheable {
       FSDataInputStream is = streamWrapper.getStream(doVerificationThruHBaseChecksum);
 
       HFileBlock blk = readBlockDataInternal(is, offset,
-                         onDiskSizeWithHeaderL,
-                         uncompressedSize, pread,
+                         onDiskSizeWithHeaderL, pread,
                          doVerificationThruHBaseChecksum);
       if (blk == null) {
         HFile.LOG.warn("HBase checksum verification failed for file " +
@@ -1576,8 +1553,7 @@ public class HFileBlock implements Cacheable {
         // a few more than precisely this number.
         is = this.streamWrapper.fallbackToFsChecksum(CHECKSUM_VERIFICATION_NUM_IO_THRESHOLD);
         doVerificationThruHBaseChecksum = false;
-        blk = readBlockDataInternal(is, offset, onDiskSizeWithHeaderL,
-                                    uncompressedSize, pread,
+        blk = readBlockDataInternal(is, offset, onDiskSizeWithHeaderL, pread,
                                     doVerificationThruHBaseChecksum);
         if (blk != null) {
           HFile.LOG.warn("HDFS checksum verification suceeded for file " +
@@ -1605,175 +1581,139 @@ public class HFileBlock implements Cacheable {
     }
 
     /**
+     * @return Check <code>onDiskSizeWithHeaderL</code> size is healthy and then return it as an int
+     * @throws IOException
+     */
+    private static int checkAndGetSizeAsInt(final long onDiskSizeWithHeaderL, final int hdrSize)
+    throws IOException {
+      if ((onDiskSizeWithHeaderL < hdrSize && onDiskSizeWithHeaderL != -1)
+          || onDiskSizeWithHeaderL >= Integer.MAX_VALUE) {
+        throw new IOException("Invalid onDisksize=" + onDiskSizeWithHeaderL
+            + ": expected to be at least " + hdrSize
+            + " and at most " + Integer.MAX_VALUE + ", or -1");
+      }
+      return (int)onDiskSizeWithHeaderL;
+    }
+
+    /**
+     * Check threadlocal cache for this block's header; we usually read it on the tail of reading
+     * the previous block to save a seek. Otherwise, we have to do a seek to read the header before
+     * we can pull in the block.
+     * @return The cached block header or null if not found.
+     * @see #cacheNextBlockHeader(long, byte[], int, int)
+     */
+    private ByteBuffer getCachedHeader(final long offset) {
+      PrefetchedHeader prefetchedHeader = prefetchedHeaderForThread.get();
+      // PrefetchedHeader prefetchedHeader = prefetchedHeaderForThread.get();
+      return prefetchedHeader != null && prefetchedHeader.offset == offset?
+          prefetchedHeader.buf: null;
+    }
+
+    /**
+     * Save away the next blocks header in thread local.
+     * @see #getCachedHeader(long)
+     */
+    private void cacheNextBlockHeader(final long nextBlockOffset,
+        final byte [] header, final int headerOffset, final int headerLength) {
+      PrefetchedHeader prefetchedHeader = prefetchedHeaderForThread.get();
+      prefetchedHeader.offset = nextBlockOffset;
+      System.arraycopy(header, headerOffset, prefetchedHeader.header, 0, headerLength);
+    }
+
+    /**
+     * Verify the passed in onDiskSizeWithHeader aligns with what is in the header else something
+     * is not right.
+     * @throws IOException
+     */
+    private void verifyOnDiskSizeMatchesHeader(final int passedIn, final ByteBuffer headerBuf,
+        final long offset)
+    throws IOException {
+      // Assert size provided aligns with what is in the header
+      int fromHeader = getOnDiskSizeWithHeader(headerBuf);
+      if (passedIn != fromHeader) {
+        throw new IOException("Passed in onDiskSizeWithHeader=" + passedIn + " != " + fromHeader +
+            ", offset=" + offset + ", fileContext=" + this.fileContext);
+      }
+    }
+
+    /**
      * Reads a version 2 block.
      *
      * @param offset the offset in the stream to read at
      * @param onDiskSizeWithHeaderL the on-disk size of the block, including
-     *          the header, or -1 if unknown
-     * @param uncompressedSize the uncompressed size of the the block. Always
-     *          expected to be -1. This parameter is only used in version 1.
+     *          the header and checksums if present or -1 if unknown
      * @param pread whether to use a positional read
      * @param verifyChecksum Whether to use HBase checksums.
      *        If HBase checksum is switched off, then use HDFS checksum.
      * @return the HFileBlock or null if there is a HBase checksum mismatch
      */
     private HFileBlock readBlockDataInternal(FSDataInputStream is, long offset,
-        long onDiskSizeWithHeaderL, int uncompressedSize, boolean pread,
-        boolean verifyChecksum)
+        long onDiskSizeWithHeaderL, boolean pread, boolean verifyChecksum)
     throws IOException {
       if (offset < 0) {
         throw new IOException("Invalid offset=" + offset + " trying to read "
-            + "block (onDiskSize=" + onDiskSizeWithHeaderL
-            + ", uncompressedSize=" + uncompressedSize + ")");
-      }
-
-      if (uncompressedSize != -1) {
-        throw new IOException("Version 2 block reader API does not need " +
-            "the uncompressed size parameter");
+            + "block (onDiskSize=" + onDiskSizeWithHeaderL + ")");
       }
-
-      if ((onDiskSizeWithHeaderL < hdrSize && onDiskSizeWithHeaderL != -1)
-          || onDiskSizeWithHeaderL >= Integer.MAX_VALUE) {
-        throw new IOException("Invalid onDisksize=" + onDiskSizeWithHeaderL
-            + ": expected to be at least " + hdrSize
-            + " and at most " + Integer.MAX_VALUE + ", or -1 (offset="
-            + offset + ", uncompressedSize=" + uncompressedSize + ")");
-      }
-
-      int onDiskSizeWithHeader = (int) onDiskSizeWithHeaderL;
-
-      // See if we can avoid reading the header. This is desirable, because
-      // we will not incur a backward seek operation if we have already
-      // read this block's header as part of the previous read's look-ahead.
-      // And we also want to skip reading the header again if it has already
-      // been read.
-      // TODO: How often does this optimization fire? Has to be same thread so the thread local
-      // is pertinent and we have to be reading next block as in a big scan.
-      ByteBuffer headerBuf = null;
-      PrefetchedHeader prefetchedHeader = prefetchedHeaderForThread.get();
-      boolean preReadHeader = false;
-      if (prefetchedHeader != null && prefetchedHeader.offset == offset) {
-        headerBuf = prefetchedHeader.buf;
-        preReadHeader = true;
+      int onDiskSizeWithHeader = checkAndGetSizeAsInt(onDiskSizeWithHeaderL, hdrSize);
+      ByteBuffer headerBuf = getCachedHeader(offset);
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Reading " + this.fileContext.getHFileName() + " at offset=" + offset +
+          ", pread=" + pread + ", verifyChecksum=" + verifyChecksum + ", cachedHeader=" +
+          headerBuf + ", onDiskSizeWithHeader=" + onDiskSizeWithHeader);
       }
-      // Allocate enough space to fit the next block's header too.
-      int nextBlockOnDiskSize = 0;
-      byte[] onDiskBlock = null;
-
-      HFileBlock b = null;
-      boolean fastPath = false;
-      boolean readHdrOnly = false;
-      if (onDiskSizeWithHeader > 0) {
-        fastPath = true;
-        // We know the total on-disk size. Read the entire block into memory,
-        // then parse the header. This code path is used when
-        // doing a random read operation relying on the block index, as well as
-        // when the client knows the on-disk size from peeking into the next
-        // block's header (e.g. this block's header) when reading the previous
-        // block. This is the faster and more preferable case.
-
-        // Size that we have to skip in case we have already read the header.
-        int preReadHeaderSize = headerBuf == null ? 0 : hdrSize;
-        onDiskBlock = new byte[onDiskSizeWithHeader + hdrSize]; // room for this block plus the
-                                                                // next block's header
-        nextBlockOnDiskSize = readAtOffset(is, onDiskBlock,
-            preReadHeaderSize, onDiskSizeWithHeader - preReadHeaderSize,
-            true, offset + preReadHeaderSize, pread);
-        if (headerBuf != null) {
-          // the header has been read when reading the previous block, copy
-          // to this block's header
-          // headerBuf is HBB
-          assert headerBuf.hasArray();
-          System.arraycopy(headerBuf.array(),
-              headerBuf.arrayOffset(), onDiskBlock, 0, hdrSize);
-        } else {
-          headerBuf = ByteBuffer.wrap(onDiskBlock, 0, hdrSize);
-        }
-        // We know the total on-disk size but not the uncompressed size. Parse the header.
-        try {
-          // TODO: FIX!!! Expensive parse just to get a length
-          b = new HFileBlock(headerBuf, fileContext.isUseHBaseChecksum());
-        } catch (IOException ex) {
-          // Seen in load testing. Provide comprehensive debug info.
-          throw new IOException("Failed to read compressed block at "
-              + offset
-              + ", onDiskSizeWithoutHeader="
-              + onDiskSizeWithHeader
-              + ", preReadHeaderSize="
-              + hdrSize
-              + ", header.length="
-              + prefetchedHeader.header.length
-              + ", header bytes: "
-              + Bytes.toStringBinary(prefetchedHeader.header, 0,
-                  hdrSize), ex);
-        }
-        // if the caller specifies a onDiskSizeWithHeader, validate it.
-        int onDiskSizeWithoutHeader = onDiskSizeWithHeader - hdrSize;
-        assert onDiskSizeWithoutHeader >= 0;
-        b.validateOnDiskSizeWithoutHeader(onDiskSizeWithoutHeader);
-      } else {
-        // Check headerBuf to see if we have read this block's header as part of
-        // reading the previous block. This is an optimization of peeking into
-        // the next block's header (e.g.this block's header) when reading the
-        // previous block. This is the faster and more preferable case. If the
-        // header is already there, don't read the header again.
-
-        // Unfortunately, we still have to do a separate read operation to
-        // read the header.
+      if (onDiskSizeWithHeader <= 0) {
+        // We were not passed the block size. Need to get it from the header. If header was not in
+        // cache, need to seek to pull it in. This latter might happen when we are doing the first
+        // read in a series of reads or a random read, and we don't have access to the block index.
+        // This is costly and should happen very rarely.
         if (headerBuf == null) {
-          readHdrOnly = true;
-          // From the header, determine the on-disk size of the given hfile
-          // block, and read the remaining data, thereby incurring two read
-          // operations. This might happen when we are doing the first read
-          // in a series of reads or a random read, and we don't have access
-          // to the block index. This is costly and should happen very rarely.
           headerBuf = ByteBuffer.allocate(hdrSize);
-          // headerBuf is HBB
-          readAtOffset(is, headerBuf.array(), headerBuf.arrayOffset(),
-              hdrSize, false, offset, pread);
+          readAtOffset(is, headerBuf.array(), headerBuf.arrayOffset(), hdrSize, false,
+              offset, pread);
         }
-        // TODO: FIX!!! Expensive parse just to get a length
-        b = new HFileBlock(headerBuf, fileContext.isUseHBaseChecksum());
-        // onDiskBlock is whole block + header + checksums then extra hdrSize to read next header
-        onDiskBlock = new byte[b.getOnDiskSizeWithHeader() + hdrSize];
-        // headerBuf is HBB. Copy hdr into onDiskBlock
-        System.arraycopy(headerBuf.array(), headerBuf.arrayOffset(), onDiskBlock, 0, hdrSize);
-        nextBlockOnDiskSize = readAtOffset(is, onDiskBlock, hdrSize,
-            b.getOnDiskSizeWithHeader() - hdrSize, true, offset + hdrSize, pread);
-        onDiskSizeWithHeader = b.onDiskSizeWithoutHeader + hdrSize;
-      }
-
-      if (!fileContext.isCompressedOrEncrypted()) {
-        b.assumeUncompressed();
+        onDiskSizeWithHeader = getOnDiskSizeWithHeader(headerBuf);
       }
-
-      if (verifyChecksum && !validateBlockChecksum(b, offset, onDiskBlock, hdrSize)) {
-        return null;             // checksum mismatch
+      int preReadHeaderSize = headerBuf == null? 0 : hdrSize;
+      // Allocate enough space to fit the next block's header too; saves a seek next time through.
+      // onDiskBlock is whole block + header + checksums then extra hdrSize to read next header;
+      // onDiskSizeWithHeader is header, body, and any checksums if present.
+      // TODO: Make this ByteBuffer-based. Will make it easier to go to HDFS with BBPool (offheap).
+      byte[] onDiskBlock = new byte[onDiskSizeWithHeader + hdrSize];
+      int nextBlockOnDiskSize = readAtOffset(is, onDiskBlock, preReadHeaderSize,
+          onDiskSizeWithHeader - preReadHeaderSize, true, offset + preReadHeaderSize, pread);
+      if (headerBuf != null) {
+        // The header has been read when reading the previous block OR in a distinct header-only
+        // read. Copy to this block's header.
+        System.arraycopy(headerBuf.array(), headerBuf.arrayOffset(), onDiskBlock, 0, hdrSize);
+      } else {
+        headerBuf = ByteBuffer.wrap(onDiskBlock, 0, hdrSize);
       }
-
+      // Do a few checks before we go instantiate HFileBlock.
+      assert onDiskSizeWithHeader > this.hdrSize;
+      verifyOnDiskSizeMatchesHeader(onDiskSizeWithHeader, headerBuf, offset);
       // The onDiskBlock will become the headerAndDataBuffer for this block.
       // If nextBlockOnDiskSizeWithHeader is not zero, the onDiskBlock already
-      // contains the header of next block, so no need to set next
-      // block's header in it.
-      b = new HFileBlock(ByteBuffer.wrap(onDiskBlock, 0, onDiskSizeWithHeader),
-        this.fileContext.isUseHBaseChecksum());
-
-      b.nextBlockOnDiskSizeWithHeader = nextBlockOnDiskSize;
-
-      // Set prefetched header
-      if (b.hasNextBlockHeader()) {
-        prefetchedHeader.offset = offset + b.getOnDiskSizeWithHeader();
-        System.arraycopy(onDiskBlock, onDiskSizeWithHeader, prefetchedHeader.header, 0, hdrSize);
+      // contains the header of next block, so no need to set next block's header in it.
+      HFileBlock hFileBlock =
+          new HFileBlock(new SingleByteBuff(ByteBuffer.wrap(onDiskBlock, 0, onDiskSizeWithHeader)),
+              this.fileContext.isUseHBaseChecksum(), MemoryType.EXCLUSIVE, offset,
+              nextBlockOnDiskSize, fileContext);
+      // Run check on uncompressed sizings.
+      if (!fileContext.isCompressedOrEncrypted()) {
+        hFileBlock.sanityCheckUncompressed();
+      }
+      if (verifyChecksum && !validateBlockChecksum(hFileBlock, offset, onDiskBlock, hdrSize)) {
+        return null;
       }
-
-      b.offset = offset;
-      b.fileContext.setIncludesTags(this.fileContext.isIncludesTags());
-      b.fileContext.setIncludesMvcc(this.fileContext.isIncludesMvcc());
       if (LOG.isTraceEnabled()) {
-        LOG.trace("Read preReadHeader=" + preReadHeader + ", fastPath=" + fastPath +
-            ", readHdrOnly=" + readHdrOnly + ", " + b);
+        LOG.trace("Read " + hFileBlock);
+      }
+      // Cache next block header if we read it for the next time through here.
+      if (nextBlockOnDiskSize != -1) {
+        cacheNextBlockHeader(offset + hFileBlock.getOnDiskSizeWithHeader(),
+            onDiskBlock, onDiskSizeWithHeader, hdrSize);
       }
-      return b;
+      return hFileBlock;
     }
 
     @Override
@@ -1819,42 +1759,73 @@ public class HFileBlock implements Cacheable {
     }
   }
 
+  /** An additional sanity-check in case no compression or encryption is being used. */
+  void sanityCheckUncompressed() throws IOException {
+    if (onDiskSizeWithoutHeader != uncompressedSizeWithoutHeader +
+        totalChecksumBytes()) {
+      throw new IOException("Using no compression but "
+          + "onDiskSizeWithoutHeader=" + onDiskSizeWithoutHeader + ", "
+          + "uncompressedSizeWithoutHeader=" + uncompressedSizeWithoutHeader
+          + ", numChecksumbytes=" + totalChecksumBytes());
+    }
+  }
+
+  // Cacheable implementation
   @Override
   public int getSerializedLength() {
     if (buf != null) {
-      // include extra bytes for the next header when it's available.
-      int extraSpace = hasNextBlockHeader() ? headerSize() : 0;
-      return this.buf.limit() + extraSpace + HFileBlock.EXTRA_SERIALIZATION_SPACE;
+      // Include extra bytes for block metadata.
+      return this.buf.limit() + BLOCK_METADATA_SPACE;
     }
     return 0;
   }
 
+  // Cacheable implementation
   @Override
   public void serialize(ByteBuffer destination) {
-    this.buf.get(destination, 0, getSerializedLength() - EXTRA_SERIALIZATION_SPACE);
-    serializeExtraInfo(destination);
+    // BE CAREFUL!! There is a custom version of this serialization over in BucketCache#doDrain.
+    // Make sure any changes in here are reflected over there.
+    this.buf.get(destination, 0, getSerializedLength() - BLOCK_METADATA_SPACE);
+    destination = addMetaData(destination);
+
+    // Make it ready for reading. flip sets position to zero and limit to current position which
+    // is what we want if we do not want to serialize the block plus checksums if present plus
+    // metadata.
+    destination.flip();
+  }
+
+  /**
+   * For use by bucketcache. This exposes internals.
+   */
+  public ByteBuffer getMetaData() {
+    ByteBuffer bb = ByteBuffer.allocate(BLOCK_METADATA_SPACE);
+    bb = addMetaData(bb);
+    bb.flip();
+    return bb;
   }
 
   /**
-   * Write out the content of EXTRA_SERIALIZATION_SPACE. Public so can be accessed by BucketCache.
+   * Adds metadata at current position (position is moved forward). Does not flip or reset.
+   * @return The passed <code>destination</code> with metadata added.
    */
-  public void serializeExtraInfo(ByteBuffer destination) {
+  private ByteBuffer addMetaData(final ByteBuffer destination) {
     destination.put(this.fileContext.isUseHBaseChecksum() ? (byte) 1 : (byte) 0);
     destination.putLong(this.offset);
-    destination.putInt(this.nextBlockOnDiskSizeWithHeader);
-    destination.rewind();
+    destination.putInt(this.nextBlockOnDiskSize);
+    return destination;
   }
 
+  // Cacheable implementation
   @Override
   public CacheableDeserializer<Cacheable> getDeserializer() {
-    return HFileBlock.blockDeserializer;
+    return HFileBlock.BLOCK_DESERIALIZER;
   }
 
   @Override
   public int hashCode() {
     int result = 1;
     result = result * 31 + blockType.hashCode();
-    result = result * 31 + nextBlockOnDiskSizeWithHeader;
+    result = result * 31 + nextBlockOnDiskSize;
     result = result * 31 + (int) (offset ^ (offset >>> 32));
     result = result * 31 + onDiskSizeWithoutHeader;
     result = result * 31 + (int) (prevBlockOffset ^ (prevBlockOffset >>> 32));
@@ -1880,9 +1851,10 @@ public class HFileBlock implements Cacheable {
     if (castedComparison.blockType != this.blockType) {
       return false;
     }
-    if (castedComparison.nextBlockOnDiskSizeWithHeader != this.nextBlockOnDiskSizeWithHeader) {
+    if (castedComparison.nextBlockOnDiskSize != this.nextBlockOnDiskSize) {
       return false;
     }
+    // Offset is important. Needed when we have to remake cachekey when block is returned to cache.
     if (castedComparison.offset != this.offset) {
       return false;
     }
@@ -1968,7 +1940,7 @@ public class HFileBlock implements Cacheable {
   }
 
   /**
-   * @return the HFileContext used to create this HFileBlock. Not necessary the
+   * @return This HFileBlocks fileContext which will a derivative of the
    * fileContext for the file from which this block's data was originally read.
    */
   HFileContext getHFileContext() {
@@ -1992,6 +1964,7 @@ public class HFileBlock implements Cacheable {
    * This is mostly helpful for debugging. This assumes that the block
    * has minor version > 0.
    */
+  @VisibleForTesting
   static String toStringHeader(ByteBuff buf) throws IOException {
     byte[] magicBuf = new byte[Math.min(buf.limit() - buf.position(), BlockType.MAGIC_LENGTH)];
     buf.get(magicBuf);

http://git-wip-us.apache.org/repos/asf/hbase/blob/12f66e30/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
index 9f29f97..506f08d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
@@ -60,7 +60,7 @@ import org.apache.hadoop.util.StringUtils;
  * Examples of how to use the block index writer can be found in
  * {@link org.apache.hadoop.hbase.io.hfile.CompoundBloomFilterWriter} and
  *  {@link HFileWriterImpl}. Examples of how to use the reader can be
- *  found in {@link HFileWriterImpl} and
+ *  found in {@link HFileReaderImpl} and
  *  {@link org.apache.hadoop.hbase.io.hfile.TestHFileBlockIndex}.
  */
 @InterfaceAudience.Private

http://git-wip-us.apache.org/repos/asf/hbase/blob/12f66e30/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
index 8f5040e..d71911f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
@@ -252,18 +252,20 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
           long end = 0;
           try {
             end = getTrailer().getLoadOnOpenDataOffset();
-            HFileBlock prevBlock = null;
             if (LOG.isTraceEnabled()) {
               LOG.trace("File=" + path.toString() + ", offset=" + offset + ", end=" + end);
             }
+            // TODO: Could we use block iterator in here? Would that get stuff into the cache?
+            HFileBlock prevBlock = null;
             while (offset < end) {
               if (Thread.interrupted()) {
                 break;
               }
-              long onDiskSize = -1;
-              if (prevBlock != null) {
-                onDiskSize = prevBlock.getNextBlockOnDiskSizeWithHeader();
-              }
+              // Perhaps we got our block from cache? Unlikely as this may be, if it happens, then
+              // the internal-to-hfileblock thread local which holds the overread that gets the
+              // next header, will not have happened...so, pass in the onDiskSize gotten from the
+              // cached block. This 'optimization' triggers extremely rarely I'd say.
+              long onDiskSize = prevBlock != null? prevBlock.getNextBlockOnDiskSize(): -1;
               HFileBlock block = readBlock(offset, onDiskSize, true, false, false, false,
                 null, null);
               // Need not update the current block. Ideally here the readBlock won't find the
@@ -903,9 +905,8 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
 
         // We are reading the next block without block type validation, because
         // it might turn out to be a non-data block.
-        block = reader.readBlock(block.getOffset()
-            + block.getOnDiskSizeWithHeader(),
-            block.getNextBlockOnDiskSizeWithHeader(), cacheBlocks, pread,
+        block = reader.readBlock(block.getOffset() + block.getOnDiskSizeWithHeader(),
+            block.getNextBlockOnDiskSize(), cacheBlocks, pread,
             isCompaction, true, null, getEffectiveDataBlockEncoding());
         if (block != null && !block.getBlockType().isData()) { // Findbugs: NP_NULL_ON_SOME_PATH
           // Whatever block we read we will be returning it unless
@@ -1439,8 +1440,8 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
         // Cache Miss, please load.
       }
 
-      HFileBlock metaBlock = fsBlockReader.readBlockData(metaBlockOffset,
-          blockSize, -1, true).unpack(hfileContext, fsBlockReader);
+      HFileBlock metaBlock = fsBlockReader.readBlockData(metaBlockOffset, blockSize, true).
+          unpack(hfileContext, fsBlockReader);
 
       // Cache the block
       if (cacheBlock) {
@@ -1526,8 +1527,8 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
           traceScope.getSpan().addTimelineAnnotation("blockCacheMiss");
         }
         // Load block from filesystem.
-        HFileBlock hfileBlock = fsBlockReader.readBlockData(dataBlockOffset, onDiskBlockSize, -1,
-            pread);
+        HFileBlock hfileBlock =
+            fsBlockReader.readBlockData(dataBlockOffset, onDiskBlockSize, pread);
         validateBlockType(hfileBlock, expectedBlockType);
         HFileBlock unpacked = hfileBlock.unpack(hfileContext, fsBlockReader);
         BlockType.BlockCategory category = hfileBlock.getBlockType().getCategory();
@@ -1871,6 +1872,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
    * @return Scanner on this file.
    */
   @Override
+  @VisibleForTesting
   public HFileScanner getScanner(boolean cacheBlocks, final boolean pread) {
     return getScanner(cacheBlocks, pread, false);
   }


[16/50] [abbrv] hbase git commit: Revert "HBASE-15392 Single Cell Get reads two HFileBlocks M hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java moreRowsMayExistAfterCell Exploit the fact a Scan is a Get Scan. Also save

Posted by sy...@apache.org.
Revert "HBASE-15392 Single Cell Get reads two HFileBlocks M hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java moreRowsMayExistAfterCell Exploit the fact a Scan is a Get Scan. Also save compares if no non-default stopRow."

Revert mistaken commit

This reverts commit 7073f69993e446c2a63308f7827639d675a2da58.


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

Branch: refs/heads/hbase-12439
Commit: a13d6e000ddef2c524b641e04f7969950b2b1811
Parents: c49d0ca
Author: stack <st...@apache.org>
Authored: Tue Mar 22 18:39:22 2016 -0700
Committer: stack <st...@apache.org>
Committed: Tue Mar 22 18:39:22 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/CellComparator.java |   2 +-
 .../hbase/io/hfile/CombinedBlockCache.java      |  13 +-
 .../hadoop/hbase/io/hfile/HFileWriterImpl.java  |   2 +-
 .../hbase/regionserver/KeyValueScanner.java     |  10 +-
 .../hbase/regionserver/ScanQueryMatcher.java    |  60 ++---
 .../hadoop/hbase/regionserver/StoreScanner.java |  87 +------
 .../hbase/util/CollectionBackedScanner.java     |   3 +-
 .../hbase/regionserver/KeyValueScanFixture.java |   8 +-
 .../regionserver/TestKeyValueScanFixture.java   |   3 +-
 .../hbase/regionserver/TestStoreScanner.java    | 244 +------------------
 10 files changed, 62 insertions(+), 370 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a13d6e00/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 4a5c0b7..a5e26cf 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
@@ -433,7 +433,7 @@ public class CellComparator implements Comparator<Cell>, Serializable {
   /**
    * Used to compare two cells based on the column hint provided. This is specifically
    * used when we need to optimize the seeks based on the next indexed key. This is an
-   * advanced usage API specifically needed for some optimizations.
+   * advance usage API specifically needed for some optimizations.
    * @param nextIndexedCell the next indexed cell 
    * @param currentCell the cell to be compared
    * @param foff the family offset of the currentCell

http://git-wip-us.apache.org/repos/asf/hbase/blob/a13d6e00/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java
index 666b357..22bffee 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java
@@ -63,8 +63,8 @@ public class CombinedBlockCache implements ResizableBlockCache, HeapSize {
   @Override
   public void cacheBlock(BlockCacheKey cacheKey, Cacheable buf, boolean inMemory,
       final boolean cacheDataInL1) {
-    boolean metaBlock = buf.getBlockType().getCategory() != BlockCategory.DATA;
-    if (metaBlock || cacheDataInL1) {
+    boolean isMetaBlock = buf.getBlockType().getCategory() != BlockCategory.DATA;
+    if (isMetaBlock || cacheDataInL1) {
       lruCache.cacheBlock(cacheKey, buf, inMemory, cacheDataInL1);
     } else {
       l2Cache.cacheBlock(cacheKey, buf, inMemory, false);
@@ -81,9 +81,12 @@ public class CombinedBlockCache implements ResizableBlockCache, HeapSize {
       boolean repeat, boolean updateCacheMetrics) {
     // TODO: is there a hole here, or just awkwardness since in the lruCache getBlock
     // we end up calling l2Cache.getBlock.
-    return lruCache.containsBlock(cacheKey)?
-        lruCache.getBlock(cacheKey, caching, repeat, updateCacheMetrics):
-        l2Cache.getBlock(cacheKey, caching, repeat, updateCacheMetrics);
+    if (lruCache.containsBlock(cacheKey)) {
+      return lruCache.getBlock(cacheKey, caching, repeat, updateCacheMetrics);
+    }
+    Cacheable result = l2Cache.getBlock(cacheKey, caching, repeat, updateCacheMetrics);
+
+    return result;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/a13d6e00/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
index 9ab46cf..d310d13 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
@@ -416,7 +416,7 @@ public class HFileWriterImpl implements HFile.Writer {
     // No opportunity for optimization. Just return right key.
     return right;
   }
-
+  
   /**
    * @param leftArray
    * @param leftOffset

http://git-wip-us.apache.org/repos/asf/hbase/blob/a13d6e00/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java
index e26022e..eae713f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java
@@ -29,9 +29,6 @@ import org.apache.hadoop.hbase.client.Scan;
  * Scanner that returns the next KeyValue.
  */
 @InterfaceAudience.Private
-// TODO: Change name from KeyValueScanner to CellScanner only we already have a simple CellScanner
-// so this should be something else altogether, a decoration on our base CellScanner. TODO.
-// This class shows in CPs so do it all in one swell swoop. HBase-2.0.0.
 public interface KeyValueScanner extends Shipper {
   /**
    * The byte array represents for NO_NEXT_INDEXED_KEY;
@@ -164,9 +161,8 @@ public interface KeyValueScanner extends Shipper {
   public boolean seekToLastRow() throws IOException;
 
   /**
-   * @return the next key in the index, usually the first key of next block OR a key that falls
-   * between last key of current block and first key of next block..
-   * see HFileWriterImpl#getMidpoint, or null if not known.
+   * @return the next key in the index (the key to seek to the next block)
+   * if known, or null otherwise
    */
   public Cell getNextIndexedKey();
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a13d6e00/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java
index 706fc5b..c220b5c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java
@@ -93,7 +93,7 @@ public class ScanQueryMatcher {
   /* row is not private for tests */
   /** Row the query is on */
   Cell curCell;
-
+  
   /**
    * Oldest put in any of the involved store files
    * Used to decide whether it is ok to delete
@@ -119,7 +119,7 @@ public class ScanQueryMatcher {
    * first column.
    * */
   private boolean hasNullColumn = true;
-
+  
   private RegionCoprocessorHost regionCoprocessorHost= null;
 
   // By default, when hbase.hstore.time.to.purge.deletes is 0ms, a delete
@@ -140,22 +140,22 @@ public class ScanQueryMatcher {
   // currently influencing. This is because Puts, that this delete can
   // influence.  may appear out of order.
   private final long timeToPurgeDeletes;
-
+  
   private final boolean isUserScan;
 
   private final boolean isReversed;
 
   /**
-   * True if we are doing a 'Get' Scan. Every Get is actually a one-row Scan.
-   */
-  private final boolean get;
-
-  /**
    * Construct a QueryMatcher for a scan
+   * @param scan
    * @param scanInfo The store's immutable scan info
+   * @param columns
    * @param scanType Type of the scan
    * @param earliestPutTs Earliest put seen in any of the store files.
-   * @param oldestUnexpiredTS the oldest timestamp we are interested in, based on TTL
+   * @param oldestUnexpiredTS the oldest timestamp we are interested in,
+   *  based on TTL
+   * @param regionCoprocessorHost 
+   * @throws IOException 
    */
   public ScanQueryMatcher(Scan scan, ScanInfo scanInfo, NavigableSet<byte[]> columns,
       ScanType scanType, long readPointToUse, long earliestPutTs, long oldestUnexpiredTS,
@@ -166,7 +166,6 @@ public class ScanQueryMatcher {
     } else {
       this.tr = timeRange;
     }
-    this.get = scan.isGetScan();
     this.rowComparator = scanInfo.getComparator();
     this.regionCoprocessorHost = regionCoprocessorHost;
     this.deletes =  instantiateDeleteTracker();
@@ -235,8 +234,8 @@ public class ScanQueryMatcher {
    * @param now the current server time
    * @param dropDeletesFromRow The inclusive left bound of the range; can be EMPTY_START_ROW.
    * @param dropDeletesToRow The exclusive right bound of the range; can be EMPTY_END_ROW.
-   * @param regionCoprocessorHost
-   * @throws IOException
+   * @param regionCoprocessorHost 
+   * @throws IOException 
    */
   public ScanQueryMatcher(Scan scan, ScanInfo scanInfo, NavigableSet<byte[]> columns,
       long readPointToUse, long earliestPutTs, long oldestUnexpiredTS, long now,
@@ -281,7 +280,7 @@ public class ScanQueryMatcher {
    *      caused by a data corruption.
    */
   public MatchCode match(Cell cell) throws IOException {
-    if (filter != null && filter.filterAllRemaining()) {
+      if (filter != null && filter.filterAllRemaining()) {
       return MatchCode.DONE_SCAN;
     }
     if (curCell != null) {
@@ -325,7 +324,7 @@ public class ScanQueryMatcher {
     // check if the cell is expired by cell TTL
     if (HStore.isCellTTLExpired(cell, this.oldestUnexpiredTS, this.now)) {
       return MatchCode.SKIP;
-    }
+    }    
 
     /*
      * The delete logic is pretty complicated now.
@@ -360,10 +359,10 @@ public class ScanQueryMatcher {
         }
         // Can't early out now, because DelFam come before any other keys
       }
-
+     
       if ((!isUserScan)
           && timeToPurgeDeletes > 0
-          && (EnvironmentEdgeManager.currentTime() - timestamp)
+          && (EnvironmentEdgeManager.currentTime() - timestamp) 
             <= timeToPurgeDeletes) {
         return MatchCode.INCLUDE;
       } else if (retainDeletesInOutput || mvccVersion > maxReadPointToTrackVersions) {
@@ -504,27 +503,22 @@ public class ScanQueryMatcher {
     }
   }
 
-  /**
-   * @return Returns false if we know there are no more rows to be scanned (We've reached the
-   * <code>stopRow</code> or we are scanning on row only because this Scan is for a Get, etc.
-   */
   public boolean moreRowsMayExistAfter(Cell kv) {
-    // If a 'get' Scan -- we are doing a Get (every Get is a single-row Scan in implementation) --
-    // then we are looking at one row only, the one specified in the Get coordinate..so we know
-    // for sure that there are no more rows on this Scan
-    if (this.get) {
-      return false;
+    if (this.isReversed) {
+      if (rowComparator.compareRows(kv, stopRow, 0, stopRow.length) <= 0) {
+        return false;
+      } else {
+        return true;
+      }
     }
-    // If no stopRow, return that there may be more rows. The tests that follow depend on a
-    // non-empty, non-default stopRow so this little test below short-circuits out doing the
-    // following compares.
-    if (this.stopRow == null || this.stopRow == HConstants.EMPTY_BYTE_ARRAY) {
+    if (!Bytes.equals(stopRow , HConstants.EMPTY_END_ROW) &&
+        rowComparator.compareRows(kv, stopRow, 0, stopRow.length) >= 0) {
+      // KV >= STOPROW
+      // then NO there is nothing left.
+      return false;
+    } else {
       return true;
     }
-    return this.isReversed?
-      rowComparator.compareRows(kv, stopRow, 0, stopRow.length) > 0:
-      Bytes.equals(stopRow, HConstants.EMPTY_END_ROW) ||
-        rowComparator.compareRows(kv, stopRow, 0, stopRow.length) < 0;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a13d6e00/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
index 66f846a..2f0d284 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
@@ -133,7 +133,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
   protected List<KeyValueScanner> currentScanners = new ArrayList<KeyValueScanner>();
   // flush update lock
   private ReentrantLock flushLock = new ReentrantLock();
-
+  
   protected final long readPt;
 
   // used by the injection framework to test race between StoreScanner construction and compaction
@@ -600,12 +600,6 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
         continue;
 
       case DONE:
-        // Optimization for Gets! If DONE, no more to get on this row, early exit!
-        if (this.scan.isGetScan()) {
-          // Then no more to this row... exit.
-          close(false);// Do all cleanup except heap.close()
-          return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
-        }
         matcher.curCell = null;
         return scannerContext.setScannerState(NextState.MORE_VALUES).hasMoreValues();
 
@@ -655,67 +649,18 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
     return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
   }
 
-  /**
-   * See if we should actually SEEK or rather just SKIP to the next Cell (see HBASE-13109).
-   * This method works together with ColumnTrackers and Filters. ColumnTrackers may issue SEEK
-   * hints, such as seek to next column, next row, or seek to an arbitrary seek key.
-   * This method intercepts these qcodes and decides whether a seek is the most efficient _actual_
-   * way to get us to the requested cell (SEEKs are more expensive than SKIP, SKIP, SKIP inside the
-   * current, loaded block).
-   * It does this by looking at the next indexed key of the current HFile. This key
-   * is then compared with the _SEEK_ key, where a SEEK key is an artificial 'last possible key
-   * on the row' (only in here, we avoid actually creating a SEEK key; in the compare we work with
-   * the current Cell but compare as though it were a seek key; see down in
-   * matcher.compareKeyForNextRow, etc). If the compare gets us onto the
-   * next block we *_SEEK, otherwise we just INCLUDE or SKIP, and let the ColumnTrackers or Filters
-   * go through the next Cell, and so on)
-   *
-   * <p>The ColumnTrackers and Filters must behave correctly in all cases, i.e. if they are past the
-   * Cells they care about they must issues a SKIP or SEEK.
-   *
-   * <p>Other notes:
-   * <ul>
-   * <li>Rows can straddle block boundaries</li>
-   * <li>Versions of columns can straddle block boundaries (i.e. column C1 at T1 might be in a
-   * different block than column C1 at T2)</li>
-   * <li>We want to SKIP and INCLUDE if the chance is high that we'll find the desired Cell after a
-   * few SKIPs...</li>
-   * <li>We want to INCLUDE_AND_SEEK and SEEK when the chance is high that we'll be able to seek
-   * past many Cells, especially if we know we need to go to the next block.</li>
-   * </ul>
-   * <p>A good proxy (best effort) to determine whether INCLUDE/SKIP is better than SEEK is whether
-   * we'll likely end up seeking to the next block (or past the next block) to get our next column.
-   * Example:
-   * <pre>
-   * |    BLOCK 1              |     BLOCK 2                   |
-   * |  r1/c1, r1/c2, r1/c3    |    r1/c4, r1/c5, r2/c1        |
-   *                                   ^         ^
-   *                                   |         |
-   *                           Next Index Key   SEEK_NEXT_ROW (before r2/c1)
-   *
-   *
-   * |    BLOCK 1                       |     BLOCK 2                      |
-   * |  r1/c1/t5, r1/c1/t4, r1/c1/t3    |    r1/c1/t2, r1/c1/T1, r1/c2/T3  |
-   *                                            ^              ^
-   *                                            |              |
-   *                                    Next Index Key        SEEK_NEXT_COL
-   * </pre>
-   * Now imagine we want columns c1 and c3 (see first diagram above), the 'Next Index Key' of r1/c4
-   * is > r1/c3 so we should seek to get to the c1 on the next row, r2. In second case, say we only
-   * want one version of c1, after we have it, a SEEK_COL will be issued to get to c2. Looking at
-   * the 'Next Index Key', it would land us in the next block, so we should SEEK. In other scenarios
-   * where the SEEK will not land us in the next block, it is very likely better to issues a series
-   * of SKIPs.
+  /*
+   * See if we should actually SEEK or rather just SKIP to the next Cell.
+   * (see HBASE-13109)
    */
-  @VisibleForTesting
-  protected ScanQueryMatcher.MatchCode optimize(ScanQueryMatcher.MatchCode qcode, Cell cell) {
+  private ScanQueryMatcher.MatchCode optimize(ScanQueryMatcher.MatchCode qcode, Cell cell) {
     switch(qcode) {
     case INCLUDE_AND_SEEK_NEXT_COL:
     case SEEK_NEXT_COL:
     {
       Cell nextIndexedKey = getNextIndexedKey();
       if (nextIndexedKey != null && nextIndexedKey != KeyValueScanner.NO_NEXT_INDEXED_KEY
-          && matcher.compareKeyForNextColumn(nextIndexedKey, cell) > 0) {
+          && matcher.compareKeyForNextColumn(nextIndexedKey, cell) >= 0) {
         return qcode == MatchCode.SEEK_NEXT_COL ? MatchCode.SKIP : MatchCode.INCLUDE;
       }
       break;
@@ -723,16 +668,10 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
     case INCLUDE_AND_SEEK_NEXT_ROW:
     case SEEK_NEXT_ROW:
     {
-      // If it is a Get Scan, then we know that we are done with this row; there are no more
-      // rows beyond the current one: don't try to optimize. We are DONE. Return the *_NEXT_ROW
-      // qcode as is. When the caller gets these flags on a Get Scan, it knows it can shut down the
-      // Scan.
-      if (!this.scan.isGetScan()) {
-        Cell nextIndexedKey = getNextIndexedKey();
-        if (nextIndexedKey != null && nextIndexedKey != KeyValueScanner.NO_NEXT_INDEXED_KEY
-            && matcher.compareKeyForNextRow(nextIndexedKey, cell) > 0) {
-          return qcode == MatchCode.SEEK_NEXT_ROW ? MatchCode.SKIP : MatchCode.INCLUDE;
-        }
+      Cell nextIndexedKey = getNextIndexedKey();
+      if (nextIndexedKey != null && nextIndexedKey != KeyValueScanner.NO_NEXT_INDEXED_KEY
+          && matcher.compareKeyForNextRow(nextIndexedKey, cell) >= 0) {
+        return qcode == MatchCode.SEEK_NEXT_ROW ? MatchCode.SKIP : MatchCode.INCLUDE;
       }
       break;
     }
@@ -870,10 +809,10 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
     // check the var without any lock. Suppose even if we see the old
     // value here still it is ok to continue because we will not be resetting
     // the heap but will continue with the referenced memstore's snapshot. For compactions
-    // any way we don't need the updateReaders at all to happen as we still continue with
+    // any way we don't need the updateReaders at all to happen as we still continue with 
     // the older files
     if (flushed) {
-      // If there is a flush and the current scan is notified on the flush ensure that the
+      // If there is a flush and the current scan is notified on the flush ensure that the 
       // scan's heap gets reset and we do a seek on the newly flushed file.
       if(!this.closing) {
         this.lastTop = this.peek();
@@ -903,7 +842,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
     if (scanners.isEmpty()) return;
     int storeFileScannerCount = scanners.size();
     CountDownLatch latch = new CountDownLatch(storeFileScannerCount);
-    List<ParallelSeekHandler> handlers =
+    List<ParallelSeekHandler> handlers = 
         new ArrayList<ParallelSeekHandler>(storeFileScannerCount);
     for (KeyValueScanner scanner : scanners) {
       if (scanner instanceof StoreFileScanner) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a13d6e00/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CollectionBackedScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CollectionBackedScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CollectionBackedScanner.java
index 4720880..9fc068f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CollectionBackedScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CollectionBackedScanner.java
@@ -30,7 +30,8 @@ import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.regionserver.NonReversedNonLazyKeyValueScanner;
 
 /**
- * Utility scanner that wraps a sortable collection and serves as a KeyValueScanner.
+ * Utility scanner that wraps a sortable collection and serves
+ * as a KeyValueScanner.
  */
 @InterfaceAudience.Private
 public class CollectionBackedScanner extends NonReversedNonLazyKeyValueScanner {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a13d6e00/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/KeyValueScanFixture.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/KeyValueScanFixture.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/KeyValueScanFixture.java
index a4e7f9b..3f87a00 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/KeyValueScanFixture.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/KeyValueScanFixture.java
@@ -22,7 +22,6 @@ package org.apache.hadoop.hbase.regionserver;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.util.CollectionBackedScanner;
@@ -34,8 +33,9 @@ import org.apache.hadoop.hbase.util.CollectionBackedScanner;
  * to be a store file scanner.
  */
 public class KeyValueScanFixture extends CollectionBackedScanner {
-  public KeyValueScanFixture(CellComparator comparator, Cell... cells) {
-    super(comparator, cells);
+  public KeyValueScanFixture(CellComparator comparator,
+                             KeyValue... incData) {
+    super(comparator, incData);
   }
 
   public static List<KeyValueScanner> scanFixture(KeyValue[] ... kvArrays) {
@@ -45,4 +45,4 @@ public class KeyValueScanFixture extends CollectionBackedScanner {
     }
     return scanners;
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a13d6e00/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeyValueScanFixture.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeyValueScanFixture.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeyValueScanFixture.java
index 0e96682..a8c2c65 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeyValueScanFixture.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeyValueScanFixture.java
@@ -46,7 +46,8 @@ public class TestKeyValueScanFixture extends TestCase {
         KeyValueTestUtil.create("RowB", "family", "qf1",
             10, KeyValue.Type.Put, "value-10")
     };
-    KeyValueScanner scan = new KeyValueScanFixture(CellComparator.COMPARATOR, kvs);
+    KeyValueScanner scan = new KeyValueScanFixture(
+        CellComparator.COMPARATOR, kvs);
 
     KeyValue kv = KeyValueUtil.createFirstOnRow(Bytes.toBytes("RowA"));
     // should seek to this:

http://git-wip-us.apache.org/repos/asf/hbase/blob/a13d6e00/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java
index 4c594b0..92c85aa 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java
@@ -20,7 +20,6 @@
 package org.apache.hadoop.hbase.regionserver;
 
 import static org.apache.hadoop.hbase.regionserver.KeyValueScanFixture.scanFixture;
-import static org.junit.Assert.*;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -28,21 +27,16 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.NavigableSet;
 import java.util.TreeSet;
-import java.util.concurrent.atomic.AtomicInteger;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CategoryBasedTimeout;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
-import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeepDeletedCells;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueTestUtil;
-import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
@@ -59,113 +53,16 @@ import org.junit.rules.TestRule;
 // Can't be small as it plays with EnvironmentEdgeManager
 @Category({RegionServerTests.class, MediumTests.class})
 public class TestStoreScanner {
-  private static final Log LOG = LogFactory.getLog(TestStoreScanner.class);
   @Rule public TestName name = new TestName();
   @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()).
       withLookingForStuckThread(true).build();
   private static final String CF_STR = "cf";
-  private static final byte [] CF = Bytes.toBytes(CF_STR);
+  final byte [] CF = Bytes.toBytes(CF_STR);
   static Configuration CONF = HBaseConfiguration.create();
   private ScanInfo scanInfo = new ScanInfo(CONF, CF, 0, Integer.MAX_VALUE,
       Long.MAX_VALUE, KeepDeletedCells.FALSE, 0, CellComparator.COMPARATOR);
   private ScanType scanType = ScanType.USER_SCAN;
 
-  /**
-   * From here on down, we have a bunch of defines and specific CELL_GRID of Cells. The
-   * CELL_GRID then has a Scanner that can fake out 'block' transitions. All this elaborate
-   * setup is for tests that ensure we don't overread, and that the
-   * {@link StoreScanner#optimize(org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.MatchCode,
-   * Cell)} is not overly enthusiastic.
-   */
-  private static final byte [] ZERO = new byte [] {'0'};
-  private static final byte [] ZERO_POINT_ZERO = new byte [] {'0', '.', '0'};
-  private static final byte [] ONE = new byte [] {'1'};
-  private static final byte [] TWO = new byte [] {'2'};
-  private static final byte [] TWO_POINT_TWO = new byte [] {'2', '.', '2'};
-  private static final byte [] THREE = new byte [] {'3'};
-  private static final byte [] FOUR = new byte [] {'4'};
-  private static final byte [] FIVE = new byte [] {'5'};
-  private static final byte [] VALUE = new byte [] {'v'};
-  private static final int CELL_GRID_BLOCK2_BOUNDARY = 4;
-  private static final int CELL_GRID_BLOCK3_BOUNDARY = 11;
-  private static final int CELL_GRID_BLOCK4_BOUNDARY = 15;
-  private static final int CELL_GRID_BLOCK5_BOUNDARY = 19;
-
-  /**
-   * Five rows by four columns distinguished by column qualifier (column qualifier is one of the
-   * four rows... ONE, TWO, etc.). Exceptions are a weird row after TWO; it is TWO_POINT_TWO.
-   * And then row FOUR has five columns finishing w/ row FIVE having a single column.
-   * We will use this to test scan does the right thing as it
-   * we do Gets, StoreScanner#optimize, and what we do on (faked) block boundaries.
-   */
-  private static final Cell [] CELL_GRID = new Cell [] {
-    CellUtil.createCell(ONE, CF, ONE, 1L, KeyValue.Type.Put.getCode(), VALUE),
-    CellUtil.createCell(ONE, CF, TWO, 1L, KeyValue.Type.Put.getCode(), VALUE),
-    CellUtil.createCell(ONE, CF, THREE, 1L, KeyValue.Type.Put.getCode(), VALUE),
-    CellUtil.createCell(ONE, CF, FOUR, 1L, KeyValue.Type.Put.getCode(), VALUE),
-    // Offset 4 CELL_GRID_BLOCK2_BOUNDARY
-    CellUtil.createCell(TWO, CF, ONE, 1L, KeyValue.Type.Put.getCode(), VALUE),
-    CellUtil.createCell(TWO, CF, TWO, 1L, KeyValue.Type.Put.getCode(), VALUE),
-    CellUtil.createCell(TWO, CF, THREE, 1L, KeyValue.Type.Put.getCode(), VALUE),
-    CellUtil.createCell(TWO, CF, FOUR, 1L, KeyValue.Type.Put.getCode(), VALUE),
-    CellUtil.createCell(TWO_POINT_TWO, CF, ZERO, 1L, KeyValue.Type.Put.getCode(), VALUE),
-    CellUtil.createCell(TWO_POINT_TWO, CF, ZERO_POINT_ZERO, 1L, KeyValue.Type.Put.getCode(), VALUE),
-    CellUtil.createCell(TWO_POINT_TWO, CF, FIVE, 1L, KeyValue.Type.Put.getCode(), VALUE),
-    // Offset 11! CELL_GRID_BLOCK3_BOUNDARY
-    CellUtil.createCell(THREE, CF, ONE, 1L, KeyValue.Type.Put.getCode(), VALUE),
-    CellUtil.createCell(THREE, CF, TWO, 1L, KeyValue.Type.Put.getCode(), VALUE),
-    CellUtil.createCell(THREE, CF, THREE, 1L, KeyValue.Type.Put.getCode(), VALUE),
-    CellUtil.createCell(THREE, CF, FOUR, 1L, KeyValue.Type.Put.getCode(), VALUE),
-    // Offset 15 CELL_GRID_BLOCK4_BOUNDARY
-    CellUtil.createCell(FOUR, CF, ONE, 1L, KeyValue.Type.Put.getCode(), VALUE),
-    CellUtil.createCell(FOUR, CF, TWO, 1L, KeyValue.Type.Put.getCode(), VALUE),
-    CellUtil.createCell(FOUR, CF, THREE, 1L, KeyValue.Type.Put.getCode(), VALUE),
-    CellUtil.createCell(FOUR, CF, FOUR, 1L, KeyValue.Type.Put.getCode(), VALUE),
-    // Offset 19 CELL_GRID_BLOCK5_BOUNDARY
-    CellUtil.createCell(FOUR, CF, FIVE, 1L, KeyValue.Type.Put.getCode(), VALUE),
-    CellUtil.createCell(FIVE, CF, ZERO, 1L, KeyValue.Type.Put.getCode(), VALUE),
-  };
-
-  /**
-   * A StoreScanner for our CELL_GRID above. Fakes the block transitions. Does counts of
-   * calls to optimize and counts of when optimize actually did an optimize.
-   */
-  private static class CellGridStoreScanner extends StoreScanner {
-    // Count of how often optimize is called and of how often it does an optimize.
-    final AtomicInteger count = new AtomicInteger(0);
-    final AtomicInteger optimization = new AtomicInteger(0);
-
-    CellGridStoreScanner(final Scan scan, ScanInfo scanInfo, ScanType scanType)
-    throws IOException {
-      super(scan, scanInfo, scanType, scan.getFamilyMap().get(CF),
-        Arrays.<KeyValueScanner>asList(
-          new KeyValueScanner[] {new KeyValueScanFixture(CellComparator.COMPARATOR, CELL_GRID)}));
-    }
-
-    protected ScanQueryMatcher.MatchCode optimize(ScanQueryMatcher.MatchCode qcode, Cell cell) {
-      count.incrementAndGet();
-      ScanQueryMatcher.MatchCode after = super.optimize(qcode, cell);
-      LOG.info("Cell=" + cell + ", nextIndex=" + CellUtil.toString(getNextIndexedKey(), false) +
-          ", before=" + qcode + ", after=" + after);
-      if (qcode != after) {
-        optimization.incrementAndGet();
-      }
-      return after;
-    };
-
-    @Override
-    public Cell getNextIndexedKey() {
-      // Fake block boundaries by having index of next block change as we go through scan.
-      return count.get() > CELL_GRID_BLOCK4_BOUNDARY?
-          CellUtil.createFirstOnRow(CELL_GRID[CELL_GRID_BLOCK5_BOUNDARY]):
-            count.get() > CELL_GRID_BLOCK3_BOUNDARY?
-                CellUtil.createFirstOnRow(CELL_GRID[CELL_GRID_BLOCK4_BOUNDARY]):
-                  count.get() > CELL_GRID_BLOCK2_BOUNDARY?
-                      CellUtil.createFirstOnRow(CELL_GRID[CELL_GRID_BLOCK3_BOUNDARY]):
-                        CellUtil.createFirstOnRow(CELL_GRID[CELL_GRID_BLOCK2_BOUNDARY]);
-    }
-  };
-
   /*
    * Test utility for building a NavigableSet for scanners.
    * @param strCols
@@ -181,145 +78,6 @@ public class TestStoreScanner {
   }
 
   @Test
-  public void testFullRowGetDoesNotOverreadWhenRowInsideOneBlock() throws IOException {
-    // Do a Get against row two. Row two is inside a block that starts with row TWO but ends with
-    // row TWO_POINT_TWO. We should read one block only.
-    Get get = new Get(TWO);
-    Scan scan = new Scan(get);
-    CellGridStoreScanner scanner = new CellGridStoreScanner(scan, this.scanInfo, this.scanType);
-    try {
-      List<Cell> results = new ArrayList<Cell>();
-      while (scanner.next(results)) {
-        continue;
-      }
-      // Should be four results of column 1 (though there are 5 rows in the CELL_GRID -- the
-      // TWO_POINT_TWO row does not have a a column ONE.
-      Assert.assertEquals(4, results.size());
-      // We should have gone the optimize route 5 times totally... an INCLUDE for the four cells
-      // in the row plus the DONE on the end.
-      Assert.assertEquals(5, scanner.count.get());
-      // For a full row Get, there should be no opportunity for scanner optimization.
-      Assert.assertEquals(0, scanner.optimization.get());
-    } finally {
-      scanner.close();
-    }
-  }
-
-  @Test
-  public void testFullRowSpansBlocks() throws IOException {
-    // Do a Get against row FOUR. It spans two blocks.
-    Get get = new Get(FOUR);
-    Scan scan = new Scan(get);
-    CellGridStoreScanner scanner = new CellGridStoreScanner(scan, this.scanInfo, this.scanType);
-    try {
-      List<Cell> results = new ArrayList<Cell>();
-      while (scanner.next(results)) {
-        continue;
-      }
-      // Should be four results of column 1 (though there are 5 rows in the CELL_GRID -- the
-      // TWO_POINT_TWO row does not have a a column ONE.
-      Assert.assertEquals(5, results.size());
-      // We should have gone the optimize route 6 times totally... an INCLUDE for the five cells
-      // in the row plus the DONE on the end.
-      Assert.assertEquals(6, scanner.count.get());
-      // For a full row Get, there should be no opportunity for scanner optimization.
-      Assert.assertEquals(0, scanner.optimization.get());
-    } finally {
-      scanner.close();
-    }
-  }
-
-  /**
-   * Test optimize in StoreScanner. Test that we skip to the next 'block' when we it makes sense
-   * reading the block 'index'.
-   * @throws IOException
-   */
-  @Test
-  public void testOptimize() throws IOException {
-    Scan scan = new Scan();
-    // A scan that just gets the first qualifier on each row of the CELL_GRID
-    scan.addColumn(CF, ONE);
-    CellGridStoreScanner scanner = new CellGridStoreScanner(scan, this.scanInfo, this.scanType);
-    try {
-      List<Cell> results = new ArrayList<Cell>();
-      while (scanner.next(results)) {
-        continue;
-      }
-      // Should be four results of column 1 (though there are 5 rows in the CELL_GRID -- the
-      // TWO_POINT_TWO row does not have a a column ONE.
-      Assert.assertEquals(4, results.size());
-      for (Cell cell: results) {
-        assertTrue(Bytes.equals(ONE, 0, ONE.length,
-            cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength()));
-      }
-      Assert.assertTrue("Optimize should do some optimizations", scanner.optimization.get() > 0);
-    } finally {
-      scanner.close();
-    }
-  }
-
-  /**
-   * Ensure the optimize Scan method in StoreScanner does not get in the way of a Get doing minimum
-   * work... seeking to start of block and then SKIPPING until we find the wanted Cell.
-   * This 'simple' scenario mimics case of all Cells fitting inside a single HFileBlock.
-   * See HBASE-15392. This test is a little cryptic. Takes a bit of staring to figure what it up to.
-   * @throws IOException
-   */
-  @Test
-  public void testOptimizeAndGet() throws IOException {
-    // First test a Get of two columns in the row R2. Every Get is a Scan. Get columns named
-    // R2 and R3.
-    Get get = new Get(TWO);
-    get.addColumn(CF, TWO);
-    get.addColumn(CF, THREE);
-    Scan scan = new Scan(get);
-    CellGridStoreScanner scanner = new CellGridStoreScanner(scan, this.scanInfo, this.scanType);
-    try {
-      List<Cell> results = new ArrayList<Cell>();
-      // For a Get there should be no more next's after the first call.
-      Assert.assertEquals(false, scanner.next(results));
-      // Should be one result only.
-      Assert.assertEquals(2, results.size());
-      // And we should have gone through optimize twice only.
-      Assert.assertEquals("First qcode is SEEK_NEXT_COL and second INCLUDE_AND_SEEK_NEXT_ROW",
-        3, scanner.count.get());
-    } finally {
-      scanner.close();
-    }
-  }
-
-  /**
-   * Ensure that optimize does not cause the Get to do more seeking than required. Optimize
-   * (see HBASE-15392) was causing us to seek all Cells in a block when a Get Scan if the next block
-   * index/start key was a different row to the current one. A bug. We'd call next too often
-   * because we had to exhaust all Cells in the current row making us load the next block just to
-   * discard what we read there. This test is a little cryptic. Takes a bit of staring to figure
-   * what it up to.
-   * @throws IOException
-   */
-  @Test
-  public void testOptimizeAndGetWithFakedNextBlockIndexStart() throws IOException {
-    // First test a Get of second column in the row R2. Every Get is a Scan. Second column has a
-    // qualifier of R2.
-    Get get = new Get(THREE);
-    get.addColumn(CF, TWO);
-    Scan scan = new Scan(get);
-    CellGridStoreScanner scanner = new CellGridStoreScanner(scan, this.scanInfo, this.scanType);
-    try {
-      List<Cell> results = new ArrayList<Cell>();
-      // For a Get there should be no more next's after the first call.
-      Assert.assertEquals(false, scanner.next(results));
-      // Should be one result only.
-      Assert.assertEquals(1, results.size());
-      // And we should have gone through optimize twice only.
-      Assert.assertEquals("First qcode is SEEK_NEXT_COL and second INCLUDE_AND_SEEK_NEXT_ROW",
-        2, scanner.count.get());
-    } finally {
-      scanner.close();
-    }
-  }
-
-  @Test
   public void testScanTimeRange() throws IOException {
     String r1 = "R1";
     // returns only 1 of these 2 even though same timestamp


[11/50] [abbrv] hbase git commit: HBASE-15416 TestHFileBackedByBucketCache is flakey since it went in

Posted by sy...@apache.org.
HBASE-15416 TestHFileBackedByBucketCache is flakey since it went in


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

Branch: refs/heads/hbase-12439
Commit: 8af9ed743352b0dc691d30b6902379bdef67df43
Parents: b3fe4ed
Author: stack <st...@apache.org>
Authored: Tue Mar 22 15:44:13 2016 -0700
Committer: stack <st...@apache.org>
Committed: Tue Mar 22 15:44:25 2016 -0700

----------------------------------------------------------------------
 .../io/hfile/TestHFileBackedByBucketCache.java  | 237 -------------------
 1 file changed, 237 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/8af9ed74/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBackedByBucketCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBackedByBucketCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBackedByBucketCache.java
deleted file mode 100644
index 2759078..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBackedByBucketCache.java
+++ /dev/null
@@ -1,237 +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 static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.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.CategoryBasedTimeout;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellComparator;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.io.compress.Compression;
-import org.apache.hadoop.hbase.io.hfile.bucket.BucketAllocator;
-import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache;
-import org.apache.hadoop.hbase.testclassification.IOTests;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.rules.TestName;
-import org.junit.rules.TestRule;
-
-/**
- * Test for file-backed BucketCache.
- */
-// This is marked a LargeTest so it runs in its own JVM. We do this because we are making use of
-// the cache and the cache is global. We don't want any other concurrent test polluting ours which
-// can happen if more than one test in a single JVM which can happen when tests are small.
-@Category({IOTests.class, LargeTests.class})
-public class TestHFileBackedByBucketCache {
-  private static final Log LOG = LogFactory.getLog(TestHFileBackedByBucketCache.class);
-  @Rule public TestName name = new TestName();
-  @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()).
-      withLookingForStuckThread(true).build();
-  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private static final int ROW_LENGTH = 4;
-  private Configuration conf;
-  private FileSystem fs;
-
-  // MATH! SIZING FOR THE TEST!
-  // Set bucketcache to be smallest size possible which is 1MB. We do that in the test
-  // @Before <code>before</code> method. Into out 1MB cache, have it so only one bucket. If
-  // bucketsize is set to 125kb in size, we will have one bucket in our 1MB bucketcache. It is
-  // cryptic how this comes about but basically comes down to
-  // {@link BucketAllocator#FEWEST_ITEMS_IN_BUCKET} being '4'... so 4 * 125 = just over 500k or so
-  // which makes for one bucket only in 1M which you can see from TRACE logging:
-  //
-  // Cache totalSize=532480, buckets=1, bucket capacity=
-  //   532480=(4*133120)=(FEWEST_ITEMS_IN_BUCKET*(largest configured bucketcache size))
-  //
-  // Now into this one big bucket, we write hfileblocks....Each hfileblock has two keys because
-  // first is under the BLOCKSIZE of 64k and then the second puts us over the 64k...
-  // so two Cells per block...
-  /**
-   * Default size.
-   */
-  private static final int BLOCKSIZE = 64 * 1024;
-
-  /**
-   * Bucket sizes get multiplied by 4 for actual bucket size.
-   * See {@link BucketAllocator#FEWEST_ITEMS_IN_BUCKET}.
-   */
-  private static final int BUCKETSIZE = 125 * 1024;
-
-  /**
-   * Make it so one Cell is just under a BLOCKSIZE. The second Cell puts us over the BLOCKSIZE
-   * so we have two Cells per HFilBlock.
-   */
-  private static final int VALUE_SIZE = 33 * 1024;
-
-  @Before
-  public void before() throws IOException {
-    // Do setup of a bucketcache that has one bucket only. Enable trace-level logging for
-    // key classes.
-    this.conf = TEST_UTIL.getConfiguration();
-    this.fs = FileSystem.get(conf);
-
-    // Set BucketCache and HFileBlock to log at trace level.
-    setTraceLevel(BucketCache.class);
-    setTraceLevel(HFileBlock.class);
-    setTraceLevel(HFileReaderImpl.class);
-    setTraceLevel(BucketAllocator.class);
-  }
-
-  //  Assumes log4j logging.
-  private static void setTraceLevel(final Class<?> clazz) {
-    Log testlog = LogFactory.getLog(clazz.getName());
-    ((org.apache.commons.logging.impl.Log4JLogger)testlog).getLogger().
-      setLevel(org.apache.log4j.Level.TRACE);
-  }
-
-  /**
-   * Test that bucketcache is caching and that the persist of in-memory map works
-   * @throws IOException
-   */
-  @Test
-  public void testBucketCacheCachesAndPersists() throws IOException {
-    // Set up a bucket cache. Set up one that will persist by passing a
-    // hbase.bucketcache.persistent.path value to store the in-memory map of what is out in
-    // the file-backed bucketcache. Set bucketcache to have one size only, BUCKETSIZE.
-    // See "MATH! SIZING FOR THE TEST!" note above around declaration of BUCKETSIZE
-    String bucketCacheDataFile =
-      (new Path(TEST_UTIL.getDataTestDir(), "bucketcache.data")).toString();
-    (new File(bucketCacheDataFile)).getParentFile().mkdirs();
-    this.conf.set("hbase.bucketcache.ioengine", "file:" + bucketCacheDataFile);
-    this.conf.set("hbase.bucketcache.persistent.path", bucketCacheDataFile + ".map");
-    this.conf.setStrings("hbase.bucketcache.bucket.sizes", Integer.toString(BUCKETSIZE));
-    // This is minimum bucketcache size.... 1MB.
-    this.conf.setInt("hbase.bucketcache.size", 1);
-    // Write 8 entries which should make for four hfileBlocks.
-    final int count = 8;
-    final int hfileBlockCount = 4;
-    Path hfilePath = new Path(TEST_UTIL.getDataTestDir(), this.name.getMethodName());
-    // Clear out any existing global cache instance. Will pollute our tests below. Any concurrent
-    // running test will pollute our results below.
-    CacheConfig.GLOBAL_BLOCK_CACHE_INSTANCE = null;
-    CacheConfig cacheConfig = new CacheConfig(conf);
-    List<Cell> writtenCells = writeFile(hfilePath, Compression.Algorithm.NONE, cacheConfig, count);
-    CacheStats stats = cacheConfig.getBlockCache().getStats();
-    List<Cell> readCells = readFile(hfilePath, cacheConfig);
-    assertTrue(!writtenCells.isEmpty());
-    assertEquals(writtenCells.size(), readCells.size());
-    assertEquals(hfileBlockCount, stats.getMissCount());
-    assertEquals(1, stats.getHitCount()); // readFile will read first block is from cache.
-
-    // Now, close out the cache and then reopen and verify that cache still has our blocks.
-    // Assert that persistence works.
-    cacheConfig.getBlockCache().shutdown();
-    // Need to clear the global cache else the new CacheConfig won't create a bucketcache but
-    // just reuse the old one.
-    CacheConfig.GLOBAL_BLOCK_CACHE_INSTANCE = null;
-    cacheConfig = new CacheConfig(conf);
-    stats = cacheConfig.getBlockCache().getStats();
-    assertEquals(0, stats.getHitCachingCount());
-    readCells = readFile(hfilePath, cacheConfig);
-    // readFile will read all hfileblocs in the file, hfileBlockCount, and then one more, so + 1.
-    assertEquals(hfileBlockCount + 1, stats.getHitCachingCount());
-  }
-
-  /**
-   * Write a file with <code>count</code> entries.
-   * @return The Cells written to the file.
-   * @throws IOException
-   */
-  private List<Cell> writeFile(final Path hfilePath, final Compression.Algorithm compressAlgo,
-      final CacheConfig cacheConfig, final int count)
-  throws IOException {
-    List<Cell> cells = new ArrayList<Cell>(count);
-    HFileContext context =
-        new HFileContextBuilder().withBlockSize(BLOCKSIZE).withCompression(compressAlgo).build();
-    try (HFile.Writer writer = new HFile.WriterFactory(conf, cacheConfig).
-        withPath(fs, hfilePath).
-        withFileContext(context).
-        withComparator(CellComparator.COMPARATOR).
-        create()) {
-      byte [] valueBytes = new byte [VALUE_SIZE];
-      for (int i = 0; i < valueBytes.length; i++) valueBytes[i] = '0';
-      for (int i = 0; i < count; ++i) {
-        byte[] keyBytes = format(i);
-        KeyValue keyValue = new KeyValue(keyBytes, HConstants.CATALOG_FAMILY, keyBytes,
-            HConstants.LATEST_TIMESTAMP, valueBytes);
-        writer.append(keyValue);
-        cells.add(keyValue);
-      }
-    }
-    return cells;
-  }
-
-  /**
-   * Read the whole file, then read the first block so we get something from cache for sure.
-   * So... there are TOTAL_BLOCKS_IN_FILE read + 1. See math at head of this class.
-   * @return The Cells read from the file.
-   */
-  private List<Cell> readFile(final Path hfilePath, final CacheConfig cacheConfig)
-  throws IOException {
-    List<Cell> cells = new ArrayList<Cell>();
-    try (HFile.Reader reader = HFile.createReader(this.fs, hfilePath, cacheConfig, this.conf);
-        HFileScanner scanner = reader.getScanner(true, true)) {
-      scanner.seekTo();
-      do {
-        cells.add(scanner.getCell());
-        LOG.info(scanner.getKey());
-      } while (scanner.next());
-      // Do a random seek just so we see a block coming from cache.
-      scanner.seekTo(reader.getFirstKey());
-      scanner.next();
-      LOG.info(scanner.getCell());
-    }
-    return cells;
-  }
-
-  /*
-   * Format passed integer.
-   * @param number
-   * @return Returns zero-prefixed ROW_LENGTH-byte wide decimal version of passed
-   * number (Does absolute in case number is negative).
-   */
-  private static byte [] format(final int number) {
-    byte [] b = new byte[ROW_LENGTH];
-    int d = Math.abs(number);
-    for (int i = b.length - 1; i >= 0; i--) {
-      b[i] = (byte)((d % 10) + '0');
-      d /= 10;
-    }
-    return b;
-  }
-}


[05/50] [abbrv] hbase git commit: Add javadoc on how BBIOEngine works (by Anoop Sam John)

Posted by sy...@apache.org.
Add javadoc on how BBIOEngine works (by Anoop Sam John)


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

Branch: refs/heads/hbase-12439
Commit: 891569bbd92f95797033c0706c9de9617228415b
Parents: 7073f69
Author: stack <st...@apache.org>
Authored: Tue Mar 22 13:01:13 2016 -0700
Committer: stack <st...@apache.org>
Committed: Tue Mar 22 13:01:13 2016 -0700

----------------------------------------------------------------------
 .../io/hfile/bucket/ByteBufferIOEngine.java     | 33 +++++++++++++++++++-
 1 file changed, 32 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/891569bb/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/ByteBufferIOEngine.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/ByteBufferIOEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/ByteBufferIOEngine.java
index 45ed1ae..63de32c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/ByteBufferIOEngine.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/ByteBufferIOEngine.java
@@ -31,7 +31,38 @@ import org.apache.hadoop.hbase.util.ByteBufferArray;
 
 /**
  * IO engine that stores data in memory using an array of ByteBuffers
- * {@link ByteBufferArray}
+ * {@link ByteBufferArray}.
+ *
+ *<h2>How it Works</h2>
+ * First, see {@link ByteBufferArray} and how it gives a view across multiple ByteBuffers managed
+ * by it internally. This class does the physical BB create and the write and read to the
+ * underlying BBs. So we will create N BBs based on the total BC capacity specified on create
+ * of the ByteBufferArray. So say we have 10 GB of off heap BucketCache, we will create 2560 such
+ * BBs inside our ByteBufferArray.
+ * 
+ * <p>Now the way BucketCache works is that the entire 10 GB is split into diff sized buckets: by
+ * default from 5 KB to 513 KB. Within each bucket of a particular size, there are
+ * usually more than one bucket 'block'. The way it is calculate in bucketcache is that the total
+ * bucketcache size is divided by 4 (hard-coded currently) * max size option. So using defaults,
+ * buckets will be is 4 * 513kb (the biggest default value) = 2052kb. A bucket of 2052kb at offset
+ * zero will serve out bucket 'blocks' of 5kb, the next bucket will do the next size up and so on
+ * up to the maximum (default) of 513kb).
+ * 
+ * <p>When we write blocks to the bucketcache, we will see which bucket size group it best fits.
+ * So a 4 KB block size goes to the 5 KB size group. Each of the block writes, writes within its
+ * appropriate bucket. Though the bucket is '4kb' in size, it will occupy one of the 
+ * 5 KB bucket 'blocks' (even if actual size of the bucket is less). Bucket 'blocks' will not span
+ * buckets.
+ * 
+ * <p>But you can see the physical memory under the bucket 'blocks' can be split across the
+ * underlying backing BBs from ByteBufferArray. All is split into 4 MB sized BBs.
+ * 
+ * <p>Each Bucket knows its offset in the entire space of BC and when block is written the offset
+ * arrives at ByteBufferArray and it figures which BB to write to. It may so happen that the entire
+ * block to be written does not fit a particular backing ByteBufferArray so the remainder goes to
+ * another BB. See {@link ByteBufferArray#putMultiple(long, int, byte[])}.
+
+So said all these, when we read a block it may be possible that the bytes of that blocks is physically placed in 2 adjucent BBs.  In such case also, we avoid any copy need by having the MBB...
  */
 @InterfaceAudience.Private
 public class ByteBufferIOEngine implements IOEngine {


[28/50] [abbrv] hbase git commit: HBASE-15475 Addendum

Posted by sy...@apache.org.
HBASE-15475 Addendum


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

Branch: refs/heads/hbase-12439
Commit: e71bc71c84d3562c9caca8863bbb63fe85c52f12
Parents: 52fd705
Author: Elliott Clark <ec...@apache.org>
Authored: Thu Mar 24 09:11:19 2016 -0700
Committer: Elliott Clark <ec...@apache.org>
Committed: Thu Mar 24 09:11:19 2016 -0700

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/e71bc71c/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
index 8c080c1..73e580c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
@@ -411,13 +411,13 @@ public class TestHFile  {
     Cell left = CellUtil.createCell(Bytes.toBytes("a"),
         Bytes.toBytes("a"),
         Bytes.toBytes("a"),
-        9,
+        11,
         KeyValue.Type.Maximum.getCode(),
         HConstants.EMPTY_BYTE_ARRAY);
     Cell right = CellUtil.createCell(Bytes.toBytes("a"),
         Bytes.toBytes("a"),
         Bytes.toBytes("a"),
-        11,
+        9,
         KeyValue.Type.Maximum.getCode(),
         HConstants.EMPTY_BYTE_ARRAY);
     Cell mid = HFileWriterImpl.getMidpoint(CellComparator.COMPARATOR, left, right);


[41/50] [abbrv] hbase git commit: HBASE-15548 SyncTable: sourceHashDir is supposed to be optional but won't work without (Dave Latham)

Posted by sy...@apache.org.
HBASE-15548 SyncTable: sourceHashDir is supposed to be optional but won't work without (Dave Latham)


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

Branch: refs/heads/hbase-12439
Commit: 7793bc54c2d20ce12b74d2ce013a2dbff449c221
Parents: c96b642
Author: tedyu <yu...@gmail.com>
Authored: Mon Mar 28 16:52:44 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Mon Mar 28 16:52:44 2016 -0700

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/7793bc54/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/SyncTable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/SyncTable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/SyncTable.java
index 1658ba4..d062f6c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/SyncTable.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/SyncTable.java
@@ -685,7 +685,7 @@ public class SyncTable extends Configured implements Tool {
     System.err.println();
     System.err.println("Args:");
     System.err.println(" sourcehashdir    path to HashTable output dir for source table");
-    System.err.println("                  if not specified, then all data will be scanned");
+    System.err.println("                  (see org.apache.hadoop.hbase.mapreduce.HashTable)");
     System.err.println(" sourcetable      Name of the source table to sync from");
     System.err.println(" targettable      Name of the target table to sync to");
     System.err.println();


[31/50] [abbrv] hbase git commit: HBASE-15526 Make SnapshotManager accessible through MasterServices

Posted by sy...@apache.org.
HBASE-15526 Make SnapshotManager accessible through MasterServices


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

Branch: refs/heads/hbase-12439
Commit: 07c597c8a2fe4f5b2b5412eea479b75e37de8f1a
Parents: ad3feaa
Author: tedyu <yu...@gmail.com>
Authored: Thu Mar 24 15:59:46 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Thu Mar 24 15:59:46 2016 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hbase/master/HMaster.java  | 3 +--
 .../java/org/apache/hadoop/hbase/master/MasterServices.java    | 6 ++++++
 .../org/apache/hadoop/hbase/master/TestCatalogJanitor.java     | 6 ++++++
 .../hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java    | 4 ++--
 4 files changed, 15 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/07c597c8/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 7942b28..221dcd5 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
@@ -2452,10 +2452,9 @@ public class HMaster extends HRegionServer implements MasterServices {
   }
 
   /**
-   * Exposed for TESTING!
    * @return the underlying snapshot manager
    */
-  public SnapshotManager getSnapshotManagerForTesting() {
+  public SnapshotManager getSnapshotManager() {
     return this.snapshotManager;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/07c597c8/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
index 8ce21b0..4d71117 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.executor.ExecutorService;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
 import org.apache.hadoop.hbase.security.User;
@@ -46,6 +47,11 @@ import com.google.protobuf.Service;
 @InterfaceAudience.Private
 public interface MasterServices extends Server {
   /**
+   * @return the underlying snapshot manager
+   */
+  SnapshotManager getSnapshotManager();
+
+  /**
    * @return Master's instance of {@link ClusterSchema}
    */
   ClusterSchema getClusterSchema();

http://git-wip-us.apache.org/repos/asf/hbase/blob/07c597c8/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 2949369..65bc50d 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
@@ -64,6 +64,7 @@ import org.apache.hadoop.hbase.io.Reference;
 import org.apache.hadoop.hbase.master.CatalogJanitor.SplitParentFirstComparator;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
@@ -243,6 +244,11 @@ public class TestCatalogJanitor {
     }
 
     @Override
+    public SnapshotManager getSnapshotManager() {
+      return null;
+    }
+
+    @Override
     public AssignmentManager getAssignmentManager() {
       return this.asm;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/07c597c8/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java
index a6b6e4c..b95089b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java
@@ -131,7 +131,7 @@ public class TestSnapshotFromMaster {
   @Before
   public void setup() throws Exception {
     UTIL.createTable(TABLE_NAME, TEST_FAM);
-    master.getSnapshotManagerForTesting().setSnapshotHandlerForTesting(TABLE_NAME, null);
+    master.getSnapshotManager().setSnapshotHandlerForTesting(TABLE_NAME, null);
   }
 
   @After
@@ -185,7 +185,7 @@ public class TestSnapshotFromMaster {
     Mockito.when(mockHandler.getCompletionTimestamp())
       .thenReturn(EnvironmentEdgeManager.currentTime());
 
-    master.getSnapshotManagerForTesting()
+    master.getSnapshotManager()
         .setSnapshotHandlerForTesting(TABLE_NAME, mockHandler);
 
     // if we do a lookup without a snapshot name, we should fail - you should always know your name


[13/50] [abbrv] hbase git commit: Revert "HBASE-15477 Purge 'next block header' from cached blocks"

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/54a543de/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
index 69c42c3..66aced0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
@@ -1317,22 +1317,25 @@ public class BucketCache implements BlockCache, HeapSize {
         final AtomicLong realCacheSize) throws CacheFullException, IOException,
         BucketAllocatorException {
       int len = data.getSerializedLength();
-      // This cacheable thing can't be serialized
+      // This cacheable thing can't be serialized...
       if (len == 0) return null;
       long offset = bucketAllocator.allocateBlock(len);
       BucketEntry bucketEntry = new BucketEntry(offset, len, accessCounter, inMemory);
       bucketEntry.setDeserialiserReference(data.getDeserializer(), deserialiserMap);
       try {
         if (data instanceof HFileBlock) {
-          // If an instance of HFileBlock, save on some allocations.
-          HFileBlock block = (HFileBlock)data;
-          ByteBuff sliceBuf = block.getBufferReadOnly();
-          ByteBuffer metadata = block.getMetaData();
+          HFileBlock block = (HFileBlock) data;
+          ByteBuff sliceBuf = block.getBufferReadOnlyWithHeader();
+          sliceBuf.rewind();
+          assert len == sliceBuf.limit() + HFileBlock.EXTRA_SERIALIZATION_SPACE ||
+            len == sliceBuf.limit() + block.headerSize() + HFileBlock.EXTRA_SERIALIZATION_SPACE;
+          ByteBuffer extraInfoBuffer = ByteBuffer.allocate(HFileBlock.EXTRA_SERIALIZATION_SPACE);
+          block.serializeExtraInfo(extraInfoBuffer);
           if (LOG.isTraceEnabled()) {
             LOG.trace("Write offset=" + offset + ", len=" + len);
           }
           ioEngine.write(sliceBuf, offset);
-          ioEngine.write(metadata, offset + len - metadata.limit());
+          ioEngine.write(extraInfoBuffer, offset + len - HFileBlock.EXTRA_SERIALIZATION_SPACE);
         } else {
           ByteBuffer bb = ByteBuffer.allocate(len);
           data.serialize(bb);

http://git-wip-us.apache.org/repos/asf/hbase/blob/54a543de/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java
index ed86a83..e26022e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java
@@ -18,7 +18,6 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
-import java.io.Closeable;
 import java.io.IOException;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -33,7 +32,7 @@ import org.apache.hadoop.hbase.client.Scan;
 // TODO: Change name from KeyValueScanner to CellScanner only we already have a simple CellScanner
 // so this should be something else altogether, a decoration on our base CellScanner. TODO.
 // This class shows in CPs so do it all in one swell swoop. HBase-2.0.0.
-public interface KeyValueScanner extends Shipper, Closeable {
+public interface KeyValueScanner extends Shipper {
   /**
    * The byte array represents for NO_NEXT_INDEXED_KEY;
    * The actual value is irrelevant because this is always compared by reference.
@@ -75,7 +74,6 @@ public interface KeyValueScanner extends Shipper, Closeable {
    * The default implementation for this would be to return 0. A file having
    * lower sequence id will be considered to be the older one.
    */
-  // TODO: Implement SequenceId Interface instead.
   long getSequenceID();
 
   /**
@@ -139,11 +137,11 @@ public interface KeyValueScanner extends Shipper, Closeable {
    * peek KeyValue of scanner has the same row with specified Cell,
    * otherwise seek the scanner at the first Cell of the row which is the
    * previous row of specified KeyValue
-   *
+   * 
    * @param key seek KeyValue
    * @return true if the scanner is at the valid KeyValue, false if such
    *         KeyValue does not exist
-   *
+   * 
    */
   public boolean backwardSeek(Cell key) throws IOException;
 
@@ -158,7 +156,7 @@ public interface KeyValueScanner extends Shipper, Closeable {
 
   /**
    * Seek the scanner at the first KeyValue of last row
-   *
+   * 
    * @return true if scanner has values left, false if the underlying data is
    *         empty
    * @throws IOException
@@ -171,4 +169,4 @@ public interface KeyValueScanner extends Shipper, Closeable {
    * see HFileWriterImpl#getMidpoint, or null if not known.
    */
   public Cell getNextIndexedKey();
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/54a543de/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
index b6164b2..61eb9b8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
@@ -1271,7 +1271,7 @@ public class StoreFile {
     }
 
     /**
-     * @deprecated Do not write further code which depends on this call. Instead
+     * Warning: Do not write further code which depends on this call. Instead
      * use getStoreFileScanner() which uses the StoreFileScanner class/interface
      * which is the preferred way to scan a store with higher level concepts.
      *
@@ -1285,7 +1285,7 @@ public class StoreFile {
     }
 
     /**
-     * @deprecated Do not write further code which depends on this call. Instead
+     * Warning: Do not write further code which depends on this call. Instead
      * use getStoreFileScanner() which uses the StoreFileScanner class/interface
      * which is the preferred way to scan a store with higher level concepts.
      *

http://git-wip-us.apache.org/repos/asf/hbase/blob/54a543de/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java
index 040685d..69671e2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.io.HeapSize;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache;
 import org.apache.hadoop.hbase.nio.ByteBuff;
+import org.apache.hadoop.hbase.nio.SingleByteBuff;
 import org.apache.hadoop.hbase.util.ChecksumType;
 
 public class CacheTestUtils {
@@ -65,7 +66,6 @@ public class CacheTestUtils {
     /*Post eviction, heapsize should be the same */
     assertEquals(heapSize, ((HeapSize) toBeTested).heapSize());
   }
-
   public static void testCacheMultiThreaded(final BlockCache toBeTested,
       final int blockSize, final int numThreads, final int numQueries,
       final double passingScore) throws Exception {
@@ -339,16 +339,25 @@ public class CacheTestUtils {
   }
 
 
-  private static HFileBlockPair[] generateHFileBlocks(int blockSize, int numBlocks) {
+  private static HFileBlockPair[] generateHFileBlocks(int blockSize,
+      int numBlocks) {
     HFileBlockPair[] returnedBlocks = new HFileBlockPair[numBlocks];
     Random rand = new Random();
     HashSet<String> usedStrings = new HashSet<String>();
     for (int i = 0; i < numBlocks; i++) {
-      ByteBuffer cachedBuffer = ByteBuffer.allocate(blockSize);
+
+      // The buffer serialized size needs to match the size of BlockSize. So we
+      // declare our data size to be smaller than it by the serialization space
+      // required.
+
+      SingleByteBuff cachedBuffer = new SingleByteBuff(ByteBuffer.allocate(blockSize
+          - HFileBlock.EXTRA_SERIALIZATION_SPACE));
       rand.nextBytes(cachedBuffer.array());
       cachedBuffer.rewind();
-      int onDiskSizeWithoutHeader = blockSize;
-      int uncompressedSizeWithoutHeader = blockSize;
+      int onDiskSizeWithoutHeader = blockSize
+          - HFileBlock.EXTRA_SERIALIZATION_SPACE;
+      int uncompressedSizeWithoutHeader = blockSize
+          - HFileBlock.EXTRA_SERIALIZATION_SPACE;
       long prevBlockOffset = rand.nextLong();
       BlockType.DATA.write(cachedBuffer);
       cachedBuffer.putInt(onDiskSizeWithoutHeader);
@@ -367,7 +376,7 @@ public class CacheTestUtils {
           onDiskSizeWithoutHeader, uncompressedSizeWithoutHeader,
           prevBlockOffset, cachedBuffer, HFileBlock.DONT_FILL_HEADER,
           blockSize,
-          onDiskSizeWithoutHeader + HConstants.HFILEBLOCK_HEADER_SIZE, -1, meta);
+          onDiskSizeWithoutHeader + HConstants.HFILEBLOCK_HEADER_SIZE, meta);
 
       String strKey;
       /* No conflicting keys */
@@ -386,4 +395,4 @@ public class CacheTestUtils {
     BlockCacheKey blockName;
     HFileBlock block;
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/54a543de/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
index a9d8258..5158e35 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
@@ -259,6 +259,7 @@ public class TestCacheOnWrite {
     assertTrue(testDescription, scanner.seekTo());
 
     long offset = 0;
+    HFileBlock prevBlock = null;
     EnumMap<BlockType, Integer> blockCountByType =
         new EnumMap<BlockType, Integer>(BlockType.class);
 
@@ -266,10 +267,14 @@ public class TestCacheOnWrite {
     List<Long> cachedBlocksOffset = new ArrayList<Long>();
     Map<Long, HFileBlock> cachedBlocks = new HashMap<Long, HFileBlock>();
     while (offset < reader.getTrailer().getLoadOnOpenDataOffset()) {
+      long onDiskSize = -1;
+      if (prevBlock != null) {
+         onDiskSize = prevBlock.getNextBlockOnDiskSizeWithHeader();
+      }
       // Flags: don't cache the block, use pread, this is not a compaction.
       // Also, pass null for expected block type to avoid checking it.
-      HFileBlock block = reader.readBlock(offset, -1, false, true, false, true, null,
-          encodingInCache);
+      HFileBlock block = reader.readBlock(offset, onDiskSize, false, true,
+        false, true, null, encodingInCache);
       BlockCacheKey blockCacheKey = new BlockCacheKey(reader.getName(),
           offset);
       HFileBlock fromCache = (HFileBlock) blockCache.getBlock(blockCacheKey, true, false, true);
@@ -302,6 +307,7 @@ public class TestCacheOnWrite {
         assertEquals(
           block.getUncompressedSizeWithoutHeader(), fromCache.getUncompressedSizeWithoutHeader());
       }
+      prevBlock = block;
       offset += block.getOnDiskSizeWithHeader();
       BlockType bt = block.getBlockType();
       Integer count = blockCountByType.get(bt);

http://git-wip-us.apache.org/repos/asf/hbase/blob/54a543de/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestChecksum.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestChecksum.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestChecksum.java
index d91a811..91ab8c0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestChecksum.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestChecksum.java
@@ -94,7 +94,7 @@ public class TestChecksum {
     meta = new HFileContextBuilder().withHBaseCheckSum(true).build();
     HFileBlock.FSReader hbr = new HFileBlock.FSReaderImpl(
         is, totalSize, (HFileSystem) fs, path, meta);
-    HFileBlock b = hbr.readBlockData(0, -1, false);
+    HFileBlock b = hbr.readBlockData(0, -1, -1, false);
     assertEquals(b.getChecksumType(), ChecksumType.getDefaultChecksumType().getCode());
   }
 
@@ -108,14 +108,12 @@ public class TestChecksum {
       ChecksumType cktype = itr.next();
       Path path = new Path(TEST_UTIL.getDataTestDir(), "checksum" + cktype.getName());
       FSDataOutputStream os = fs.create(path);
-      HFileContext meta = new HFileContextBuilder().
-          withChecksumType(cktype).
-          build();
+      HFileContext meta = new HFileContextBuilder()
+          .withChecksumType(cktype).build();
       HFileBlock.Writer hbw = new HFileBlock.Writer(null, meta);
       DataOutputStream dos = hbw.startWriting(BlockType.DATA);
-      for (int i = 0; i < 1000; ++i) {
+      for (int i = 0; i < 1000; ++i)
         dos.writeInt(i);
-      }
       hbw.writeHeaderAndData(os);
       int totalSize = hbw.getOnDiskSizeWithHeader();
       os.close();
@@ -127,7 +125,7 @@ public class TestChecksum {
       meta = new HFileContextBuilder().withHBaseCheckSum(true).build();
       HFileBlock.FSReader hbr = new HFileBlock.FSReaderImpl(
           is, totalSize, (HFileSystem) fs, path, meta);
-      HFileBlock b = hbr.readBlockData(0, -1, false);
+      HFileBlock b = hbr.readBlockData(0, -1, -1, false);
       ByteBuff data = b.getBufferWithoutHeader();
       for (int i = 0; i < 1000; i++) {
         assertEquals(i, data.getInt());
@@ -190,7 +188,7 @@ public class TestChecksum {
               .withHBaseCheckSum(true)
               .build();
         HFileBlock.FSReader hbr = new FSReaderImplTest(is, totalSize, fs, path, meta);
-        HFileBlock b = hbr.readBlockData(0, -1, pread);
+        HFileBlock b = hbr.readBlockData(0, -1, -1, pread);
         b.sanityCheck();
         assertEquals(4936, b.getUncompressedSizeWithoutHeader());
         assertEquals(algo == GZ ? 2173 : 4936, 
@@ -211,17 +209,17 @@ public class TestChecksum {
         // requests. Verify that this is correct.
         for (int i = 0; i < 
              HFileBlock.CHECKSUM_VERIFICATION_NUM_IO_THRESHOLD + 1; i++) {
-          b = hbr.readBlockData(0, -1, pread);
+          b = hbr.readBlockData(0, -1, -1, pread);
           assertEquals(0, HFile.getChecksumFailuresCount());
         }
         // The next read should have hbase checksum verification reanabled,
         // we verify this by assertng that there was a hbase-checksum failure.
-        b = hbr.readBlockData(0, -1, pread);
+        b = hbr.readBlockData(0, -1, -1, pread);
         assertEquals(1, HFile.getChecksumFailuresCount());
 
         // Since the above encountered a checksum failure, we switch
         // back to not checking hbase checksums.
-        b = hbr.readBlockData(0, -1, pread);
+        b = hbr.readBlockData(0, -1, -1, pread);
         assertEquals(0, HFile.getChecksumFailuresCount());
         is.close();
 
@@ -232,7 +230,7 @@ public class TestChecksum {
         assertEquals(false, newfs.useHBaseChecksum());
         is = new FSDataInputStreamWrapper(newfs, path);
         hbr = new FSReaderImplTest(is, totalSize, newfs, path, meta);
-        b = hbr.readBlockData(0, -1, pread);
+        b = hbr.readBlockData(0, -1, -1, pread);
         is.close();
         b.sanityCheck();
         b = b.unpack(meta, hbr);
@@ -316,7 +314,7 @@ public class TestChecksum {
                .build();
         HFileBlock.FSReader hbr = new HFileBlock.FSReaderImpl(new FSDataInputStreamWrapper(
             is, nochecksum), totalSize, hfs, path, meta);
-        HFileBlock b = hbr.readBlockData(0, -1, pread);
+        HFileBlock b = hbr.readBlockData(0, -1, -1, pread);
         is.close();
         b.sanityCheck();
         assertEquals(dataSize, b.getUncompressedSizeWithoutHeader());
@@ -356,4 +354,5 @@ public class TestChecksum {
       return false;  // checksum validation failure
     }
   }
-}
\ No newline at end of file
+}
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/54a543de/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java
index eb87a0c..6748efc 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java
@@ -320,7 +320,7 @@ public class TestHFileBlock {
         .withIncludesTags(includesTag)
         .withCompression(algo).build();
         HFileBlock.FSReader hbr = new HFileBlock.FSReaderImpl(is, totalSize, meta);
-        HFileBlock b = hbr.readBlockData(0, -1, pread);
+        HFileBlock b = hbr.readBlockData(0, -1, -1, pread);
         is.close();
         assertEquals(0, HFile.getChecksumFailuresCount());
 
@@ -334,15 +334,17 @@ public class TestHFileBlock {
           is = fs.open(path);
           hbr = new HFileBlock.FSReaderImpl(is, totalSize, meta);
           b = hbr.readBlockData(0, 2173 + HConstants.HFILEBLOCK_HEADER_SIZE +
-                                b.totalChecksumBytes(), pread);
+                                b.totalChecksumBytes(), -1, pread);
           assertEquals(expected, b);
           int wrongCompressedSize = 2172;
           try {
             b = hbr.readBlockData(0, wrongCompressedSize
-                + HConstants.HFILEBLOCK_HEADER_SIZE, pread);
+                + HConstants.HFILEBLOCK_HEADER_SIZE, -1, pread);
             fail("Exception expected");
           } catch (IOException ex) {
-            String expectedPrefix = "Passed in onDiskSizeWithHeader=";
+            String expectedPrefix = "On-disk size without header provided is "
+                + wrongCompressedSize + ", but block header contains "
+                + b.getOnDiskSizeWithoutHeader() + ".";
             assertTrue("Invalid exception message: '" + ex.getMessage()
                 + "'.\nMessage is expected to start with: '" + expectedPrefix
                 + "'", ex.getMessage().startsWith(expectedPrefix));
@@ -422,7 +424,7 @@ public class TestHFileBlock {
           HFileBlock blockFromHFile, blockUnpacked;
           int pos = 0;
           for (int blockId = 0; blockId < numBlocks; ++blockId) {
-            blockFromHFile = hbr.readBlockData(pos, -1, pread);
+            blockFromHFile = hbr.readBlockData(pos, -1, -1, pread);
             assertEquals(0, HFile.getChecksumFailuresCount());
             blockFromHFile.sanityCheck();
             pos += blockFromHFile.getOnDiskSizeWithHeader();
@@ -558,7 +560,7 @@ public class TestHFileBlock {
             if (detailedLogging) {
               LOG.info("Reading block #" + i + " at offset " + curOffset);
             }
-            HFileBlock b = hbr.readBlockData(curOffset, -1, pread);
+            HFileBlock b = hbr.readBlockData(curOffset, -1, -1, pread);
             if (detailedLogging) {
               LOG.info("Block #" + i + ": " + b);
             }
@@ -572,7 +574,8 @@ public class TestHFileBlock {
 
             // Now re-load this block knowing the on-disk size. This tests a
             // different branch in the loader.
-            HFileBlock b2 = hbr.readBlockData(curOffset, b.getOnDiskSizeWithHeader(), pread);
+            HFileBlock b2 = hbr.readBlockData(curOffset,
+                b.getOnDiskSizeWithHeader(), -1, pread);
             b2.sanityCheck();
 
             assertEquals(b.getBlockType(), b2.getBlockType());
@@ -598,7 +601,7 @@ public class TestHFileBlock {
               b = b.unpack(meta, hbr);
               // b's buffer has header + data + checksum while
               // expectedContents have header + data only
-              ByteBuff bufRead = b.getBufferReadOnly();
+              ByteBuff bufRead = b.getBufferWithHeader();
               ByteBuffer bufExpected = expectedContents.get(i);
               boolean bytesAreCorrect = Bytes.compareTo(bufRead.array(),
                   bufRead.arrayOffset(),
@@ -681,7 +684,7 @@ public class TestHFileBlock {
         HFileBlock b;
         try {
           long onDiskSizeArg = withOnDiskSize ? expectedSize : -1;
-          b = hbr.readBlockData(offset, onDiskSizeArg, pread);
+          b = hbr.readBlockData(offset, onDiskSizeArg, -1, pread);
         } catch (IOException ex) {
           LOG.error("Error in client " + clientId + " trying to read block at "
               + offset + ", pread=" + pread + ", withOnDiskSize=" +
@@ -716,7 +719,8 @@ public class TestHFileBlock {
   protected void testConcurrentReadingInternals() throws IOException,
       InterruptedException, ExecutionException {
     for (Compression.Algorithm compressAlgo : COMPRESSION_ALGORITHMS) {
-      Path path = new Path(TEST_UTIL.getDataTestDir(), "concurrent_reading");
+      Path path =
+          new Path(TEST_UTIL.getDataTestDir(), "concurrent_reading");
       Random rand = defaultRandom();
       List<Long> offsets = new ArrayList<Long>();
       List<BlockType> types = new ArrayList<BlockType>();
@@ -839,7 +843,8 @@ public class TestHFileBlock {
                           .withBytesPerCheckSum(HFile.DEFAULT_BYTES_PER_CHECKSUM)
                           .withChecksumType(ChecksumType.NULL).build();
       HFileBlock block = new HFileBlock(BlockType.DATA, size, size, -1, buf,
-          HFileBlock.FILL_HEADER, -1, 0, -1, meta);
+          HFileBlock.FILL_HEADER, -1,
+          0, meta);
       long byteBufferExpectedSize = ClassSize.align(ClassSize.estimateBase(
           new MultiByteBuff(buf).getClass(), true)
           + HConstants.HFILEBLOCK_HEADER_SIZE + size);

http://git-wip-us.apache.org/repos/asf/hbase/blob/54a543de/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockCompatibility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockCompatibility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockCompatibility.java
new file mode 100644
index 0000000..16607b9
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockCompatibility.java
@@ -0,0 +1,750 @@
+/*
+ *
+ * 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 static org.apache.hadoop.hbase.io.compress.Compression.Algorithm.GZ;
+import static org.apache.hadoop.hbase.io.compress.Compression.Algorithm.NONE;
+import static org.junit.Assert.*;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+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.FSDataInputStreamWrapper;
+import org.apache.hadoop.hbase.io.compress.Compression;
+import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
+import org.apache.hadoop.hbase.io.encoding.HFileBlockDefaultEncodingContext;
+import org.apache.hadoop.hbase.io.encoding.HFileBlockEncodingContext;
+import org.apache.hadoop.hbase.io.hfile.HFileBlock.BlockWritable;
+import org.apache.hadoop.hbase.nio.ByteBuff;
+import org.apache.hadoop.hbase.nio.SingleByteBuff;
+import org.apache.hadoop.hbase.testclassification.IOTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.ChecksumType;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.io.compress.Compressor;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * This class has unit tests to prove that older versions of
+ * HFiles (without checksums) are compatible with current readers.
+ */
+@Category({IOTests.class, SmallTests.class})
+@RunWith(Parameterized.class)
+public class TestHFileBlockCompatibility {
+
+  private static final Log LOG = LogFactory.getLog(TestHFileBlockCompatibility.class);
+  private static final Compression.Algorithm[] COMPRESSION_ALGORITHMS = {
+      NONE, GZ };
+
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private HFileSystem fs;
+
+  private final boolean includesMemstoreTS;
+  private final boolean includesTag;
+
+  public TestHFileBlockCompatibility(boolean includesMemstoreTS, boolean includesTag) {
+    this.includesMemstoreTS = includesMemstoreTS;
+    this.includesTag = includesTag;
+  }
+
+  @Parameters
+  public static Collection<Object[]> parameters() {
+    return HBaseTestingUtility.MEMSTORETS_TAGS_PARAMETRIZED;
+  }
+
+  @Before
+  public void setUp() throws IOException {
+    fs = (HFileSystem)HFileSystem.get(TEST_UTIL.getConfiguration());
+  }
+
+  public byte[] createTestV1Block(Compression.Algorithm algo)
+      throws IOException {
+    Compressor compressor = algo.getCompressor();
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    OutputStream os = algo.createCompressionStream(baos, compressor, 0);
+    DataOutputStream dos = new DataOutputStream(os);
+    BlockType.META.write(dos); // Let's make this a meta block.
+    TestHFileBlock.writeTestBlockContents(dos);
+    dos.flush();
+    algo.returnCompressor(compressor);
+    return baos.toByteArray();
+  }
+
+  private Writer createTestV2Block(Compression.Algorithm algo)
+      throws IOException {
+    final BlockType blockType = BlockType.DATA;
+    Writer hbw = new Writer(algo, null,
+        includesMemstoreTS, includesTag);
+    DataOutputStream dos = hbw.startWriting(blockType);
+    TestHFileBlock.writeTestBlockContents(dos);
+    // make sure the block is ready by calling hbw.getHeaderAndData()
+    hbw.getHeaderAndData();
+    assertEquals(1000 * 4, hbw.getUncompressedSizeWithoutHeader());
+    hbw.releaseCompressor();
+    return hbw;
+  }
+
+ private String createTestBlockStr(Compression.Algorithm algo,
+      int correctLength) throws IOException {
+    Writer hbw = createTestV2Block(algo);
+    byte[] testV2Block = hbw.getHeaderAndData();
+    int osOffset = HConstants.HFILEBLOCK_HEADER_SIZE_NO_CHECKSUM + 9;
+    if (testV2Block.length == correctLength) {
+      // Force-set the "OS" field of the gzip header to 3 (Unix) to avoid
+      // variations across operating systems.
+      // See http://www.gzip.org/zlib/rfc-gzip.html for gzip format.
+      testV2Block[osOffset] = 3;
+    }
+    return Bytes.toStringBinary(testV2Block);
+  }
+
+  @Test
+  public void testNoCompression() throws IOException {
+    assertEquals(4000, createTestV2Block(NONE).getBlockForCaching().
+        getUncompressedSizeWithoutHeader());
+  }
+
+  @Test
+  public void testGzipCompression() throws IOException {
+    final String correctTestBlockStr =
+        "DATABLK*\\x00\\x00\\x00:\\x00\\x00\\x0F\\xA0\\xFF\\xFF\\xFF\\xFF"
+            + "\\xFF\\xFF\\xFF\\xFF"
+            // gzip-compressed block: http://www.gzip.org/zlib/rfc-gzip.html
+            + "\\x1F\\x8B"  // gzip magic signature
+            + "\\x08"  // Compression method: 8 = "deflate"
+            + "\\x00"  // Flags
+            + "\\x00\\x00\\x00\\x00"  // mtime
+            + "\\x00"  // XFL (extra flags)
+            // OS (0 = FAT filesystems, 3 = Unix). However, this field
+            // sometimes gets set to 0 on Linux and Mac, so we reset it to 3.
+            + "\\x03"
+            + "\\xED\\xC3\\xC1\\x11\\x00 \\x08\\xC00DD\\xDD\\x7Fa"
+            + "\\xD6\\xE8\\xA3\\xB9K\\x84`\\x96Q\\xD3\\xA8\\xDB\\xA8e\\xD4c"
+            + "\\xD46\\xEA5\\xEA3\\xEA7\\xE7\\x00LI\\x5Cs\\xA0\\x0F\\x00\\x00";
+    final int correctGzipBlockLength = 82;
+
+    String returnedStr = createTestBlockStr(GZ, correctGzipBlockLength);
+    assertEquals(correctTestBlockStr, returnedStr);
+  }
+
+  @Test
+  public void testReaderV2() throws IOException {
+    if(includesTag) {
+      TEST_UTIL.getConfiguration().setInt("hfile.format.version", 3);
+    }
+    for (Compression.Algorithm algo : COMPRESSION_ALGORITHMS) {
+      for (boolean pread : new boolean[] { false, true }) {
+          LOG.info("testReaderV2: Compression algorithm: " + algo +
+                   ", pread=" + pread);
+        Path path = new Path(TEST_UTIL.getDataTestDir(), "blocks_v2_"
+            + algo);
+        FSDataOutputStream os = fs.create(path);
+        Writer hbw = new Writer(algo, null,
+            includesMemstoreTS, includesTag);
+        long totalSize = 0;
+        for (int blockId = 0; blockId < 2; ++blockId) {
+          DataOutputStream dos = hbw.startWriting(BlockType.DATA);
+          for (int i = 0; i < 1234; ++i)
+            dos.writeInt(i);
+          hbw.writeHeaderAndData(os);
+          totalSize += hbw.getOnDiskSizeWithHeader();
+        }
+        os.close();
+
+        FSDataInputStream is = fs.open(path);
+        HFileContext meta = new HFileContextBuilder()
+                           .withHBaseCheckSum(false)
+                           .withIncludesMvcc(includesMemstoreTS)
+                           .withIncludesTags(includesTag)
+                           .withCompression(algo)
+                           .build();
+        HFileBlock.FSReader hbr =
+          new HFileBlock.FSReaderImpl(new FSDataInputStreamWrapper(is), totalSize, fs, path, meta);
+        HFileBlock b = hbr.readBlockData(0, -1, -1, pread);
+        is.close();
+
+        b.sanityCheck();
+        assertEquals(4936, b.getUncompressedSizeWithoutHeader());
+        assertEquals(algo == GZ ? 2173 : 4936,
+                     b.getOnDiskSizeWithoutHeader() - b.totalChecksumBytes());
+        HFileBlock expected = b;
+
+        if (algo == GZ) {
+          is = fs.open(path);
+          hbr = new HFileBlock.FSReaderImpl(new FSDataInputStreamWrapper(is), totalSize, fs, path,
+              meta);
+          b = hbr.readBlockData(0, 2173 + HConstants.HFILEBLOCK_HEADER_SIZE_NO_CHECKSUM +
+                                b.totalChecksumBytes(), -1, pread);
+          assertEquals(expected, b);
+          int wrongCompressedSize = 2172;
+          try {
+            b = hbr.readBlockData(0, wrongCompressedSize
+                + HConstants.HFILEBLOCK_HEADER_SIZE_NO_CHECKSUM, -1, pread);
+            fail("Exception expected");
+          } catch (IOException ex) {
+            String expectedPrefix = "On-disk size without header provided is "
+                + wrongCompressedSize + ", but block header contains "
+                + b.getOnDiskSizeWithoutHeader() + ".";
+            assertTrue("Invalid exception message: '" + ex.getMessage()
+                + "'.\nMessage is expected to start with: '" + expectedPrefix
+                + "'", ex.getMessage().startsWith(expectedPrefix));
+          }
+          is.close();
+        }
+      }
+    }
+  }
+
+  /**
+   * Test encoding/decoding data blocks.
+   * @throws IOException a bug or a problem with temporary files.
+   */
+  @Test
+  public void testDataBlockEncoding() throws IOException {
+    if(includesTag) {
+      TEST_UTIL.getConfiguration().setInt("hfile.format.version", 3);
+    }
+    final int numBlocks = 5;
+    for (Compression.Algorithm algo : COMPRESSION_ALGORITHMS) {
+      for (boolean pread : new boolean[] { false, true }) {
+        for (DataBlockEncoding encoding : DataBlockEncoding.values()) {
+          LOG.info("testDataBlockEncoding algo " + algo +
+                   " pread = " + pread +
+                   " encoding " + encoding);
+          Path path = new Path(TEST_UTIL.getDataTestDir(), "blocks_v2_"
+              + algo + "_" + encoding.toString());
+          FSDataOutputStream os = fs.create(path);
+          HFileDataBlockEncoder dataBlockEncoder = (encoding != DataBlockEncoding.NONE) ?
+              new HFileDataBlockEncoderImpl(encoding) : NoOpDataBlockEncoder.INSTANCE;
+          TestHFileBlockCompatibility.Writer hbw =
+              new TestHFileBlockCompatibility.Writer(algo,
+                  dataBlockEncoder, includesMemstoreTS, includesTag);
+          long totalSize = 0;
+          final List<Integer> encodedSizes = new ArrayList<Integer>();
+          final List<ByteBuffer> encodedBlocks = new ArrayList<ByteBuffer>();
+          for (int blockId = 0; blockId < numBlocks; ++blockId) {
+            hbw.startWriting(BlockType.DATA);
+            TestHFileBlock.writeTestKeyValues(hbw, blockId, pread, includesTag);
+            hbw.writeHeaderAndData(os);
+            int headerLen = HConstants.HFILEBLOCK_HEADER_SIZE_NO_CHECKSUM;
+            byte[] encodedResultWithHeader = hbw.getUncompressedDataWithHeader();
+            final int encodedSize = encodedResultWithHeader.length - headerLen;
+            if (encoding != DataBlockEncoding.NONE) {
+              // We need to account for the two-byte encoding algorithm ID that
+              // comes after the 24-byte block header but before encoded KVs.
+              headerLen += DataBlockEncoding.ID_SIZE;
+            }
+            byte[] encodedDataSection =
+                new byte[encodedResultWithHeader.length - headerLen];
+            System.arraycopy(encodedResultWithHeader, headerLen,
+                encodedDataSection, 0, encodedDataSection.length);
+            final ByteBuffer encodedBuf =
+                ByteBuffer.wrap(encodedDataSection);
+            encodedSizes.add(encodedSize);
+            encodedBlocks.add(encodedBuf);
+            totalSize += hbw.getOnDiskSizeWithHeader();
+          }
+          os.close();
+
+          FSDataInputStream is = fs.open(path);
+          HFileContext meta = new HFileContextBuilder()
+                              .withHBaseCheckSum(false)
+                              .withIncludesMvcc(includesMemstoreTS)
+                              .withIncludesTags(includesTag)
+                              .withCompression(algo)
+                              .build();
+          HFileBlock.FSReaderImpl hbr = new HFileBlock.FSReaderImpl(new FSDataInputStreamWrapper(is),
+              totalSize, fs, path, meta);
+          hbr.setDataBlockEncoder(dataBlockEncoder);
+          hbr.setIncludesMemstoreTS(includesMemstoreTS);
+
+          HFileBlock b;
+          int pos = 0;
+          for (int blockId = 0; blockId < numBlocks; ++blockId) {
+            b = hbr.readBlockData(pos, -1, -1, pread);
+            b.sanityCheck();
+            if (meta.isCompressedOrEncrypted()) {
+              assertFalse(b.isUnpacked());
+              b = b.unpack(meta, hbr);
+            }
+            pos += b.getOnDiskSizeWithHeader();
+
+            assertEquals((int) encodedSizes.get(blockId),
+                b.getUncompressedSizeWithoutHeader());
+            ByteBuff actualBuffer = b.getBufferWithoutHeader();
+            if (encoding != DataBlockEncoding.NONE) {
+              // We expect a two-byte big-endian encoding id.
+              assertEquals(0, actualBuffer.get(0));
+              assertEquals(encoding.getId(), actualBuffer.get(1));
+              actualBuffer.position(2);
+              actualBuffer = actualBuffer.slice();
+            }
+
+            ByteBuffer expectedBuffer = encodedBlocks.get(blockId);
+            expectedBuffer.rewind();
+
+            // test if content matches, produce nice message
+            TestHFileBlock.assertBuffersEqual(new SingleByteBuff(expectedBuffer), actualBuffer,
+              algo, encoding, pread);
+          }
+          is.close();
+        }
+      }
+    }
+  }
+  /**
+   * This is the version of the HFileBlock.Writer that is used to
+   * create V2 blocks with minor version 0. These blocks do not
+   * have hbase-level checksums. The code is here to test
+   * backward compatibility. The reason we do not inherit from
+   * HFileBlock.Writer is because we never ever want to change the code
+   * in this class but the code in HFileBlock.Writer will continually
+   * evolve.
+   */
+  public static final class Writer extends HFileBlock.Writer {
+
+    // These constants are as they were in minorVersion 0.
+    private static final int HEADER_SIZE = HConstants.HFILEBLOCK_HEADER_SIZE_NO_CHECKSUM;
+    private static final boolean DONT_FILL_HEADER = HFileBlock.DONT_FILL_HEADER;
+    private static final byte[] DUMMY_HEADER = HFileBlock.DUMMY_HEADER_NO_CHECKSUM;
+
+    private enum State {
+      INIT,
+      WRITING,
+      BLOCK_READY
+    };
+
+    /** Writer state. Used to ensure the correct usage protocol. */
+    private State state = State.INIT;
+
+    /** Compression algorithm for all blocks this instance writes. */
+    private final Compression.Algorithm compressAlgo;
+
+    /** Data block encoder used for data blocks */
+    private final HFileDataBlockEncoder dataBlockEncoder;
+
+    private HFileBlockEncodingContext dataBlockEncodingCtx;
+    /** block encoding context for non-data blocks */
+    private HFileBlockDefaultEncodingContext defaultBlockEncodingCtx;
+
+    /**
+     * The stream we use to accumulate data in uncompressed format for each
+     * block. We reset this stream at the end of each block and reuse it. The
+     * header is written as the first {@link #HEADER_SIZE} bytes into this
+     * stream.
+     */
+    private ByteArrayOutputStream baosInMemory;
+
+    /** Compressor, which is also reused between consecutive blocks. */
+    private Compressor compressor;
+
+    /**
+     * Current block type. Set in {@link #startWriting(BlockType)}. Could be
+     * changed in {@link #encodeDataBlockForDisk()} from {@link BlockType#DATA}
+     * to {@link BlockType#ENCODED_DATA}.
+     */
+    private BlockType blockType;
+
+    /**
+     * A stream that we write uncompressed bytes to, which compresses them and
+     * writes them to {@link #baosInMemory}.
+     */
+    private DataOutputStream userDataStream;
+
+    /**
+     * Bytes to be written to the file system, including the header. Compressed
+     * if compression is turned on.
+     */
+    private byte[] onDiskBytesWithHeader;
+
+    /**
+     * Valid in the READY state. Contains the header and the uncompressed (but
+     * potentially encoded, if this is a data block) bytes, so the length is
+     * {@link #uncompressedSizeWithoutHeader} + {@link org.apache.hadoop.hbase.HConstants#HFILEBLOCK_HEADER_SIZE}.
+     */
+    private byte[] uncompressedBytesWithHeader;
+
+    /**
+     * Current block's start offset in the {@link HFile}. Set in
+     * {@link #writeHeaderAndData(FSDataOutputStream)}.
+     */
+    private long startOffset;
+
+    /**
+     * Offset of previous block by block type. Updated when the next block is
+     * started.
+     */
+    private long[] prevOffsetByType;
+
+    /** The offset of the previous block of the same type */
+    private long prevOffset;
+
+    private int unencodedDataSizeWritten;
+
+    public Writer(Compression.Algorithm compressionAlgorithm,
+        HFileDataBlockEncoder dataBlockEncoder, boolean includesMemstoreTS, boolean includesTag) {
+      this(dataBlockEncoder, new HFileContextBuilder().withHBaseCheckSum(false)
+          .withIncludesMvcc(includesMemstoreTS).withIncludesTags(includesTag)
+          .withCompression(compressionAlgorithm).build());
+    }
+
+    public Writer(HFileDataBlockEncoder dataBlockEncoder, HFileContext meta) {
+      super(dataBlockEncoder, meta);
+      compressAlgo = meta.getCompression() == null ? NONE : meta.getCompression();
+      this.dataBlockEncoder = dataBlockEncoder != null ? dataBlockEncoder
+          : NoOpDataBlockEncoder.INSTANCE;
+      defaultBlockEncodingCtx = new HFileBlockDefaultEncodingContext(null, DUMMY_HEADER, meta);
+      dataBlockEncodingCtx = this.dataBlockEncoder.newDataBlockEncodingContext(DUMMY_HEADER, meta);
+      baosInMemory = new ByteArrayOutputStream();
+
+      prevOffsetByType = new long[BlockType.values().length];
+      for (int i = 0; i < prevOffsetByType.length; ++i)
+        prevOffsetByType[i] = -1;
+    }
+
+    /**
+     * Starts writing into the block. The previous block's data is discarded.
+     *
+     * @return the stream the user can write their data into
+     * @throws IOException
+     */
+    public DataOutputStream startWriting(BlockType newBlockType)
+        throws IOException {
+      if (state == State.BLOCK_READY && startOffset != -1) {
+        // We had a previous block that was written to a stream at a specific
+        // offset. Save that offset as the last offset of a block of that type.
+        prevOffsetByType[blockType.getId()] = startOffset;
+      }
+
+      startOffset = -1;
+      blockType = newBlockType;
+
+      baosInMemory.reset();
+      baosInMemory.write(DUMMY_HEADER);
+
+      state = State.WRITING;
+
+      // We will compress it later in finishBlock()
+      userDataStream = new DataOutputStream(baosInMemory);
+      if (newBlockType == BlockType.DATA) {
+        this.dataBlockEncoder.startBlockEncoding(dataBlockEncodingCtx, userDataStream);
+      }
+      this.unencodedDataSizeWritten = 0;
+      return userDataStream;
+    }
+
+    @Override
+    public void write(Cell c) throws IOException {
+      KeyValue kv = KeyValueUtil.ensureKeyValue(c);
+      expectState(State.WRITING);
+      this.dataBlockEncoder.encode(kv, dataBlockEncodingCtx, this.userDataStream);
+      this.unencodedDataSizeWritten += kv.getLength();
+      if (dataBlockEncodingCtx.getHFileContext().isIncludesMvcc()) {
+        this.unencodedDataSizeWritten += WritableUtils.getVIntSize(kv.getSequenceId());
+      }
+    }
+
+    /**
+     * Returns the stream for the user to write to. The block writer takes care
+     * of handling compression and buffering for caching on write. Can only be
+     * called in the "writing" state.
+     *
+     * @return the data output stream for the user to write to
+     */
+    DataOutputStream getUserDataStream() {
+      expectState(State.WRITING);
+      return userDataStream;
+    }
+
+    /**
+     * Transitions the block writer from the "writing" state to the "block
+     * ready" state.  Does nothing if a block is already finished.
+     */
+    void ensureBlockReady() throws IOException {
+      Preconditions.checkState(state != State.INIT,
+          "Unexpected state: " + state);
+
+      if (state == State.BLOCK_READY)
+        return;
+
+      // This will set state to BLOCK_READY.
+      finishBlock();
+    }
+
+    /**
+     * An internal method that flushes the compressing stream (if using
+     * compression), serializes the header, and takes care of the separate
+     * uncompressed stream for caching on write, if applicable. Sets block
+     * write state to "block ready".
+     */
+    void finishBlock() throws IOException {
+      if (blockType == BlockType.DATA) {
+        this.dataBlockEncoder.endBlockEncoding(dataBlockEncodingCtx, userDataStream,
+            baosInMemory.toByteArray(), blockType);
+        blockType = dataBlockEncodingCtx.getBlockType();
+      }
+      userDataStream.flush();
+      // This does an array copy, so it is safe to cache this byte array.
+      uncompressedBytesWithHeader = baosInMemory.toByteArray();
+      prevOffset = prevOffsetByType[blockType.getId()];
+
+      // We need to set state before we can package the block up for
+      // cache-on-write. In a way, the block is ready, but not yet encoded or
+      // compressed.
+      state = State.BLOCK_READY;
+      if (blockType == BlockType.DATA || blockType == BlockType.ENCODED_DATA) {
+        onDiskBytesWithHeader = dataBlockEncodingCtx
+            .compressAndEncrypt(uncompressedBytesWithHeader);
+      } else {
+        onDiskBytesWithHeader = defaultBlockEncodingCtx
+            .compressAndEncrypt(uncompressedBytesWithHeader);
+      }
+
+      // put the header for on disk bytes
+      putHeader(onDiskBytesWithHeader, 0,
+          onDiskBytesWithHeader.length,
+          uncompressedBytesWithHeader.length);
+      //set the header for the uncompressed bytes (for cache-on-write)
+      putHeader(uncompressedBytesWithHeader, 0,
+          onDiskBytesWithHeader.length,
+        uncompressedBytesWithHeader.length);
+    }
+
+    /**
+     * Put the header into the given byte array at the given offset.
+     * @param onDiskSize size of the block on disk
+     * @param uncompressedSize size of the block after decompression (but
+     *          before optional data block decoding)
+     */
+    private void putHeader(byte[] dest, int offset, int onDiskSize,
+        int uncompressedSize) {
+      offset = blockType.put(dest, offset);
+      offset = Bytes.putInt(dest, offset, onDiskSize - HEADER_SIZE);
+      offset = Bytes.putInt(dest, offset, uncompressedSize - HEADER_SIZE);
+      Bytes.putLong(dest, offset, prevOffset);
+    }
+
+    /**
+     * Similar to {@link #writeHeaderAndData(FSDataOutputStream)}, but records
+     * the offset of this block so that it can be referenced in the next block
+     * of the same type.
+     *
+     * @param out
+     * @throws IOException
+     */
+    public void writeHeaderAndData(FSDataOutputStream out) throws IOException {
+      long offset = out.getPos();
+      if (startOffset != -1 && offset != startOffset) {
+        throw new IOException("A " + blockType + " block written to a "
+            + "stream twice, first at offset " + startOffset + ", then at "
+            + offset);
+      }
+      startOffset = offset;
+
+      writeHeaderAndData((DataOutputStream) out);
+    }
+
+    /**
+     * Writes the header and the compressed data of this block (or uncompressed
+     * data when not using compression) into the given stream. Can be called in
+     * the "writing" state or in the "block ready" state. If called in the
+     * "writing" state, transitions the writer to the "block ready" state.
+     *
+     * @param out the output stream to write the
+     * @throws IOException
+     */
+    private void writeHeaderAndData(DataOutputStream out) throws IOException {
+      ensureBlockReady();
+      out.write(onDiskBytesWithHeader);
+    }
+
+    /**
+     * Returns the header or the compressed data (or uncompressed data when not
+     * using compression) as a byte array. Can be called in the "writing" state
+     * or in the "block ready" state. If called in the "writing" state,
+     * transitions the writer to the "block ready" state.
+     *
+     * @return header and data as they would be stored on disk in a byte array
+     * @throws IOException
+     */
+    public byte[] getHeaderAndData() throws IOException {
+      ensureBlockReady();
+      return onDiskBytesWithHeader;
+    }
+
+    /**
+     * Releases the compressor this writer uses to compress blocks into the
+     * compressor pool. Needs to be called before the writer is discarded.
+     */
+    public void releaseCompressor() {
+      if (compressor != null) {
+        compressAlgo.returnCompressor(compressor);
+        compressor = null;
+      }
+    }
+
+    /**
+     * Returns the on-disk size of the data portion of the block. This is the
+     * compressed size if compression is enabled. Can only be called in the
+     * "block ready" state. Header is not compressed, and its size is not
+     * included in the return value.
+     *
+     * @return the on-disk size of the block, not including the header.
+     */
+    public int getOnDiskSizeWithoutHeader() {
+      expectState(State.BLOCK_READY);
+      return onDiskBytesWithHeader.length - HEADER_SIZE;
+    }
+
+    /**
+     * Returns the on-disk size of the block. Can only be called in the
+     * "block ready" state.
+     *
+     * @return the on-disk size of the block ready to be written, including the
+     *         header size
+     */
+    public int getOnDiskSizeWithHeader() {
+      expectState(State.BLOCK_READY);
+      return onDiskBytesWithHeader.length;
+    }
+
+    /**
+     * The uncompressed size of the block data. Does not include header size.
+     */
+    public int getUncompressedSizeWithoutHeader() {
+      expectState(State.BLOCK_READY);
+      return uncompressedBytesWithHeader.length - HEADER_SIZE;
+    }
+
+    /**
+     * The uncompressed size of the block data, including header size.
+     */
+    public int getUncompressedSizeWithHeader() {
+      expectState(State.BLOCK_READY);
+      return uncompressedBytesWithHeader.length;
+    }
+
+    /** @return true if a block is being written  */
+    public boolean isWriting() {
+      return state == State.WRITING;
+    }
+
+    /**
+     * Returns the number of bytes written into the current block so far, or
+     * zero if not writing the block at the moment. Note that this will return
+     * zero in the "block ready" state as well.
+     *
+     * @return the number of bytes written
+     */
+    public int blockSizeWritten() {
+      if (state != State.WRITING)
+        return 0;
+      return this.unencodedDataSizeWritten;
+    }
+
+    /**
+     * Returns the header followed by the uncompressed data, even if using
+     * compression. This is needed for storing uncompressed blocks in the block
+     * cache. Can be called in the "writing" state or the "block ready" state.
+     *
+     * @return uncompressed block bytes for caching on write
+     */
+    private byte[] getUncompressedDataWithHeader() {
+      expectState(State.BLOCK_READY);
+
+      return uncompressedBytesWithHeader;
+    }
+
+    private void expectState(State expectedState) {
+      if (state != expectedState) {
+        throw new IllegalStateException("Expected state: " + expectedState +
+            ", actual state: " + state);
+      }
+    }
+
+    /**
+     * Similar to {@link #getUncompressedBufferWithHeader()} but returns a byte
+     * buffer.
+     *
+     * @return uncompressed block for caching on write in the form of a buffer
+     */
+    public ByteBuffer getUncompressedBufferWithHeader() {
+      byte[] b = getUncompressedDataWithHeader();
+      return ByteBuffer.wrap(b, 0, b.length);
+    }
+
+    /**
+     * Takes the given {@link BlockWritable} instance, creates a new block of
+     * its appropriate type, writes the writable into this block, and flushes
+     * the block into the output stream. The writer is instructed not to buffer
+     * uncompressed bytes for cache-on-write.
+     *
+     * @param bw the block-writable object to write as a block
+     * @param out the file system output stream
+     * @throws IOException
+     */
+    public void writeBlock(BlockWritable bw, FSDataOutputStream out)
+        throws IOException {
+      bw.writeToBlock(startWriting(bw.getBlockType()));
+      writeHeaderAndData(out);
+    }
+
+    /**
+     * Creates a new HFileBlock.
+     */
+    public HFileBlock getBlockForCaching() {
+      HFileContext meta = new HFileContextBuilder()
+             .withHBaseCheckSum(false)
+             .withChecksumType(ChecksumType.NULL)
+             .withBytesPerCheckSum(0)
+             .build();
+      return new HFileBlock(blockType, getOnDiskSizeWithoutHeader(),
+          getUncompressedSizeWithoutHeader(), prevOffset,
+          getUncompressedBufferWithHeader(), DONT_FILL_HEADER, startOffset,
+          getOnDiskSizeWithoutHeader(), meta);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/54a543de/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 470d483..687d3cd 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
@@ -185,7 +185,8 @@ public class TestHFileBlockIndex {
       }
 
       missCount += 1;
-      prevBlock = realReader.readBlockData(offset, onDiskSize, pread);
+      prevBlock = realReader.readBlockData(offset, onDiskSize,
+          -1, pread);
       prevOffset = offset;
       prevOnDiskSize = onDiskSize;
       prevPread = pread;

http://git-wip-us.apache.org/repos/asf/hbase/blob/54a543de/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileDataBlockEncoder.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileDataBlockEncoder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileDataBlockEncoder.java
index 387514e..6f434bb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileDataBlockEncoder.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileDataBlockEncoder.java
@@ -92,7 +92,8 @@ public class TestHFileDataBlockEncoder {
 
     if (blockEncoder.getDataBlockEncoding() ==
         DataBlockEncoding.NONE) {
-      assertEquals(block.getBufferReadOnly(), returnedBlock.getBufferReadOnly());
+      assertEquals(block.getBufferWithHeader(),
+          returnedBlock.getBufferWithHeader());
     } else {
       if (BlockType.ENCODED_DATA != returnedBlock.getBlockType()) {
         System.out.println(blockEncoder);
@@ -126,7 +127,7 @@ public class TestHFileDataBlockEncoder {
                         .build();
     HFileBlock block = new HFileBlock(BlockType.DATA, size, size, -1, buf,
         HFileBlock.FILL_HEADER, 0,
-        0, -1, hfileContext);
+        0, hfileContext);
     HFileBlock cacheBlock = createBlockOnDisk(kvs, block, useTags);
     assertEquals(headerSize, cacheBlock.getDummyHeaderForVersion().length);
   }
@@ -197,7 +198,7 @@ public class TestHFileDataBlockEncoder {
                         .build();
     HFileBlock b = new HFileBlock(BlockType.DATA, size, size, -1, buf,
         HFileBlock.FILL_HEADER, 0, 
-         0, -1, meta);
+         0, meta);
     return b;
   }
 
@@ -219,8 +220,7 @@ public class TestHFileDataBlockEncoder {
     byte[] encodedBytes = baos.toByteArray();
     size = encodedBytes.length - block.getDummyHeaderForVersion().length;
     return new HFileBlock(context.getBlockType(), size, size, -1, ByteBuffer.wrap(encodedBytes),
-        HFileBlock.FILL_HEADER, 0, block.getOnDiskDataSizeWithHeader(), -1,
-        block.getHFileContext());
+        HFileBlock.FILL_HEADER, 0, block.getOnDiskDataSizeWithHeader(), block.getHFileContext());
   }
 
   private void writeBlock(List<Cell> kvs, HFileContext fileContext, boolean useTags)

http://git-wip-us.apache.org/repos/asf/hbase/blob/54a543de/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileEncryption.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileEncryption.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileEncryption.java
index 3264558..ba3a344 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileEncryption.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileEncryption.java
@@ -99,7 +99,7 @@ public class TestHFileEncryption {
 
   private long readAndVerifyBlock(long pos, HFileContext ctx, HFileBlock.FSReaderImpl hbr, int size)
       throws IOException {
-    HFileBlock b = hbr.readBlockData(pos, -1, false);
+    HFileBlock b = hbr.readBlockData(pos, -1, -1, false);
     assertEquals(0, HFile.getChecksumFailuresCount());
     b.sanityCheck();
     assertFalse(b.isUnpacked());

http://git-wip-us.apache.org/repos/asf/hbase/blob/54a543de/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java
index 983ec2f..c7eb11b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java
@@ -218,7 +218,7 @@ public class TestHFileWriterV3 {
     fsdis.seek(0);
     long curBlockPos = 0;
     while (curBlockPos <= trailer.getLastDataBlockOffset()) {
-      HFileBlock block = blockReader.readBlockData(curBlockPos, -1, false)
+      HFileBlock block = blockReader.readBlockData(curBlockPos, -1, -1, false)
         .unpack(context, blockReader);
       assertEquals(BlockType.DATA, block.getBlockType());
       ByteBuff buf = block.getBufferWithoutHeader();
@@ -279,14 +279,13 @@ public class TestHFileWriterV3 {
     while (fsdis.getPos() < trailer.getLoadOnOpenDataOffset()) {
       LOG.info("Current offset: " + fsdis.getPos() + ", scanning until " +
           trailer.getLoadOnOpenDataOffset());
-      HFileBlock block = blockReader.readBlockData(curBlockPos, -1, false)
+      HFileBlock block = blockReader.readBlockData(curBlockPos, -1, -1, false)
         .unpack(context, blockReader);
       assertEquals(BlockType.META, block.getBlockType());
       Text t = new Text();
       ByteBuff buf = block.getBufferWithoutHeader();
       if (Writables.getWritable(buf.array(), buf.arrayOffset(), buf.limit(), t) == null) {
-        throw new IOException("Failed to deserialize block " + this +
-            " into a " + t.getClass().getSimpleName());
+        throw new IOException("Failed to deserialize block " + this + " into a " + t.getClass().getSimpleName());
       }
       Text expectedText =
           (metaCounter == 0 ? new Text("Paris") : metaCounter == 1 ? new Text(

http://git-wip-us.apache.org/repos/asf/hbase/blob/54a543de/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java
index d20ba2b..69a77bf 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java
@@ -78,8 +78,14 @@ public class TestPrefetch {
     // Check that all of the data blocks were preloaded
     BlockCache blockCache = cacheConf.getBlockCache();
     long offset = 0;
+    HFileBlock prevBlock = null;
     while (offset < reader.getTrailer().getLoadOnOpenDataOffset()) {
-      HFileBlock block = reader.readBlock(offset, -1, false, true, false, true, null, null);
+      long onDiskSize = -1;
+      if (prevBlock != null) {
+         onDiskSize = prevBlock.getNextBlockOnDiskSizeWithHeader();
+      }
+      HFileBlock block = reader.readBlock(offset, onDiskSize, false, true, false, true, null,
+        null);
       BlockCacheKey blockCacheKey = new BlockCacheKey(reader.getName(), offset);
       boolean isCached = blockCache.getBlock(blockCacheKey, true, false, true) != null;
       if (block.getBlockType() == BlockType.DATA ||
@@ -87,6 +93,7 @@ public class TestPrefetch {
           block.getBlockType() == BlockType.INTERMEDIATE_INDEX) {
         assertTrue(isCached);
       }
+      prevBlock = block;
       offset += block.getOnDiskSizeWithHeader();
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/54a543de/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
index 2357bef..0916fe6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
@@ -227,10 +227,15 @@ public class TestCacheOnWriteInSchema {
       assertTrue(testDescription, scanner.seekTo());
       // Cribbed from io.hfile.TestCacheOnWrite
       long offset = 0;
+      HFileBlock prevBlock = null;
       while (offset < reader.getTrailer().getLoadOnOpenDataOffset()) {
+        long onDiskSize = -1;
+        if (prevBlock != null) {
+          onDiskSize = prevBlock.getNextBlockOnDiskSizeWithHeader();
+        }
         // Flags: don't cache the block, use pread, this is not a compaction.
         // Also, pass null for expected block type to avoid checking it.
-        HFileBlock block = reader.readBlock(offset, -1, false, true,
+        HFileBlock block = reader.readBlock(offset, onDiskSize, false, true,
           false, true, null, DataBlockEncoding.NONE);
         BlockCacheKey blockCacheKey = new BlockCacheKey(reader.getName(),
           offset);
@@ -244,6 +249,7 @@ public class TestCacheOnWriteInSchema {
             "block: " + block + "\n" +
             "blockCacheKey: " + blockCacheKey);
         }
+        prevBlock = block;
         offset += block.getOnDiskSizeWithHeader();
       }
     } finally {


[10/50] [abbrv] hbase git commit: HBASE-15412 Add average region size metric (Alicia Ying Shu)

Posted by sy...@apache.org.
HBASE-15412 Add average region size metric (Alicia Ying Shu)


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

Branch: refs/heads/hbase-12439
Commit: b3fe4ed16c45e6411f7163099a8bc4c18c39779e
Parents: 2348478
Author: Enis Soztutar <en...@apache.org>
Authored: Tue Mar 22 14:46:27 2016 -0700
Committer: Enis Soztutar <en...@apache.org>
Committed: Tue Mar 22 14:46:27 2016 -0700

----------------------------------------------------------------------
 .../regionserver/MetricsRegionServerSource.java |  3 +++
 .../MetricsRegionServerWrapper.java             |  5 +++++
 .../MetricsRegionServerSourceImpl.java          |  2 +-
 .../MetricsRegionServerWrapperImpl.java         | 13 +++++++++++-
 .../MetricsRegionServerWrapperStub.java         |  5 +++++
 .../regionserver/TestRegionServerMetrics.java   | 21 ++++++++++++++++++++
 6 files changed, 47 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b3fe4ed1/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java
index f097296..9693bba 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java
@@ -463,4 +463,7 @@ public interface MetricsRegionServerSource extends BaseSource {
   String RPC_MUTATE_REQUEST_COUNT = "rpcMutateRequestCount";
   String RPC_MUTATE_REQUEST_COUNT_DESC =
       "Number of rpc mutation requests this region server has answered.";
+  String AVERAGE_REGION_SIZE = "averageRegionSize";
+  String AVERAGE_REGION_SIZE_DESC = 
+      "Average region size over the region server including memstore and storefile sizes.";
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b3fe4ed1/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java
index 3ae6f9c..5ecda04 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java
@@ -412,4 +412,9 @@ public interface MetricsRegionServerWrapper {
    * Get the number of rpc mutate requests to this region server.
    */
   long getRpcMutateRequestsCount();
+
+  /**
+   * Get the average region size to this region server.
+   */
+  long getAverageRegionSize();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b3fe4ed1/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
index 0c24cb4..c625d49 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
@@ -328,6 +328,7 @@ public class MetricsRegionServerSourceImpl
               rsWrap.getNumReferenceFiles())
           .addGauge(Interns.info(RS_START_TIME_NAME, RS_START_TIME_DESC),
               rsWrap.getStartCode())
+          .addGauge(Interns.info(AVERAGE_REGION_SIZE, AVERAGE_REGION_SIZE_DESC), rsWrap.getAverageRegionSize())
           .addCounter(Interns.info(TOTAL_REQUEST_COUNT, TOTAL_REQUEST_COUNT_DESC),
               rsWrap.getTotalRequestCount())
           .addCounter(Interns.info(READ_REQUEST_COUNT, READ_REQUEST_COUNT_DESC),
@@ -450,7 +451,6 @@ public class MetricsRegionServerSourceImpl
           .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())
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b3fe4ed1/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
index 441fea6..6e6d07d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
@@ -104,6 +104,7 @@ class MetricsRegionServerWrapperImpl
   private volatile long mobFileCacheEvictedCount = 0;
   private volatile long mobFileCacheCount = 0;
   private volatile long blockedRequestsCount = 0L;
+  private volatile long averageRegionSize = 0L;
 
   private CacheStats cacheStats;
   private ScheduledExecutorService executor;
@@ -668,7 +669,7 @@ class MetricsRegionServerWrapperImpl
         long tempMobScanCellsCount = 0;
         long tempMobScanCellsSize = 0;
         long tempBlockedRequestsCount = 0;
-
+        int regionCount = 0;
         for (Region r : regionServer.getOnlineRegionsLocalContext()) {
           tempNumMutationsWithoutWAL += r.getNumMutationsWithoutWAL();
           tempDataInMemoryWithoutWAL += r.getDataInMemoryWithoutWAL();
@@ -726,6 +727,7 @@ class MetricsRegionServerWrapperImpl
           if (r.getRegionInfo().getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
             hdfsBlocksDistributionSecondaryRegions.add(distro);
           }
+          regionCount++;
         }
         float localityIndex = hdfsBlocksDistribution.getBlockLocalityIndex(
             regionServer.getServerName().getHostname());
@@ -765,6 +767,9 @@ class MetricsRegionServerWrapperImpl
         memstoreSize = tempMemstoreSize;
         storeFileSize = tempStoreFileSize;
         maxStoreFileAge = tempMaxStoreFileAge;
+        if (regionCount > 0) {
+          averageRegionSize = (memstoreSize + storeFileSize) / regionCount;
+        }
         if (tempMinStoreFileAge != Long.MAX_VALUE) {
           minStoreFileAge = tempMinStoreFileAge;
         }
@@ -828,4 +833,10 @@ class MetricsRegionServerWrapperImpl
   public long getBlockedRequestsCount() {
     return blockedRequestsCount;
   }
+
+  @Override
+  public long getAverageRegionSize() {
+    return averageRegionSize;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b3fe4ed1/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java
index 65c6b76..9aa49ba 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java
@@ -404,4 +404,9 @@ public class MetricsRegionServerWrapperStub implements MetricsRegionServerWrappe
   public double getMobFileCacheHitPercent() {
     return 50;
   }
+
+  @Override
+  public long getAverageRegionSize() {
+    return 10000000;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b3fe4ed1/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
index b9e54f0..2087097 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
@@ -676,4 +676,25 @@ public class TestRegionServerMetrics {
 
     table.close();
   }
+
+  @Test
+  public void testAverageRegionSize() throws Exception {
+    TableName tableName = TableName.valueOf("testAverageRegionSize");
+    byte[] cf = Bytes.toBytes("d");
+    byte[] row = Bytes.toBytes("rk");
+    byte[] qualifier = Bytes.toBytes("qual");
+    byte[] val = Bytes.toBytes("Value");
+
+    //Force a hfile.
+    Table t = TEST_UTIL.createTable(tableName, cf);
+    Put p = new Put(row);
+    p.addColumn(cf, qualifier, val);
+    t.put(p);
+    TEST_UTIL.getHBaseAdmin().flush(tableName);
+
+    metricsRegionServer.getRegionServerWrapper().forceRecompute();
+    assertTrue(metricsHelper.getGaugeDouble("averageRegionSize", serverSource) > 0.0);
+
+    t.close();
+  }
 }


[35/50] [abbrv] hbase git commit: HBASE-15496 Throw RowTooBigException only for user scan/get (Guanghao Zhang)

Posted by sy...@apache.org.
HBASE-15496 Throw RowTooBigException only for user scan/get (Guanghao Zhang)


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

Branch: refs/heads/hbase-12439
Commit: b865fb9434f3fc700cced9d6cdc5235794f8b5ba
Parents: 11d11d3
Author: tedyu <yu...@gmail.com>
Authored: Fri Mar 25 18:18:35 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Fri Mar 25 18:18:35 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java   | 4 ++++
 .../java/org/apache/hadoop/hbase/regionserver/StoreScanner.java  | 4 ++--
 2 files changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b865fb94/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java
index 706fc5b..bfd399b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java
@@ -610,6 +610,10 @@ public class ScanQueryMatcher {
     }
   }
 
+  boolean isUserScan() {
+    return this.isUserScan;
+  }
+
   //Used only for testing purposes
   static MatchCode checkColumn(ColumnTracker columnTracker, byte[] bytes, int offset,
       int length, long ttl, byte type, boolean ignoreCount) throws IOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/b865fb94/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
index 8dd3d7d..ecae787 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
@@ -351,7 +351,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
       if (!isParallelSeek) {
         long totalScannersSoughtBytes = 0;
         for (KeyValueScanner scanner : scanners) {
-          if (totalScannersSoughtBytes >= maxRowSize) {
+          if (matcher.isUserScan() && totalScannersSoughtBytes >= maxRowSize) {
             throw new RowTooBigException("Max row size allowed: " + maxRowSize
               + ", but row is bigger than that");
           }
@@ -572,7 +572,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
           scannerContext.incrementSizeProgress(CellUtil.estimatedHeapSizeOf(cell));
           scannerContext.incrementBatchProgress(1);
 
-          if (totalBytesRead > maxRowSize) {
+          if (matcher.isUserScan() && totalBytesRead > maxRowSize) {
             throw new RowTooBigException(
                 "Max row size allowed: " + maxRowSize + ", but the row is bigger than that.");
           }


[03/50] [abbrv] hbase git commit: HBASE-15447 Improve javadocs description for Delete methods

Posted by sy...@apache.org.
HBASE-15447 Improve javadocs description for Delete methods <Wellington Chevreuil>


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

Branch: refs/heads/hbase-12439
Commit: b6e1f630743a7e09f319145e357242a56580f0d7
Parents: 47471c3
Author: Misty Stanley-Jones <ms...@cloudera.com>
Authored: Tue Mar 22 12:50:51 2016 -0700
Committer: Misty Stanley-Jones <ms...@cloudera.com>
Committed: Tue Mar 22 12:56:30 2016 -0700

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/b6e1f630/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java
index 1e4f79f..c886b34 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java
@@ -76,7 +76,9 @@ public class Delete extends Mutation implements Comparable<Row> {
    * <p>
    * If no further operations are done, this will delete everything
    * associated with the specified row (all versions of all columns in all
-   * families).
+   * families), with timestamp from current point in time to the past.
+   * Cells defining timestamp for a future point in time
+   * (timestamp > current time) will not be deleted.
    * @param row row key
    */
   public Delete(byte [] row) {


[21/50] [abbrv] hbase git commit: HBASE-15508 Add command for exporting snapshot in hbase command script (Yufeng Jiang)

Posted by sy...@apache.org.
HBASE-15508 Add command for exporting snapshot in hbase command script (Yufeng Jiang)


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

Branch: refs/heads/hbase-12439
Commit: 7177e4be64d134144ddc64b8635b06c7d3141f0c
Parents: cadfb21
Author: Matteo Bertozzi <ma...@cloudera.com>
Authored: Wed Mar 23 10:26:32 2016 -0700
Committer: Matteo Bertozzi <ma...@cloudera.com>
Committed: Wed Mar 23 10:26:32 2016 -0700

----------------------------------------------------------------------
 bin/hbase                                       | 35 +++++++++++++++-----
 .../hadoop/hbase/snapshot/ExportSnapshot.java   |  4 +--
 .../hadoop/hbase/snapshot/SnapshotInfo.java     |  4 +--
 3 files changed, 30 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/7177e4be/bin/hbase
----------------------------------------------------------------------
diff --git a/bin/hbase b/bin/hbase
index 7faaa26..be519ac 100755
--- a/bin/hbase
+++ b/bin/hbase
@@ -71,20 +71,22 @@ if [ -d "${HBASE_HOME}/target" ]; then
   in_dev_env=true
 fi
 
+read -d '' options_string << EOF
+Options:
+  --config DIR     Configuration direction to use. Default: ./conf
+  --hosts HOSTS    Override the list in 'regionservers' file
+  --auth-as-server Authenticate to ZooKeeper using servers configuration
+EOF
 # if no args specified, show usage
 if [ $# = 0 ]; then
   echo "Usage: hbase [<options>] <command> [<args>]"
-  echo "Options:"
-  echo "  --config DIR    Configuration direction to use. Default: ./conf"
-  echo "  --hosts HOSTS   Override the list in 'regionservers' file"
-  echo "  --auth-as-server Authenticate to ZooKeeper using servers configuration"
+  echo "$options_string"
   echo ""
   echo "Commands:"
   echo "Some commands take arguments. Pass no args or -h for usage."
   echo "  shell           Run the HBase shell"
   echo "  hbck            Run the hbase 'fsck' tool"
-  echo "  snapshot        Create a new snapshot of a table"
-  echo "  snapshotinfo    Tool for dumping snapshot information"
+  echo "  snapshot        Tool for managing snapshots"
   echo "  wal             Write-ahead-log analyzer"
   echo "  hfile           Store file analyzer"
   echo "  zkcli           Run the ZooKeeper shell"
@@ -318,9 +320,24 @@ elif [ "$COMMAND" = "upgrade" ] ; then
   echo "Please follow the documentation at http://hbase.apache.org/book.html#upgrading."
   exit 1
 elif [ "$COMMAND" = "snapshot" ] ; then
-  CLASS="org.apache.hadoop.hbase.snapshot.CreateSnapshot"
-elif [ "$COMMAND" = "snapshotinfo" ] ; then
-  CLASS="org.apache.hadoop.hbase.snapshot.SnapshotInfo"
+  SUBCOMMAND=$1
+  shift
+  if [ "$SUBCOMMAND" = "create" ] ; then
+    CLASS="org.apache.hadoop.hbase.snapshot.CreateSnapshot"
+  elif [ "$SUBCOMMAND" = "info" ] ; then
+    CLASS="org.apache.hadoop.hbase.snapshot.SnapshotInfo"
+  elif [ "$SUBCOMMAND" = "export" ] ; then
+    CLASS="org.apache.hadoop.hbase.snapshot.ExportSnapshot"
+  else
+    echo "Usage: hbase [<options>] snapshot <subcommand> [<args>]"
+    echo "$options_string"
+    echo ""
+    echo "Subcommands:"
+    echo "  create          Create a new snapshot of a table"
+    echo "  info            Tool for dumping snapshot information"
+    echo "  export          Export an existing snapshot"
+    exit 1
+  fi
 elif [ "$COMMAND" = "master" ] ; then
   CLASS='org.apache.hadoop.hbase.master.HMaster'
   if [ "$1" != "stop" ] && [ "$1" != "clear" ] ; then

http://git-wip-us.apache.org/repos/asf/hbase/blob/7177e4be/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
index d17e36b..9734f43 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
@@ -1057,11 +1057,11 @@ public class ExportSnapshot extends Configured implements Tool {
     System.err.println("  -bandwidth              Limit bandwidth to this value in MB/second.");
     System.err.println();
     System.err.println("Examples:");
-    System.err.println("  hbase " + getClass().getName() + " \\");
+    System.err.println("  hbase snapshot export \\");
     System.err.println("    -snapshot MySnapshot -copy-to hdfs://srv2:8082/hbase \\");
     System.err.println("    -chuser MyUser -chgroup MyGroup -chmod 700 -mappers 16");
     System.err.println();
-    System.err.println("  hbase " + getClass().getName() + " \\");
+    System.err.println("  hbase snapshot export \\");
     System.err.println("    -snapshot MySnapshot -copy-from hdfs://srv2:8082/hbase \\");
     System.err.println("    -copy-to hdfs://srv1:50070/hbase \\");
     System.exit(1);

http://git-wip-us.apache.org/repos/asf/hbase/blob/7177e4be/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotInfo.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotInfo.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotInfo.java
index 95803f5..0a359f8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotInfo.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotInfo.java
@@ -467,7 +467,7 @@ public final class SnapshotInfo extends Configured implements Tool {
   }
 
   private void printUsageAndExit() {
-    System.err.printf("Usage: bin/hbase %s [options]%n", getClass().getName());
+    System.err.printf("Usage: bin/hbase snapshot info [options]%n");
     System.err.println(" where [options] are:");
     System.err.println("  -h|-help                Show this help and exit.");
     System.err.println("  -remote-dir             Root directory that contains the snapshots.");
@@ -479,7 +479,7 @@ public final class SnapshotInfo extends Configured implements Tool {
     System.err.println("  -schema                 Describe the snapshotted table.");
     System.err.println();
     System.err.println("Examples:");
-    System.err.println("  hbase " + getClass() + " \\");
+    System.err.println("  hbase snapshot info \\");
     System.err.println("    -snapshot MySnapshot -files");
     System.exit(1);
   }


[38/50] [abbrv] hbase git commit: HBASE-15265 Implement an asynchronous FSHLog

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java
index 06318f0..24f195d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java
@@ -48,17 +48,20 @@ import org.apache.hadoop.hbase.wal.WALKey;
 class FSWALEntry extends Entry {
   // The below data members are denoted 'transient' just to highlight these are not persisted;
   // they are only in memory and held here while passing over the ring buffer.
-  private final transient long sequence;
+  private final transient long txid;
   private final transient boolean inMemstore;
   private final transient HRegionInfo hri;
-  private final Set<byte[]> familyNames;
+  private final transient Set<byte[]> familyNames;
+  // In the new WAL logic, we will rewrite failed WAL entries to new WAL file, so we need to avoid
+  // calling stampRegionSequenceId again.
+  private transient boolean stamped = false;
 
-  FSWALEntry(final long sequence, final WALKey key, final WALEdit edit,
+  FSWALEntry(final long txid, final WALKey key, final WALEdit edit,
       final HRegionInfo hri, final boolean inMemstore) {
     super(key, edit);
     this.inMemstore = inMemstore;
     this.hri = hri;
-    this.sequence = sequence;
+    this.txid = txid;
     if (inMemstore) {
       // construct familyNames here to reduce the work of log sinker.
       ArrayList<Cell> cells = this.getEdit().getCells();
@@ -80,7 +83,7 @@ class FSWALEntry extends Entry {
   }
 
   public String toString() {
-    return "sequence=" + this.sequence + ", " + super.toString();
+    return "sequence=" + this.txid + ", " + super.toString();
   };
 
   boolean isInMemstore() {
@@ -92,10 +95,10 @@ class FSWALEntry extends Entry {
   }
 
   /**
-   * @return The sequence on the ring buffer when this edit was added.
+   * @return The transaction id of this edit.
    */
-  long getSequence() {
-    return this.sequence;
+  long getTxid() {
+    return this.txid;
   }
 
   /**
@@ -103,9 +106,12 @@ class FSWALEntry extends Entry {
    * SIDE-EFFECT is our stamping the sequenceid into every Cell AND setting the sequenceid into the
    * MVCC WriteEntry!!!!
    * @return The sequenceid we stamped on this edit.
-   * @throws IOException
    */
   long stampRegionSequenceId() throws IOException {
+    if (stamped) {
+      return getKey().getSequenceId();
+    }
+    stamped = true;
     long regionSequenceId = WALKey.NO_SEQUENCE_ID;
     MultiVersionConcurrencyControl mvcc = getKey().getMvcc();
     MultiVersionConcurrencyControl.WriteEntry we = null;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogWriter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogWriter.java
index 42abeae..7161e1e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogWriter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogWriter.java
@@ -19,108 +19,42 @@
 
 package org.apache.hadoop.hbase.regionserver.wal;
 
-import static org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader.DEFAULT_WAL_TRAILER_WARN_SIZE;
-import static org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader.WAL_TRAILER_WARN_SIZE;
-
 import java.io.IOException;
+import java.io.OutputStream;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-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.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.codec.Codec;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALHeader;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALTrailer;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.wal.DefaultWALProvider;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
 
 /**
  * Writer for protobuf-based WAL.
  */
-@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
-public class ProtobufLogWriter extends WriterBase {
-  private static final Log LOG = LogFactory.getLog(ProtobufLogWriter.class);
-  protected FSDataOutputStream output;
-  protected Codec.Encoder cellEncoder;
-  protected WALCellCodec.ByteStringCompressor compressor;
-  private boolean trailerWritten;
-  private WALTrailer trailer;
-  // maximum size of the wal Trailer in bytes. If a user writes/reads a trailer with size larger
-  // than this size, it is written/read respectively, with a WARN message in the log.
-  private int trailerWarnSize;
-
-  public ProtobufLogWriter() {
-    super();
-  }
-
-  protected WALCellCodec getCodec(Configuration conf, CompressionContext compressionContext)
-      throws IOException {
-    return WALCellCodec.create(conf, null, compressionContext);
-  }
-
-  protected WALHeader buildWALHeader(Configuration conf, WALHeader.Builder builder)
-      throws IOException {
-    if (!builder.hasWriterClsName()) {
-      builder.setWriterClsName(ProtobufLogWriter.class.getSimpleName());
-    }
-    if (!builder.hasCellCodecClsName()) {
-      builder.setCellCodecClsName(WALCellCodec.getWALCellCodecClass(conf));
-    }
-    return builder.build();
-  }
+@InterfaceAudience.Private
+public class ProtobufLogWriter extends AbstractProtobufLogWriter
+    implements DefaultWALProvider.Writer {
 
-  @Override
-  @SuppressWarnings("deprecation")
-  public void init(FileSystem fs, Path path, Configuration conf, boolean overwritable)
-  throws IOException {
-    super.init(fs, path, conf, overwritable);
-    assert this.output == null;
-    boolean doCompress = initializeCompressionContext(conf, path);
-    this.trailerWarnSize = conf.getInt(WAL_TRAILER_WARN_SIZE, DEFAULT_WAL_TRAILER_WARN_SIZE);
-    int bufferSize = FSUtils.getDefaultBufferSize(fs);
-    short replication = (short)conf.getInt(
-        "hbase.regionserver.hlog.replication", FSUtils.getDefaultReplication(fs, path));
-    long blockSize = conf.getLong("hbase.regionserver.hlog.blocksize",
-        FSUtils.getDefaultBlockSize(fs, path));
-    output = fs.createNonRecursive(path, overwritable, bufferSize, replication, blockSize, null);
-    output.write(ProtobufLogReader.PB_WAL_MAGIC);
-    boolean doTagCompress = doCompress
-        && conf.getBoolean(CompressionContext.ENABLE_WAL_TAGS_COMPRESSION, true);
-    buildWALHeader(conf,
-        WALHeader.newBuilder().setHasCompression(doCompress).setHasTagCompression(doTagCompress))
-        .writeDelimitedTo(output);
-
-    initAfterHeader(doCompress);
-
-    // instantiate trailer to default value.
-    trailer = WALTrailer.newBuilder().build();
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Initialized protobuf WAL=" + path + ", compression=" + doCompress);
-    }
-  }
+  private static final Log LOG = LogFactory.getLog(ProtobufLogWriter.class);
 
-  protected void initAfterHeader(boolean doCompress) throws IOException {
-    WALCellCodec codec = getCodec(conf, this.compressionContext);
-    this.cellEncoder = codec.getEncoder(this.output);
-    if (doCompress) {
-      this.compressor = codec.getByteStringCompressor();
-    }
-  }
+  protected FSDataOutputStream output;
 
   @Override
   public void append(Entry entry) throws IOException {
     entry.setCompressionContext(compressionContext);
-    entry.getKey().getBuilder(compressor).
-      setFollowingKvCount(entry.getEdit().size()).build().writeDelimitedTo(output);
+    entry.getKey().getBuilder(compressor).setFollowingKvCount(entry.getEdit().size()).build()
+        .writeDelimitedTo(output);
     for (Cell cell : entry.getEdit().getCells()) {
       // cellEncoder must assume little about the stream, since we write PB and cells in turn.
       cellEncoder.write(cell);
     }
+    length.set(output.getPos());
   }
 
   @Override
@@ -137,32 +71,6 @@ public class ProtobufLogWriter extends WriterBase {
     }
   }
 
-  WALTrailer buildWALTrailer(WALTrailer.Builder builder) {
-    return builder.build();
-  }
-
-  private void writeWALTrailer() {
-    try {
-      int trailerSize = 0;
-      if (this.trailer == null) {
-        // use default trailer.
-        LOG.warn("WALTrailer is null. Continuing with default.");
-        this.trailer = buildWALTrailer(WALTrailer.newBuilder());
-        trailerSize = this.trailer.getSerializedSize();
-      } else if ((trailerSize = this.trailer.getSerializedSize()) > this.trailerWarnSize) {
-        // continue writing after warning the user.
-        LOG.warn("Please investigate WALTrailer usage. Trailer size > maximum size : " +
-          trailerSize + " > " + this.trailerWarnSize);
-      }
-      this.trailer.writeTo(output);
-      output.writeInt(trailerSize);
-      output.write(ProtobufLogReader.PB_WAL_COMPLETE_MAGIC);
-      this.trailerWritten = true;
-    } catch (IOException ioe) {
-      LOG.warn("Failed to write trailer, non-fatal, continuing...", ioe);
-    }
-  }
-
   @Override
   public void sync() throws IOException {
     FSDataOutputStream fsdos = this.output;
@@ -171,21 +79,35 @@ public class ProtobufLogWriter extends WriterBase {
     fsdos.hflush();
   }
 
+  public FSDataOutputStream getStream() {
+    return this.output;
+  }
+
+  @SuppressWarnings("deprecation")
   @Override
-  public long getLength() throws IOException {
-    try {
-      return this.output.getPos();
-    } catch (NullPointerException npe) {
-      // Concurrent close...
-      throw new IOException(npe);
-    }
+  protected void initOutput(FileSystem fs, Path path, boolean overwritable, int bufferSize,
+      short replication, long blockSize) throws IOException {
+    this.output = fs.createNonRecursive(path, overwritable, bufferSize, replication, blockSize,
+      null);
   }
 
-  public FSDataOutputStream getStream() {
+  @Override
+  protected long writeMagicAndWALHeader(byte[] magic, WALHeader header) throws IOException {
+    output.write(magic);
+    header.writeDelimitedTo(output);
+    return output.getPos();
+  }
+
+  @Override
+  protected OutputStream getOutputStreamForCellEncoder() {
     return this.output;
   }
 
-  void setWALTrailer(WALTrailer walTrailer) {
-    this.trailer = walTrailer;
+  @Override
+  protected long writeWALTrailerAndMagic(WALTrailer trailer, byte[] magic) throws IOException {
+    trailer.writeTo(output);
+    output.writeInt(trailer.getSerializedSize());
+    output.write(magic);
+    return output.getPos();
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SyncFuture.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SyncFuture.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SyncFuture.java
index 7de8367..b5c9a2e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SyncFuture.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SyncFuture.java
@@ -18,33 +18,28 @@
 package org.apache.hadoop.hbase.regionserver.wal;
 
 import java.util.concurrent.ExecutionException;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.htrace.Span;
 
 /**
- * A Future on a filesystem sync call.  It given to a client or 'Handler' for it to wait on till
- * the sync completes.
- *
- * <p>Handlers coming in call append, append, append, and then do a flush/sync of
- * the edits they have appended the WAL before returning. Since sync takes a while to
- * complete, we give the Handlers back this sync future to wait on until the
- * actual HDFS sync completes. Meantime this sync future goes across the ringbuffer and into a
- * sync runner thread; when it completes, it finishes up the future, the handler get or failed
- * check completes and the Handler can then progress.
+ * A Future on a filesystem sync call. It given to a client or 'Handler' for it to wait on till the
+ * sync completes.
+ * <p>
+ * Handlers coming in call append, append, append, and then do a flush/sync of the edits they have
+ * appended the WAL before returning. Since sync takes a while to complete, we give the Handlers
+ * back this sync future to wait on until the actual HDFS sync completes. Meantime this sync future
+ * goes across a queue and is handled by a background thread; when it completes, it finishes up the
+ * future, the handler get or failed check completes and the Handler can then progress.
  * <p>
- * This is just a partial implementation of Future; we just implement get and
- * failure.  Unimplemented methods throw {@link UnsupportedOperationException}.
+ * This is just a partial implementation of Future; we just implement get and failure.
  * <p>
- * There is not a one-to-one correlation between dfs sync invocations and
- * instances of this class. A single dfs sync call may complete and mark many
- * SyncFutures as done; i.e. we batch up sync calls rather than do a dfs sync
- * call every time a Handler asks for it.
+ * There is not a one-to-one correlation between dfs sync invocations and instances of this class. A
+ * single dfs sync call may complete and mark many SyncFutures as done; i.e. we batch up sync calls
+ * rather than do a dfs sync call every time a Handler asks for it.
  * <p>
- * SyncFutures are immutable but recycled. Call #reset(long, Span) before use even
- * if it the first time, start the sync, then park the 'hitched' thread on a call to
- * #get().
+ * SyncFutures are immutable but recycled. Call #reset(long, Span) before use even if it the first
+ * time, start the sync, then park the 'hitched' thread on a call to #get().
  */
 @InterfaceAudience.Private
 class SyncFuture {
@@ -54,17 +49,17 @@ class SyncFuture {
   private static final long NOT_DONE = 0;
 
   /**
-   * The sequence at which we were added to the ring buffer.
+   * The transaction id of this operation, monotonically increases.
    */
-  private long ringBufferSequence;
+  private long txid;
 
   /**
-   * The sequence that was set in here when we were marked done. Should be equal
-   * or > ringBufferSequence.  Put this data member into the NOT_DONE state while this
-   * class is in use.  But for the first position on construction, let it be -1 so we can
-   * immediately call {@link #reset(long, Span)} below and it will work.
+   * The transaction id that was set in here when we were marked done. Should be equal or > txnId.
+   * Put this data member into the NOT_DONE state while this class is in use. But for the first
+   * position on construction, let it be -1 so we can immediately call {@link #reset(long, Span)}
+   * below and it will work.
    */
-  private long doneSequence = -1;
+  private long doneTxid = -1;
 
   /**
    * If error, the associated throwable. Set when the future is 'done'.
@@ -79,80 +74,83 @@ class SyncFuture {
   private Span span;
 
   /**
-   * Call this method to clear old usage and get it ready for new deploy. Call
-   * this method even if it is being used for the first time.
-   *
-   * @param sequence sequenceId from this Future's position in the RingBuffer
+   * Call this method to clear old usage and get it ready for new deploy. Call this method even if
+   * it is being used for the first time.
+   * @param txnId the new transaction id
    * @return this
    */
-  synchronized SyncFuture reset(final long sequence) {
-    return reset(sequence, null);
+  synchronized SyncFuture reset(final long txnId) {
+    return reset(txnId, null);
   }
 
   /**
-   * Call this method to clear old usage and get it ready for new deploy. Call
-   * this method even if it is being used for the first time.
-   *
+   * Call this method to clear old usage and get it ready for new deploy. Call this method even if
+   * it is being used for the first time.
    * @param sequence sequenceId from this Future's position in the RingBuffer
-   * @param span curren span, detached from caller. Don't forget to attach it when
-   *             resuming after a call to {@link #get()}.
+   * @param span curren span, detached from caller. Don't forget to attach it when resuming after a
+   *          call to {@link #get()}.
    * @return this
    */
-  synchronized SyncFuture reset(final long sequence, Span span) {
-    if (t != null && t != Thread.currentThread()) throw new IllegalStateException();
+  synchronized SyncFuture reset(final long txnId, Span span) {
+    if (t != null && t != Thread.currentThread()) {
+      throw new IllegalStateException();
+    }
     t = Thread.currentThread();
-    if (!isDone()) throw new IllegalStateException("" + sequence + " " + Thread.currentThread());
-    this.doneSequence = NOT_DONE;
-    this.ringBufferSequence = sequence;
+    if (!isDone()) {
+      throw new IllegalStateException("" + txnId + " " + Thread.currentThread());
+    }
+    this.doneTxid = NOT_DONE;
+    this.txid = txnId;
     this.span = span;
     return this;
   }
 
   @Override
   public synchronized String toString() {
-    return "done=" + isDone() + ", ringBufferSequence=" + this.ringBufferSequence;
+    return "done=" + isDone() + ", txid=" + this.txid;
   }
 
-  synchronized long getRingBufferSequence() {
-    return this.ringBufferSequence;
+  synchronized long getTxid() {
+    return this.txid;
   }
 
   /**
-   * Retrieve the {@code span} instance from this Future. EventHandler calls
-   * this method to continue the span. Thread waiting on this Future musn't call
-   * this method until AFTER calling {@link #get()} and the future has been
-   * released back to the originating thread.
+   * Retrieve the {@code span} instance from this Future. EventHandler calls this method to continue
+   * the span. Thread waiting on this Future musn't call this method until AFTER calling
+   * {@link #get()} and the future has been released back to the originating thread.
    */
   synchronized Span getSpan() {
     return this.span;
   }
 
   /**
-   * Used to re-attach a {@code span} to the Future. Called by the EventHandler
-   * after a it has completed processing and detached the span from its scope.
+   * Used to re-attach a {@code span} to the Future. Called by the EventHandler after a it has
+   * completed processing and detached the span from its scope.
    */
   synchronized void setSpan(Span span) {
     this.span = span;
   }
 
   /**
-   * @param sequence Sync sequence at which this future 'completed'.
-   * @param t Can be null.  Set if we are 'completing' on error (and this 't' is the error).
-   * @return True if we successfully marked this outstanding future as completed/done.
-   * Returns false if this future is already 'done' when this method called.
+   * @param txid the transaction id at which this future 'completed'.
+   * @param t Can be null. Set if we are 'completing' on error (and this 't' is the error).
+   * @return True if we successfully marked this outstanding future as completed/done. Returns false
+   *         if this future is already 'done' when this method called.
    */
-  synchronized boolean done(final long sequence, final Throwable t) {
-    if (isDone()) return false;
+  synchronized boolean done(final long txid, final Throwable t) {
+    if (isDone()) {
+      return false;
+    }
     this.throwable = t;
-    if (sequence < this.ringBufferSequence) {
+    if (txid < this.txid) {
       // Something badly wrong.
       if (throwable == null) {
-        this.throwable = new IllegalStateException("sequence=" + sequence +
-          ", ringBufferSequence=" + this.ringBufferSequence);
+        this.throwable =
+            new IllegalStateException("done txid=" + txid + ", my txid=" + this.txid);
       }
     }
     // Mark done.
-    this.doneSequence = sequence;
+    this.doneTxid = txid;
     // Wake up waiting threads.
     notify();
     return true;
@@ -166,21 +164,14 @@ class SyncFuture {
     while (!isDone()) {
       wait(1000);
     }
-    if (this.throwable != null) throw new ExecutionException(this.throwable);
-    return this.doneSequence;
-  }
-
-  public Long get(long timeout, TimeUnit unit)
-  throws InterruptedException, ExecutionException {
-    throw new UnsupportedOperationException();
-  }
-
-  public boolean isCancelled() {
-    throw new UnsupportedOperationException();
+    if (this.throwable != null) {
+      throw new ExecutionException(this.throwable);
+    }
+    return this.doneTxid;
   }
 
   synchronized boolean isDone() {
-    return this.doneSequence != NOT_DONE;
+    return this.doneTxid != NOT_DONE;
   }
 
   synchronized boolean isThrowable() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WriterBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WriterBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WriterBase.java
deleted file mode 100644
index 8188e02..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WriterBase.java
+++ /dev/null
@@ -1,62 +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.wal;
-
-import java.io.IOException;
-
-import org.apache.hadoop.hbase.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.HConstants;
-import org.apache.hadoop.hbase.io.util.LRUDictionary;
-import org.apache.hadoop.hbase.util.FSUtils;
-
-import org.apache.hadoop.hbase.wal.DefaultWALProvider;
-
-/**
- * Context used by our wal dictionary compressor. Null if we're not to do our
- * custom dictionary compression.
- */
-@InterfaceAudience.Private
-public abstract class WriterBase implements DefaultWALProvider.Writer {
-
-  protected CompressionContext compressionContext;
-  protected Configuration conf;
-
-  @Override
-  public void init(FileSystem fs, Path path, Configuration conf, boolean overwritable) throws IOException {
-    this.conf = conf;
-  }
-
-  public boolean initializeCompressionContext(Configuration conf, Path path) throws IOException {
-    boolean doCompress = conf.getBoolean(HConstants.ENABLE_WAL_COMPRESSION, false);
-    if (doCompress) {
-      try {
-        this.compressionContext = new CompressionContext(LRUDictionary.class,
-            FSUtils.isRecoveredEdits(path), conf.getBoolean(
-                CompressionContext.ENABLE_WAL_TAGS_COMPRESSION, true));
-      } catch (Exception e) {
-        throw new IOException("Failed to initiate CompressionContext", e);
-      }
-    }
-    return doCompress;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FanOutOneBlockAsyncDFSOutputHelper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FanOutOneBlockAsyncDFSOutputHelper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FanOutOneBlockAsyncDFSOutputHelper.java
index ea71701..32fe48b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FanOutOneBlockAsyncDFSOutputHelper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FanOutOneBlockAsyncDFSOutputHelper.java
@@ -72,6 +72,7 @@ import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
 import org.apache.hadoop.io.EnumSetWritable;
+import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.util.DataChecksum;
 
@@ -417,7 +418,6 @@ public class FanOutOneBlockAsyncDFSOutputHelper {
     public boolean progress() {
       return DFS_CLIENT_ADAPTOR.isClientRunning(client);
     }
-
   }
 
   static {
@@ -579,6 +579,18 @@ public class FanOutOneBlockAsyncDFSOutputHelper {
     return futureList;
   }
 
+  /**
+   * Exception other than RemoteException thrown when calling create on namenode
+   */
+  public static class NameNodeException extends IOException {
+
+    private static final long serialVersionUID = 3143237406477095390L;
+
+    public NameNodeException(Throwable cause) {
+      super(cause);
+    }
+  }
+
   private static FanOutOneBlockAsyncDFSOutput createOutput(DistributedFileSystem dfs, String src,
       boolean overwrite, boolean createParent, short replication, long blockSize,
       EventLoop eventLoop) throws IOException {
@@ -587,11 +599,20 @@ public class FanOutOneBlockAsyncDFSOutputHelper {
     DFSClient client = dfs.getClient();
     String clientName = client.getClientName();
     ClientProtocol namenode = client.getNamenode();
-    HdfsFileStatus stat = FILE_CREATER.create(namenode, src,
-      FsPermission.getFileDefault().applyUMask(FsPermission.getUMask(conf)), clientName,
-      new EnumSetWritable<CreateFlag>(
-          overwrite ? EnumSet.of(CREATE, OVERWRITE) : EnumSet.of(CREATE)),
-      createParent, replication, blockSize);
+    HdfsFileStatus stat;
+    try {
+      stat = FILE_CREATER.create(namenode, src,
+        FsPermission.getFileDefault().applyUMask(FsPermission.getUMask(conf)), clientName,
+        new EnumSetWritable<CreateFlag>(
+            overwrite ? EnumSet.of(CREATE, OVERWRITE) : EnumSet.of(CREATE)),
+        createParent, replication, blockSize);
+    } catch (Exception e) {
+      if (e instanceof RemoteException) {
+        throw (RemoteException) e;
+      } else {
+        throw new NameNodeException(e);
+      }
+    }
     beginFileLease(client, src, stat.getFileId());
     boolean succ = false;
     LocatedBlock locatedBlock = null;
@@ -656,6 +677,13 @@ public class FanOutOneBlockAsyncDFSOutputHelper {
     }.resolve(dfs, f);
   }
 
+  public static boolean shouldRetryCreate(RemoteException e) {
+    // RetryStartFileException is introduced in HDFS 2.6+, so here we can only use the class name.
+    // For exceptions other than this, we just throw it out. This is same with
+    // DFSOutputStream.newStreamForCreate.
+    return e.getClassName().endsWith("RetryStartFileException");
+  }
+
   static void completeFile(DFSClient client, ClientProtocol namenode, String src, String clientName,
       ExtendedBlock block, long fileId) {
     for (int retry = 0;; retry++) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
new file mode 100644
index 0000000..2f5c299
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
@@ -0,0 +1,361 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.wal;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Pattern;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL;
+import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Base class of a WAL Provider that returns a single thread safe WAL that writes to HDFS. By
+ * default, this implementation picks a directory in HDFS based on a combination of
+ * <ul>
+ * <li>the HBase root directory
+ * <li>HConstants.HREGION_LOGDIR_NAME
+ * <li>the given factory's factoryId (usually identifying the regionserver by host:port)
+ * </ul>
+ * It also uses the providerId to differentiate among files.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public abstract class AbstractFSWALProvider<T extends AbstractFSWAL<?>> implements WALProvider {
+
+  private static final Log LOG = LogFactory.getLog(AbstractFSWALProvider.class);
+
+  // Only public so classes back in regionserver.wal can access
+  public interface Reader extends WAL.Reader {
+    /**
+     * @param fs File system.
+     * @param path Path.
+     * @param c Configuration.
+     * @param s Input stream that may have been pre-opened by the caller; may be null.
+     */
+    void init(FileSystem fs, Path path, Configuration c, FSDataInputStream s) throws IOException;
+  }
+
+  protected volatile T wal;
+  protected WALFactory factory = null;
+  protected Configuration conf = null;
+  protected List<WALActionsListener> listeners = null;
+  protected String providerId = null;
+  protected AtomicBoolean initialized = new AtomicBoolean(false);
+  // for default wal provider, logPrefix won't change
+  protected String logPrefix = null;
+
+  /**
+   * we synchronized on walCreateLock to prevent wal recreation in different threads
+   */
+  private final Object walCreateLock = new Object();
+
+  /**
+   * @param factory factory that made us, identity used for FS layout. may not be null
+   * @param conf may not be null
+   * @param listeners may be null
+   * @param providerId differentiate between providers from one facotry, used for FS layout. may be
+   *          null
+   */
+  @Override
+  public void init(WALFactory factory, Configuration conf, List<WALActionsListener> listeners,
+      String providerId) throws IOException {
+    if (!initialized.compareAndSet(false, true)) {
+      throw new IllegalStateException("WALProvider.init should only be called once.");
+    }
+    this.factory = factory;
+    this.conf = conf;
+    this.listeners = listeners;
+    this.providerId = providerId;
+    // get log prefix
+    StringBuilder sb = new StringBuilder().append(factory.factoryId);
+    if (providerId != null) {
+      if (providerId.startsWith(WAL_FILE_NAME_DELIMITER)) {
+        sb.append(providerId);
+      } else {
+        sb.append(WAL_FILE_NAME_DELIMITER).append(providerId);
+      }
+    }
+    logPrefix = sb.toString();
+    doInit(conf);
+  }
+
+  @Override
+  public WAL getWAL(byte[] identifier, byte[] namespace) throws IOException {
+    T walCopy = wal;
+    if (walCopy == null) {
+      // only lock when need to create wal, and need to lock since
+      // creating hlog on fs is time consuming
+      synchronized (walCreateLock) {
+        walCopy = wal;
+        if (walCopy == null) {
+          walCopy = createWAL();
+          wal = walCopy;
+        }
+      }
+    }
+    return walCopy;
+  }
+
+  protected abstract T createWAL() throws IOException;
+
+  protected abstract void doInit(Configuration conf) throws IOException;
+
+  @Override
+  public void shutdown() throws IOException {
+    T log = this.wal;
+    if (log != null) {
+      log.shutdown();
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    T log = this.wal;
+    if (log != null) {
+      log.close();
+    }
+  }
+
+  /**
+   * iff the given WALFactory is using the DefaultWALProvider for meta and/or non-meta, count the
+   * number of files (rolled and active). if either of them aren't, count 0 for that provider.
+   */
+  @Override
+  public long getNumLogFiles() {
+    T log = this.wal;
+    return log == null ? 0 : log.getNumLogFiles();
+  }
+
+  /**
+   * iff the given WALFactory is using the DefaultWALProvider for meta and/or non-meta, count the
+   * size of files (rolled and active). if either of them aren't, count 0 for that provider.
+   */
+  @Override
+  public long getLogFileSize() {
+    T log = this.wal;
+    return log == null ? 0 : log.getLogFileSize();
+  }
+
+  /**
+   * returns the number of rolled WAL files.
+   */
+  @VisibleForTesting
+  public static int getNumRolledLogFiles(WAL wal) {
+    return ((AbstractFSWAL<?>) wal).getNumRolledLogFiles();
+  }
+
+  /**
+   * return the current filename from the current wal.
+   */
+  @VisibleForTesting
+  public static Path getCurrentFileName(final WAL wal) {
+    return ((AbstractFSWAL<?>) wal).getCurrentFileName();
+  }
+
+  /**
+   * request a log roll, but don't actually do it.
+   */
+  @VisibleForTesting
+  static void requestLogRoll(final WAL wal) {
+    ((AbstractFSWAL<?>) wal).requestLogRoll();
+  }
+
+  // should be package private; more visible for use in AbstractFSWAL
+  public static final String WAL_FILE_NAME_DELIMITER = ".";
+  /** The hbase:meta region's WAL filename extension */
+  @VisibleForTesting
+  public static final String META_WAL_PROVIDER_ID = ".meta";
+  static final String DEFAULT_PROVIDER_ID = "default";
+
+  // Implementation details that currently leak in tests or elsewhere follow
+  /** File Extension used while splitting an WAL into regions (HBASE-2312) */
+  public static final String SPLITTING_EXT = "-splitting";
+
+  /**
+   * It returns the file create timestamp from the file name. For name format see
+   * {@link #validateWALFilename(String)} public until remaining tests move to o.a.h.h.wal
+   * @param wal must not be null
+   * @return the file number that is part of the WAL file name
+   */
+  @VisibleForTesting
+  public static long extractFileNumFromWAL(final WAL wal) {
+    final Path walName = ((AbstractFSWAL<?>) wal).getCurrentFileName();
+    if (walName == null) {
+      throw new IllegalArgumentException("The WAL path couldn't be null");
+    }
+    final String[] walPathStrs = walName.toString().split("\\" + WAL_FILE_NAME_DELIMITER);
+    return Long.parseLong(walPathStrs[walPathStrs.length - (isMetaFile(walName) ? 2 : 1)]);
+  }
+
+  /**
+   * Pattern used to validate a WAL file name see {@link #validateWALFilename(String)} for
+   * description.
+   */
+  private static final Pattern pattern = Pattern
+      .compile(".*\\.\\d*(" + META_WAL_PROVIDER_ID + ")*");
+
+  /**
+   * A WAL file name is of the format: &lt;wal-name&gt;{@link #WAL_FILE_NAME_DELIMITER}
+   * &lt;file-creation-timestamp&gt;[.meta]. provider-name is usually made up of a server-name and a
+   * provider-id
+   * @param filename name of the file to validate
+   * @return <tt>true</tt> if the filename matches an WAL, <tt>false</tt> otherwise
+   */
+  public static boolean validateWALFilename(String filename) {
+    return pattern.matcher(filename).matches();
+  }
+
+  /**
+   * Construct the directory name for all WALs on a given server.
+   * @param serverName Server name formatted as described in {@link ServerName}
+   * @return the relative WAL directory name, e.g. <code>.logs/1.example.org,60030,12345</code> if
+   *         <code>serverName</code> passed is <code>1.example.org,60030,12345</code>
+   */
+  public static String getWALDirectoryName(final String serverName) {
+    StringBuilder dirName = new StringBuilder(HConstants.HREGION_LOGDIR_NAME);
+    dirName.append("/");
+    dirName.append(serverName);
+    return dirName.toString();
+  }
+
+  /**
+   * Pulls a ServerName out of a Path generated according to our layout rules. In the below layouts,
+   * this method ignores the format of the logfile component. Current format: [base directory for
+   * hbase]/hbase/.logs/ServerName/logfile or [base directory for
+   * hbase]/hbase/.logs/ServerName-splitting/logfile Expected to work for individual log files and
+   * server-specific directories.
+   * @return null if it's not a log file. Returns the ServerName of the region server that created
+   *         this log file otherwise.
+   */
+  public static ServerName getServerNameFromWALDirectoryName(Configuration conf, String path)
+      throws IOException {
+    if (path == null || path.length() <= HConstants.HREGION_LOGDIR_NAME.length()) {
+      return null;
+    }
+
+    if (conf == null) {
+      throw new IllegalArgumentException("parameter conf must be set");
+    }
+
+    final String rootDir = conf.get(HConstants.HBASE_DIR);
+    if (rootDir == null || rootDir.isEmpty()) {
+      throw new IllegalArgumentException(HConstants.HBASE_DIR + " key not found in conf.");
+    }
+
+    final StringBuilder startPathSB = new StringBuilder(rootDir);
+    if (!rootDir.endsWith("/")) {
+      startPathSB.append('/');
+    }
+    startPathSB.append(HConstants.HREGION_LOGDIR_NAME);
+    if (!HConstants.HREGION_LOGDIR_NAME.endsWith("/")) {
+      startPathSB.append('/');
+    }
+    final String startPath = startPathSB.toString();
+
+    String fullPath;
+    try {
+      fullPath = FileSystem.get(conf).makeQualified(new Path(path)).toString();
+    } catch (IllegalArgumentException e) {
+      LOG.info("Call to makeQualified failed on " + path + " " + e.getMessage());
+      return null;
+    }
+
+    if (!fullPath.startsWith(startPath)) {
+      return null;
+    }
+
+    final String serverNameAndFile = fullPath.substring(startPath.length());
+
+    if (serverNameAndFile.indexOf('/') < "a,0,0".length()) {
+      // Either it's a file (not a directory) or it's not a ServerName format
+      return null;
+    }
+
+    Path p = new Path(path);
+    return getServerNameFromWALDirectoryName(p);
+  }
+
+  /**
+   * This function returns region server name from a log file name which is in one of the following
+   * formats:
+   * <ul>
+   * <li>hdfs://&lt;name node&gt;/hbase/.logs/&lt;server name&gt;-splitting/...</li>
+   * <li>hdfs://&lt;name node&gt;/hbase/.logs/&lt;server name&gt;/...</li>
+   * </ul>
+   * @return null if the passed in logFile isn't a valid WAL file path
+   */
+  public static ServerName getServerNameFromWALDirectoryName(Path logFile) {
+    String logDirName = logFile.getParent().getName();
+    // We were passed the directory and not a file in it.
+    if (logDirName.equals(HConstants.HREGION_LOGDIR_NAME)) {
+      logDirName = logFile.getName();
+    }
+    ServerName serverName = null;
+    if (logDirName.endsWith(SPLITTING_EXT)) {
+      logDirName = logDirName.substring(0, logDirName.length() - SPLITTING_EXT.length());
+    }
+    try {
+      serverName = ServerName.parseServerName(logDirName);
+    } catch (IllegalArgumentException ex) {
+      serverName = null;
+      LOG.warn("Cannot parse a server name from path=" + logFile + "; " + ex.getMessage());
+    }
+    if (serverName != null && serverName.getStartcode() < 0) {
+      LOG.warn("Invalid log file path=" + logFile);
+      serverName = null;
+    }
+    return serverName;
+  }
+
+  public static boolean isMetaFile(Path p) {
+    return isMetaFile(p.getName());
+  }
+
+  public static boolean isMetaFile(String p) {
+    if (p != null && p.endsWith(META_WAL_PROVIDER_ID)) {
+      return true;
+    }
+    return false;
+  }
+
+  /**
+   * Get prefix of the log from its name, assuming WAL name in format of
+   * log_prefix.filenumber.log_suffix
+   * @param name Name of the WAL to parse
+   * @return prefix of the log
+   * @see AbstractFSWAL#getCurrentFileName()
+   */
+  public static String getWALPrefixFromWALName(String name) {
+    int endIndex = name.replaceAll(META_WAL_PROVIDER_ID, "").lastIndexOf(".");
+    return name.substring(0, endIndex);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java
new file mode 100644
index 0000000..bc142ce
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java
@@ -0,0 +1,73 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.wal;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL;
+import org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.Threads;
+
+import io.netty.channel.EventLoop;
+import io.netty.channel.EventLoopGroup;
+import io.netty.channel.nio.NioEventLoopGroup;
+
+/**
+ * A WAL provider that use {@link AsyncFSWAL}.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class AsyncFSWALProvider extends AbstractFSWALProvider<AsyncFSWAL> {
+
+  // Only public so classes back in regionserver.wal can access
+  public interface AsyncWriter extends WALProvider.AsyncWriter {
+    void init(FileSystem fs, Path path, Configuration c, boolean overwritable) throws IOException;
+  }
+
+  private EventLoopGroup eventLoopGroup = null;
+
+  @Override
+  protected AsyncFSWAL createWAL() throws IOException {
+    return new AsyncFSWAL(FileSystem.get(conf), FSUtils.getRootDir(conf),
+        getWALDirectoryName(factory.factoryId), HConstants.HREGION_OLDLOGDIR_NAME, conf, listeners,
+        true, logPrefix, META_WAL_PROVIDER_ID.equals(providerId) ? META_WAL_PROVIDER_ID : null,
+        eventLoopGroup.next());
+  }
+
+  @Override
+  protected void doInit(Configuration conf) throws IOException {
+    eventLoopGroup = new NioEventLoopGroup(1, Threads.newDaemonThreadFactory("AsyncFSWAL"));
+  }
+
+  /**
+   * public because of AsyncFSWAL. Should be package-private
+   */
+  public static AsyncWriter createAsyncWriter(Configuration conf, FileSystem fs, Path path,
+      boolean overwritable, EventLoop eventLoop) throws IOException {
+    AsyncWriter writer = new AsyncProtobufLogWriter(eventLoop);
+    writer.init(fs, path, conf, overwritable);
+    return writer;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DefaultWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DefaultWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DefaultWALProvider.java
index 027e7a2..9f0d0ea 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DefaultWALProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DefaultWALProvider.java
@@ -19,357 +19,42 @@
 package org.apache.hadoop.hbase.wal;
 
 import java.io.IOException;
-import java.util.List;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.regex.Pattern;
-
-import com.google.common.annotations.VisibleForTesting;
 
 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.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.util.FSUtils;
-
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 // imports for things that haven't moved from regionserver.wal yet.
 import org.apache.hadoop.hbase.regionserver.wal.FSHLog;
 import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogWriter;
-import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
+import org.apache.hadoop.hbase.util.FSUtils;
 
 /**
- * A WAL Provider that returns a single thread safe WAL that writes to HDFS.
- * By default, this implementation picks a directory in HDFS based on a combination of
- * <ul>
- *   <li>the HBase root directory
- *   <li>HConstants.HREGION_LOGDIR_NAME
- *   <li>the given factory's factoryId (usually identifying the regionserver by host:port)
- * </ul>
- * It also uses the providerId to diffentiate among files.
- *
+ * A WAL provider that use {@link FSHLog}.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-public class DefaultWALProvider implements WALProvider {
-  private static final Log LOG = LogFactory.getLog(DefaultWALProvider.class);
+public class DefaultWALProvider extends AbstractFSWALProvider<FSHLog> {
 
-  // Only public so classes back in regionserver.wal can access
-  public interface Reader extends WAL.Reader {
-    /**
-     * @param fs File system.
-     * @param path Path.
-     * @param c Configuration.
-     * @param s Input stream that may have been pre-opened by the caller; may be null.
-     */
-    void init(FileSystem fs, Path path, Configuration c, FSDataInputStream s) throws IOException;
-  }
+  private static final Log LOG = LogFactory.getLog(DefaultWALProvider.class);
 
   // Only public so classes back in regionserver.wal can access
   public interface Writer extends WALProvider.Writer {
     void init(FileSystem fs, Path path, Configuration c, boolean overwritable) throws IOException;
   }
 
-  protected volatile FSHLog log = null;
-  private WALFactory factory = null;
-  private Configuration conf = null;
-  private List<WALActionsListener> listeners = null;
-  private String providerId = null;
-  private AtomicBoolean initialized = new AtomicBoolean(false);
-  // for default wal provider, logPrefix won't change
-  private String logPrefix = null;
-
-  /**
-   * we synchronized on walCreateLock to prevent wal recreation in different threads
-   */
-  private final Object walCreateLock = new Object();
-
-  /**
-   * @param factory factory that made us, identity used for FS layout. may not be null
-   * @param conf may not be null
-   * @param listeners may be null
-   * @param providerId differentiate between providers from one facotry, used for FS layout. may be
-   *                   null
-   */
-  @Override
-  public void init(final WALFactory factory, final Configuration conf,
-      final List<WALActionsListener> listeners, String providerId) throws IOException {
-    if (!initialized.compareAndSet(false, true)) {
-      throw new IllegalStateException("WALProvider.init should only be called once.");
-    }
-    this.factory = factory;
-    this.conf = conf;
-    this.listeners = listeners;
-    this.providerId = providerId;
-    // get log prefix
-    StringBuilder sb = new StringBuilder().append(factory.factoryId);
-    if (providerId != null) {
-      if (providerId.startsWith(WAL_FILE_NAME_DELIMITER)) {
-        sb.append(providerId);
-      } else {
-        sb.append(WAL_FILE_NAME_DELIMITER).append(providerId);
-      }
-    }
-    logPrefix = sb.toString();
-  }
-
-  @Override
-  public WAL getWAL(final byte[] identifier, byte[] namespace) throws IOException {
-    if (log == null) {
-      // only lock when need to create wal, and need to lock since
-      // creating hlog on fs is time consuming
-      synchronized (walCreateLock) {
-        if (log == null) {
-          log = new FSHLog(FileSystem.get(conf), FSUtils.getRootDir(conf),
-              getWALDirectoryName(factory.factoryId), HConstants.HREGION_OLDLOGDIR_NAME, conf,
-              listeners, true, logPrefix,
-              META_WAL_PROVIDER_ID.equals(providerId) ? META_WAL_PROVIDER_ID : null);
-        }
-      }
-    }
-    return log;
-  }
-
-  @Override
-  public void close() throws IOException {
-    if (log != null) log.close();
-  }
-
-  @Override
-  public void shutdown() throws IOException {
-    if (log != null) log.shutdown();
-  }
-
-  // should be package private; more visible for use in FSHLog
-  public static final String WAL_FILE_NAME_DELIMITER = ".";
-  /** The hbase:meta region's WAL filename extension */
-  @VisibleForTesting
-  public static final String META_WAL_PROVIDER_ID = ".meta";
-  static final String DEFAULT_PROVIDER_ID = "default";
-
-  // Implementation details that currently leak in tests or elsewhere follow
-  /** File Extension used while splitting an WAL into regions (HBASE-2312) */
-  public static final String SPLITTING_EXT = "-splitting";
-
-  /**
-   * iff the given WALFactory is using the DefaultWALProvider for meta and/or non-meta,
-   * count the number of files (rolled and active). if either of them aren't, count 0
-   * for that provider.
-   */
-  @Override
-  public long getNumLogFiles() {
-    return log == null ? 0 : this.log.getNumLogFiles();
-  }
-
-  /**
-   * iff the given WALFactory is using the DefaultWALProvider for meta and/or non-meta,
-   * count the size of files (rolled and active). if either of them aren't, count 0
-   * for that provider.
-   */
-  @Override
-  public long getLogFileSize() {
-    return log == null ? 0 : this.log.getLogFileSize();
-  }
-
-  /**
-   * returns the number of rolled WAL files.
-   */
-  @VisibleForTesting
-  public static int getNumRolledLogFiles(WAL wal) {
-    return ((FSHLog)wal).getNumRolledLogFiles();
-  }
-
-  /**
-   * return the current filename from the current wal.
-   */
-  @VisibleForTesting
-  public static Path getCurrentFileName(final WAL wal) {
-    return ((FSHLog)wal).getCurrentFileName();
-  }
-
-  /**
-   * request a log roll, but don't actually do it.
-   */
-  @VisibleForTesting
-  static void requestLogRoll(final WAL wal) {
-    ((FSHLog)wal).requestLogRoll();
-  }
-
-  /**
-   * It returns the file create timestamp from the file name.
-   * For name format see {@link #validateWALFilename(String)}
-   * public until remaining tests move to o.a.h.h.wal
-   * @param wal must not be null
-   * @return the file number that is part of the WAL file name
-   */
-  @VisibleForTesting
-  public static long extractFileNumFromWAL(final WAL wal) {
-    final Path walName = ((FSHLog)wal).getCurrentFileName();
-    if (walName == null) {
-      throw new IllegalArgumentException("The WAL path couldn't be null");
-    }
-    final String[] walPathStrs = walName.toString().split("\\" + WAL_FILE_NAME_DELIMITER);
-    return Long.parseLong(walPathStrs[walPathStrs.length - (isMetaFile(walName) ? 2:1)]);
-  }
-
-  /**
-   * Pattern used to validate a WAL file name
-   * see {@link #validateWALFilename(String)} for description.
-   */
-  private static final Pattern pattern = Pattern.compile(".*\\.\\d*("+META_WAL_PROVIDER_ID+")*");
-
-  /**
-   * A WAL file name is of the format:
-   * &lt;wal-name&gt;{@link #WAL_FILE_NAME_DELIMITER}&lt;file-creation-timestamp&gt;[.meta].
-   *
-   * provider-name is usually made up of a server-name and a provider-id
-   *
-   * @param filename name of the file to validate
-   * @return <tt>true</tt> if the filename matches an WAL, <tt>false</tt>
-   *         otherwise
-   */
-  public static boolean validateWALFilename(String filename) {
-    return pattern.matcher(filename).matches();
-  }
-
-  /**
-   * Construct the directory name for all WALs on a given server.
-   *
-   * @param serverName
-   *          Server name formatted as described in {@link ServerName}
-   * @return the relative WAL directory name, e.g.
-   *         <code>.logs/1.example.org,60030,12345</code> if
-   *         <code>serverName</code> passed is
-   *         <code>1.example.org,60030,12345</code>
-   */
-  public static String getWALDirectoryName(final String serverName) {
-    StringBuilder dirName = new StringBuilder(HConstants.HREGION_LOGDIR_NAME);
-    dirName.append("/");
-    dirName.append(serverName);
-    return dirName.toString();
-  }
-
-  /**
-   * Pulls a ServerName out of a Path generated according to our layout rules.
-   *
-   * In the below layouts, this method ignores the format of the logfile component.
-   *
-   * Current format:
-   *
-   * [base directory for hbase]/hbase/.logs/ServerName/logfile
-   *      or
-   * [base directory for hbase]/hbase/.logs/ServerName-splitting/logfile
-   *
-   * Expected to work for individual log files and server-specific directories.
-   *
-   * @return null if it's not a log file. Returns the ServerName of the region
-   *         server that created this log file otherwise.
-   */
-  public static ServerName getServerNameFromWALDirectoryName(Configuration conf, String path)
-      throws IOException {
-    if (path == null
-        || path.length() <= HConstants.HREGION_LOGDIR_NAME.length()) {
-      return null;
-    }
-
-    if (conf == null) {
-      throw new IllegalArgumentException("parameter conf must be set");
-    }
-
-    final String rootDir = conf.get(HConstants.HBASE_DIR);
-    if (rootDir == null || rootDir.isEmpty()) {
-      throw new IllegalArgumentException(HConstants.HBASE_DIR
-          + " key not found in conf.");
-    }
-
-    final StringBuilder startPathSB = new StringBuilder(rootDir);
-    if (!rootDir.endsWith("/"))
-      startPathSB.append('/');
-    startPathSB.append(HConstants.HREGION_LOGDIR_NAME);
-    if (!HConstants.HREGION_LOGDIR_NAME.endsWith("/"))
-      startPathSB.append('/');
-    final String startPath = startPathSB.toString();
-
-    String fullPath;
-    try {
-      fullPath = FileSystem.get(conf).makeQualified(new Path(path)).toString();
-    } catch (IllegalArgumentException e) {
-      LOG.info("Call to makeQualified failed on " + path + " " + e.getMessage());
-      return null;
-    }
-
-    if (!fullPath.startsWith(startPath)) {
-      return null;
-    }
-
-    final String serverNameAndFile = fullPath.substring(startPath.length());
-
-    if (serverNameAndFile.indexOf('/') < "a,0,0".length()) {
-      // Either it's a file (not a directory) or it's not a ServerName format
-      return null;
-    }
-
-    Path p = new Path(path);
-    return getServerNameFromWALDirectoryName(p);
-  }
-
-  /**
-   * This function returns region server name from a log file name which is in one of the following
-   * formats:
-   * <ul>
-   *   <li>hdfs://&lt;name node&gt;/hbase/.logs/&lt;server name&gt;-splitting/...</li>
-   *   <li>hdfs://&lt;name node&gt;/hbase/.logs/&lt;server name&gt;/...</li>
-   * </ul>
-   * @param logFile
-   * @return null if the passed in logFile isn't a valid WAL file path
-   */
-  public static ServerName getServerNameFromWALDirectoryName(Path logFile) {
-    String logDirName = logFile.getParent().getName();
-    // We were passed the directory and not a file in it.
-    if (logDirName.equals(HConstants.HREGION_LOGDIR_NAME)) {
-      logDirName = logFile.getName();
-    }
-    ServerName serverName = null;
-    if (logDirName.endsWith(SPLITTING_EXT)) {
-      logDirName = logDirName.substring(0, logDirName.length() - SPLITTING_EXT.length());
-    }
-    try {
-      serverName = ServerName.parseServerName(logDirName);
-    } catch (IllegalArgumentException ex) {
-      serverName = null;
-      LOG.warn("Cannot parse a server name from path=" + logFile + "; " + ex.getMessage());
-    }
-    if (serverName != null && serverName.getStartcode() < 0) {
-      LOG.warn("Invalid log file path=" + logFile);
-      serverName = null;
-    }
-    return serverName;
-  }
-
-  public static boolean isMetaFile(Path p) {
-    return isMetaFile(p.getName());
-  }
-
-  public static boolean isMetaFile(String p) {
-    if (p != null && p.endsWith(META_WAL_PROVIDER_ID)) {
-      return true;
-    }
-    return false;
-  }
-
   /**
    * public because of FSHLog. Should be package-private
    */
   public static Writer createWriter(final Configuration conf, final FileSystem fs, final Path path,
-      final boolean overwritable)
-      throws IOException {
+      final boolean overwritable) throws IOException {
     // Configuration already does caching for the Class lookup.
     Class<? extends Writer> logWriterClass = conf.getClass("hbase.regionserver.hlog.writer.impl",
-        ProtobufLogWriter.class, Writer.class);
+      ProtobufLogWriter.class, Writer.class);
     try {
       Writer writer = logWriterClass.newInstance();
       writer.init(fs, path, conf, overwritable);
@@ -380,15 +65,14 @@ public class DefaultWALProvider implements WALProvider {
     }
   }
 
-  /**
-   * Get prefix of the log from its name, assuming WAL name in format of
-   * log_prefix.filenumber.log_suffix @see {@link FSHLog#getCurrentFileName()}
-   * @param name Name of the WAL to parse
-   * @return prefix of the log
-   */
-  public static String getWALPrefixFromWALName(String name) {
-    int endIndex = name.replaceAll(META_WAL_PROVIDER_ID, "").lastIndexOf(".");
-    return name.substring(0, endIndex);
+  @Override
+  protected FSHLog createWAL() throws IOException {
+    return new FSHLog(FileSystem.get(conf), FSUtils.getRootDir(conf),
+        getWALDirectoryName(factory.factoryId), HConstants.HREGION_OLDLOGDIR_NAME, conf, listeners,
+        true, logPrefix, META_WAL_PROVIDER_ID.equals(providerId) ? META_WAL_PROVIDER_ID : null);
   }
 
+  @Override
+  protected void doInit(Configuration conf) throws IOException {
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java
index c3d4b2c..028c60b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java
@@ -215,6 +215,10 @@ class DisabledWALProvider implements WALProvider {
     public String toString() {
       return "WAL disabled.";
     }
+
+    @Override
+    public void logRollerExited() {
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java
index 0b83528..051ce54 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java
@@ -199,6 +199,13 @@ public interface WAL {
   String toString();
 
   /**
+   * In some WAL implementation, we will write WAL entries to new file if sync failed, which means,
+   * the fail recovery is depended on log roller. So here we tell the WAL that log roller has
+   * already been exited so the WAL cloud give up recovery.
+   */
+  void logRollerExited();
+
+  /**
    * When outside clients need to consume persisted WALs, they rely on a provided
    * Reader.
    */
@@ -268,7 +275,5 @@ public interface WAL {
     public String toString() {
       return this.key + "=" + this.edit;
     }
-
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
index 08f42aa..a2761df 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
@@ -76,7 +76,8 @@ public class WALFactory {
   static enum Providers {
     defaultProvider(DefaultWALProvider.class),
     filesystem(DefaultWALProvider.class),
-    multiwal(RegionGroupingProvider.class);
+    multiwal(RegionGroupingProvider.class),
+    asyncfs(AsyncFSWALProvider.class);
 
     Class<? extends WALProvider> clazz;
     Providers(Class<? extends WALProvider> clazz) {
@@ -350,9 +351,10 @@ public class WALFactory {
 
   /**
    * Create a writer for the WAL.
+   * <p>
    * should be package-private. public only for tests and
    * {@link org.apache.hadoop.hbase.regionserver.wal.Compressor}
-   * @return A WAL writer.  Close when done with it.
+   * @return A WAL writer. Close when done with it.
    * @throws IOException
    */
   public Writer createWALWriter(final FileSystem fs, final Path path) throws IOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALProvider.java
index 2c500dc..ad79485 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALProvider.java
@@ -20,11 +20,11 @@ package org.apache.hadoop.hbase.wal;
 
 import java.io.Closeable;
 import java.io.IOException;
+import java.nio.channels.CompletionHandler;
 import java.util.List;
 
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
-
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 // imports for things that haven't moved from regionserver.wal yet.
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 
@@ -80,6 +80,12 @@ public interface WALProvider {
     long getLength() throws IOException;
   }
 
+  interface AsyncWriter extends Closeable {
+    <A> void sync(CompletionHandler<Long, A> handler, A attachment);
+    void append(WAL.Entry entry);
+    long getLength();
+  }
+
   /**
    * Get number of the log files this provider is managing
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java
new file mode 100644
index 0000000..7abdef9
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java
@@ -0,0 +1,332 @@
+/**
+ *
+ * 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.wal;
+
+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.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.MiniHBaseCluster;
+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.Get;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.regionserver.Region;
+import org.apache.hadoop.hbase.regionserver.Store;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Threads;
+import org.apache.hadoop.hbase.wal.DefaultWALProvider;
+import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+
+/**
+ * Test log deletion as logs are rolled.
+ */
+public abstract class AbstractTestLogRolling  {
+  private static final Log LOG = LogFactory.getLog(AbstractTestLogRolling.class);
+  protected HRegionServer server;
+  protected String tableName;
+  protected byte[] value;
+  protected FileSystem fs;
+  protected MiniDFSCluster dfsCluster;
+  protected Admin admin;
+  protected MiniHBaseCluster cluster;
+  protected static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  @Rule public final TestName name = new TestName();
+
+  public AbstractTestLogRolling()  {
+    this.server = null;
+    this.tableName = null;
+
+    String className = this.getClass().getName();
+    StringBuilder v = new StringBuilder(className);
+    while (v.length() < 1000) {
+      v.append(className);
+    }
+    this.value = Bytes.toBytes(v.toString());
+  }
+
+  // Need to override this setup so we can edit the config before it gets sent
+  // to the HDFS & HBase cluster startup.
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+
+
+    /**** configuration for testLogRolling ****/
+    // Force a region split after every 768KB
+    TEST_UTIL.getConfiguration().setLong(HConstants.HREGION_MAX_FILESIZE, 768L * 1024L);
+
+    // We roll the log after every 32 writes
+    TEST_UTIL.getConfiguration().setInt("hbase.regionserver.maxlogentries", 32);
+
+    TEST_UTIL.getConfiguration().setInt("hbase.regionserver.logroll.errors.tolerated", 2);
+    TEST_UTIL.getConfiguration().setInt("hbase.rpc.timeout", 10 * 1000);
+
+    // For less frequently updated regions flush after every 2 flushes
+    TEST_UTIL.getConfiguration().setInt("hbase.hregion.memstore.optionalflushcount", 2);
+
+    // We flush the cache after every 8192 bytes
+    TEST_UTIL.getConfiguration().setInt(
+        HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 8192);
+
+    // Increase the amount of time between client retries
+    TEST_UTIL.getConfiguration().setLong("hbase.client.pause", 10 * 1000);
+
+    // Reduce thread wake frequency so that other threads can get
+    // a chance to run.
+    TEST_UTIL.getConfiguration().setInt(HConstants.THREAD_WAKE_FREQUENCY, 2 * 1000);
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    TEST_UTIL.startMiniCluster(1, 1, 2);
+
+    cluster = TEST_UTIL.getHBaseCluster();
+    dfsCluster = TEST_UTIL.getDFSCluster();
+    fs = TEST_UTIL.getTestFileSystem();
+    admin = TEST_UTIL.getHBaseAdmin();
+
+    // disable region rebalancing (interferes with log watching)
+    cluster.getMaster().balanceSwitch(false);
+  }
+
+  @After
+  public void tearDown() throws Exception  {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  protected void startAndWriteData() throws IOException, InterruptedException {
+    // When the hbase:meta table can be opened, the region servers are running
+    TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME);
+    this.server = cluster.getRegionServerThreads().get(0).getRegionServer();
+
+    Table table = createTestTable(this.tableName);
+
+    server = TEST_UTIL.getRSForFirstRegionInTable(table.getName());
+    for (int i = 1; i <= 256; i++) {    // 256 writes should cause 8 log rolls
+      doPut(table, i);
+      if (i % 32 == 0) {
+        // After every 32 writes sleep to let the log roller run
+        try {
+          Thread.sleep(2000);
+        } catch (InterruptedException e) {
+          // continue
+        }
+      }
+    }
+  }
+
+  /**
+   * Tests that log rolling doesn't hang when no data is written.
+   */
+  @Test(timeout=120000)
+  public void testLogRollOnNothingWritten() throws Exception {
+    final Configuration conf = TEST_UTIL.getConfiguration();
+    final WALFactory wals = new WALFactory(conf, null,
+        ServerName.valueOf("test.com",8080, 1).toString());
+    final WAL newLog = wals.getWAL(new byte[]{}, null);
+    try {
+      // Now roll the log before we write anything.
+      newLog.rollWriter(true);
+    } finally {
+      wals.close();
+    }
+  }
+
+  /**
+   * Tests that logs are deleted
+   * @throws IOException
+   * @throws org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException
+   */
+  @Test
+  public void testLogRolling() throws Exception {
+    this.tableName = getName();
+    // TODO: Why does this write data take for ever?
+    startAndWriteData();
+    HRegionInfo region =
+        server.getOnlineRegions(TableName.valueOf(tableName)).get(0).getRegionInfo();
+    final WAL log = server.getWAL(region);
+    LOG.info("after writing there are " + DefaultWALProvider.getNumRolledLogFiles(log) +
+        " log files");
+
+      // flush all regions
+      for (Region r: server.getOnlineRegionsLocalContext()) {
+        r.flush(true);
+      }
+
+      // Now roll the log
+      log.rollWriter();
+
+    int count = DefaultWALProvider.getNumRolledLogFiles(log);
+    LOG.info("after flushing all regions and rolling logs there are " + count + " log files");
+      assertTrue(("actual count: " + count), count <= 2);
+  }
+
+  protected String getName() {
+    return "TestLogRolling-" + name.getMethodName();
+  }
+
+  void writeData(Table table, int rownum) throws IOException {
+    doPut(table, rownum);
+
+    // sleep to let the log roller run (if it needs to)
+    try {
+      Thread.sleep(2000);
+    } catch (InterruptedException e) {
+      // continue
+    }
+  }
+
+  void validateData(Table table, int rownum) throws IOException {
+    String row = "row" + String.format("%1$04d", rownum);
+    Get get = new Get(Bytes.toBytes(row));
+    get.addFamily(HConstants.CATALOG_FAMILY);
+    Result result = table.get(get);
+    assertTrue(result.size() == 1);
+    assertTrue(Bytes.equals(value,
+                result.getValue(HConstants.CATALOG_FAMILY, null)));
+    LOG.info("Validated row " + row);
+  }
+
+  void batchWriteAndWait(Table table, final FSHLog log, int start, boolean expect, int timeout)
+      throws IOException {
+    for (int i = 0; i < 10; i++) {
+      Put put = new Put(Bytes.toBytes("row"
+          + String.format("%1$04d", (start + i))));
+      put.addColumn(HConstants.CATALOG_FAMILY, null, value);
+      table.put(put);
+    }
+    Put tmpPut = new Put(Bytes.toBytes("tmprow"));
+    tmpPut.addColumn(HConstants.CATALOG_FAMILY, null, value);
+    long startTime = System.currentTimeMillis();
+    long remaining = timeout;
+    while (remaining > 0) {
+      if (log.isLowReplicationRollEnabled() == expect) {
+        break;
+      } else {
+        // Trigger calling FSHlog#checkLowReplication()
+        table.put(tmpPut);
+        try {
+          Thread.sleep(200);
+        } catch (InterruptedException e) {
+          // continue
+        }
+        remaining = timeout - (System.currentTimeMillis() - startTime);
+      }
+    }
+  }
+
+  /**
+   * Tests that logs are deleted when some region has a compaction
+   * record in WAL and no other records. See HBASE-8597.
+   */
+  @Test
+  public void testCompactionRecordDoesntBlockRolling() throws Exception {
+    Table table = null;
+
+    // When the hbase:meta table can be opened, the region servers are running
+    Table t = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME);
+    try {
+      table = createTestTable(getName());
+
+      server = TEST_UTIL.getRSForFirstRegionInTable(table.getName());
+      Region region = server.getOnlineRegions(table.getName()).get(0);
+      final WAL log = server.getWAL(region.getRegionInfo());
+      Store s = region.getStore(HConstants.CATALOG_FAMILY);
+
+      //have to flush namespace to ensure it doesn't affect wall tests
+      admin.flush(TableName.NAMESPACE_TABLE_NAME);
+
+      // Put some stuff into table, to make sure we have some files to compact.
+      for (int i = 1; i <= 2; ++i) {
+        doPut(table, i);
+        admin.flush(table.getName());
+      }
+      doPut(table, 3); // don't flush yet, or compaction might trigger before we roll WAL
+      assertEquals("Should have no WAL after initial writes", 0,
+          DefaultWALProvider.getNumRolledLogFiles(log));
+      assertEquals(2, s.getStorefilesCount());
+
+      // Roll the log and compact table, to have compaction record in the 2nd WAL.
+      log.rollWriter();
+      assertEquals("Should have WAL; one table is not flushed", 1,
+          DefaultWALProvider.getNumRolledLogFiles(log));
+      admin.flush(table.getName());
+      region.compact(false);
+      // Wait for compaction in case if flush triggered it before us.
+      Assert.assertNotNull(s);
+      for (int waitTime = 3000; s.getStorefilesCount() > 1 && waitTime > 0; waitTime -= 200) {
+        Threads.sleepWithoutInterrupt(200);
+      }
+      assertEquals("Compaction didn't happen", 1, s.getStorefilesCount());
+
+      // Write some value to the table so the WAL cannot be deleted until table is flushed.
+      doPut(table, 0); // Now 2nd WAL will have both compaction and put record for table.
+      log.rollWriter(); // 1st WAL deleted, 2nd not deleted yet.
+      assertEquals("Should have WAL; one table is not flushed", 1,
+          DefaultWALProvider.getNumRolledLogFiles(log));
+
+      // Flush table to make latest WAL obsolete; write another record, and roll again.
+      admin.flush(table.getName());
+      doPut(table, 1);
+      log.rollWriter(); // Now 2nd WAL is deleted and 3rd is added.
+      assertEquals("Should have 1 WALs at the end", 1,
+          DefaultWALProvider.getNumRolledLogFiles(log));
+    } finally {
+      if (t != null) t.close();
+      if (table != null) table.close();
+    }
+  }
+
+  protected void doPut(Table table, int i) throws IOException {
+    Put put = new Put(Bytes.toBytes("row" + String.format("%1$04d", i)));
+    put.addColumn(HConstants.CATALOG_FAMILY, null, value);
+    table.put(put);
+  }
+
+  protected Table createTestTable(String tableName) throws IOException {
+    // Create the test table and open it
+    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
+    desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
+    admin.createTable(desc);
+    return TEST_UTIL.getConnection().getTable(desc.getTableName());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestProtobufLog.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestProtobufLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestProtobufLog.java
new file mode 100644
index 0000000..a4267a0
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestProtobufLog.java
@@ -0,0 +1,209 @@
+/**
+ *
+ * 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.wal;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+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.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.coprocessor.SampleRegionWALObserver;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.apache.hadoop.hbase.wal.WALKey;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+
+/**
+ * WAL tests that can be reused across providers.
+ */
+public abstract class AbstractTestProtobufLog<W extends Closeable> {
+  protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+  protected FileSystem fs;
+  protected Path dir;
+  protected WALFactory wals;
+
+  @Rule
+  public final TestName currentTest = new TestName();
+
+  @Before
+  public void setUp() throws Exception {
+    fs = TEST_UTIL.getDFSCluster().getFileSystem();
+    dir = new Path(TEST_UTIL.createRootDir(), currentTest.getMethodName());
+    wals = new WALFactory(TEST_UTIL.getConfiguration(), null, currentTest.getMethodName());
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    wals.close();
+    FileStatus[] entries = fs.listStatus(new Path("/"));
+    for (FileStatus dir : entries) {
+      fs.delete(dir.getPath(), true);
+    }
+  }
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    // Make block sizes small.
+    TEST_UTIL.getConfiguration().setInt("dfs.blocksize", 1024 * 1024);
+    // needed for testAppendClose()
+    TEST_UTIL.getConfiguration().setBoolean("dfs.support.broken.append", true);
+    TEST_UTIL.getConfiguration().setBoolean("dfs.support.append", true);
+    // quicker heartbeat interval for faster DN death notification
+    TEST_UTIL.getConfiguration().setInt("dfs.namenode.heartbeat.recheck-interval", 5000);
+    TEST_UTIL.getConfiguration().setInt("dfs.heartbeat.interval", 1);
+    TEST_UTIL.getConfiguration().setInt("dfs.client.socket-timeout", 5000);
+
+    // faster failover with cluster.shutdown();fs.close() idiom
+    TEST_UTIL.getConfiguration()
+        .setInt("hbase.ipc.client.connect.max.retries", 1);
+    TEST_UTIL.getConfiguration().setInt(
+        "dfs.client.block.recovery.retries", 1);
+    TEST_UTIL.getConfiguration().setInt(
+      "hbase.ipc.client.connection.maxidletime", 500);
+    TEST_UTIL.getConfiguration().set(CoprocessorHost.WAL_COPROCESSOR_CONF_KEY,
+        SampleRegionWALObserver.class.getName());
+    TEST_UTIL.startMiniDFSCluster(3);
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  /**
+   * Reads the WAL with and without WALTrailer.
+   * @throws IOException
+   */
+  @Test
+  public void testWALTrailer() throws IOException {
+    // read With trailer.
+    doRead(true);
+    // read without trailer
+    doRead(false);
+  }
+
+  /**
+   * Appends entries in the WAL and reads it.
+   * @param withTrailer If 'withTrailer' is true, it calls a close on the WALwriter before reading
+   *          so that a trailer is appended to the WAL. Otherwise, it starts reading after the sync
+   *          call. This means that reader is not aware of the trailer. In this scenario, if the
+   *          reader tries to read the trailer in its next() call, it returns false from
+   *          ProtoBufLogReader.
+   * @throws IOException
+   */
+  private void doRead(boolean withTrailer) throws IOException {
+    final int columnCount = 5;
+    final int recordCount = 5;
+    final TableName tableName =
+        TableName.valueOf("tablename");
+    final byte[] row = Bytes.toBytes("row");
+    long timestamp = System.currentTimeMillis();
+    Path path = new Path(dir, "tempwal");
+    // delete the log if already exists, for test only
+    fs.delete(path, true);
+    W writer = null;
+    ProtobufLogReader reader = null;
+    try {
+      HRegionInfo hri = new HRegionInfo(tableName,
+          HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
+      HTableDescriptor htd = new HTableDescriptor(tableName);
+      fs.mkdirs(dir);
+      // Write log in pb format.
+      writer = createWriter(path);
+      for (int i = 0; i < recordCount; ++i) {
+        WALKey key = new WALKey(
+            hri.getEncodedNameAsBytes(), tableName, i, timestamp, HConstants.DEFAULT_CLUSTER_ID);
+        WALEdit edit = new WALEdit();
+        for (int j = 0; j < columnCount; ++j) {
+          if (i == 0) {
+            htd.addFamily(new HColumnDescriptor("column" + j));
+          }
+          String value = i + "" + j;
+          edit.add(new KeyValue(row, row, row, timestamp, Bytes.toBytes(value)));
+        }
+        append(writer, new WAL.Entry(key, edit));
+      }
+      sync(writer);
+      if (withTrailer) writer.close();
+
+      // Now read the log using standard means.
+      reader = (ProtobufLogReader) wals.createReader(fs, path);
+      if (withTrailer) {
+        assertNotNull(reader.trailer);
+      } else {
+        assertNull(reader.trailer);
+      }
+      for (int i = 0; i < recordCount; ++i) {
+        WAL.Entry entry = reader.next();
+        assertNotNull(entry);
+        assertEquals(columnCount, entry.getEdit().size());
+        assertArrayEquals(hri.getEncodedNameAsBytes(), entry.getKey().getEncodedRegionName());
+        assertEquals(tableName, entry.getKey().getTablename());
+        int idx = 0;
+        for (Cell val : entry.getEdit().getCells()) {
+          assertTrue(Bytes.equals(row, 0, row.length, val.getRowArray(), val.getRowOffset(),
+            val.getRowLength()));
+          String value = i + "" + idx;
+          assertArrayEquals(Bytes.toBytes(value), CellUtil.cloneValue(val));
+          idx++;
+        }
+      }
+      WAL.Entry entry = reader.next();
+      assertNull(entry);
+    } finally {
+      if (writer != null) {
+        writer.close();
+      }
+      if (reader != null) {
+        reader.close();
+      }
+    }
+  }
+
+  protected abstract W createWriter(Path path) throws IOException;
+
+  protected abstract void append(W writer, WAL.Entry entry) throws IOException;
+
+  protected abstract void sync(W writer) throws IOException;
+}


[50/50] [abbrv] hbase git commit: Add ashishsinghi to pom.xml

Posted by sy...@apache.org.
Add ashishsinghi to pom.xml

Change-Id: Ib0709d92622350c50bee7e8a0bae0554d40df882


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

Branch: refs/heads/hbase-12439
Commit: f1fc5208aa724de7e31cdd4e2e4a696cf823929c
Parents: afdfd1b
Author: Ashish Singhi <as...@apache.org>
Authored: Wed Mar 30 11:08:21 2016 +0530
Committer: Ashish Singhi <as...@huawei.com>
Committed: Wed Mar 30 11:09:30 2016 +0530

----------------------------------------------------------------------
 pom.xml | 8 ++++++++
 1 file changed, 8 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/f1fc5208/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 450275c..0324c1c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -169,6 +169,14 @@
       <organizationUrl>http://www.facebook.com/</organizationUrl>
     </developer>
     <developer>
+      <id>ashishsinghi</id>
+      <name>Ashish Singhi</name>
+      <email>ashishsinghi@apache.org</email>
+      <timezone>+5</timezone>
+      <organization>Huawei</organization>
+      <organizationUrl>http://www.huawei.com/en/</organizationUrl>
+    </developer>
+    <developer>
       <id>busbey</id>
       <name>Sean Busbey</name>
       <email>busbey@apache.org</email>


[47/50] [abbrv] hbase git commit: HBASE-11393 Replication TableCfs should be a PB object rather than a string

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/7f39baf0/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java
index 0240a67..f64d0c1 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java
@@ -3947,6 +3947,719 @@ public final class ZooKeeperProtos {
     // @@protoc_insertion_point(class_scope:hbase.pb.DeprecatedTableState)
   }
 
+  public interface TableCFOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // optional .hbase.pb.TableName table_name = 1;
+    /**
+     * <code>optional .hbase.pb.TableName table_name = 1;</code>
+     */
+    boolean hasTableName();
+    /**
+     * <code>optional .hbase.pb.TableName table_name = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName();
+    /**
+     * <code>optional .hbase.pb.TableName table_name = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder();
+
+    // repeated bytes families = 2;
+    /**
+     * <code>repeated bytes families = 2;</code>
+     */
+    java.util.List<com.google.protobuf.ByteString> getFamiliesList();
+    /**
+     * <code>repeated bytes families = 2;</code>
+     */
+    int getFamiliesCount();
+    /**
+     * <code>repeated bytes families = 2;</code>
+     */
+    com.google.protobuf.ByteString getFamilies(int index);
+  }
+  /**
+   * Protobuf type {@code hbase.pb.TableCF}
+   */
+  public static final class TableCF extends
+      com.google.protobuf.GeneratedMessage
+      implements TableCFOrBuilder {
+    // Use TableCF.newBuilder() to construct.
+    private TableCF(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private TableCF(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final TableCF defaultInstance;
+    public static TableCF getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public TableCF getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private TableCF(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000001) == 0x00000001)) {
+                subBuilder = tableName_.toBuilder();
+              }
+              tableName_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(tableName_);
+                tableName_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000001;
+              break;
+            }
+            case 18: {
+              if (!((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
+                families_ = new java.util.ArrayList<com.google.protobuf.ByteString>();
+                mutable_bitField0_ |= 0x00000002;
+              }
+              families_.add(input.readBytes());
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
+          families_ = java.util.Collections.unmodifiableList(families_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_TableCF_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_TableCF_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.class, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<TableCF> PARSER =
+        new com.google.protobuf.AbstractParser<TableCF>() {
+      public TableCF parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new TableCF(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<TableCF> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // optional .hbase.pb.TableName table_name = 1;
+    public static final int TABLE_NAME_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName tableName_;
+    /**
+     * <code>optional .hbase.pb.TableName table_name = 1;</code>
+     */
+    public boolean hasTableName() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional .hbase.pb.TableName table_name = 1;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName() {
+      return tableName_;
+    }
+    /**
+     * <code>optional .hbase.pb.TableName table_name = 1;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
+      return tableName_;
+    }
+
+    // repeated bytes families = 2;
+    public static final int FAMILIES_FIELD_NUMBER = 2;
+    private java.util.List<com.google.protobuf.ByteString> families_;
+    /**
+     * <code>repeated bytes families = 2;</code>
+     */
+    public java.util.List<com.google.protobuf.ByteString>
+        getFamiliesList() {
+      return families_;
+    }
+    /**
+     * <code>repeated bytes families = 2;</code>
+     */
+    public int getFamiliesCount() {
+      return families_.size();
+    }
+    /**
+     * <code>repeated bytes families = 2;</code>
+     */
+    public com.google.protobuf.ByteString getFamilies(int index) {
+      return families_.get(index);
+    }
+
+    private void initFields() {
+      tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+      families_ = java.util.Collections.emptyList();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (hasTableName()) {
+        if (!getTableName().isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeMessage(1, tableName_);
+      }
+      for (int i = 0; i < families_.size(); i++) {
+        output.writeBytes(2, families_.get(i));
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, tableName_);
+      }
+      {
+        int dataSize = 0;
+        for (int i = 0; i < families_.size(); i++) {
+          dataSize += com.google.protobuf.CodedOutputStream
+            .computeBytesSizeNoTag(families_.get(i));
+        }
+        size += dataSize;
+        size += 1 * getFamiliesList().size();
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF other = (org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF) obj;
+
+      boolean result = true;
+      result = result && (hasTableName() == other.hasTableName());
+      if (hasTableName()) {
+        result = result && getTableName()
+            .equals(other.getTableName());
+      }
+      result = result && getFamiliesList()
+          .equals(other.getFamiliesList());
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasTableName()) {
+        hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER;
+        hash = (53 * hash) + getTableName().hashCode();
+      }
+      if (getFamiliesCount() > 0) {
+        hash = (37 * hash) + FAMILIES_FIELD_NUMBER;
+        hash = (53 * hash) + getFamiliesList().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code hbase.pb.TableCF}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_TableCF_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_TableCF_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.class, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getTableNameFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        if (tableNameBuilder_ == null) {
+          tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+        } else {
+          tableNameBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        families_ = java.util.Collections.emptyList();
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_TableCF_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF build() {
+        org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF result = new org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (tableNameBuilder_ == null) {
+          result.tableName_ = tableName_;
+        } else {
+          result.tableName_ = tableNameBuilder_.build();
+        }
+        if (((bitField0_ & 0x00000002) == 0x00000002)) {
+          families_ = java.util.Collections.unmodifiableList(families_);
+          bitField0_ = (bitField0_ & ~0x00000002);
+        }
+        result.families_ = families_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.getDefaultInstance()) return this;
+        if (other.hasTableName()) {
+          mergeTableName(other.getTableName());
+        }
+        if (!other.families_.isEmpty()) {
+          if (families_.isEmpty()) {
+            families_ = other.families_;
+            bitField0_ = (bitField0_ & ~0x00000002);
+          } else {
+            ensureFamiliesIsMutable();
+            families_.addAll(other.families_);
+          }
+          onChanged();
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (hasTableName()) {
+          if (!getTableName().isInitialized()) {
+            
+            return false;
+          }
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // optional .hbase.pb.TableName table_name = 1;
+      private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public boolean hasTableName() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName() {
+        if (tableNameBuilder_ == null) {
+          return tableName_;
+        } else {
+          return tableNameBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public Builder setTableName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
+        if (tableNameBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          tableName_ = value;
+          onChanged();
+        } else {
+          tableNameBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public Builder setTableName(
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) {
+        if (tableNameBuilder_ == null) {
+          tableName_ = builderForValue.build();
+          onChanged();
+        } else {
+          tableNameBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public Builder mergeTableName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
+        if (tableNameBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              tableName_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance()) {
+            tableName_ =
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.newBuilder(tableName_).mergeFrom(value).buildPartial();
+          } else {
+            tableName_ = value;
+          }
+          onChanged();
+        } else {
+          tableNameBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public Builder clearTableName() {
+        if (tableNameBuilder_ == null) {
+          tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+          onChanged();
+        } else {
+          tableNameBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder getTableNameBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getTableNameFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
+        if (tableNameBuilder_ != null) {
+          return tableNameBuilder_.getMessageOrBuilder();
+        } else {
+          return tableName_;
+        }
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
+          getTableNameFieldBuilder() {
+        if (tableNameBuilder_ == null) {
+          tableNameBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
+                  tableName_,
+                  getParentForChildren(),
+                  isClean());
+          tableName_ = null;
+        }
+        return tableNameBuilder_;
+      }
+
+      // repeated bytes families = 2;
+      private java.util.List<com.google.protobuf.ByteString> families_ = java.util.Collections.emptyList();
+      private void ensureFamiliesIsMutable() {
+        if (!((bitField0_ & 0x00000002) == 0x00000002)) {
+          families_ = new java.util.ArrayList<com.google.protobuf.ByteString>(families_);
+          bitField0_ |= 0x00000002;
+         }
+      }
+      /**
+       * <code>repeated bytes families = 2;</code>
+       */
+      public java.util.List<com.google.protobuf.ByteString>
+          getFamiliesList() {
+        return java.util.Collections.unmodifiableList(families_);
+      }
+      /**
+       * <code>repeated bytes families = 2;</code>
+       */
+      public int getFamiliesCount() {
+        return families_.size();
+      }
+      /**
+       * <code>repeated bytes families = 2;</code>
+       */
+      public com.google.protobuf.ByteString getFamilies(int index) {
+        return families_.get(index);
+      }
+      /**
+       * <code>repeated bytes families = 2;</code>
+       */
+      public Builder setFamilies(
+          int index, com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureFamiliesIsMutable();
+        families_.set(index, value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated bytes families = 2;</code>
+       */
+      public Builder addFamilies(com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureFamiliesIsMutable();
+        families_.add(value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated bytes families = 2;</code>
+       */
+      public Builder addAllFamilies(
+          java.lang.Iterable<? extends com.google.protobuf.ByteString> values) {
+        ensureFamiliesIsMutable();
+        super.addAll(values, families_);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated bytes families = 2;</code>
+       */
+      public Builder clearFamilies() {
+        families_ = java.util.Collections.emptyList();
+        bitField0_ = (bitField0_ & ~0x00000002);
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:hbase.pb.TableCF)
+    }
+
+    static {
+      defaultInstance = new TableCF(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.TableCF)
+  }
+
   public interface ReplicationPeerOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
 
@@ -4044,6 +4757,31 @@ public final class ZooKeeperProtos {
      */
     org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPairOrBuilder getConfigurationOrBuilder(
         int index);
+
+    // repeated .hbase.pb.TableCF table_cfs = 5;
+    /**
+     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF> 
+        getTableCfsList();
+    /**
+     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF getTableCfs(int index);
+    /**
+     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+     */
+    int getTableCfsCount();
+    /**
+     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFOrBuilder> 
+        getTableCfsOrBuilderList();
+    /**
+     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFOrBuilder getTableCfsOrBuilder(
+        int index);
   }
   /**
    * Protobuf type {@code hbase.pb.ReplicationPeer}
@@ -4127,6 +4865,14 @@ public final class ZooKeeperProtos {
               configuration_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair.PARSER, extensionRegistry));
               break;
             }
+            case 42: {
+              if (!((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
+                tableCfs_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF>();
+                mutable_bitField0_ |= 0x00000010;
+              }
+              tableCfs_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.PARSER, extensionRegistry));
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -4141,6 +4887,9 @@ public final class ZooKeeperProtos {
         if (((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
           configuration_ = java.util.Collections.unmodifiableList(configuration_);
         }
+        if (((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
+          tableCfs_ = java.util.Collections.unmodifiableList(tableCfs_);
+        }
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
@@ -4346,11 +5095,48 @@ public final class ZooKeeperProtos {
       return configuration_.get(index);
     }
 
+    // repeated .hbase.pb.TableCF table_cfs = 5;
+    public static final int TABLE_CFS_FIELD_NUMBER = 5;
+    private java.util.List<org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF> tableCfs_;
+    /**
+     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+     */
+    public java.util.List<org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF> getTableCfsList() {
+      return tableCfs_;
+    }
+    /**
+     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+     */
+    public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFOrBuilder> 
+        getTableCfsOrBuilderList() {
+      return tableCfs_;
+    }
+    /**
+     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+     */
+    public int getTableCfsCount() {
+      return tableCfs_.size();
+    }
+    /**
+     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF getTableCfs(int index) {
+      return tableCfs_.get(index);
+    }
+    /**
+     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFOrBuilder getTableCfsOrBuilder(
+        int index) {
+      return tableCfs_.get(index);
+    }
+
     private void initFields() {
       clusterkey_ = "";
       replicationEndpointImpl_ = "";
       data_ = java.util.Collections.emptyList();
       configuration_ = java.util.Collections.emptyList();
+      tableCfs_ = java.util.Collections.emptyList();
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -4373,6 +5159,12 @@ public final class ZooKeeperProtos {
           return false;
         }
       }
+      for (int i = 0; i < getTableCfsCount(); i++) {
+        if (!getTableCfs(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
       memoizedIsInitialized = 1;
       return true;
     }
@@ -4392,6 +5184,9 @@ public final class ZooKeeperProtos {
       for (int i = 0; i < configuration_.size(); i++) {
         output.writeMessage(4, configuration_.get(i));
       }
+      for (int i = 0; i < tableCfs_.size(); i++) {
+        output.writeMessage(5, tableCfs_.get(i));
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -4417,6 +5212,10 @@ public final class ZooKeeperProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeMessageSize(4, configuration_.get(i));
       }
+      for (int i = 0; i < tableCfs_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(5, tableCfs_.get(i));
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -4454,6 +5253,8 @@ public final class ZooKeeperProtos {
           .equals(other.getDataList());
       result = result && getConfigurationList()
           .equals(other.getConfigurationList());
+      result = result && getTableCfsList()
+          .equals(other.getTableCfsList());
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -4483,6 +5284,10 @@ public final class ZooKeeperProtos {
         hash = (37 * hash) + CONFIGURATION_FIELD_NUMBER;
         hash = (53 * hash) + getConfigurationList().hashCode();
       }
+      if (getTableCfsCount() > 0) {
+        hash = (37 * hash) + TABLE_CFS_FIELD_NUMBER;
+        hash = (53 * hash) + getTableCfsList().hashCode();
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -4591,6 +5396,7 @@ public final class ZooKeeperProtos {
         if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
           getDataFieldBuilder();
           getConfigurationFieldBuilder();
+          getTableCfsFieldBuilder();
         }
       }
       private static Builder create() {
@@ -4615,6 +5421,12 @@ public final class ZooKeeperProtos {
         } else {
           configurationBuilder_.clear();
         }
+        if (tableCfsBuilder_ == null) {
+          tableCfs_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000010);
+        } else {
+          tableCfsBuilder_.clear();
+        }
         return this;
       }
 
@@ -4669,6 +5481,15 @@ public final class ZooKeeperProtos {
         } else {
           result.configuration_ = configurationBuilder_.build();
         }
+        if (tableCfsBuilder_ == null) {
+          if (((bitField0_ & 0x00000010) == 0x00000010)) {
+            tableCfs_ = java.util.Collections.unmodifiableList(tableCfs_);
+            bitField0_ = (bitField0_ & ~0x00000010);
+          }
+          result.tableCfs_ = tableCfs_;
+        } else {
+          result.tableCfs_ = tableCfsBuilder_.build();
+        }
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -4747,6 +5568,32 @@ public final class ZooKeeperProtos {
             }
           }
         }
+        if (tableCfsBuilder_ == null) {
+          if (!other.tableCfs_.isEmpty()) {
+            if (tableCfs_.isEmpty()) {
+              tableCfs_ = other.tableCfs_;
+              bitField0_ = (bitField0_ & ~0x00000010);
+            } else {
+              ensureTableCfsIsMutable();
+              tableCfs_.addAll(other.tableCfs_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.tableCfs_.isEmpty()) {
+            if (tableCfsBuilder_.isEmpty()) {
+              tableCfsBuilder_.dispose();
+              tableCfsBuilder_ = null;
+              tableCfs_ = other.tableCfs_;
+              bitField0_ = (bitField0_ & ~0x00000010);
+              tableCfsBuilder_ = 
+                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+                   getTableCfsFieldBuilder() : null;
+            } else {
+              tableCfsBuilder_.addAllMessages(other.tableCfs_);
+            }
+          }
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -4768,6 +5615,12 @@ public final class ZooKeeperProtos {
             return false;
           }
         }
+        for (int i = 0; i < getTableCfsCount(); i++) {
+          if (!getTableCfs(i).isInitialized()) {
+            
+            return false;
+          }
+        }
         return true;
       }
 
@@ -5448,6 +6301,246 @@ public final class ZooKeeperProtos {
         return configurationBuilder_;
       }
 
+      // repeated .hbase.pb.TableCF table_cfs = 5;
+      private java.util.List<org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF> tableCfs_ =
+        java.util.Collections.emptyList();
+      private void ensureTableCfsIsMutable() {
+        if (!((bitField0_ & 0x00000010) == 0x00000010)) {
+          tableCfs_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF>(tableCfs_);
+          bitField0_ |= 0x00000010;
+         }
+      }
+
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.Builder, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFOrBuilder> tableCfsBuilder_;
+
+      /**
+       * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF> getTableCfsList() {
+        if (tableCfsBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(tableCfs_);
+        } else {
+          return tableCfsBuilder_.getMessageList();
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+       */
+      public int getTableCfsCount() {
+        if (tableCfsBuilder_ == null) {
+          return tableCfs_.size();
+        } else {
+          return tableCfsBuilder_.getCount();
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF getTableCfs(int index) {
+        if (tableCfsBuilder_ == null) {
+          return tableCfs_.get(index);
+        } else {
+          return tableCfsBuilder_.getMessage(index);
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+       */
+      public Builder setTableCfs(
+          int index, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF value) {
+        if (tableCfsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureTableCfsIsMutable();
+          tableCfs_.set(index, value);
+          onChanged();
+        } else {
+          tableCfsBuilder_.setMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+       */
+      public Builder setTableCfs(
+          int index, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.Builder builderForValue) {
+        if (tableCfsBuilder_ == null) {
+          ensureTableCfsIsMutable();
+          tableCfs_.set(index, builderForValue.build());
+          onChanged();
+        } else {
+          tableCfsBuilder_.setMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+       */
+      public Builder addTableCfs(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF value) {
+        if (tableCfsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureTableCfsIsMutable();
+          tableCfs_.add(value);
+          onChanged();
+        } else {
+          tableCfsBuilder_.addMessage(value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+       */
+      public Builder addTableCfs(
+          int index, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF value) {
+        if (tableCfsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureTableCfsIsMutable();
+          tableCfs_.add(index, value);
+          onChanged();
+        } else {
+          tableCfsBuilder_.addMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+       */
+      public Builder addTableCfs(
+          org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.Builder builderForValue) {
+        if (tableCfsBuilder_ == null) {
+          ensureTableCfsIsMutable();
+          tableCfs_.add(builderForValue.build());
+          onChanged();
+        } else {
+          tableCfsBuilder_.addMessage(builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+       */
+      public Builder addTableCfs(
+          int index, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.Builder builderForValue) {
+        if (tableCfsBuilder_ == null) {
+          ensureTableCfsIsMutable();
+          tableCfs_.add(index, builderForValue.build());
+          onChanged();
+        } else {
+          tableCfsBuilder_.addMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+       */
+      public Builder addAllTableCfs(
+          java.lang.Iterable<? extends org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF> values) {
+        if (tableCfsBuilder_ == null) {
+          ensureTableCfsIsMutable();
+          super.addAll(values, tableCfs_);
+          onChanged();
+        } else {
+          tableCfsBuilder_.addAllMessages(values);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+       */
+      public Builder clearTableCfs() {
+        if (tableCfsBuilder_ == null) {
+          tableCfs_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000010);
+          onChanged();
+        } else {
+          tableCfsBuilder_.clear();
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+       */
+      public Builder removeTableCfs(int index) {
+        if (tableCfsBuilder_ == null) {
+          ensureTableCfsIsMutable();
+          tableCfs_.remove(index);
+          onChanged();
+        } else {
+          tableCfsBuilder_.remove(index);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.Builder getTableCfsBuilder(
+          int index) {
+        return getTableCfsFieldBuilder().getBuilder(index);
+      }
+      /**
+       * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFOrBuilder getTableCfsOrBuilder(
+          int index) {
+        if (tableCfsBuilder_ == null) {
+          return tableCfs_.get(index);  } else {
+          return tableCfsBuilder_.getMessageOrBuilder(index);
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+       */
+      public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFOrBuilder> 
+           getTableCfsOrBuilderList() {
+        if (tableCfsBuilder_ != null) {
+          return tableCfsBuilder_.getMessageOrBuilderList();
+        } else {
+          return java.util.Collections.unmodifiableList(tableCfs_);
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.Builder addTableCfsBuilder() {
+        return getTableCfsFieldBuilder().addBuilder(
+            org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.Builder addTableCfsBuilder(
+          int index) {
+        return getTableCfsFieldBuilder().addBuilder(
+            index, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.Builder> 
+           getTableCfsBuilderList() {
+        return getTableCfsFieldBuilder().getBuilderList();
+      }
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.Builder, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFOrBuilder> 
+          getTableCfsFieldBuilder() {
+        if (tableCfsBuilder_ == null) {
+          tableCfsBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.Builder, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFOrBuilder>(
+                  tableCfs_,
+                  ((bitField0_ & 0x00000010) == 0x00000010),
+                  getParentForChildren(),
+                  isClean());
+          tableCfs_ = null;
+        }
+        return tableCfsBuilder_;
+      }
+
       // @@protoc_insertion_point(builder_scope:hbase.pb.ReplicationPeer)
     }
 
@@ -8666,6 +9759,11 @@ public final class ZooKeeperProtos {
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
       internal_static_hbase_pb_DeprecatedTableState_fieldAccessorTable;
   private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_hbase_pb_TableCF_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_hbase_pb_TableCF_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ReplicationPeer_descriptor;
   private static
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
@@ -8722,23 +9820,26 @@ public final class ZooKeeperProtos {
       "Y\020\002\"\225\001\n\024DeprecatedTableState\022<\n\005state\030\001 " +
       "\002(\0162$.hbase.pb.DeprecatedTableState.Stat" +
       "e:\007ENABLED\"?\n\005State\022\013\n\007ENABLED\020\000\022\014\n\010DISA" +
-      "BLED\020\001\022\r\n\tDISABLING\020\002\022\014\n\010ENABLING\020\003\"\237\001\n\017" +
-      "ReplicationPeer\022\022\n\nclusterkey\030\001 \002(\t\022\037\n\027r",
-      "eplicationEndpointImpl\030\002 \001(\t\022&\n\004data\030\003 \003" +
-      "(\0132\030.hbase.pb.BytesBytesPair\022/\n\rconfigur" +
-      "ation\030\004 \003(\0132\030.hbase.pb.NameStringPair\"g\n" +
-      "\020ReplicationState\022/\n\005state\030\001 \002(\0162 .hbase" +
-      ".pb.ReplicationState.State\"\"\n\005State\022\013\n\007E" +
-      "NABLED\020\000\022\014\n\010DISABLED\020\001\"+\n\027ReplicationHLo" +
-      "gPosition\022\020\n\010position\030\001 \002(\003\"%\n\017Replicati" +
-      "onLock\022\022\n\nlock_owner\030\001 \002(\t\"\252\001\n\tTableLock" +
-      "\022\'\n\ntable_name\030\001 \001(\0132\023.hbase.pb.TableNam" +
-      "e\022(\n\nlock_owner\030\002 \001(\0132\024.hbase.pb.ServerN",
-      "ame\022\021\n\tthread_id\030\003 \001(\003\022\021\n\tis_shared\030\004 \001(" +
-      "\010\022\017\n\007purpose\030\005 \001(\t\022\023\n\013create_time\030\006 \001(\003\"" +
-      "\036\n\013SwitchState\022\017\n\007enabled\030\001 \001(\010BE\n*org.a" +
-      "pache.hadoop.hbase.protobuf.generatedB\017Z" +
-      "ooKeeperProtosH\001\210\001\001\240\001\001"
+      "BLED\020\001\022\r\n\tDISABLING\020\002\022\014\n\010ENABLING\020\003\"D\n\007T" +
+      "ableCF\022\'\n\ntable_name\030\001 \001(\0132\023.hbase.pb.Ta",
+      "bleName\022\020\n\010families\030\002 \003(\014\"\305\001\n\017Replicatio" +
+      "nPeer\022\022\n\nclusterkey\030\001 \002(\t\022\037\n\027replication" +
+      "EndpointImpl\030\002 \001(\t\022&\n\004data\030\003 \003(\0132\030.hbase" +
+      ".pb.BytesBytesPair\022/\n\rconfiguration\030\004 \003(" +
+      "\0132\030.hbase.pb.NameStringPair\022$\n\ttable_cfs" +
+      "\030\005 \003(\0132\021.hbase.pb.TableCF\"g\n\020Replication" +
+      "State\022/\n\005state\030\001 \002(\0162 .hbase.pb.Replicat" +
+      "ionState.State\"\"\n\005State\022\013\n\007ENABLED\020\000\022\014\n\010" +
+      "DISABLED\020\001\"+\n\027ReplicationHLogPosition\022\020\n" +
+      "\010position\030\001 \002(\003\"%\n\017ReplicationLock\022\022\n\nlo",
+      "ck_owner\030\001 \002(\t\"\252\001\n\tTableLock\022\'\n\ntable_na" +
+      "me\030\001 \001(\0132\023.hbase.pb.TableName\022(\n\nlock_ow" +
+      "ner\030\002 \001(\0132\024.hbase.pb.ServerName\022\021\n\tthrea" +
+      "d_id\030\003 \001(\003\022\021\n\tis_shared\030\004 \001(\010\022\017\n\007purpose" +
+      "\030\005 \001(\t\022\023\n\013create_time\030\006 \001(\003\"\036\n\013SwitchSta" +
+      "te\022\017\n\007enabled\030\001 \001(\010BE\n*org.apache.hadoop" +
+      ".hbase.protobuf.generatedB\017ZooKeeperProt" +
+      "osH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -8775,38 +9876,44 @@ public final class ZooKeeperProtos {
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_DeprecatedTableState_descriptor,
               new java.lang.String[] { "State", });
-          internal_static_hbase_pb_ReplicationPeer_descriptor =
+          internal_static_hbase_pb_TableCF_descriptor =
             getDescriptor().getMessageTypes().get(5);
+          internal_static_hbase_pb_TableCF_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_hbase_pb_TableCF_descriptor,
+              new java.lang.String[] { "TableName", "Families", });
+          internal_static_hbase_pb_ReplicationPeer_descriptor =
+            getDescriptor().getMessageTypes().get(6);
           internal_static_hbase_pb_ReplicationPeer_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_ReplicationPeer_descriptor,
-              new java.lang.String[] { "Clusterkey", "ReplicationEndpointImpl", "Data", "Configuration", });
+              new java.lang.String[] { "Clusterkey", "ReplicationEndpointImpl", "Data", "Configuration", "TableCfs", });
           internal_static_hbase_pb_ReplicationState_descriptor =
-            getDescriptor().getMessageTypes().get(6);
+            getDescriptor().getMessageTypes().get(7);
           internal_static_hbase_pb_ReplicationState_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_ReplicationState_descriptor,
               new java.lang.String[] { "State", });
           internal_static_hbase_pb_ReplicationHLogPosition_descriptor =
-            getDescriptor().getMessageTypes().get(7);
+            getDescriptor().getMessageTypes().get(8);
           internal_static_hbase_pb_ReplicationHLogPosition_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_ReplicationHLogPosition_descriptor,
               new java.lang.String[] { "Position", });
           internal_static_hbase_pb_ReplicationLock_descriptor =
-            getDescriptor().getMessageTypes().get(8);
+            getDescriptor().getMessageTypes().get(9);
           internal_static_hbase_pb_ReplicationLock_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_ReplicationLock_descriptor,
               new java.lang.String[] { "LockOwner", });
           internal_static_hbase_pb_TableLock_descriptor =
-            getDescriptor().getMessageTypes().get(9);
+            getDescriptor().getMessageTypes().get(10);
           internal_static_hbase_pb_TableLock_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_TableLock_descriptor,
               new java.lang.String[] { "TableName", "LockOwner", "ThreadId", "IsShared", "Purpose", "CreateTime", });
           internal_static_hbase_pb_SwitchState_descriptor =
-            getDescriptor().getMessageTypes().get(10);
+            getDescriptor().getMessageTypes().get(11);
           internal_static_hbase_pb_SwitchState_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_SwitchState_descriptor,

http://git-wip-us.apache.org/repos/asf/hbase/blob/7f39baf0/hbase-protocol/src/main/protobuf/ZooKeeper.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/ZooKeeper.proto b/hbase-protocol/src/main/protobuf/ZooKeeper.proto
index 4963c09..186d183 100644
--- a/hbase-protocol/src/main/protobuf/ZooKeeper.proto
+++ b/hbase-protocol/src/main/protobuf/ZooKeeper.proto
@@ -105,6 +105,11 @@ message DeprecatedTableState {
   required State state = 1 [default = ENABLED];
 }
 
+message TableCF {
+  optional TableName table_name = 1;
+  repeated bytes families = 2;
+}
+
 /**
  * Used by replication. Holds a replication peer key.
  */
@@ -115,6 +120,7 @@ message ReplicationPeer {
   optional string replicationEndpointImpl = 2;
   repeated BytesBytesPair data = 3;
   repeated NameStringPair configuration = 4;
+  repeated TableCF table_cfs = 5;
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/7f39baf0/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 221dcd5..9829a85 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
@@ -141,6 +141,7 @@ import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
 import org.apache.hadoop.hbase.regionserver.RegionSplitPolicy;
 import org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy;
 import org.apache.hadoop.hbase.regionserver.compactions.FIFOCompactionPolicy;
+import org.apache.hadoop.hbase.replication.master.TableCFsUpdater;
 import org.apache.hadoop.hbase.replication.regionserver.Replication;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.UserProvider;
@@ -671,6 +672,7 @@ public class HMaster extends HRegionServer implements MasterServices {
     // publish cluster ID
     status.setStatus("Publishing Cluster ID in ZooKeeper");
     ZKClusterId.setClusterId(this.zooKeeper, fileSystemManager.getClusterId());
+
     this.serverManager = createServerManager(this, this);
 
     // Invalidate all write locks held previously
@@ -680,6 +682,13 @@ public class HMaster extends HRegionServer implements MasterServices {
     status.setStatus("Initializing ZK system trackers");
     initializeZKBasedSystemTrackers();
 
+    // This is for backwards compatibility
+    // See HBASE-11393
+    status.setStatus("Update TableCFs node in ZNode");
+    TableCFsUpdater tableCFsUpdater = new TableCFsUpdater(zooKeeper,
+            conf, this.clusterConnection);
+    tableCFsUpdater.update();
+
     // initialize master side coprocessors before we start handling requests
     status.setStatus("Initializing master coprocessors");
     this.cpHost = new MasterCoprocessorHost(this, this.conf);
@@ -849,7 +858,6 @@ public class HMaster extends HRegionServer implements MasterServices {
 
     zombieDetector.interrupt();
   }
-
   /**
    * Create a {@link ServerManager} instance.
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/7f39baf0/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/TableCFsUpdater.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/TableCFsUpdater.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/TableCFsUpdater.java
new file mode 100644
index 0000000..4e28756
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/TableCFsUpdater.java
@@ -0,0 +1,122 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication.master;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.client.replication.ReplicationSerDeHelper;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.ReplicationStateZKBase;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.zookeeper.KeeperException;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * This class is used to upgrade TableCFs from HBase 1.x to HBase 2.x.
+ * It will be removed in HBase 3.x.  See HBASE-11393
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class TableCFsUpdater extends ReplicationStateZKBase {
+
+  private static final Log LOG = LogFactory.getLog(TableCFsUpdater.class);
+
+  public TableCFsUpdater(ZooKeeperWatcher zookeeper,
+                         Configuration conf, Abortable abortable) {
+    super(zookeeper, conf, abortable);
+  }
+
+  public void update() {
+    List<String> znodes = null;
+    try {
+      znodes = ZKUtil.listChildrenNoWatch(this.zookeeper, this.peersZNode);
+    } catch (KeeperException e) {
+      LOG.warn("", e);
+    }
+    if (znodes != null) {
+      for (String peerId : znodes) {
+        if (!update(peerId)) {
+          LOG.error("upgrade tableCFs failed for peerId=" + peerId);
+        }
+      }
+    }
+  }
+
+  public boolean update(String peerId) {
+    String tableCFsNode = getTableCFsNode(peerId);
+    try {
+      if (ZKUtil.checkExists(zookeeper, tableCFsNode) != -1) {
+        String peerNode = getPeerNode(peerId);
+        ReplicationPeerConfig rpc = getReplicationPeerConig(peerNode);
+        // We only need to copy data from tableCFs node to rpc Node the first time hmaster start.
+        if (rpc.getTableCFsMap() == null || rpc.getTableCFsMap().size() == 0) {
+          // we copy TableCFs node into PeerNode
+          LOG.info("copy tableCFs into peerNode:" + peerId);
+          ZooKeeperProtos.TableCF[] tableCFs =
+                  ReplicationSerDeHelper.parseTableCFs(
+                          ZKUtil.getData(this.zookeeper, tableCFsNode));
+          if (tableCFs != null && tableCFs.length > 0) {
+            rpc.setTableCFsMap(ReplicationSerDeHelper.convert2Map(tableCFs));
+            ZKUtil.setData(this.zookeeper, peerNode,
+              ReplicationSerDeHelper.toByteArray(rpc));
+          }
+        } else {
+          LOG.info("No tableCFs in peerNode:" + peerId);
+        }
+      }
+    } catch (KeeperException e) {
+      LOG.warn("NOTICE!! Update peerId failed, peerId=" + peerId, e);
+      return false;
+    } catch (InterruptedException e) {
+      LOG.warn("NOTICE!! Update peerId failed, peerId=" + peerId, e);
+      return false;
+    } catch (IOException e) {
+      LOG.warn("NOTICE!! Update peerId failed, peerId=" + peerId, e);
+      return false;
+    }
+    return true;
+  }
+
+  private ReplicationPeerConfig getReplicationPeerConig(String peerNode)
+          throws KeeperException, InterruptedException {
+    byte[] data = null;
+    data = ZKUtil.getData(this.zookeeper, peerNode);
+    if (data == null) {
+      LOG.error("Could not get configuration for " +
+              "peer because it doesn't exist. peer=" + peerNode);
+      return null;
+    }
+    try {
+      return ReplicationSerDeHelper.parsePeerFrom(data);
+    } catch (DeserializationException e) {
+      LOG.warn("Failed to parse cluster key from peer=" + peerNode);
+      return null;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/7f39baf0/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 b773b46..a79aa0a 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
@@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequ
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.regionserver.StorefileRefresherChore;
 import org.apache.hadoop.hbase.regionserver.TestHRegionServerBulkLoad;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -249,7 +250,10 @@ public class TestReplicaWithCluster {
     HTU2.getHBaseAdmin().createTable(hdt, HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE);
 
     ReplicationAdmin admin = new ReplicationAdmin(HTU.getConfiguration());
-    admin.addPeer("2", HTU2.getClusterKey());
+
+    ReplicationPeerConfig rpc = new ReplicationPeerConfig();
+    rpc.setClusterKey(HTU2.getClusterKey());
+    admin.addPeer("2", rpc, null);
     admin.close();
 
     Put p = new Put(row);

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

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/7f39baf0/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java
index e919c24..775e4a5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java
@@ -52,6 +52,7 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
+import org.apache.hadoop.hbase.client.replication.ReplicationSerDeHelper;
 import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
@@ -501,8 +502,9 @@ public class TestMasterReplication {
     try {
       replicationAdmin = new ReplicationAdmin(
           configurations[masterClusterNumber]);
-      replicationAdmin.addPeer(id,
-          utilities[slaveClusterNumber].getClusterKey());
+      ReplicationPeerConfig rpc = new ReplicationPeerConfig();
+      rpc.setClusterKey(utilities[slaveClusterNumber].getClusterKey());
+      replicationAdmin.addPeer(id, rpc, null);
     } finally {
       close(replicationAdmin);
     }
@@ -513,7 +515,10 @@ public class TestMasterReplication {
     ReplicationAdmin replicationAdmin = null;
     try {
       replicationAdmin = new ReplicationAdmin(configurations[masterClusterNumber]);
-      replicationAdmin.addPeer(id, utilities[slaveClusterNumber].getClusterKey(), tableCfs);
+      ReplicationPeerConfig replicationPeerConfig = new ReplicationPeerConfig();
+      replicationPeerConfig.setClusterKey(utilities[slaveClusterNumber].getClusterKey());
+      replicationAdmin.addPeer(id, replicationPeerConfig,
+        ReplicationSerDeHelper.parseTableCFsFromConfig(tableCfs));
     } finally {
       close(replicationAdmin);
     }

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


[32/50] [abbrv] hbase git commit: HBASE-15384 Avoid using '/tmp' directory in TestBulkLoad

Posted by sy...@apache.org.
HBASE-15384 Avoid using '/tmp' directory in TestBulkLoad


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

Branch: refs/heads/hbase-12439
Commit: e9c4f12856420b3089665751ea141298580f6c21
Parents: 07c597c
Author: chenheng <ch...@apache.org>
Authored: Thu Mar 24 15:19:02 2016 +0800
Committer: chenheng <ch...@apache.org>
Committed: Fri Mar 25 12:06:55 2016 +0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/regionserver/TestBulkLoad.java   | 11 +++++++++--
 1 file changed, 9 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e9c4f128/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
index d0633a8..6ba12a9 100644
--- 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
@@ -38,6 +38,7 @@ 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.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
@@ -81,6 +82,7 @@ public class TestBulkLoad {
 
   @ClassRule
   public static TemporaryFolder testFolder = new TemporaryFolder();
+  private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private final WAL log = mock(WAL.class);
   private final Configuration conf = HBaseConfiguration.create();
   private final Random random = new Random();
@@ -217,13 +219,18 @@ public class TestBulkLoad {
   }
 
   private Pair<byte[], String> withMissingHFileForFamily(byte[] family) {
-    return new Pair<byte[], String>(family, "/tmp/does_not_exist");
+    return new Pair<byte[], String>(family, getNotExistFilePath());
+  }
+
+  private String getNotExistFilePath() {
+    Path path = new Path(TEST_UTIL.getDataTestDir(), "does_not_exist");
+    return path.toUri().getPath();
   }
 
   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");
+    return new Pair<byte[], String>(new byte[]{0x00, 0x01, 0x02}, getNotExistFilePath());
   }
 
 


[17/50] [abbrv] hbase git commit: HBASE-15477 Purge 'next block header' from cached blocks

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/12f66e30/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileScanner.java
index c67bdd4..e0f3d74 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileScanner.java
@@ -99,18 +99,21 @@ public interface HFileScanner extends Shipper, Closeable {
    * @throws IOException
    */
   boolean seekTo() throws IOException;
+
   /**
    * Scans to the next entry in the file.
    * @return Returns false if you are at the end otherwise true if more in file.
    * @throws IOException
    */
   boolean next() throws IOException;
+
   /**
    * Gets the current key in the form of a cell. You must call
    * {@link #seekTo(Cell)} before this method.
    * @return gets the current key as a Cell.
    */
   Cell getKey();
+
   /**
    * Gets a buffer view to the current value.  You must call
    * {@link #seekTo(Cell)} before this method.
@@ -119,26 +122,35 @@ public interface HFileScanner extends Shipper, Closeable {
    * the position is 0, the start of the buffer view.
    */
   ByteBuffer getValue();
+
   /**
    * @return Instance of {@link org.apache.hadoop.hbase.Cell}.
    */
   Cell getCell();
+
   /**
    * Convenience method to get a copy of the key as a string - interpreting the
    * bytes as UTF8. You must call {@link #seekTo(Cell)} before this method.
    * @return key as a string
+   * @deprecated Since hbase-2.0.0
    */
+  @Deprecated
   String getKeyString();
+
   /**
    * Convenience method to get a copy of the value as a string - interpreting
    * the bytes as UTF8. You must call {@link #seekTo(Cell)} before this method.
    * @return value as a string
+   * @deprecated Since hbase-2.0.0
    */
+  @Deprecated
   String getValueString();
+
   /**
    * @return Reader that underlies this Scanner instance.
    */
   HFile.Reader getReader();
+
   /**
    * @return True is scanner has had one of the seek calls invoked; i.e.
    * {@link #seekBefore(Cell)} or {@link #seekTo()} or {@link #seekTo(Cell)}.

http://git-wip-us.apache.org/repos/asf/hbase/blob/12f66e30/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
index 66aced0..69c42c3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
@@ -1317,25 +1317,22 @@ public class BucketCache implements BlockCache, HeapSize {
         final AtomicLong realCacheSize) throws CacheFullException, IOException,
         BucketAllocatorException {
       int len = data.getSerializedLength();
-      // This cacheable thing can't be serialized...
+      // This cacheable thing can't be serialized
       if (len == 0) return null;
       long offset = bucketAllocator.allocateBlock(len);
       BucketEntry bucketEntry = new BucketEntry(offset, len, accessCounter, inMemory);
       bucketEntry.setDeserialiserReference(data.getDeserializer(), deserialiserMap);
       try {
         if (data instanceof HFileBlock) {
-          HFileBlock block = (HFileBlock) data;
-          ByteBuff sliceBuf = block.getBufferReadOnlyWithHeader();
-          sliceBuf.rewind();
-          assert len == sliceBuf.limit() + HFileBlock.EXTRA_SERIALIZATION_SPACE ||
-            len == sliceBuf.limit() + block.headerSize() + HFileBlock.EXTRA_SERIALIZATION_SPACE;
-          ByteBuffer extraInfoBuffer = ByteBuffer.allocate(HFileBlock.EXTRA_SERIALIZATION_SPACE);
-          block.serializeExtraInfo(extraInfoBuffer);
+          // If an instance of HFileBlock, save on some allocations.
+          HFileBlock block = (HFileBlock)data;
+          ByteBuff sliceBuf = block.getBufferReadOnly();
+          ByteBuffer metadata = block.getMetaData();
           if (LOG.isTraceEnabled()) {
             LOG.trace("Write offset=" + offset + ", len=" + len);
           }
           ioEngine.write(sliceBuf, offset);
-          ioEngine.write(extraInfoBuffer, offset + len - HFileBlock.EXTRA_SERIALIZATION_SPACE);
+          ioEngine.write(metadata, offset + len - metadata.limit());
         } else {
           ByteBuffer bb = ByteBuffer.allocate(len);
           data.serialize(bb);

http://git-wip-us.apache.org/repos/asf/hbase/blob/12f66e30/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java
index eae713f..3df284c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java
@@ -18,6 +18,7 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
+import java.io.Closeable;
 import java.io.IOException;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -29,7 +30,7 @@ import org.apache.hadoop.hbase.client.Scan;
  * Scanner that returns the next KeyValue.
  */
 @InterfaceAudience.Private
-public interface KeyValueScanner extends Shipper {
+public interface KeyValueScanner extends Shipper, Closeable {
   /**
    * The byte array represents for NO_NEXT_INDEXED_KEY;
    * The actual value is irrelevant because this is always compared by reference.
@@ -71,6 +72,7 @@ public interface KeyValueScanner extends Shipper {
    * The default implementation for this would be to return 0. A file having
    * lower sequence id will be considered to be the older one.
    */
+  // TODO: Implement SequenceId Interface instead.
   long getSequenceID();
 
   /**
@@ -165,4 +167,4 @@ public interface KeyValueScanner extends Shipper {
    * if known, or null otherwise
    */
   public Cell getNextIndexedKey();
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/12f66e30/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
index 61eb9b8..b6164b2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
@@ -1271,7 +1271,7 @@ public class StoreFile {
     }
 
     /**
-     * Warning: Do not write further code which depends on this call. Instead
+     * @deprecated Do not write further code which depends on this call. Instead
      * use getStoreFileScanner() which uses the StoreFileScanner class/interface
      * which is the preferred way to scan a store with higher level concepts.
      *
@@ -1285,7 +1285,7 @@ public class StoreFile {
     }
 
     /**
-     * Warning: Do not write further code which depends on this call. Instead
+     * @deprecated Do not write further code which depends on this call. Instead
      * use getStoreFileScanner() which uses the StoreFileScanner class/interface
      * which is the preferred way to scan a store with higher level concepts.
      *

http://git-wip-us.apache.org/repos/asf/hbase/blob/12f66e30/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java
index 69671e2..040685d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java
@@ -40,7 +40,6 @@ import org.apache.hadoop.hbase.io.HeapSize;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache;
 import org.apache.hadoop.hbase.nio.ByteBuff;
-import org.apache.hadoop.hbase.nio.SingleByteBuff;
 import org.apache.hadoop.hbase.util.ChecksumType;
 
 public class CacheTestUtils {
@@ -66,6 +65,7 @@ public class CacheTestUtils {
     /*Post eviction, heapsize should be the same */
     assertEquals(heapSize, ((HeapSize) toBeTested).heapSize());
   }
+
   public static void testCacheMultiThreaded(final BlockCache toBeTested,
       final int blockSize, final int numThreads, final int numQueries,
       final double passingScore) throws Exception {
@@ -339,25 +339,16 @@ public class CacheTestUtils {
   }
 
 
-  private static HFileBlockPair[] generateHFileBlocks(int blockSize,
-      int numBlocks) {
+  private static HFileBlockPair[] generateHFileBlocks(int blockSize, int numBlocks) {
     HFileBlockPair[] returnedBlocks = new HFileBlockPair[numBlocks];
     Random rand = new Random();
     HashSet<String> usedStrings = new HashSet<String>();
     for (int i = 0; i < numBlocks; i++) {
-
-      // The buffer serialized size needs to match the size of BlockSize. So we
-      // declare our data size to be smaller than it by the serialization space
-      // required.
-
-      SingleByteBuff cachedBuffer = new SingleByteBuff(ByteBuffer.allocate(blockSize
-          - HFileBlock.EXTRA_SERIALIZATION_SPACE));
+      ByteBuffer cachedBuffer = ByteBuffer.allocate(blockSize);
       rand.nextBytes(cachedBuffer.array());
       cachedBuffer.rewind();
-      int onDiskSizeWithoutHeader = blockSize
-          - HFileBlock.EXTRA_SERIALIZATION_SPACE;
-      int uncompressedSizeWithoutHeader = blockSize
-          - HFileBlock.EXTRA_SERIALIZATION_SPACE;
+      int onDiskSizeWithoutHeader = blockSize;
+      int uncompressedSizeWithoutHeader = blockSize;
       long prevBlockOffset = rand.nextLong();
       BlockType.DATA.write(cachedBuffer);
       cachedBuffer.putInt(onDiskSizeWithoutHeader);
@@ -376,7 +367,7 @@ public class CacheTestUtils {
           onDiskSizeWithoutHeader, uncompressedSizeWithoutHeader,
           prevBlockOffset, cachedBuffer, HFileBlock.DONT_FILL_HEADER,
           blockSize,
-          onDiskSizeWithoutHeader + HConstants.HFILEBLOCK_HEADER_SIZE, meta);
+          onDiskSizeWithoutHeader + HConstants.HFILEBLOCK_HEADER_SIZE, -1, meta);
 
       String strKey;
       /* No conflicting keys */
@@ -395,4 +386,4 @@ public class CacheTestUtils {
     BlockCacheKey blockName;
     HFileBlock block;
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/12f66e30/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
index 5158e35..a9d8258 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
@@ -259,7 +259,6 @@ public class TestCacheOnWrite {
     assertTrue(testDescription, scanner.seekTo());
 
     long offset = 0;
-    HFileBlock prevBlock = null;
     EnumMap<BlockType, Integer> blockCountByType =
         new EnumMap<BlockType, Integer>(BlockType.class);
 
@@ -267,14 +266,10 @@ public class TestCacheOnWrite {
     List<Long> cachedBlocksOffset = new ArrayList<Long>();
     Map<Long, HFileBlock> cachedBlocks = new HashMap<Long, HFileBlock>();
     while (offset < reader.getTrailer().getLoadOnOpenDataOffset()) {
-      long onDiskSize = -1;
-      if (prevBlock != null) {
-         onDiskSize = prevBlock.getNextBlockOnDiskSizeWithHeader();
-      }
       // Flags: don't cache the block, use pread, this is not a compaction.
       // Also, pass null for expected block type to avoid checking it.
-      HFileBlock block = reader.readBlock(offset, onDiskSize, false, true,
-        false, true, null, encodingInCache);
+      HFileBlock block = reader.readBlock(offset, -1, false, true, false, true, null,
+          encodingInCache);
       BlockCacheKey blockCacheKey = new BlockCacheKey(reader.getName(),
           offset);
       HFileBlock fromCache = (HFileBlock) blockCache.getBlock(blockCacheKey, true, false, true);
@@ -307,7 +302,6 @@ public class TestCacheOnWrite {
         assertEquals(
           block.getUncompressedSizeWithoutHeader(), fromCache.getUncompressedSizeWithoutHeader());
       }
-      prevBlock = block;
       offset += block.getOnDiskSizeWithHeader();
       BlockType bt = block.getBlockType();
       Integer count = blockCountByType.get(bt);

http://git-wip-us.apache.org/repos/asf/hbase/blob/12f66e30/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestChecksum.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestChecksum.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestChecksum.java
index 91ab8c0..d91a811 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestChecksum.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestChecksum.java
@@ -94,7 +94,7 @@ public class TestChecksum {
     meta = new HFileContextBuilder().withHBaseCheckSum(true).build();
     HFileBlock.FSReader hbr = new HFileBlock.FSReaderImpl(
         is, totalSize, (HFileSystem) fs, path, meta);
-    HFileBlock b = hbr.readBlockData(0, -1, -1, false);
+    HFileBlock b = hbr.readBlockData(0, -1, false);
     assertEquals(b.getChecksumType(), ChecksumType.getDefaultChecksumType().getCode());
   }
 
@@ -108,12 +108,14 @@ public class TestChecksum {
       ChecksumType cktype = itr.next();
       Path path = new Path(TEST_UTIL.getDataTestDir(), "checksum" + cktype.getName());
       FSDataOutputStream os = fs.create(path);
-      HFileContext meta = new HFileContextBuilder()
-          .withChecksumType(cktype).build();
+      HFileContext meta = new HFileContextBuilder().
+          withChecksumType(cktype).
+          build();
       HFileBlock.Writer hbw = new HFileBlock.Writer(null, meta);
       DataOutputStream dos = hbw.startWriting(BlockType.DATA);
-      for (int i = 0; i < 1000; ++i)
+      for (int i = 0; i < 1000; ++i) {
         dos.writeInt(i);
+      }
       hbw.writeHeaderAndData(os);
       int totalSize = hbw.getOnDiskSizeWithHeader();
       os.close();
@@ -125,7 +127,7 @@ public class TestChecksum {
       meta = new HFileContextBuilder().withHBaseCheckSum(true).build();
       HFileBlock.FSReader hbr = new HFileBlock.FSReaderImpl(
           is, totalSize, (HFileSystem) fs, path, meta);
-      HFileBlock b = hbr.readBlockData(0, -1, -1, false);
+      HFileBlock b = hbr.readBlockData(0, -1, false);
       ByteBuff data = b.getBufferWithoutHeader();
       for (int i = 0; i < 1000; i++) {
         assertEquals(i, data.getInt());
@@ -188,7 +190,7 @@ public class TestChecksum {
               .withHBaseCheckSum(true)
               .build();
         HFileBlock.FSReader hbr = new FSReaderImplTest(is, totalSize, fs, path, meta);
-        HFileBlock b = hbr.readBlockData(0, -1, -1, pread);
+        HFileBlock b = hbr.readBlockData(0, -1, pread);
         b.sanityCheck();
         assertEquals(4936, b.getUncompressedSizeWithoutHeader());
         assertEquals(algo == GZ ? 2173 : 4936, 
@@ -209,17 +211,17 @@ public class TestChecksum {
         // requests. Verify that this is correct.
         for (int i = 0; i < 
              HFileBlock.CHECKSUM_VERIFICATION_NUM_IO_THRESHOLD + 1; i++) {
-          b = hbr.readBlockData(0, -1, -1, pread);
+          b = hbr.readBlockData(0, -1, pread);
           assertEquals(0, HFile.getChecksumFailuresCount());
         }
         // The next read should have hbase checksum verification reanabled,
         // we verify this by assertng that there was a hbase-checksum failure.
-        b = hbr.readBlockData(0, -1, -1, pread);
+        b = hbr.readBlockData(0, -1, pread);
         assertEquals(1, HFile.getChecksumFailuresCount());
 
         // Since the above encountered a checksum failure, we switch
         // back to not checking hbase checksums.
-        b = hbr.readBlockData(0, -1, -1, pread);
+        b = hbr.readBlockData(0, -1, pread);
         assertEquals(0, HFile.getChecksumFailuresCount());
         is.close();
 
@@ -230,7 +232,7 @@ public class TestChecksum {
         assertEquals(false, newfs.useHBaseChecksum());
         is = new FSDataInputStreamWrapper(newfs, path);
         hbr = new FSReaderImplTest(is, totalSize, newfs, path, meta);
-        b = hbr.readBlockData(0, -1, -1, pread);
+        b = hbr.readBlockData(0, -1, pread);
         is.close();
         b.sanityCheck();
         b = b.unpack(meta, hbr);
@@ -314,7 +316,7 @@ public class TestChecksum {
                .build();
         HFileBlock.FSReader hbr = new HFileBlock.FSReaderImpl(new FSDataInputStreamWrapper(
             is, nochecksum), totalSize, hfs, path, meta);
-        HFileBlock b = hbr.readBlockData(0, -1, -1, pread);
+        HFileBlock b = hbr.readBlockData(0, -1, pread);
         is.close();
         b.sanityCheck();
         assertEquals(dataSize, b.getUncompressedSizeWithoutHeader());
@@ -354,5 +356,4 @@ public class TestChecksum {
       return false;  // checksum validation failure
     }
   }
-}
-
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/12f66e30/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java
index 6748efc..eb87a0c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java
@@ -320,7 +320,7 @@ public class TestHFileBlock {
         .withIncludesTags(includesTag)
         .withCompression(algo).build();
         HFileBlock.FSReader hbr = new HFileBlock.FSReaderImpl(is, totalSize, meta);
-        HFileBlock b = hbr.readBlockData(0, -1, -1, pread);
+        HFileBlock b = hbr.readBlockData(0, -1, pread);
         is.close();
         assertEquals(0, HFile.getChecksumFailuresCount());
 
@@ -334,17 +334,15 @@ public class TestHFileBlock {
           is = fs.open(path);
           hbr = new HFileBlock.FSReaderImpl(is, totalSize, meta);
           b = hbr.readBlockData(0, 2173 + HConstants.HFILEBLOCK_HEADER_SIZE +
-                                b.totalChecksumBytes(), -1, pread);
+                                b.totalChecksumBytes(), pread);
           assertEquals(expected, b);
           int wrongCompressedSize = 2172;
           try {
             b = hbr.readBlockData(0, wrongCompressedSize
-                + HConstants.HFILEBLOCK_HEADER_SIZE, -1, pread);
+                + HConstants.HFILEBLOCK_HEADER_SIZE, pread);
             fail("Exception expected");
           } catch (IOException ex) {
-            String expectedPrefix = "On-disk size without header provided is "
-                + wrongCompressedSize + ", but block header contains "
-                + b.getOnDiskSizeWithoutHeader() + ".";
+            String expectedPrefix = "Passed in onDiskSizeWithHeader=";
             assertTrue("Invalid exception message: '" + ex.getMessage()
                 + "'.\nMessage is expected to start with: '" + expectedPrefix
                 + "'", ex.getMessage().startsWith(expectedPrefix));
@@ -424,7 +422,7 @@ public class TestHFileBlock {
           HFileBlock blockFromHFile, blockUnpacked;
           int pos = 0;
           for (int blockId = 0; blockId < numBlocks; ++blockId) {
-            blockFromHFile = hbr.readBlockData(pos, -1, -1, pread);
+            blockFromHFile = hbr.readBlockData(pos, -1, pread);
             assertEquals(0, HFile.getChecksumFailuresCount());
             blockFromHFile.sanityCheck();
             pos += blockFromHFile.getOnDiskSizeWithHeader();
@@ -560,7 +558,7 @@ public class TestHFileBlock {
             if (detailedLogging) {
               LOG.info("Reading block #" + i + " at offset " + curOffset);
             }
-            HFileBlock b = hbr.readBlockData(curOffset, -1, -1, pread);
+            HFileBlock b = hbr.readBlockData(curOffset, -1, pread);
             if (detailedLogging) {
               LOG.info("Block #" + i + ": " + b);
             }
@@ -574,8 +572,7 @@ public class TestHFileBlock {
 
             // Now re-load this block knowing the on-disk size. This tests a
             // different branch in the loader.
-            HFileBlock b2 = hbr.readBlockData(curOffset,
-                b.getOnDiskSizeWithHeader(), -1, pread);
+            HFileBlock b2 = hbr.readBlockData(curOffset, b.getOnDiskSizeWithHeader(), pread);
             b2.sanityCheck();
 
             assertEquals(b.getBlockType(), b2.getBlockType());
@@ -601,7 +598,7 @@ public class TestHFileBlock {
               b = b.unpack(meta, hbr);
               // b's buffer has header + data + checksum while
               // expectedContents have header + data only
-              ByteBuff bufRead = b.getBufferWithHeader();
+              ByteBuff bufRead = b.getBufferReadOnly();
               ByteBuffer bufExpected = expectedContents.get(i);
               boolean bytesAreCorrect = Bytes.compareTo(bufRead.array(),
                   bufRead.arrayOffset(),
@@ -684,7 +681,7 @@ public class TestHFileBlock {
         HFileBlock b;
         try {
           long onDiskSizeArg = withOnDiskSize ? expectedSize : -1;
-          b = hbr.readBlockData(offset, onDiskSizeArg, -1, pread);
+          b = hbr.readBlockData(offset, onDiskSizeArg, pread);
         } catch (IOException ex) {
           LOG.error("Error in client " + clientId + " trying to read block at "
               + offset + ", pread=" + pread + ", withOnDiskSize=" +
@@ -719,8 +716,7 @@ public class TestHFileBlock {
   protected void testConcurrentReadingInternals() throws IOException,
       InterruptedException, ExecutionException {
     for (Compression.Algorithm compressAlgo : COMPRESSION_ALGORITHMS) {
-      Path path =
-          new Path(TEST_UTIL.getDataTestDir(), "concurrent_reading");
+      Path path = new Path(TEST_UTIL.getDataTestDir(), "concurrent_reading");
       Random rand = defaultRandom();
       List<Long> offsets = new ArrayList<Long>();
       List<BlockType> types = new ArrayList<BlockType>();
@@ -843,8 +839,7 @@ public class TestHFileBlock {
                           .withBytesPerCheckSum(HFile.DEFAULT_BYTES_PER_CHECKSUM)
                           .withChecksumType(ChecksumType.NULL).build();
       HFileBlock block = new HFileBlock(BlockType.DATA, size, size, -1, buf,
-          HFileBlock.FILL_HEADER, -1,
-          0, meta);
+          HFileBlock.FILL_HEADER, -1, 0, -1, meta);
       long byteBufferExpectedSize = ClassSize.align(ClassSize.estimateBase(
           new MultiByteBuff(buf).getClass(), true)
           + HConstants.HFILEBLOCK_HEADER_SIZE + size);

http://git-wip-us.apache.org/repos/asf/hbase/blob/12f66e30/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockCompatibility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockCompatibility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockCompatibility.java
deleted file mode 100644
index 16607b9..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockCompatibility.java
+++ /dev/null
@@ -1,750 +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 static org.apache.hadoop.hbase.io.compress.Compression.Algorithm.GZ;
-import static org.apache.hadoop.hbase.io.compress.Compression.Algorithm.NONE;
-import static org.junit.Assert.*;
-
-import java.io.ByteArrayOutputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HConstants;
-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.FSDataInputStreamWrapper;
-import org.apache.hadoop.hbase.io.compress.Compression;
-import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
-import org.apache.hadoop.hbase.io.encoding.HFileBlockDefaultEncodingContext;
-import org.apache.hadoop.hbase.io.encoding.HFileBlockEncodingContext;
-import org.apache.hadoop.hbase.io.hfile.HFileBlock.BlockWritable;
-import org.apache.hadoop.hbase.nio.ByteBuff;
-import org.apache.hadoop.hbase.nio.SingleByteBuff;
-import org.apache.hadoop.hbase.testclassification.IOTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.ChecksumType;
-import org.apache.hadoop.io.WritableUtils;
-import org.apache.hadoop.io.compress.Compressor;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-
-import com.google.common.base.Preconditions;
-
-/**
- * This class has unit tests to prove that older versions of
- * HFiles (without checksums) are compatible with current readers.
- */
-@Category({IOTests.class, SmallTests.class})
-@RunWith(Parameterized.class)
-public class TestHFileBlockCompatibility {
-
-  private static final Log LOG = LogFactory.getLog(TestHFileBlockCompatibility.class);
-  private static final Compression.Algorithm[] COMPRESSION_ALGORITHMS = {
-      NONE, GZ };
-
-  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private HFileSystem fs;
-
-  private final boolean includesMemstoreTS;
-  private final boolean includesTag;
-
-  public TestHFileBlockCompatibility(boolean includesMemstoreTS, boolean includesTag) {
-    this.includesMemstoreTS = includesMemstoreTS;
-    this.includesTag = includesTag;
-  }
-
-  @Parameters
-  public static Collection<Object[]> parameters() {
-    return HBaseTestingUtility.MEMSTORETS_TAGS_PARAMETRIZED;
-  }
-
-  @Before
-  public void setUp() throws IOException {
-    fs = (HFileSystem)HFileSystem.get(TEST_UTIL.getConfiguration());
-  }
-
-  public byte[] createTestV1Block(Compression.Algorithm algo)
-      throws IOException {
-    Compressor compressor = algo.getCompressor();
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    OutputStream os = algo.createCompressionStream(baos, compressor, 0);
-    DataOutputStream dos = new DataOutputStream(os);
-    BlockType.META.write(dos); // Let's make this a meta block.
-    TestHFileBlock.writeTestBlockContents(dos);
-    dos.flush();
-    algo.returnCompressor(compressor);
-    return baos.toByteArray();
-  }
-
-  private Writer createTestV2Block(Compression.Algorithm algo)
-      throws IOException {
-    final BlockType blockType = BlockType.DATA;
-    Writer hbw = new Writer(algo, null,
-        includesMemstoreTS, includesTag);
-    DataOutputStream dos = hbw.startWriting(blockType);
-    TestHFileBlock.writeTestBlockContents(dos);
-    // make sure the block is ready by calling hbw.getHeaderAndData()
-    hbw.getHeaderAndData();
-    assertEquals(1000 * 4, hbw.getUncompressedSizeWithoutHeader());
-    hbw.releaseCompressor();
-    return hbw;
-  }
-
- private String createTestBlockStr(Compression.Algorithm algo,
-      int correctLength) throws IOException {
-    Writer hbw = createTestV2Block(algo);
-    byte[] testV2Block = hbw.getHeaderAndData();
-    int osOffset = HConstants.HFILEBLOCK_HEADER_SIZE_NO_CHECKSUM + 9;
-    if (testV2Block.length == correctLength) {
-      // Force-set the "OS" field of the gzip header to 3 (Unix) to avoid
-      // variations across operating systems.
-      // See http://www.gzip.org/zlib/rfc-gzip.html for gzip format.
-      testV2Block[osOffset] = 3;
-    }
-    return Bytes.toStringBinary(testV2Block);
-  }
-
-  @Test
-  public void testNoCompression() throws IOException {
-    assertEquals(4000, createTestV2Block(NONE).getBlockForCaching().
-        getUncompressedSizeWithoutHeader());
-  }
-
-  @Test
-  public void testGzipCompression() throws IOException {
-    final String correctTestBlockStr =
-        "DATABLK*\\x00\\x00\\x00:\\x00\\x00\\x0F\\xA0\\xFF\\xFF\\xFF\\xFF"
-            + "\\xFF\\xFF\\xFF\\xFF"
-            // gzip-compressed block: http://www.gzip.org/zlib/rfc-gzip.html
-            + "\\x1F\\x8B"  // gzip magic signature
-            + "\\x08"  // Compression method: 8 = "deflate"
-            + "\\x00"  // Flags
-            + "\\x00\\x00\\x00\\x00"  // mtime
-            + "\\x00"  // XFL (extra flags)
-            // OS (0 = FAT filesystems, 3 = Unix). However, this field
-            // sometimes gets set to 0 on Linux and Mac, so we reset it to 3.
-            + "\\x03"
-            + "\\xED\\xC3\\xC1\\x11\\x00 \\x08\\xC00DD\\xDD\\x7Fa"
-            + "\\xD6\\xE8\\xA3\\xB9K\\x84`\\x96Q\\xD3\\xA8\\xDB\\xA8e\\xD4c"
-            + "\\xD46\\xEA5\\xEA3\\xEA7\\xE7\\x00LI\\x5Cs\\xA0\\x0F\\x00\\x00";
-    final int correctGzipBlockLength = 82;
-
-    String returnedStr = createTestBlockStr(GZ, correctGzipBlockLength);
-    assertEquals(correctTestBlockStr, returnedStr);
-  }
-
-  @Test
-  public void testReaderV2() throws IOException {
-    if(includesTag) {
-      TEST_UTIL.getConfiguration().setInt("hfile.format.version", 3);
-    }
-    for (Compression.Algorithm algo : COMPRESSION_ALGORITHMS) {
-      for (boolean pread : new boolean[] { false, true }) {
-          LOG.info("testReaderV2: Compression algorithm: " + algo +
-                   ", pread=" + pread);
-        Path path = new Path(TEST_UTIL.getDataTestDir(), "blocks_v2_"
-            + algo);
-        FSDataOutputStream os = fs.create(path);
-        Writer hbw = new Writer(algo, null,
-            includesMemstoreTS, includesTag);
-        long totalSize = 0;
-        for (int blockId = 0; blockId < 2; ++blockId) {
-          DataOutputStream dos = hbw.startWriting(BlockType.DATA);
-          for (int i = 0; i < 1234; ++i)
-            dos.writeInt(i);
-          hbw.writeHeaderAndData(os);
-          totalSize += hbw.getOnDiskSizeWithHeader();
-        }
-        os.close();
-
-        FSDataInputStream is = fs.open(path);
-        HFileContext meta = new HFileContextBuilder()
-                           .withHBaseCheckSum(false)
-                           .withIncludesMvcc(includesMemstoreTS)
-                           .withIncludesTags(includesTag)
-                           .withCompression(algo)
-                           .build();
-        HFileBlock.FSReader hbr =
-          new HFileBlock.FSReaderImpl(new FSDataInputStreamWrapper(is), totalSize, fs, path, meta);
-        HFileBlock b = hbr.readBlockData(0, -1, -1, pread);
-        is.close();
-
-        b.sanityCheck();
-        assertEquals(4936, b.getUncompressedSizeWithoutHeader());
-        assertEquals(algo == GZ ? 2173 : 4936,
-                     b.getOnDiskSizeWithoutHeader() - b.totalChecksumBytes());
-        HFileBlock expected = b;
-
-        if (algo == GZ) {
-          is = fs.open(path);
-          hbr = new HFileBlock.FSReaderImpl(new FSDataInputStreamWrapper(is), totalSize, fs, path,
-              meta);
-          b = hbr.readBlockData(0, 2173 + HConstants.HFILEBLOCK_HEADER_SIZE_NO_CHECKSUM +
-                                b.totalChecksumBytes(), -1, pread);
-          assertEquals(expected, b);
-          int wrongCompressedSize = 2172;
-          try {
-            b = hbr.readBlockData(0, wrongCompressedSize
-                + HConstants.HFILEBLOCK_HEADER_SIZE_NO_CHECKSUM, -1, pread);
-            fail("Exception expected");
-          } catch (IOException ex) {
-            String expectedPrefix = "On-disk size without header provided is "
-                + wrongCompressedSize + ", but block header contains "
-                + b.getOnDiskSizeWithoutHeader() + ".";
-            assertTrue("Invalid exception message: '" + ex.getMessage()
-                + "'.\nMessage is expected to start with: '" + expectedPrefix
-                + "'", ex.getMessage().startsWith(expectedPrefix));
-          }
-          is.close();
-        }
-      }
-    }
-  }
-
-  /**
-   * Test encoding/decoding data blocks.
-   * @throws IOException a bug or a problem with temporary files.
-   */
-  @Test
-  public void testDataBlockEncoding() throws IOException {
-    if(includesTag) {
-      TEST_UTIL.getConfiguration().setInt("hfile.format.version", 3);
-    }
-    final int numBlocks = 5;
-    for (Compression.Algorithm algo : COMPRESSION_ALGORITHMS) {
-      for (boolean pread : new boolean[] { false, true }) {
-        for (DataBlockEncoding encoding : DataBlockEncoding.values()) {
-          LOG.info("testDataBlockEncoding algo " + algo +
-                   " pread = " + pread +
-                   " encoding " + encoding);
-          Path path = new Path(TEST_UTIL.getDataTestDir(), "blocks_v2_"
-              + algo + "_" + encoding.toString());
-          FSDataOutputStream os = fs.create(path);
-          HFileDataBlockEncoder dataBlockEncoder = (encoding != DataBlockEncoding.NONE) ?
-              new HFileDataBlockEncoderImpl(encoding) : NoOpDataBlockEncoder.INSTANCE;
-          TestHFileBlockCompatibility.Writer hbw =
-              new TestHFileBlockCompatibility.Writer(algo,
-                  dataBlockEncoder, includesMemstoreTS, includesTag);
-          long totalSize = 0;
-          final List<Integer> encodedSizes = new ArrayList<Integer>();
-          final List<ByteBuffer> encodedBlocks = new ArrayList<ByteBuffer>();
-          for (int blockId = 0; blockId < numBlocks; ++blockId) {
-            hbw.startWriting(BlockType.DATA);
-            TestHFileBlock.writeTestKeyValues(hbw, blockId, pread, includesTag);
-            hbw.writeHeaderAndData(os);
-            int headerLen = HConstants.HFILEBLOCK_HEADER_SIZE_NO_CHECKSUM;
-            byte[] encodedResultWithHeader = hbw.getUncompressedDataWithHeader();
-            final int encodedSize = encodedResultWithHeader.length - headerLen;
-            if (encoding != DataBlockEncoding.NONE) {
-              // We need to account for the two-byte encoding algorithm ID that
-              // comes after the 24-byte block header but before encoded KVs.
-              headerLen += DataBlockEncoding.ID_SIZE;
-            }
-            byte[] encodedDataSection =
-                new byte[encodedResultWithHeader.length - headerLen];
-            System.arraycopy(encodedResultWithHeader, headerLen,
-                encodedDataSection, 0, encodedDataSection.length);
-            final ByteBuffer encodedBuf =
-                ByteBuffer.wrap(encodedDataSection);
-            encodedSizes.add(encodedSize);
-            encodedBlocks.add(encodedBuf);
-            totalSize += hbw.getOnDiskSizeWithHeader();
-          }
-          os.close();
-
-          FSDataInputStream is = fs.open(path);
-          HFileContext meta = new HFileContextBuilder()
-                              .withHBaseCheckSum(false)
-                              .withIncludesMvcc(includesMemstoreTS)
-                              .withIncludesTags(includesTag)
-                              .withCompression(algo)
-                              .build();
-          HFileBlock.FSReaderImpl hbr = new HFileBlock.FSReaderImpl(new FSDataInputStreamWrapper(is),
-              totalSize, fs, path, meta);
-          hbr.setDataBlockEncoder(dataBlockEncoder);
-          hbr.setIncludesMemstoreTS(includesMemstoreTS);
-
-          HFileBlock b;
-          int pos = 0;
-          for (int blockId = 0; blockId < numBlocks; ++blockId) {
-            b = hbr.readBlockData(pos, -1, -1, pread);
-            b.sanityCheck();
-            if (meta.isCompressedOrEncrypted()) {
-              assertFalse(b.isUnpacked());
-              b = b.unpack(meta, hbr);
-            }
-            pos += b.getOnDiskSizeWithHeader();
-
-            assertEquals((int) encodedSizes.get(blockId),
-                b.getUncompressedSizeWithoutHeader());
-            ByteBuff actualBuffer = b.getBufferWithoutHeader();
-            if (encoding != DataBlockEncoding.NONE) {
-              // We expect a two-byte big-endian encoding id.
-              assertEquals(0, actualBuffer.get(0));
-              assertEquals(encoding.getId(), actualBuffer.get(1));
-              actualBuffer.position(2);
-              actualBuffer = actualBuffer.slice();
-            }
-
-            ByteBuffer expectedBuffer = encodedBlocks.get(blockId);
-            expectedBuffer.rewind();
-
-            // test if content matches, produce nice message
-            TestHFileBlock.assertBuffersEqual(new SingleByteBuff(expectedBuffer), actualBuffer,
-              algo, encoding, pread);
-          }
-          is.close();
-        }
-      }
-    }
-  }
-  /**
-   * This is the version of the HFileBlock.Writer that is used to
-   * create V2 blocks with minor version 0. These blocks do not
-   * have hbase-level checksums. The code is here to test
-   * backward compatibility. The reason we do not inherit from
-   * HFileBlock.Writer is because we never ever want to change the code
-   * in this class but the code in HFileBlock.Writer will continually
-   * evolve.
-   */
-  public static final class Writer extends HFileBlock.Writer {
-
-    // These constants are as they were in minorVersion 0.
-    private static final int HEADER_SIZE = HConstants.HFILEBLOCK_HEADER_SIZE_NO_CHECKSUM;
-    private static final boolean DONT_FILL_HEADER = HFileBlock.DONT_FILL_HEADER;
-    private static final byte[] DUMMY_HEADER = HFileBlock.DUMMY_HEADER_NO_CHECKSUM;
-
-    private enum State {
-      INIT,
-      WRITING,
-      BLOCK_READY
-    };
-
-    /** Writer state. Used to ensure the correct usage protocol. */
-    private State state = State.INIT;
-
-    /** Compression algorithm for all blocks this instance writes. */
-    private final Compression.Algorithm compressAlgo;
-
-    /** Data block encoder used for data blocks */
-    private final HFileDataBlockEncoder dataBlockEncoder;
-
-    private HFileBlockEncodingContext dataBlockEncodingCtx;
-    /** block encoding context for non-data blocks */
-    private HFileBlockDefaultEncodingContext defaultBlockEncodingCtx;
-
-    /**
-     * The stream we use to accumulate data in uncompressed format for each
-     * block. We reset this stream at the end of each block and reuse it. The
-     * header is written as the first {@link #HEADER_SIZE} bytes into this
-     * stream.
-     */
-    private ByteArrayOutputStream baosInMemory;
-
-    /** Compressor, which is also reused between consecutive blocks. */
-    private Compressor compressor;
-
-    /**
-     * Current block type. Set in {@link #startWriting(BlockType)}. Could be
-     * changed in {@link #encodeDataBlockForDisk()} from {@link BlockType#DATA}
-     * to {@link BlockType#ENCODED_DATA}.
-     */
-    private BlockType blockType;
-
-    /**
-     * A stream that we write uncompressed bytes to, which compresses them and
-     * writes them to {@link #baosInMemory}.
-     */
-    private DataOutputStream userDataStream;
-
-    /**
-     * Bytes to be written to the file system, including the header. Compressed
-     * if compression is turned on.
-     */
-    private byte[] onDiskBytesWithHeader;
-
-    /**
-     * Valid in the READY state. Contains the header and the uncompressed (but
-     * potentially encoded, if this is a data block) bytes, so the length is
-     * {@link #uncompressedSizeWithoutHeader} + {@link org.apache.hadoop.hbase.HConstants#HFILEBLOCK_HEADER_SIZE}.
-     */
-    private byte[] uncompressedBytesWithHeader;
-
-    /**
-     * Current block's start offset in the {@link HFile}. Set in
-     * {@link #writeHeaderAndData(FSDataOutputStream)}.
-     */
-    private long startOffset;
-
-    /**
-     * Offset of previous block by block type. Updated when the next block is
-     * started.
-     */
-    private long[] prevOffsetByType;
-
-    /** The offset of the previous block of the same type */
-    private long prevOffset;
-
-    private int unencodedDataSizeWritten;
-
-    public Writer(Compression.Algorithm compressionAlgorithm,
-        HFileDataBlockEncoder dataBlockEncoder, boolean includesMemstoreTS, boolean includesTag) {
-      this(dataBlockEncoder, new HFileContextBuilder().withHBaseCheckSum(false)
-          .withIncludesMvcc(includesMemstoreTS).withIncludesTags(includesTag)
-          .withCompression(compressionAlgorithm).build());
-    }
-
-    public Writer(HFileDataBlockEncoder dataBlockEncoder, HFileContext meta) {
-      super(dataBlockEncoder, meta);
-      compressAlgo = meta.getCompression() == null ? NONE : meta.getCompression();
-      this.dataBlockEncoder = dataBlockEncoder != null ? dataBlockEncoder
-          : NoOpDataBlockEncoder.INSTANCE;
-      defaultBlockEncodingCtx = new HFileBlockDefaultEncodingContext(null, DUMMY_HEADER, meta);
-      dataBlockEncodingCtx = this.dataBlockEncoder.newDataBlockEncodingContext(DUMMY_HEADER, meta);
-      baosInMemory = new ByteArrayOutputStream();
-
-      prevOffsetByType = new long[BlockType.values().length];
-      for (int i = 0; i < prevOffsetByType.length; ++i)
-        prevOffsetByType[i] = -1;
-    }
-
-    /**
-     * Starts writing into the block. The previous block's data is discarded.
-     *
-     * @return the stream the user can write their data into
-     * @throws IOException
-     */
-    public DataOutputStream startWriting(BlockType newBlockType)
-        throws IOException {
-      if (state == State.BLOCK_READY && startOffset != -1) {
-        // We had a previous block that was written to a stream at a specific
-        // offset. Save that offset as the last offset of a block of that type.
-        prevOffsetByType[blockType.getId()] = startOffset;
-      }
-
-      startOffset = -1;
-      blockType = newBlockType;
-
-      baosInMemory.reset();
-      baosInMemory.write(DUMMY_HEADER);
-
-      state = State.WRITING;
-
-      // We will compress it later in finishBlock()
-      userDataStream = new DataOutputStream(baosInMemory);
-      if (newBlockType == BlockType.DATA) {
-        this.dataBlockEncoder.startBlockEncoding(dataBlockEncodingCtx, userDataStream);
-      }
-      this.unencodedDataSizeWritten = 0;
-      return userDataStream;
-    }
-
-    @Override
-    public void write(Cell c) throws IOException {
-      KeyValue kv = KeyValueUtil.ensureKeyValue(c);
-      expectState(State.WRITING);
-      this.dataBlockEncoder.encode(kv, dataBlockEncodingCtx, this.userDataStream);
-      this.unencodedDataSizeWritten += kv.getLength();
-      if (dataBlockEncodingCtx.getHFileContext().isIncludesMvcc()) {
-        this.unencodedDataSizeWritten += WritableUtils.getVIntSize(kv.getSequenceId());
-      }
-    }
-
-    /**
-     * Returns the stream for the user to write to. The block writer takes care
-     * of handling compression and buffering for caching on write. Can only be
-     * called in the "writing" state.
-     *
-     * @return the data output stream for the user to write to
-     */
-    DataOutputStream getUserDataStream() {
-      expectState(State.WRITING);
-      return userDataStream;
-    }
-
-    /**
-     * Transitions the block writer from the "writing" state to the "block
-     * ready" state.  Does nothing if a block is already finished.
-     */
-    void ensureBlockReady() throws IOException {
-      Preconditions.checkState(state != State.INIT,
-          "Unexpected state: " + state);
-
-      if (state == State.BLOCK_READY)
-        return;
-
-      // This will set state to BLOCK_READY.
-      finishBlock();
-    }
-
-    /**
-     * An internal method that flushes the compressing stream (if using
-     * compression), serializes the header, and takes care of the separate
-     * uncompressed stream for caching on write, if applicable. Sets block
-     * write state to "block ready".
-     */
-    void finishBlock() throws IOException {
-      if (blockType == BlockType.DATA) {
-        this.dataBlockEncoder.endBlockEncoding(dataBlockEncodingCtx, userDataStream,
-            baosInMemory.toByteArray(), blockType);
-        blockType = dataBlockEncodingCtx.getBlockType();
-      }
-      userDataStream.flush();
-      // This does an array copy, so it is safe to cache this byte array.
-      uncompressedBytesWithHeader = baosInMemory.toByteArray();
-      prevOffset = prevOffsetByType[blockType.getId()];
-
-      // We need to set state before we can package the block up for
-      // cache-on-write. In a way, the block is ready, but not yet encoded or
-      // compressed.
-      state = State.BLOCK_READY;
-      if (blockType == BlockType.DATA || blockType == BlockType.ENCODED_DATA) {
-        onDiskBytesWithHeader = dataBlockEncodingCtx
-            .compressAndEncrypt(uncompressedBytesWithHeader);
-      } else {
-        onDiskBytesWithHeader = defaultBlockEncodingCtx
-            .compressAndEncrypt(uncompressedBytesWithHeader);
-      }
-
-      // put the header for on disk bytes
-      putHeader(onDiskBytesWithHeader, 0,
-          onDiskBytesWithHeader.length,
-          uncompressedBytesWithHeader.length);
-      //set the header for the uncompressed bytes (for cache-on-write)
-      putHeader(uncompressedBytesWithHeader, 0,
-          onDiskBytesWithHeader.length,
-        uncompressedBytesWithHeader.length);
-    }
-
-    /**
-     * Put the header into the given byte array at the given offset.
-     * @param onDiskSize size of the block on disk
-     * @param uncompressedSize size of the block after decompression (but
-     *          before optional data block decoding)
-     */
-    private void putHeader(byte[] dest, int offset, int onDiskSize,
-        int uncompressedSize) {
-      offset = blockType.put(dest, offset);
-      offset = Bytes.putInt(dest, offset, onDiskSize - HEADER_SIZE);
-      offset = Bytes.putInt(dest, offset, uncompressedSize - HEADER_SIZE);
-      Bytes.putLong(dest, offset, prevOffset);
-    }
-
-    /**
-     * Similar to {@link #writeHeaderAndData(FSDataOutputStream)}, but records
-     * the offset of this block so that it can be referenced in the next block
-     * of the same type.
-     *
-     * @param out
-     * @throws IOException
-     */
-    public void writeHeaderAndData(FSDataOutputStream out) throws IOException {
-      long offset = out.getPos();
-      if (startOffset != -1 && offset != startOffset) {
-        throw new IOException("A " + blockType + " block written to a "
-            + "stream twice, first at offset " + startOffset + ", then at "
-            + offset);
-      }
-      startOffset = offset;
-
-      writeHeaderAndData((DataOutputStream) out);
-    }
-
-    /**
-     * Writes the header and the compressed data of this block (or uncompressed
-     * data when not using compression) into the given stream. Can be called in
-     * the "writing" state or in the "block ready" state. If called in the
-     * "writing" state, transitions the writer to the "block ready" state.
-     *
-     * @param out the output stream to write the
-     * @throws IOException
-     */
-    private void writeHeaderAndData(DataOutputStream out) throws IOException {
-      ensureBlockReady();
-      out.write(onDiskBytesWithHeader);
-    }
-
-    /**
-     * Returns the header or the compressed data (or uncompressed data when not
-     * using compression) as a byte array. Can be called in the "writing" state
-     * or in the "block ready" state. If called in the "writing" state,
-     * transitions the writer to the "block ready" state.
-     *
-     * @return header and data as they would be stored on disk in a byte array
-     * @throws IOException
-     */
-    public byte[] getHeaderAndData() throws IOException {
-      ensureBlockReady();
-      return onDiskBytesWithHeader;
-    }
-
-    /**
-     * Releases the compressor this writer uses to compress blocks into the
-     * compressor pool. Needs to be called before the writer is discarded.
-     */
-    public void releaseCompressor() {
-      if (compressor != null) {
-        compressAlgo.returnCompressor(compressor);
-        compressor = null;
-      }
-    }
-
-    /**
-     * Returns the on-disk size of the data portion of the block. This is the
-     * compressed size if compression is enabled. Can only be called in the
-     * "block ready" state. Header is not compressed, and its size is not
-     * included in the return value.
-     *
-     * @return the on-disk size of the block, not including the header.
-     */
-    public int getOnDiskSizeWithoutHeader() {
-      expectState(State.BLOCK_READY);
-      return onDiskBytesWithHeader.length - HEADER_SIZE;
-    }
-
-    /**
-     * Returns the on-disk size of the block. Can only be called in the
-     * "block ready" state.
-     *
-     * @return the on-disk size of the block ready to be written, including the
-     *         header size
-     */
-    public int getOnDiskSizeWithHeader() {
-      expectState(State.BLOCK_READY);
-      return onDiskBytesWithHeader.length;
-    }
-
-    /**
-     * The uncompressed size of the block data. Does not include header size.
-     */
-    public int getUncompressedSizeWithoutHeader() {
-      expectState(State.BLOCK_READY);
-      return uncompressedBytesWithHeader.length - HEADER_SIZE;
-    }
-
-    /**
-     * The uncompressed size of the block data, including header size.
-     */
-    public int getUncompressedSizeWithHeader() {
-      expectState(State.BLOCK_READY);
-      return uncompressedBytesWithHeader.length;
-    }
-
-    /** @return true if a block is being written  */
-    public boolean isWriting() {
-      return state == State.WRITING;
-    }
-
-    /**
-     * Returns the number of bytes written into the current block so far, or
-     * zero if not writing the block at the moment. Note that this will return
-     * zero in the "block ready" state as well.
-     *
-     * @return the number of bytes written
-     */
-    public int blockSizeWritten() {
-      if (state != State.WRITING)
-        return 0;
-      return this.unencodedDataSizeWritten;
-    }
-
-    /**
-     * Returns the header followed by the uncompressed data, even if using
-     * compression. This is needed for storing uncompressed blocks in the block
-     * cache. Can be called in the "writing" state or the "block ready" state.
-     *
-     * @return uncompressed block bytes for caching on write
-     */
-    private byte[] getUncompressedDataWithHeader() {
-      expectState(State.BLOCK_READY);
-
-      return uncompressedBytesWithHeader;
-    }
-
-    private void expectState(State expectedState) {
-      if (state != expectedState) {
-        throw new IllegalStateException("Expected state: " + expectedState +
-            ", actual state: " + state);
-      }
-    }
-
-    /**
-     * Similar to {@link #getUncompressedBufferWithHeader()} but returns a byte
-     * buffer.
-     *
-     * @return uncompressed block for caching on write in the form of a buffer
-     */
-    public ByteBuffer getUncompressedBufferWithHeader() {
-      byte[] b = getUncompressedDataWithHeader();
-      return ByteBuffer.wrap(b, 0, b.length);
-    }
-
-    /**
-     * Takes the given {@link BlockWritable} instance, creates a new block of
-     * its appropriate type, writes the writable into this block, and flushes
-     * the block into the output stream. The writer is instructed not to buffer
-     * uncompressed bytes for cache-on-write.
-     *
-     * @param bw the block-writable object to write as a block
-     * @param out the file system output stream
-     * @throws IOException
-     */
-    public void writeBlock(BlockWritable bw, FSDataOutputStream out)
-        throws IOException {
-      bw.writeToBlock(startWriting(bw.getBlockType()));
-      writeHeaderAndData(out);
-    }
-
-    /**
-     * Creates a new HFileBlock.
-     */
-    public HFileBlock getBlockForCaching() {
-      HFileContext meta = new HFileContextBuilder()
-             .withHBaseCheckSum(false)
-             .withChecksumType(ChecksumType.NULL)
-             .withBytesPerCheckSum(0)
-             .build();
-      return new HFileBlock(blockType, getOnDiskSizeWithoutHeader(),
-          getUncompressedSizeWithoutHeader(), prevOffset,
-          getUncompressedBufferWithHeader(), DONT_FILL_HEADER, startOffset,
-          getOnDiskSizeWithoutHeader(), meta);
-    }
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/hbase/blob/12f66e30/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 687d3cd..470d483 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
@@ -185,8 +185,7 @@ public class TestHFileBlockIndex {
       }
 
       missCount += 1;
-      prevBlock = realReader.readBlockData(offset, onDiskSize,
-          -1, pread);
+      prevBlock = realReader.readBlockData(offset, onDiskSize, pread);
       prevOffset = offset;
       prevOnDiskSize = onDiskSize;
       prevPread = pread;

http://git-wip-us.apache.org/repos/asf/hbase/blob/12f66e30/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileDataBlockEncoder.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileDataBlockEncoder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileDataBlockEncoder.java
index 6f434bb..387514e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileDataBlockEncoder.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileDataBlockEncoder.java
@@ -92,8 +92,7 @@ public class TestHFileDataBlockEncoder {
 
     if (blockEncoder.getDataBlockEncoding() ==
         DataBlockEncoding.NONE) {
-      assertEquals(block.getBufferWithHeader(),
-          returnedBlock.getBufferWithHeader());
+      assertEquals(block.getBufferReadOnly(), returnedBlock.getBufferReadOnly());
     } else {
       if (BlockType.ENCODED_DATA != returnedBlock.getBlockType()) {
         System.out.println(blockEncoder);
@@ -127,7 +126,7 @@ public class TestHFileDataBlockEncoder {
                         .build();
     HFileBlock block = new HFileBlock(BlockType.DATA, size, size, -1, buf,
         HFileBlock.FILL_HEADER, 0,
-        0, hfileContext);
+        0, -1, hfileContext);
     HFileBlock cacheBlock = createBlockOnDisk(kvs, block, useTags);
     assertEquals(headerSize, cacheBlock.getDummyHeaderForVersion().length);
   }
@@ -198,7 +197,7 @@ public class TestHFileDataBlockEncoder {
                         .build();
     HFileBlock b = new HFileBlock(BlockType.DATA, size, size, -1, buf,
         HFileBlock.FILL_HEADER, 0, 
-         0, meta);
+         0, -1, meta);
     return b;
   }
 
@@ -220,7 +219,8 @@ public class TestHFileDataBlockEncoder {
     byte[] encodedBytes = baos.toByteArray();
     size = encodedBytes.length - block.getDummyHeaderForVersion().length;
     return new HFileBlock(context.getBlockType(), size, size, -1, ByteBuffer.wrap(encodedBytes),
-        HFileBlock.FILL_HEADER, 0, block.getOnDiskDataSizeWithHeader(), block.getHFileContext());
+        HFileBlock.FILL_HEADER, 0, block.getOnDiskDataSizeWithHeader(), -1,
+        block.getHFileContext());
   }
 
   private void writeBlock(List<Cell> kvs, HFileContext fileContext, boolean useTags)

http://git-wip-us.apache.org/repos/asf/hbase/blob/12f66e30/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileEncryption.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileEncryption.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileEncryption.java
index ba3a344..3264558 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileEncryption.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileEncryption.java
@@ -99,7 +99,7 @@ public class TestHFileEncryption {
 
   private long readAndVerifyBlock(long pos, HFileContext ctx, HFileBlock.FSReaderImpl hbr, int size)
       throws IOException {
-    HFileBlock b = hbr.readBlockData(pos, -1, -1, false);
+    HFileBlock b = hbr.readBlockData(pos, -1, false);
     assertEquals(0, HFile.getChecksumFailuresCount());
     b.sanityCheck();
     assertFalse(b.isUnpacked());

http://git-wip-us.apache.org/repos/asf/hbase/blob/12f66e30/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java
index c7eb11b..983ec2f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java
@@ -218,7 +218,7 @@ public class TestHFileWriterV3 {
     fsdis.seek(0);
     long curBlockPos = 0;
     while (curBlockPos <= trailer.getLastDataBlockOffset()) {
-      HFileBlock block = blockReader.readBlockData(curBlockPos, -1, -1, false)
+      HFileBlock block = blockReader.readBlockData(curBlockPos, -1, false)
         .unpack(context, blockReader);
       assertEquals(BlockType.DATA, block.getBlockType());
       ByteBuff buf = block.getBufferWithoutHeader();
@@ -279,13 +279,14 @@ public class TestHFileWriterV3 {
     while (fsdis.getPos() < trailer.getLoadOnOpenDataOffset()) {
       LOG.info("Current offset: " + fsdis.getPos() + ", scanning until " +
           trailer.getLoadOnOpenDataOffset());
-      HFileBlock block = blockReader.readBlockData(curBlockPos, -1, -1, false)
+      HFileBlock block = blockReader.readBlockData(curBlockPos, -1, false)
         .unpack(context, blockReader);
       assertEquals(BlockType.META, block.getBlockType());
       Text t = new Text();
       ByteBuff buf = block.getBufferWithoutHeader();
       if (Writables.getWritable(buf.array(), buf.arrayOffset(), buf.limit(), t) == null) {
-        throw new IOException("Failed to deserialize block " + this + " into a " + t.getClass().getSimpleName());
+        throw new IOException("Failed to deserialize block " + this +
+            " into a " + t.getClass().getSimpleName());
       }
       Text expectedText =
           (metaCounter == 0 ? new Text("Paris") : metaCounter == 1 ? new Text(

http://git-wip-us.apache.org/repos/asf/hbase/blob/12f66e30/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java
index 69a77bf..d20ba2b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java
@@ -78,14 +78,8 @@ public class TestPrefetch {
     // Check that all of the data blocks were preloaded
     BlockCache blockCache = cacheConf.getBlockCache();
     long offset = 0;
-    HFileBlock prevBlock = null;
     while (offset < reader.getTrailer().getLoadOnOpenDataOffset()) {
-      long onDiskSize = -1;
-      if (prevBlock != null) {
-         onDiskSize = prevBlock.getNextBlockOnDiskSizeWithHeader();
-      }
-      HFileBlock block = reader.readBlock(offset, onDiskSize, false, true, false, true, null,
-        null);
+      HFileBlock block = reader.readBlock(offset, -1, false, true, false, true, null, null);
       BlockCacheKey blockCacheKey = new BlockCacheKey(reader.getName(), offset);
       boolean isCached = blockCache.getBlock(blockCacheKey, true, false, true) != null;
       if (block.getBlockType() == BlockType.DATA ||
@@ -93,7 +87,6 @@ public class TestPrefetch {
           block.getBlockType() == BlockType.INTERMEDIATE_INDEX) {
         assertTrue(isCached);
       }
-      prevBlock = block;
       offset += block.getOnDiskSizeWithHeader();
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/12f66e30/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
index 0916fe6..2357bef 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
@@ -227,15 +227,10 @@ public class TestCacheOnWriteInSchema {
       assertTrue(testDescription, scanner.seekTo());
       // Cribbed from io.hfile.TestCacheOnWrite
       long offset = 0;
-      HFileBlock prevBlock = null;
       while (offset < reader.getTrailer().getLoadOnOpenDataOffset()) {
-        long onDiskSize = -1;
-        if (prevBlock != null) {
-          onDiskSize = prevBlock.getNextBlockOnDiskSizeWithHeader();
-        }
         // Flags: don't cache the block, use pread, this is not a compaction.
         // Also, pass null for expected block type to avoid checking it.
-        HFileBlock block = reader.readBlock(offset, onDiskSize, false, true,
+        HFileBlock block = reader.readBlock(offset, -1, false, true,
           false, true, null, DataBlockEncoding.NONE);
         BlockCacheKey blockCacheKey = new BlockCacheKey(reader.getName(),
           offset);
@@ -249,7 +244,6 @@ public class TestCacheOnWriteInSchema {
             "block: " + block + "\n" +
             "blockCacheKey: " + blockCacheKey);
         }
-        prevBlock = block;
         offset += block.getOnDiskSizeWithHeader();
       }
     } finally {


[34/50] [abbrv] hbase git commit: HBASE-15389 Write out multiple files when compaction

Posted by sy...@apache.org.
HBASE-15389 Write out multiple files when compaction


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

Branch: refs/heads/hbase-12439
Commit: 11d11d3fcc591227cccf3531b911e46c68774501
Parents: e9c4f12
Author: zhangduo <zh...@apache.org>
Authored: Fri Mar 25 15:07:54 2016 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Fri Mar 25 15:07:54 2016 +0800

----------------------------------------------------------------------
 .../regionserver/AbstractMultiFileWriter.java   | 120 +++++++
 .../regionserver/DateTieredMultiFileWriter.java |  83 +++++
 .../hadoop/hbase/regionserver/StoreFile.java    |   9 +-
 .../regionserver/StripeMultiFileWriter.java     | 239 ++++++--------
 .../hbase/regionserver/StripeStoreFlusher.java  |  30 +-
 .../AbstractMultiOutputCompactor.java           | 161 +++++++++
 .../regionserver/compactions/Compactor.java     |  10 +-
 .../compactions/DateTieredCompactor.java        |  86 +++++
 .../compactions/DefaultCompactor.java           |   4 +-
 .../compactions/StripeCompactionPolicy.java     |  13 +-
 .../compactions/StripeCompactor.java            | 169 +++-------
 .../hbase/regionserver/TestStripeCompactor.java | 325 -------------------
 .../regionserver/compactions/TestCompactor.java | 212 ++++++++++++
 .../compactions/TestDateTieredCompactor.java    | 169 ++++++++++
 .../compactions/TestStripeCompactionPolicy.java |  24 +-
 .../compactions/TestStripeCompactor.java        | 223 +++++++++++++
 16 files changed, 1272 insertions(+), 605 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/11d11d3f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMultiFileWriter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMultiFileWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMultiFileWriter.java
new file mode 100644
index 0000000..4987c59
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMultiFileWriter.java
@@ -0,0 +1,120 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.regionserver.StoreFile.Writer;
+import org.apache.hadoop.hbase.regionserver.compactions.Compactor.CellSink;
+
+/**
+ * Base class for cell sink that separates the provided cells into multiple files.
+ */
+@InterfaceAudience.Private
+public abstract class AbstractMultiFileWriter implements CellSink {
+
+  private static final Log LOG = LogFactory.getLog(AbstractMultiFileWriter.class);
+
+  /** Factory that is used to produce single StoreFile.Writer-s */
+  protected WriterFactory writerFactory;
+
+  /** Source scanner that is tracking KV count; may be null if source is not StoreScanner */
+  protected StoreScanner sourceScanner;
+
+  public interface WriterFactory {
+    public StoreFile.Writer createWriter() throws IOException;
+  }
+
+  /**
+   * Initializes multi-writer before usage.
+   * @param sourceScanner Optional store scanner to obtain the information about read progress.
+   * @param factory Factory used to produce individual file writers.
+   */
+  public void init(StoreScanner sourceScanner, WriterFactory factory) {
+    this.writerFactory = factory;
+    this.sourceScanner = sourceScanner;
+  }
+
+  /**
+   * Commit all writers.
+   * <p>
+   * Notice that here we use the same <code>maxSeqId</code> for all output files since we haven't
+   * find an easy to find enough sequence ids for different output files in some corner cases. See
+   * comments in HBASE-15400 for more details.
+   */
+  public List<Path> commitWriters(long maxSeqId, boolean majorCompaction) throws IOException {
+    preCommitWriters();
+    Collection<StoreFile.Writer> writers = this.writers();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Commit " + writers.size() + " writers, maxSeqId=" + maxSeqId
+          + ", majorCompaction=" + majorCompaction);
+    }
+    List<Path> paths = new ArrayList<Path>();
+    for (Writer writer : writers) {
+      if (writer == null) {
+        continue;
+      }
+      writer.appendMetadata(maxSeqId, majorCompaction);
+      preCloseWriter(writer);
+      paths.add(writer.getPath());
+      writer.close();
+    }
+    return paths;
+  }
+
+  /**
+   * Close all writers without throwing any exceptions. This is used when compaction failed usually.
+   */
+  public List<Path> abortWriters() {
+    List<Path> paths = new ArrayList<Path>();
+    for (StoreFile.Writer writer : writers()) {
+      try {
+        if (writer != null) {
+          paths.add(writer.getPath());
+          writer.close();
+        }
+      } catch (Exception ex) {
+        LOG.error("Failed to close the writer after an unfinished compaction.", ex);
+      }
+    }
+    return paths;
+  }
+
+  protected abstract Collection<StoreFile.Writer> writers();
+
+  /**
+   * Subclasses override this method to be called at the end of a successful sequence of append; all
+   * appends are processed before this method is called.
+   */
+  protected void preCommitWriters() throws IOException {
+  }
+
+  /**
+   * Subclasses override this method to be called before we close the give writer. Usually you can
+   * append extra metadata to the writer.
+   */
+  protected void preCloseWriter(StoreFile.Writer writer) throws IOException {
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/11d11d3f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredMultiFileWriter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredMultiFileWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredMultiFileWriter.java
new file mode 100644
index 0000000..f0bd444
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredMultiFileWriter.java
@@ -0,0 +1,83 @@
+/**
+ * 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.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.TreeMap;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.regionserver.StoreFile.Writer;
+
+/**
+ * class for cell sink that separates the provided cells into multiple files for date tiered
+ * compaction.
+ */
+@InterfaceAudience.Private
+public class DateTieredMultiFileWriter extends AbstractMultiFileWriter {
+
+  private final NavigableMap<Long, StoreFile.Writer> lowerBoundary2Writer
+    = new TreeMap<Long, StoreFile.Writer>();
+
+  private final boolean needEmptyFile;
+
+  /**
+   * @param needEmptyFile whether need to create an empty store file if we haven't written out
+   *          anything.
+   */
+  public DateTieredMultiFileWriter(List<Long> lowerBoundaries, boolean needEmptyFile) {
+    for (Long lowerBoundary : lowerBoundaries) {
+      lowerBoundary2Writer.put(lowerBoundary, null);
+    }
+    this.needEmptyFile = needEmptyFile;
+  }
+
+  @Override
+  public void append(Cell cell) throws IOException {
+    Map.Entry<Long, StoreFile.Writer> entry = lowerBoundary2Writer.floorEntry(cell.getTimestamp());
+    StoreFile.Writer writer = entry.getValue();
+    if (writer == null) {
+      writer = writerFactory.createWriter();
+      lowerBoundary2Writer.put(entry.getKey(), writer);
+    }
+    writer.append(cell);
+  }
+
+  @Override
+  protected Collection<Writer> writers() {
+    return lowerBoundary2Writer.values();
+  }
+
+  @Override
+  protected void preCommitWriters() throws IOException {
+    if (!needEmptyFile) {
+      return;
+    }
+    for (StoreFile.Writer writer : lowerBoundary2Writer.values()) {
+      if (writer != null) {
+        return;
+      }
+    }
+    // we haven't written out any data, create an empty file to retain metadata
+    lowerBoundary2Writer.put(lowerBoundary2Writer.firstKey(), writerFactory.createWriter());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/11d11d3f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
index b6164b2..868bee0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
@@ -261,6 +261,13 @@ public class StoreFile {
   }
 
   /**
+   * Clone a StoreFile for opening private reader.
+   */
+  public StoreFile cloneForReader() {
+    return new StoreFile(this);
+  }
+
+  /**
    * @return the StoreFile object associated to this StoreFile.
    *         null if the StoreFile is not a reference.
    */
@@ -294,7 +301,7 @@ public class StoreFile {
    * @return True if this is HFile.
    */
   public boolean isHFile() {
-    return this.fileInfo.isHFile(this.fileInfo.getPath());
+    return StoreFileInfo.isHFile(this.fileInfo.getPath());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/11d11d3f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeMultiFileWriter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeMultiFileWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeMultiFileWriter.java
index 651b863..1c3f14c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeMultiFileWriter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeMultiFileWriter.java
@@ -20,52 +20,36 @@ package org.apache.hadoop.hbase.regionserver;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 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.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.regionserver.compactions.Compactor;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.regionserver.StoreFile.Writer;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
- * Base class for cell sink that separates the provided cells into multiple files.
+ * Base class for cell sink that separates the provided cells into multiple files for stripe
+ * compaction.
  */
 @InterfaceAudience.Private
-public abstract class StripeMultiFileWriter implements Compactor.CellSink {
-  private static final Log LOG = LogFactory.getLog(StripeMultiFileWriter.class);
+public abstract class StripeMultiFileWriter extends AbstractMultiFileWriter {
 
-  /** Factory that is used to produce single StoreFile.Writer-s */
-  protected WriterFactory writerFactory;
-  protected CellComparator comparator;
+  private static final Log LOG = LogFactory.getLog(StripeMultiFileWriter.class);
 
+  protected final CellComparator comparator;
   protected List<StoreFile.Writer> existingWriters;
   protected List<byte[]> boundaries;
-  /** Source scanner that is tracking KV count; may be null if source is not StoreScanner */
-  protected StoreScanner sourceScanner;
 
   /** Whether to write stripe metadata */
   private boolean doWriteStripeMetadata = true;
 
-  public interface WriterFactory {
-    public StoreFile.Writer createWriter() throws IOException;
-  }
-
-  /**
-   * Initializes multi-writer before usage.
-   * @param sourceScanner Optional store scanner to obtain the information about read progress.
-   * @param factory Factory used to produce individual file writers.
-   * @param comparator Comparator used to compare rows.
-   */
-  public void init(StoreScanner sourceScanner, WriterFactory factory, CellComparator comparator)
-      throws IOException {
-    this.writerFactory = factory;
-    this.sourceScanner = sourceScanner;
+  public StripeMultiFileWriter(CellComparator comparator) {
     this.comparator = comparator;
   }
 
@@ -73,41 +57,35 @@ public abstract class StripeMultiFileWriter implements Compactor.CellSink {
     this.doWriteStripeMetadata = false;
   }
 
-  public List<Path> commitWriters(long maxSeqId, boolean isMajor) throws IOException {
+  @Override
+  protected Collection<Writer> writers() {
+    return existingWriters;
+  }
+
+  protected abstract void preCommitWritersInternal() throws IOException;
+
+  @Override
+  protected final void preCommitWriters() throws IOException {
+    // do some sanity check here.
     assert this.existingWriters != null;
-    commitWritersInternal();
+    preCommitWritersInternal();
     assert this.boundaries.size() == (this.existingWriters.size() + 1);
-    LOG.debug((this.doWriteStripeMetadata ? "W" : "Not w")
-      + "riting out metadata for " + this.existingWriters.size() + " writers");
-    List<Path> paths = new ArrayList<Path>();
-    for (int i = 0; i < this.existingWriters.size(); ++i) {
-      StoreFile.Writer writer = this.existingWriters.get(i);
-      if (writer == null) continue; // writer was skipped due to 0 KVs
-      if (doWriteStripeMetadata) {
-        writer.appendFileInfo(StripeStoreFileManager.STRIPE_START_KEY, this.boundaries.get(i));
-        writer.appendFileInfo(StripeStoreFileManager.STRIPE_END_KEY, this.boundaries.get(i + 1));
-      }
-      writer.appendMetadata(maxSeqId, isMajor);
-      paths.add(writer.getPath());
-      writer.close();
-    }
-    this.existingWriters = null;
-    return paths;
   }
 
-  public List<Path> abortWriters() {
-    assert this.existingWriters != null;
-    List<Path> paths = new ArrayList<Path>();
-    for (StoreFile.Writer writer : this.existingWriters) {
-      try {
-        paths.add(writer.getPath());
-        writer.close();
-      } catch (Exception ex) {
-        LOG.error("Failed to close the writer after an unfinished compaction.", ex);
+  @Override
+  protected void preCloseWriter(Writer writer) throws IOException {
+    if (doWriteStripeMetadata) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Write stripe metadata for " + writer.getPath().toString());
+      }
+      int index = existingWriters.indexOf(writer);
+      writer.appendFileInfo(StripeStoreFileManager.STRIPE_START_KEY, boundaries.get(index));
+      writer.appendFileInfo(StripeStoreFileManager.STRIPE_END_KEY, boundaries.get(index + 1));
+    } else {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Skip writing stripe metadata for " + writer.getPath().toString());
       }
     }
-    this.existingWriters = null;
-    return paths;
   }
 
   /**
@@ -115,13 +93,12 @@ public abstract class StripeMultiFileWriter implements Compactor.CellSink {
    * @param left The left boundary of the writer.
    * @param cell The cell whose row has to be checked.
    */
-  protected void sanityCheckLeft(
-      byte[] left, Cell cell) throws IOException {
-    if (!Arrays.equals(StripeStoreFileManager.OPEN_KEY, left) &&
-        comparator.compareRows(cell, left, 0, left.length) < 0) {
-      String error = "The first row is lower than the left boundary of [" + Bytes.toString(left)
-          + "]: [" + Bytes.toString(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength())
-          + "]";
+  protected void sanityCheckLeft(byte[] left, Cell cell) throws IOException {
+    if (!Arrays.equals(StripeStoreFileManager.OPEN_KEY, left)
+        && comparator.compareRows(cell, left, 0, left.length) < 0) {
+      String error =
+          "The first row is lower than the left boundary of [" + Bytes.toString(left) + "]: ["
+              + Bytes.toString(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()) + "]";
       LOG.error(error);
       throw new IOException(error);
     }
@@ -131,28 +108,22 @@ public abstract class StripeMultiFileWriter implements Compactor.CellSink {
    * Subclasses can call this method to make sure the last KV is within multi-writer range.
    * @param right The right boundary of the writer.
    */
-  protected void sanityCheckRight(
-      byte[] right, Cell cell) throws IOException {
-    if (!Arrays.equals(StripeStoreFileManager.OPEN_KEY, right) &&
-        comparator.compareRows(cell, right, 0, right.length) >= 0) {
-      String error = "The last row is higher or equal than the right boundary of ["
-          + Bytes.toString(right) + "]: ["
-          + Bytes.toString(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()) + "]";
+  protected void sanityCheckRight(byte[] right, Cell cell) throws IOException {
+    if (!Arrays.equals(StripeStoreFileManager.OPEN_KEY, right)
+        && comparator.compareRows(cell, right, 0, right.length) >= 0) {
+      String error =
+          "The last row is higher or equal than the right boundary of [" + Bytes.toString(right)
+              + "]: ["
+              + Bytes.toString(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()) + "]";
       LOG.error(error);
       throw new IOException(error);
     }
   }
 
   /**
-   * Subclasses override this method to be called at the end of a successful sequence of
-   * append; all appends are processed before this method is called.
-   */
-  protected abstract void commitWritersInternal() throws IOException;
-
-  /**
-   * MultiWriter that separates the cells based on fixed row-key boundaries.
-   * All the KVs between each pair of neighboring boundaries from the list supplied to ctor
-   * will end up in one file, and separate from all other such pairs.
+   * MultiWriter that separates the cells based on fixed row-key boundaries. All the KVs between
+   * each pair of neighboring boundaries from the list supplied to ctor will end up in one file, and
+   * separate from all other such pairs.
    */
   public static class BoundaryMultiWriter extends StripeMultiFileWriter {
     private StoreFile.Writer currentWriter;
@@ -165,31 +136,28 @@ public abstract class StripeMultiFileWriter implements Compactor.CellSink {
 
     /**
      * @param targetBoundaries The boundaries on which writers/files are separated.
-     * @param majorRangeFrom Major range is the range for which at least one file should be
-     *                       written (because all files are included in compaction).
-     *                       majorRangeFrom is the left boundary.
+     * @param majorRangeFrom Major range is the range for which at least one file should be written
+     *          (because all files are included in compaction). majorRangeFrom is the left boundary.
      * @param majorRangeTo The right boundary of majorRange (see majorRangeFrom).
      */
-    public BoundaryMultiWriter(List<byte[]> targetBoundaries,
+    public BoundaryMultiWriter(CellComparator comparator, List<byte[]> targetBoundaries,
         byte[] majorRangeFrom, byte[] majorRangeTo) throws IOException {
-      super();
+      super(comparator);
       this.boundaries = targetBoundaries;
       this.existingWriters = new ArrayList<StoreFile.Writer>(this.boundaries.size() - 1);
       // "major" range (range for which all files are included) boundaries, if any,
       // must match some target boundaries, let's find them.
-      assert  (majorRangeFrom == null) == (majorRangeTo == null);
+      assert (majorRangeFrom == null) == (majorRangeTo == null);
       if (majorRangeFrom != null) {
-        majorRangeFromIndex = Arrays.equals(majorRangeFrom, StripeStoreFileManager.OPEN_KEY)
-                                ? 0
-                                : Collections.binarySearch(boundaries, majorRangeFrom,
-                                                           Bytes.BYTES_COMPARATOR);
-        majorRangeToIndex = Arrays.equals(majorRangeTo, StripeStoreFileManager.OPEN_KEY)
-                              ? boundaries.size()
-                              : Collections.binarySearch(boundaries, majorRangeTo,
-                                                         Bytes.BYTES_COMPARATOR);
+        majorRangeFromIndex =
+            Arrays.equals(majorRangeFrom, StripeStoreFileManager.OPEN_KEY) ? 0 : Collections
+                .binarySearch(boundaries, majorRangeFrom, Bytes.BYTES_COMPARATOR);
+        majorRangeToIndex =
+            Arrays.equals(majorRangeTo, StripeStoreFileManager.OPEN_KEY) ? boundaries.size()
+                : Collections.binarySearch(boundaries, majorRangeTo, Bytes.BYTES_COMPARATOR);
         if (this.majorRangeFromIndex < 0 || this.majorRangeToIndex < 0) {
-          throw new IOException("Major range does not match writer boundaries: [" +
-              Bytes.toString(majorRangeFrom) + "] [" + Bytes.toString(majorRangeTo) + "]; from "
+          throw new IOException("Major range does not match writer boundaries: ["
+              + Bytes.toString(majorRangeFrom) + "] [" + Bytes.toString(majorRangeTo) + "]; from "
               + majorRangeFromIndex + " to " + majorRangeToIndex);
         }
       }
@@ -199,8 +167,7 @@ public abstract class StripeMultiFileWriter implements Compactor.CellSink {
     public void append(Cell cell) throws IOException {
       if (currentWriter == null && existingWriters.isEmpty()) {
         // First append ever, do a sanity check.
-        sanityCheckLeft(this.boundaries.get(0),
-            cell);
+        sanityCheckLeft(this.boundaries.get(0), cell);
       }
       prepareWriterFor(cell);
       currentWriter.append(cell);
@@ -209,19 +176,18 @@ public abstract class StripeMultiFileWriter implements Compactor.CellSink {
     }
 
     private boolean isCellAfterCurrentWriter(Cell cell) {
-      return !Arrays.equals(currentWriterEndKey, StripeStoreFileManager.OPEN_KEY) &&
-            (comparator.compareRows(cell, currentWriterEndKey, 0, currentWriterEndKey.length) >= 0);
+      return !Arrays.equals(currentWriterEndKey, StripeStoreFileManager.OPEN_KEY)
+        && (comparator.compareRows(cell, currentWriterEndKey, 0, currentWriterEndKey.length) >= 0);
     }
 
     @Override
-    protected void commitWritersInternal() throws IOException {
+    protected void preCommitWritersInternal() throws IOException {
       stopUsingCurrentWriter();
       while (existingWriters.size() < boundaries.size() - 1) {
         createEmptyWriter();
       }
       if (lastCell != null) {
-        sanityCheckRight(boundaries.get(boundaries.size() - 1),
-            lastCell);
+        sanityCheckRight(boundaries.get(boundaries.size() - 1), lastCell);
       }
     }
 
@@ -241,14 +207,13 @@ public abstract class StripeMultiFileWriter implements Compactor.CellSink {
     }
 
     /**
-     * Called if there are no cells for some stripe.
-     * We need to have something in the writer list for this stripe, so that writer-boundary
-     * list indices correspond to each other. We can insert null in the writer list for that
-     * purpose, except in the following cases where we actually need a file:
-     * 1) If we are in range for which we are compacting all the files, we need to create an
-     * empty file to preserve stripe metadata.
-     * 2) If we have not produced any file at all for this compactions, and this is the
-     * last chance (the last stripe), we need to preserve last seqNum (see also HBASE-6059).
+     * Called if there are no cells for some stripe. We need to have something in the writer list
+     * for this stripe, so that writer-boundary list indices correspond to each other. We can insert
+     * null in the writer list for that purpose, except in the following cases where we actually
+     * need a file: 1) If we are in range for which we are compacting all the files, we need to
+     * create an empty file to preserve stripe metadata. 2) If we have not produced any file at all
+     * for this compactions, and this is the last chance (the last stripe), we need to preserve last
+     * seqNum (see also HBASE-6059).
      */
     private void createEmptyWriter() throws IOException {
       int index = existingWriters.size();
@@ -258,12 +223,13 @@ public abstract class StripeMultiFileWriter implements Compactor.CellSink {
       boolean needEmptyFile = isInMajorRange || isLastWriter;
       existingWriters.add(needEmptyFile ? writerFactory.createWriter() : null);
       hasAnyWriter |= needEmptyFile;
-      currentWriterEndKey = (existingWriters.size() + 1 == boundaries.size())
-          ? null : boundaries.get(existingWriters.size() + 1);
+      currentWriterEndKey =
+          (existingWriters.size() + 1 == boundaries.size()) ? null : boundaries.get(existingWriters
+              .size() + 1);
     }
 
     private void checkCanCreateWriter() throws IOException {
-      int maxWriterCount =  boundaries.size() - 1;
+      int maxWriterCount = boundaries.size() - 1;
       assert existingWriters.size() <= maxWriterCount;
       if (existingWriters.size() >= maxWriterCount) {
         throw new IOException("Cannot create any more writers (created " + existingWriters.size()
@@ -280,16 +246,16 @@ public abstract class StripeMultiFileWriter implements Compactor.CellSink {
         cellsInCurrentWriter = 0;
       }
       currentWriter = null;
-      currentWriterEndKey = (existingWriters.size() + 1 == boundaries.size())
-          ? null : boundaries.get(existingWriters.size() + 1);
+      currentWriterEndKey =
+          (existingWriters.size() + 1 == boundaries.size()) ? null : boundaries.get(existingWriters
+              .size() + 1);
     }
   }
 
   /**
-   * MultiWriter that separates the cells based on target cell number per file and file count.
-   * New file is started every time the target number of KVs is reached, unless the fixed
-   * count of writers has already been created (in that case all the remaining KVs go into
-   * the last writer).
+   * MultiWriter that separates the cells based on target cell number per file and file count. New
+   * file is started every time the target number of KVs is reached, unless the fixed count of
+   * writers has already been created (in that case all the remaining KVs go into the last writer).
    */
   public static class SizeMultiWriter extends StripeMultiFileWriter {
     private int targetCount;
@@ -310,8 +276,9 @@ public abstract class StripeMultiFileWriter implements Compactor.CellSink {
      * @param left The left boundary of the first writer.
      * @param right The right boundary of the last writer.
      */
-    public SizeMultiWriter(int targetCount, long targetKvs, byte[] left, byte[] right) {
-      super();
+    public SizeMultiWriter(CellComparator comparator, int targetCount, long targetKvs, byte[] left,
+        byte[] right) {
+      super(comparator);
       this.targetCount = targetCount;
       this.targetCells = targetKvs;
       this.left = left;
@@ -331,11 +298,11 @@ public abstract class StripeMultiFileWriter implements Compactor.CellSink {
         sanityCheckLeft(left, cell);
         doCreateWriter = true;
       } else if (lastRowInCurrentWriter != null
-          && !CellUtil.matchingRow(cell,
-              lastRowInCurrentWriter, 0, lastRowInCurrentWriter.length)) {
+          && !CellUtil.matchingRow(cell, lastRowInCurrentWriter, 0,
+            lastRowInCurrentWriter.length)) {
         if (LOG.isDebugEnabled()) {
           LOG.debug("Stopping to use a writer after [" + Bytes.toString(lastRowInCurrentWriter)
-              + "] row; wrote out "  + cellsInCurrentWriter + " kvs");
+              + "] row; wrote out " + cellsInCurrentWriter + " kvs");
         }
         lastRowInCurrentWriter = null;
         cellsInCurrentWriter = 0;
@@ -343,7 +310,8 @@ public abstract class StripeMultiFileWriter implements Compactor.CellSink {
         doCreateWriter = true;
       }
       if (doCreateWriter) {
-        byte[] boundary = existingWriters.isEmpty() ? left : CellUtil.cloneRow(cell); // make a copy
+        // make a copy
+        byte[] boundary = existingWriters.isEmpty() ? left : CellUtil.cloneRow(cell);
         if (LOG.isDebugEnabled()) {
           LOG.debug("Creating new writer starting at [" + Bytes.toString(boundary) + "]");
         }
@@ -357,34 +325,35 @@ public abstract class StripeMultiFileWriter implements Compactor.CellSink {
       ++cellsInCurrentWriter;
       cellsSeen = cellsInCurrentWriter;
       if (this.sourceScanner != null) {
-        cellsSeen = Math.max(cellsSeen,
-            this.sourceScanner.getEstimatedNumberOfKvsScanned() - cellsSeenInPrevious);
+        cellsSeen =
+            Math.max(cellsSeen, this.sourceScanner.getEstimatedNumberOfKvsScanned()
+                - cellsSeenInPrevious);
       }
 
       // If we are not already waiting for opportunity to close, start waiting if we can
       // create any more writers and if the current one is too big.
-      if (lastRowInCurrentWriter == null
-          && existingWriters.size() < targetCount
+      if (lastRowInCurrentWriter == null && existingWriters.size() < targetCount
           && cellsSeen >= targetCells) {
         lastRowInCurrentWriter = CellUtil.cloneRow(cell); // make a copy
         if (LOG.isDebugEnabled()) {
-          LOG.debug("Preparing to start a new writer after [" + Bytes.toString(
-              lastRowInCurrentWriter) + "] row; observed " + cellsSeen + " kvs and wrote out "
-              + cellsInCurrentWriter + " kvs");
+          LOG.debug("Preparing to start a new writer after ["
+              + Bytes.toString(lastRowInCurrentWriter) + "] row; observed " + cellsSeen
+              + " kvs and wrote out " + cellsInCurrentWriter + " kvs");
         }
       }
     }
 
     @Override
-    protected void commitWritersInternal() throws IOException {
+    protected void preCommitWritersInternal() throws IOException {
       if (LOG.isDebugEnabled()) {
-        LOG.debug("Stopping with "  + cellsInCurrentWriter + " kvs in last writer" +
-            ((this.sourceScanner == null) ? "" : ("; observed estimated "
+        LOG.debug("Stopping with "
+            + cellsInCurrentWriter
+            + " kvs in last writer"
+            + ((this.sourceScanner == null) ? "" : ("; observed estimated "
                 + this.sourceScanner.getEstimatedNumberOfKvsScanned() + " KVs total")));
       }
       if (lastCell != null) {
-        sanityCheckRight(
-            right, lastCell);
+        sanityCheckRight(right, lastCell);
       }
 
       // When expired stripes were going to be merged into one, and if no writer was created during

http://git-wip-us.apache.org/repos/asf/hbase/blob/11d11d3f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java
index 9a06a88..34e8497 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java
@@ -26,6 +26,7 @@ import java.util.List;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -69,7 +70,8 @@ public class StripeStoreFlusher extends StoreFlusher {
     }
 
     // Let policy select flush method.
-    StripeFlushRequest req = this.policy.selectFlush(this.stripes, cellsCount);
+    StripeFlushRequest req = this.policy.selectFlush(store.getComparator(), this.stripes,
+      cellsCount);
 
     boolean success = false;
     StripeMultiFileWriter mw = null;
@@ -78,7 +80,7 @@ public class StripeStoreFlusher extends StoreFlusher {
       StripeMultiFileWriter.WriterFactory factory = createWriterFactory(
           snapshot.getTimeRangeTracker(), cellsCount);
       StoreScanner storeScanner = (scanner instanceof StoreScanner) ? (StoreScanner)scanner : null;
-      mw.init(storeScanner, factory, store.getComparator());
+      mw.init(storeScanner, factory);
 
       synchronized (flushLock) {
         performFlush(scanner, mw, smallestReadPoint, throughputController);
@@ -123,10 +125,17 @@ public class StripeStoreFlusher extends StoreFlusher {
 
   /** Stripe flush request wrapper that writes a non-striped file. */
   public static class StripeFlushRequest {
+
+    protected final CellComparator comparator;
+
+    public StripeFlushRequest(CellComparator comparator) {
+      this.comparator = comparator;
+    }
+
     @VisibleForTesting
     public StripeMultiFileWriter createWriter() throws IOException {
-      StripeMultiFileWriter writer =
-          new StripeMultiFileWriter.SizeMultiWriter(1, Long.MAX_VALUE, OPEN_KEY, OPEN_KEY);
+      StripeMultiFileWriter writer = new StripeMultiFileWriter.SizeMultiWriter(comparator, 1,
+          Long.MAX_VALUE, OPEN_KEY, OPEN_KEY);
       writer.setNoStripeMetadata();
       return writer;
     }
@@ -137,13 +146,15 @@ public class StripeStoreFlusher extends StoreFlusher {
     private final List<byte[]> targetBoundaries;
 
     /** @param targetBoundaries New files should be written with these boundaries. */
-    public BoundaryStripeFlushRequest(List<byte[]> targetBoundaries) {
+    public BoundaryStripeFlushRequest(CellComparator comparator, List<byte[]> targetBoundaries) {
+      super(comparator);
       this.targetBoundaries = targetBoundaries;
     }
 
     @Override
     public StripeMultiFileWriter createWriter() throws IOException {
-      return new StripeMultiFileWriter.BoundaryMultiWriter(targetBoundaries, null, null);
+      return new StripeMultiFileWriter.BoundaryMultiWriter(comparator, targetBoundaries, null,
+          null);
     }
   }
 
@@ -157,15 +168,16 @@ public class StripeStoreFlusher extends StoreFlusher {
      * @param targetKvs The KV count of each segment. If targetKvs*targetCount is less than
      *                  total number of kvs, all the overflow data goes into the last stripe.
      */
-    public SizeStripeFlushRequest(int targetCount, long targetKvs) {
+    public SizeStripeFlushRequest(CellComparator comparator, int targetCount, long targetKvs) {
+      super(comparator);
       this.targetCount = targetCount;
       this.targetKvs = targetKvs;
     }
 
     @Override
     public StripeMultiFileWriter createWriter() throws IOException {
-      return new StripeMultiFileWriter.SizeMultiWriter(
-          this.targetCount, this.targetKvs, OPEN_KEY, OPEN_KEY);
+      return new StripeMultiFileWriter.SizeMultiWriter(comparator, this.targetCount, this.targetKvs,
+          OPEN_KEY, OPEN_KEY);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/11d11d3f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/AbstractMultiOutputCompactor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/AbstractMultiOutputCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/AbstractMultiOutputCompactor.java
new file mode 100644
index 0000000..29d8561
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/AbstractMultiOutputCompactor.java
@@ -0,0 +1,161 @@
+/**
+ * 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.compactions;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+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.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.regionserver.AbstractMultiFileWriter;
+import org.apache.hadoop.hbase.regionserver.AbstractMultiFileWriter.WriterFactory;
+import org.apache.hadoop.hbase.regionserver.InternalScanner;
+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.StoreScanner;
+import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
+import org.apache.hadoop.hbase.security.User;
+
+import com.google.common.io.Closeables;
+
+/**
+ * Base class for implementing a Compactor which will generate multiple output files after
+ * compaction.
+ */
+@InterfaceAudience.Private
+public abstract class AbstractMultiOutputCompactor<T extends AbstractMultiFileWriter>
+    extends Compactor {
+
+  private static final Log LOG = LogFactory.getLog(AbstractMultiOutputCompactor.class);
+
+  public AbstractMultiOutputCompactor(Configuration conf, Store store) {
+    super(conf, store);
+  }
+
+  protected interface InternalScannerFactory {
+
+    ScanType getScanType(CompactionRequest request);
+
+    InternalScanner createScanner(List<StoreFileScanner> scanners, ScanType scanType,
+        FileDetails fd, long smallestReadPoint) throws IOException;
+  }
+
+  protected List<Path> compact(T writer, final CompactionRequest request,
+      InternalScannerFactory scannerFactory, ThroughputController throughputController, User user)
+          throws IOException {
+    final FileDetails fd = getFileDetails(request.getFiles(), request.isAllFiles());
+    this.progress = new CompactionProgress(fd.maxKeyCount);
+
+    // Find the smallest read point across all the Scanners.
+    long smallestReadPoint = getSmallestReadPoint();
+
+    List<StoreFileScanner> scanners;
+    Collection<StoreFile> readersToClose;
+    if (this.conf.getBoolean("hbase.regionserver.compaction.private.readers", true)) {
+      // clone all StoreFiles, so we'll do the compaction on a independent copy of StoreFiles,
+      // HFiles, and their readers
+      readersToClose = new ArrayList<StoreFile>(request.getFiles().size());
+      for (StoreFile f : request.getFiles()) {
+        readersToClose.add(f.cloneForReader());
+      }
+      scanners = createFileScanners(readersToClose, smallestReadPoint,
+        store.throttleCompaction(request.getSize()));
+    } else {
+      readersToClose = Collections.emptyList();
+      scanners = createFileScanners(request.getFiles(), smallestReadPoint,
+        store.throttleCompaction(request.getSize()));
+    }
+    InternalScanner scanner = null;
+    boolean finished = false;
+    try {
+      /* Include deletes, unless we are doing a major compaction */
+      ScanType scanType = scannerFactory.getScanType(request);
+      scanner = preCreateCoprocScanner(request, scanType, fd.earliestPutTs, scanners);
+      if (scanner == null) {
+        scanner = scannerFactory.createScanner(scanners, scanType, fd, smallestReadPoint);
+      }
+      scanner = postCreateCoprocScanner(request, scanType, scanner, user);
+      if (scanner == null) {
+        // NULL scanner returned from coprocessor hooks means skip normal processing.
+        return new ArrayList<Path>();
+      }
+      boolean cleanSeqId = false;
+      if (fd.minSeqIdToKeep > 0) {
+        smallestReadPoint = Math.min(fd.minSeqIdToKeep, smallestReadPoint);
+        cleanSeqId = true;
+      }
+      // Create the writer factory for compactions.
+      final boolean needMvcc = fd.maxMVCCReadpoint >= 0;
+      WriterFactory writerFactory = new WriterFactory() {
+        @Override
+        public Writer createWriter() throws IOException {
+          return store.createWriterInTmp(fd.maxKeyCount, compactionCompression, true, needMvcc,
+            fd.maxTagsLength > 0, store.throttleCompaction(request.getSize()));
+        }
+      };
+      // Prepare multi-writer, and perform the compaction using scanner and writer.
+      // It is ok here if storeScanner is null.
+      StoreScanner storeScanner
+        = (scanner instanceof StoreScanner) ? (StoreScanner) scanner : null;
+      writer.init(storeScanner, writerFactory);
+      finished = performCompaction(fd, scanner, writer, smallestReadPoint, cleanSeqId,
+        throughputController, request.isAllFiles());
+      if (!finished) {
+        throw new InterruptedIOException("Aborting compaction of store " + store + " in region "
+            + store.getRegionInfo().getRegionNameAsString() + " because it was interrupted.");
+      }
+    } finally {
+      Closeables.close(scanner, true);
+      for (StoreFile f : readersToClose) {
+        try {
+          f.closeReader(true);
+        } catch (IOException e) {
+          LOG.warn("Exception closing " + f, e);
+        }
+      }
+      if (!finished) {
+        FileSystem fs = store.getFileSystem();
+        for (Path leftoverFile : writer.abortWriters()) {
+          try {
+            fs.delete(leftoverFile, false);
+          } catch (IOException e) {
+            LOG.error("Failed to delete the leftover file " + leftoverFile
+                + " after an unfinished compaction.",
+              e);
+          }
+        }
+      }
+    }
+    assert finished : "We should have exited the method on all error paths";
+    return commitMultiWriter(writer, fd, request);
+  }
+
+  protected abstract List<Path> commitMultiWriter(T writer, FileDetails fd,
+      CompactionRequest request) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/11d11d3f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
index 0e6ab05..9125684 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
@@ -64,14 +64,14 @@ public abstract class Compactor {
   private static final Log LOG = LogFactory.getLog(Compactor.class);
   private static final long COMPACTION_PROGRESS_LOG_INTERVAL = 60 * 1000;
   protected CompactionProgress progress;
-  protected Configuration conf;
-  protected Store store;
+  protected final Configuration conf;
+  protected final Store store;
 
-  protected int compactionKVMax;
-  protected Compression.Algorithm compactionCompression;
+  protected final int compactionKVMax;
+  protected final Compression.Algorithm compactionCompression;
 
   /** specify how many days to keep MVCC values during major compaction **/ 
-  protected int keepSeqIdPeriod;
+  protected final int keepSeqIdPeriod;
 
   //TODO: depending on Store is not good but, realistically, all compactors currently do.
   Compactor(final Configuration conf, final Store store) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/11d11d3f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactor.java
new file mode 100644
index 0000000..413b29c
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactor.java
@@ -0,0 +1,86 @@
+/**
+ * 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.compactions;
+
+import java.io.IOException;
+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.Path;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.regionserver.DateTieredMultiFileWriter;
+import org.apache.hadoop.hbase.regionserver.InternalScanner;
+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.StoreFileScanner;
+import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
+import org.apache.hadoop.hbase.security.User;
+
+/**
+ * This compactor will generate StoreFile for different time ranges.
+ */
+@InterfaceAudience.Private
+public class DateTieredCompactor extends AbstractMultiOutputCompactor<DateTieredMultiFileWriter> {
+
+  private static final Log LOG = LogFactory.getLog(DateTieredCompactor.class);
+
+  public DateTieredCompactor(Configuration conf, Store store) {
+    super(conf, store);
+  }
+
+  private boolean needEmptyFile(CompactionRequest request) {
+    // if we are going to compact the last N files, then we need to emit an empty file to retain the
+    // maxSeqId if we haven't written out anything.
+    return StoreFile.getMaxSequenceIdInList(request.getFiles()) == store.getMaxSequenceId();
+  }
+
+  public List<Path> compact(final CompactionRequest request, List<Long> lowerBoundaries,
+      ThroughputController throughputController, User user) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Executing compaction with " + lowerBoundaries.size()
+          + "windows, lower boundaries: " + lowerBoundaries);
+    }
+
+    DateTieredMultiFileWriter writer =
+        new DateTieredMultiFileWriter(lowerBoundaries, needEmptyFile(request));
+    return compact(writer, request, new InternalScannerFactory() {
+
+      @Override
+      public ScanType getScanType(CompactionRequest request) {
+        return request.isRetainDeleteMarkers() ? ScanType.COMPACT_RETAIN_DELETES
+            : ScanType.COMPACT_DROP_DELETES;
+      }
+
+      @Override
+      public InternalScanner createScanner(List<StoreFileScanner> scanners, ScanType scanType,
+          FileDetails fd, long smallestReadPoint) throws IOException {
+        return DateTieredCompactor.this.createScanner(store, scanners, scanType, smallestReadPoint,
+          fd.earliestPutTs);
+      }
+    }, throughputController, user);
+  }
+
+  @Override
+  protected List<Path> commitMultiWriter(DateTieredMultiFileWriter writer, FileDetails fd,
+      CompactionRequest request) throws IOException {
+    return writer.commitWriters(fd.maxSeqId, request.isAllFiles());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/11d11d3f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java
----------------------------------------------------------------------
diff --git 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
index e7e0cca..22a45b1 100644
--- 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
@@ -65,10 +65,10 @@ public class DefaultCompactor extends Compactor {
     Collection<StoreFile> readersToClose;
     if (this.conf.getBoolean("hbase.regionserver.compaction.private.readers", true)) {
       // clone all StoreFiles, so we'll do the compaction on a independent copy of StoreFiles,
-      // HFileFiles, and their readers
+      // HFiles, and their readers
       readersToClose = new ArrayList<StoreFile>(request.getFiles().size());
       for (StoreFile f : request.getFiles()) {
-        readersToClose.add(new StoreFile(f));
+        readersToClose.add(f.cloneForReader());
       }
       scanners = createFileScanners(readersToClose, smallestReadPoint,
           store.throttleCompaction(request.getSize()));

http://git-wip-us.apache.org/repos/asf/hbase/blob/11d11d3f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/StripeCompactionPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/StripeCompactionPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/StripeCompactionPolicy.java
index 2bb8fc8..e8a4340 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/StripeCompactionPolicy.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/StripeCompactionPolicy.java
@@ -27,9 +27,10 @@ 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.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
 import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreUtils;
@@ -84,18 +85,20 @@ public class StripeCompactionPolicy extends CompactionPolicy {
         request, OPEN_KEY, OPEN_KEY, targetKvsAndCount.getSecond(), targetKvsAndCount.getFirst());
   }
 
-  public StripeStoreFlusher.StripeFlushRequest selectFlush(
+  public StripeStoreFlusher.StripeFlushRequest selectFlush(CellComparator comparator,
       StripeInformationProvider si, int kvCount) {
     if (this.config.isUsingL0Flush()) {
-      return new StripeStoreFlusher.StripeFlushRequest(); // L0 is used, return dumb request.
+      // L0 is used, return dumb request.
+      return new StripeStoreFlusher.StripeFlushRequest(comparator);
     }
     if (si.getStripeCount() == 0) {
       // No stripes - start with the requisite count, derive KVs per stripe.
       int initialCount = this.config.getInitialCount();
-      return new StripeStoreFlusher.SizeStripeFlushRequest(initialCount, kvCount / initialCount);
+      return new StripeStoreFlusher.SizeStripeFlushRequest(comparator, initialCount,
+          kvCount / initialCount);
     }
     // There are stripes - do according to the boundaries.
-    return new StripeStoreFlusher.BoundaryStripeFlushRequest(si.getStripeBoundaries());
+    return new StripeStoreFlusher.BoundaryStripeFlushRequest(comparator, si.getStripeBoundaries());
   }
 
   public StripeCompactionRequest selectCompaction(StripeInformationProvider si,

http://git-wip-us.apache.org/repos/asf/hbase/blob/11d11d3f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/StripeCompactor.java
----------------------------------------------------------------------
diff --git 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
index fd0e2b2..1364ce0 100644
--- 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
@@ -18,50 +18,65 @@
 package org.apache.hadoop.hbase.regionserver.compactions;
 
 import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.util.ArrayList;
-import java.util.Collection;
 import java.util.List;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.io.compress.Compression;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
 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.StoreFileScanner;
-import org.apache.hadoop.hbase.regionserver.StoreScanner;
 import org.apache.hadoop.hbase.regionserver.StripeMultiFileWriter;
-import org.apache.hadoop.hbase.regionserver.StoreFile.Writer;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
- * This is the placeholder for stripe compactor. The implementation,
- * as well as the proper javadoc, will be added in HBASE-7967.
+ * This is the placeholder for stripe compactor. The implementation, as well as the proper javadoc,
+ * will be added in HBASE-7967.
  */
 @InterfaceAudience.Private
-public class StripeCompactor extends Compactor {
+public class StripeCompactor extends AbstractMultiOutputCompactor<StripeMultiFileWriter> {
   private static final Log LOG = LogFactory.getLog(StripeCompactor.class);
+
   public StripeCompactor(Configuration conf, Store store) {
     super(conf, store);
   }
 
-  public List<Path> compact(CompactionRequest request, List<byte[]> targetBoundaries,
-    byte[] majorRangeFromRow, byte[] majorRangeToRow,
-    ThroughputController throughputController) throws IOException {
-    return compact(request, targetBoundaries, majorRangeFromRow, majorRangeToRow,
-      throughputController, null);
+  private final class StripeInternalScannerFactory implements InternalScannerFactory {
+
+    private final byte[] majorRangeFromRow;
+
+    private final byte[] majorRangeToRow;
+
+    public StripeInternalScannerFactory(byte[] majorRangeFromRow, byte[] majorRangeToRow) {
+      this.majorRangeFromRow = majorRangeFromRow;
+      this.majorRangeToRow = majorRangeToRow;
+    }
+
+    @Override
+    public ScanType getScanType(CompactionRequest request) {
+      // If majorRangeFromRow and majorRangeToRow are not null, then we will not use the return
+      // value to create InternalScanner. See the createScanner method below. The return value is
+      // also used when calling coprocessor hooks.
+      return ScanType.COMPACT_RETAIN_DELETES;
+    }
+
+    @Override
+    public InternalScanner createScanner(List<StoreFileScanner> scanners, ScanType scanType,
+        FileDetails fd, long smallestReadPoint) throws IOException {
+      return (majorRangeFromRow == null) ? StripeCompactor.this.createScanner(store, scanners,
+        scanType, smallestReadPoint, fd.earliestPutTs) : StripeCompactor.this.createScanner(store,
+        scanners, smallestReadPoint, fd.earliestPutTs, majorRangeFromRow, majorRangeToRow);
+    }
   }
 
   public List<Path> compact(CompactionRequest request, List<byte[]> targetBoundaries,
-      byte[] majorRangeFromRow, byte[] majorRangeToRow,
-      ThroughputController throughputController, User user) throws IOException {
+      byte[] majorRangeFromRow, byte[] majorRangeToRow, ThroughputController throughputController,
+      User user) throws IOException {
     if (LOG.isDebugEnabled()) {
       StringBuilder sb = new StringBuilder();
       sb.append("Executing compaction with " + targetBoundaries.size() + " boundaries:");
@@ -70,116 +85,32 @@ public class StripeCompactor extends Compactor {
       }
       LOG.debug(sb.toString());
     }
-    StripeMultiFileWriter writer = new StripeMultiFileWriter.BoundaryMultiWriter(
-        targetBoundaries, majorRangeFromRow, majorRangeToRow);
-    return compactInternal(writer, request, majorRangeFromRow, majorRangeToRow,
-      throughputController, user);
-  }
-
-  public List<Path> compact(CompactionRequest request, int targetCount, long targetSize,
-    byte[] left, byte[] right, byte[] majorRangeFromRow, byte[] majorRangeToRow,
-    ThroughputController throughputController) throws IOException {
-    return compact(request, targetCount, targetSize, left, right, majorRangeFromRow,
-      majorRangeToRow, throughputController, null);
+    StripeMultiFileWriter writer =
+        new StripeMultiFileWriter.BoundaryMultiWriter(store.getComparator(), targetBoundaries,
+            majorRangeFromRow, majorRangeToRow);
+    return compact(writer, request, new StripeInternalScannerFactory(majorRangeFromRow,
+        majorRangeToRow), throughputController, user);
   }
 
   public List<Path> compact(CompactionRequest request, int targetCount, long targetSize,
       byte[] left, byte[] right, byte[] majorRangeFromRow, byte[] majorRangeToRow,
       ThroughputController throughputController, User user) throws IOException {
     if (LOG.isDebugEnabled()) {
-      LOG.debug("Executing compaction with " + targetSize
-          + " target file size, no more than " + targetCount + " files, in ["
-          + Bytes.toString(left) + "] [" + Bytes.toString(right) + "] range");
+      LOG.debug("Executing compaction with " + targetSize + " target file size, no more than "
+          + targetCount + " files, in [" + Bytes.toString(left) + "] [" + Bytes.toString(right)
+          + "] range");
     }
-    StripeMultiFileWriter writer = new StripeMultiFileWriter.SizeMultiWriter(
-        targetCount, targetSize, left, right);
-    return compactInternal(writer, request, majorRangeFromRow, majorRangeToRow,
-      throughputController, user);
+    StripeMultiFileWriter writer =
+        new StripeMultiFileWriter.SizeMultiWriter(store.getComparator(), targetCount, targetSize,
+            left, right);
+    return compact(writer, request, new StripeInternalScannerFactory(majorRangeFromRow,
+        majorRangeToRow), throughputController, user);
   }
 
-  private List<Path> compactInternal(StripeMultiFileWriter mw, final CompactionRequest request,
-      byte[] majorRangeFromRow, byte[] majorRangeToRow,
-      ThroughputController throughputController, User user) throws IOException {
-    final Collection<StoreFile> filesToCompact = request.getFiles();
-    final FileDetails fd = getFileDetails(filesToCompact, request.isMajor());
-    this.progress = new CompactionProgress(fd.maxKeyCount);
-
-    long smallestReadPoint = getSmallestReadPoint();
-    List<StoreFileScanner> scanners = createFileScanners(filesToCompact,
-        smallestReadPoint, store.throttleCompaction(request.getSize()));
-
-    boolean finished = false;
-    InternalScanner scanner = null;
-    boolean cleanSeqId = false;
-    try {
-      // Get scanner to use.
-      ScanType coprocScanType = ScanType.COMPACT_RETAIN_DELETES;
-      scanner = preCreateCoprocScanner(request, coprocScanType, fd.earliestPutTs, scanners, user);
-      if (scanner == null) {
-        scanner = (majorRangeFromRow == null)
-            ? createScanner(store, scanners,
-                ScanType.COMPACT_RETAIN_DELETES, smallestReadPoint, fd.earliestPutTs)
-            : createScanner(store, scanners,
-                smallestReadPoint, fd.earliestPutTs, majorRangeFromRow, majorRangeToRow);
-      }
-      scanner = postCreateCoprocScanner(request, coprocScanType, scanner, user);
-      if (scanner == null) {
-        // NULL scanner returned from coprocessor hooks means skip normal processing.
-        return new ArrayList<Path>();
-      }
-
-      // Create the writer factory for compactions.
-      if(fd.minSeqIdToKeep > 0) {
-        smallestReadPoint = Math.min(fd.minSeqIdToKeep, smallestReadPoint);
-        cleanSeqId = true;
-      }
-
-      final boolean needMvcc = fd.maxMVCCReadpoint > 0;
-
-      final Compression.Algorithm compression = store.getFamily().getCompactionCompressionType();
-      StripeMultiFileWriter.WriterFactory factory = new StripeMultiFileWriter.WriterFactory() {
-        @Override
-        public Writer createWriter() throws IOException {
-          return store.createWriterInTmp(
-              fd.maxKeyCount, compression, true, needMvcc, fd.maxTagsLength > 0,
-              store.throttleCompaction(request.getSize()));
-        }
-      };
-
-      // Prepare multi-writer, and perform the compaction using scanner and writer.
-      // 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, throughputController,
-                  request.isMajor());
-      if (!finished) {
-        throw new InterruptedIOException( "Aborting compaction of store " + store +
-            " in region " + store.getRegionInfo().getRegionNameAsString() +
-            " because it was interrupted.");
-      }
-    } finally {
-      if (scanner != null) {
-        try {
-          scanner.close();
-        } catch (Throwable t) {
-          // Don't fail the compaction if this fails.
-          LOG.error("Failed to close scanner after compaction.", t);
-        }
-      }
-      if (!finished) {
-        for (Path leftoverFile : mw.abortWriters()) {
-          try {
-            store.getFileSystem().delete(leftoverFile, false);
-          } catch (Exception ex) {
-            LOG.error("Failed to delete the leftover file after an unfinished compaction.", ex);
-          }
-        }
-      }
-    }
-
-    assert finished : "We should have exited the method on all error paths";
-    List<Path> newFiles = mw.commitWriters(fd.maxSeqId, request.isMajor());
+  @Override
+  protected List<Path> commitMultiWriter(StripeMultiFileWriter writer, FileDetails fd,
+      CompactionRequest request) throws IOException {
+    List<Path> newFiles = writer.commitWriters(fd.maxSeqId, request.isMajor());
     assert !newFiles.isEmpty() : "Should have produced an empty file to preserve metadata.";
     return newFiles;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/11d11d3f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeCompactor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeCompactor.java
deleted file mode 100644
index cb586f3..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeCompactor.java
+++ /dev/null
@@ -1,325 +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.apache.hadoop.hbase.regionserver.StripeStoreFileManager.OPEN_KEY;
-import static org.apache.hadoop.hbase.regionserver.StripeStoreFileManager.STRIPE_END_KEY;
-import static org.apache.hadoop.hbase.regionserver.StripeStoreFileManager.STRIPE_START_KEY;
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyBoolean;
-import static org.mockito.Matchers.anyLong;
-import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.TreeMap;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellComparator;
-import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.io.compress.Compression;
-import org.apache.hadoop.hbase.io.hfile.HFile;
-import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
-import org.apache.hadoop.hbase.regionserver.compactions.StripeCompactor;
-import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
-import org.apache.hadoop.hbase.testclassification.RegionServerTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-
-@Category({RegionServerTests.class, SmallTests.class})
-public class TestStripeCompactor {
-  private static final byte[] NAME_OF_THINGS = Bytes.toBytes("foo");
-  private static final TableName TABLE_NAME = TableName.valueOf(NAME_OF_THINGS, NAME_OF_THINGS);
-
-  private static final byte[] KEY_B = Bytes.toBytes("bbb");
-  private static final byte[] KEY_C = Bytes.toBytes("ccc");
-  private static final byte[] KEY_D = Bytes.toBytes("ddd");
-
-  private static final KeyValue KV_A = kvAfter(Bytes.toBytes("aaa"));
-  private static final KeyValue KV_B = kvAfter(KEY_B);
-  private static final KeyValue KV_C = kvAfter(KEY_C);
-  private static final KeyValue KV_D = kvAfter(KEY_D);
-
-  private static KeyValue kvAfter(byte[] key) {
-    return new KeyValue(Arrays.copyOf(key, key.length + 1), 0L);
-  }
-
-  private static <T> T[] a(T... a) {
-    return a;
-  }
-
-  private static KeyValue[] e() {
-    return TestStripeCompactor.<KeyValue>a();
-  }
-
-  @Test
-  public void testBoundaryCompactions() throws Exception {
-    // General verification
-    verifyBoundaryCompaction(a(KV_A, KV_A, KV_B, KV_B, KV_C, KV_D),
-        a(OPEN_KEY, KEY_B, KEY_D, OPEN_KEY), a(a(KV_A, KV_A), a(KV_B, KV_B, KV_C), a(KV_D)));
-    verifyBoundaryCompaction(a(KV_B, KV_C), a(KEY_B, KEY_C, KEY_D), a(a(KV_B), a(KV_C)));
-    verifyBoundaryCompaction(a(KV_B, KV_C), a(KEY_B, KEY_D), new KeyValue[][] { a(KV_B, KV_C) });
-  }
-
-  @Test
-  public void testBoundaryCompactionEmptyFiles() throws Exception {
-    // No empty file if there're already files.
-    verifyBoundaryCompaction(
-        a(KV_B), a(KEY_B, KEY_C, KEY_D, OPEN_KEY), a(a(KV_B), null, null), null, null, false);
-    verifyBoundaryCompaction(a(KV_A, KV_C),
-        a(OPEN_KEY, KEY_B, KEY_C, KEY_D), a(a(KV_A), null, a(KV_C)), null, null, false);
-    // But should be created if there are no file.
-    verifyBoundaryCompaction(
-        e(), a(OPEN_KEY, KEY_B, KEY_C, OPEN_KEY), a(null, null, e()), null, null, false);
-    // In major range if there's major range.
-    verifyBoundaryCompaction(
-        e(), a(OPEN_KEY, KEY_B, KEY_C, OPEN_KEY), a(null, e(), null), KEY_B, KEY_C, false);
-    verifyBoundaryCompaction(
-        e(), a(OPEN_KEY, KEY_B, KEY_C, OPEN_KEY), a(e(), e(), null), OPEN_KEY, KEY_C, false);
-    // Major range should have files regardless of KVs.
-    verifyBoundaryCompaction(a(KV_A), a(OPEN_KEY, KEY_B, KEY_C, KEY_D, OPEN_KEY),
-        a(a(KV_A), e(), e(), null), KEY_B, KEY_D, false);
-    verifyBoundaryCompaction(a(KV_C), a(OPEN_KEY, KEY_B, KEY_C, KEY_D, OPEN_KEY),
-        a(null, null, a(KV_C), e()), KEY_C, OPEN_KEY, false);
-
-  }
-
-  public static void verifyBoundaryCompaction(
-      KeyValue[] input, byte[][] boundaries, KeyValue[][] output) throws Exception {
-    verifyBoundaryCompaction(input, boundaries, output, null, null, true);
-  }
-
-  public static void verifyBoundaryCompaction(KeyValue[] input, byte[][] boundaries,
-      KeyValue[][] output, byte[] majorFrom, byte[] majorTo, boolean allFiles)
-          throws Exception {
-    StoreFileWritersCapture writers = new StoreFileWritersCapture();
-    StripeCompactor sc = createCompactor(writers, input);
-    List<Path> paths =
-        sc.compact(createDummyRequest(), Arrays.asList(boundaries), majorFrom, majorTo,
-          NoLimitThroughputController.INSTANCE);
-    writers.verifyKvs(output, allFiles, true);
-    if (allFiles) {
-      assertEquals(output.length, paths.size());
-      writers.verifyBoundaries(boundaries);
-    }
-  }
-
-  @Test
-  public void testSizeCompactions() throws Exception {
-    // General verification with different sizes.
-    verifySizeCompaction(a(KV_A, KV_A, KV_B, KV_C, KV_D), 3, 2, OPEN_KEY, OPEN_KEY,
-        a(a(KV_A, KV_A), a(KV_B, KV_C), a(KV_D)));
-    verifySizeCompaction(a(KV_A, KV_B, KV_C, KV_D), 4, 1, OPEN_KEY, OPEN_KEY,
-        a(a(KV_A), a(KV_B), a(KV_C), a(KV_D)));
-    verifySizeCompaction(a(KV_B, KV_C), 2, 1, KEY_B, KEY_D, a(a(KV_B), a(KV_C)));
-    // Verify row boundaries are preserved.
-    verifySizeCompaction(a(KV_A, KV_A, KV_A, KV_C, KV_D), 3, 2, OPEN_KEY, OPEN_KEY,
-        a(a(KV_A, KV_A, KV_A), a(KV_C, KV_D)));
-    verifySizeCompaction(a(KV_A, KV_B, KV_B, KV_C), 3, 1, OPEN_KEY, OPEN_KEY,
-        a(a(KV_A), a(KV_B, KV_B), a(KV_C)));
-    // Too much data, count limits the number of files.
-    verifySizeCompaction(a(KV_A, KV_B, KV_C, KV_D), 2, 1, OPEN_KEY, OPEN_KEY,
-        a(a(KV_A), a(KV_B, KV_C, KV_D)));
-    verifySizeCompaction(a(KV_A, KV_B, KV_C), 1, Long.MAX_VALUE, OPEN_KEY, KEY_D,
-        new KeyValue[][] { a(KV_A, KV_B, KV_C) });
-    // Too little data/large count, no extra files.
-    verifySizeCompaction(a(KV_A, KV_B, KV_C, KV_D), Integer.MAX_VALUE, 2, OPEN_KEY, OPEN_KEY,
-        a(a(KV_A, KV_B), a(KV_C, KV_D)));
-  }
-
-  public static void verifySizeCompaction(KeyValue[] input, int targetCount, long targetSize,
-      byte[] left, byte[] right, KeyValue[][] output) throws Exception {
-    StoreFileWritersCapture writers = new StoreFileWritersCapture();
-    StripeCompactor sc = createCompactor(writers, input);
-    List<Path> paths =
-        sc.compact(createDummyRequest(), targetCount, targetSize, left, right, null, null,
-          NoLimitThroughputController.INSTANCE);
-    assertEquals(output.length, paths.size());
-    writers.verifyKvs(output, true, true);
-    List<byte[]> boundaries = new ArrayList<byte[]>();
-    boundaries.add(left);
-    for (int i = 1; i < output.length; ++i) {
-      boundaries.add(CellUtil.cloneRow(output[i][0]));
-    }
-    boundaries.add(right);
-    writers.verifyBoundaries(boundaries.toArray(new byte[][] {}));
-  }
-
-  private static StripeCompactor createCompactor(
-      StoreFileWritersCapture writers, KeyValue[] input) throws Exception {
-    Configuration conf = HBaseConfiguration.create();
-    final Scanner scanner = new Scanner(input);
-
-    // Create store mock that is satisfactory for compactor.
-    HColumnDescriptor col = new HColumnDescriptor(NAME_OF_THINGS);
-    ScanInfo si = new ScanInfo(conf, col, Long.MAX_VALUE, 0, CellComparator.COMPARATOR);
-    Store store = mock(Store.class);
-    when(store.getFamily()).thenReturn(col);
-    when(store.getScanInfo()).thenReturn(si);
-    when(store.areWritesEnabled()).thenReturn(true);
-    when(store.getFileSystem()).thenReturn(mock(FileSystem.class));
-    when(store.getRegionInfo()).thenReturn(new HRegionInfo(TABLE_NAME));
-    when(store.createWriterInTmp(anyLong(), any(Compression.Algorithm.class),
-        anyBoolean(), anyBoolean(), anyBoolean(), anyBoolean())).thenAnswer(writers);
-    when(store.getComparator()).thenReturn(CellComparator.COMPARATOR);
-
-    return new StripeCompactor(conf, store) {
-      @Override
-      protected InternalScanner createScanner(Store store, List<StoreFileScanner> scanners,
-          long smallestReadPoint, long earliestPutTs, byte[] dropDeletesFromRow,
-          byte[] dropDeletesToRow) throws IOException {
-        return scanner;
-      }
-
-      @Override
-      protected InternalScanner createScanner(Store store, List<StoreFileScanner> scanners,
-          ScanType scanType, long smallestReadPoint, long earliestPutTs) throws IOException {
-        return scanner;
-      }
-    };
-  }
-
-  private static CompactionRequest createDummyRequest() throws Exception {
-    // "Files" are totally unused, it's Scanner class below that gives compactor fake KVs.
-    // But compaction depends on everything under the sun, so stub everything with dummies.
-    StoreFile sf = mock(StoreFile.class);
-    StoreFile.Reader r = mock(StoreFile.Reader.class);
-    when(r.length()).thenReturn(1L);
-    when(r.getBloomFilterType()).thenReturn(BloomType.NONE);
-    when(r.getHFileReader()).thenReturn(mock(HFile.Reader.class));
-    when(r.getStoreFileScanner(anyBoolean(), anyBoolean(), anyBoolean(), anyLong()))
-      .thenReturn(mock(StoreFileScanner.class));
-    when(sf.getReader()).thenReturn(r);
-    when(sf.createReader()).thenReturn(r);
-    when(sf.createReader(anyBoolean())).thenReturn(r);
-    return new CompactionRequest(Arrays.asList(sf));
-  }
-
-  private static class Scanner implements InternalScanner {
-    private final ArrayList<KeyValue> kvs;
-    public Scanner(KeyValue... kvs) {
-      this.kvs = new ArrayList<KeyValue>(Arrays.asList(kvs));
-    }
-
-    @Override
-    public boolean next(List<Cell> results) throws IOException {
-      if (kvs.isEmpty()) return false;
-      results.add(kvs.remove(0));
-      return !kvs.isEmpty();
-    }
-
-    @Override
-    public boolean next(List<Cell> result, ScannerContext scannerContext)
-        throws IOException {
-      return next(result);
-    }
-
-    @Override
-    public void close() throws IOException {}
-  }
-
-  // StoreFile.Writer has private ctor and is unwieldy, so this has to be convoluted.
-  public static class StoreFileWritersCapture implements
-    Answer<StoreFile.Writer>, StripeMultiFileWriter.WriterFactory {
-    public static class Writer {
-      public ArrayList<KeyValue> kvs = new ArrayList<KeyValue>();
-      public TreeMap<byte[], byte[]> data = new TreeMap<byte[], byte[]>(Bytes.BYTES_COMPARATOR);
-    }
-
-    private List<Writer> writers = new ArrayList<Writer>();
-
-    @Override
-    public StoreFile.Writer createWriter() throws IOException {
-      final Writer realWriter = new Writer();
-      writers.add(realWriter);
-      StoreFile.Writer writer = mock(StoreFile.Writer.class);
-      doAnswer(new Answer<Object>() {
-        public Object answer(InvocationOnMock invocation) {
-          return realWriter.kvs.add((KeyValue)invocation.getArguments()[0]);
-        }}).when(writer).append(any(KeyValue.class));
-      doAnswer(new Answer<Object>() {
-        public Object answer(InvocationOnMock invocation) {
-          Object[] args = invocation.getArguments();
-          return realWriter.data.put((byte[])args[0], (byte[])args[1]);
-        }}).when(writer).appendFileInfo(any(byte[].class), any(byte[].class));
-      return writer;
-    }
-
-    @Override
-    public StoreFile.Writer answer(InvocationOnMock invocation) throws Throwable {
-      return createWriter();
-    }
-
-    public void verifyKvs(KeyValue[][] kvss, boolean allFiles, boolean requireMetadata) {
-      if (allFiles) {
-        assertEquals(kvss.length, writers.size());
-      }
-      int skippedWriters = 0;
-      for (int i = 0; i < kvss.length; ++i) {
-        KeyValue[] kvs = kvss[i];
-        if (kvs != null) {
-          Writer w = writers.get(i - skippedWriters);
-          if (requireMetadata) {
-            assertNotNull(w.data.get(STRIPE_START_KEY));
-            assertNotNull(w.data.get(STRIPE_END_KEY));
-          } else {
-            assertNull(w.data.get(STRIPE_START_KEY));
-            assertNull(w.data.get(STRIPE_END_KEY));
-          }
-          assertEquals(kvs.length, w.kvs.size());
-          for (int j = 0; j < kvs.length; ++j) {
-            assertEquals(kvs[j], w.kvs.get(j));
-          }
-        } else {
-          assertFalse(allFiles);
-          ++skippedWriters;
-        }
-      }
-    }
-
-    public void verifyBoundaries(byte[][] boundaries) {
-      assertEquals(boundaries.length - 1, writers.size());
-      for (int i = 0; i < writers.size(); ++i) {
-        assertArrayEquals("i = " + i, boundaries[i], writers.get(i).data.get(STRIPE_START_KEY));
-        assertArrayEquals("i = " + i, boundaries[i + 1], writers.get(i).data.get(STRIPE_END_KEY));
-      }
-    }
-  }
-}


[24/50] [abbrv] hbase git commit: HBASE-15475 Allow TimestampsFilter to provide a seek hint

Posted by sy...@apache.org.
HBASE-15475 Allow TimestampsFilter to provide a seek hint

Summary:
Allow TestTimestampFilterSeekHint to provide a seek next hint.
This can be incorrect as it might skip deletes. However it can
make things much much faster.

Test Plan: Added a unit test.

Differential Revision: https://reviews.facebook.net/D55617


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

Branch: refs/heads/hbase-12439
Commit: 925c18596928bc985946ee2f24136a4b34fd21f5
Parents: d7a4499
Author: Elliott Clark <ec...@apache.org>
Authored: Wed Mar 16 21:14:55 2016 -0700
Committer: Elliott Clark <ec...@apache.org>
Committed: Wed Mar 23 18:56:52 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/filter/TimestampsFilter.java   |  73 ++++++++++--
 .../hbase/protobuf/generated/ClientProtos.java  |  34 +++---
 .../hbase/protobuf/generated/FilterProtos.java  | 117 +++++++++++++++++--
 hbase-protocol/src/main/protobuf/Filter.proto   |   1 +
 .../apache/hadoop/hbase/io/hfile/TestHFile.java |  20 ++++
 .../TestTimestampFilterSeekHint.java            | 106 +++++++++++++++++
 6 files changed, 311 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/925c1859/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java
index 0ce0447..f0e5afe 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java
@@ -23,6 +23,7 @@ import java.util.List;
 import java.util.TreeSet;
 
 import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
@@ -44,6 +45,7 @@ import com.google.protobuf.InvalidProtocolBufferException;
 @InterfaceStability.Stable
 public class TimestampsFilter extends FilterBase {
 
+  private final boolean canHint;
   TreeSet<Long> timestamps;
   private static final int MAX_LOG_TIMESTAMPS = 5;
 
@@ -52,16 +54,29 @@ public class TimestampsFilter extends FilterBase {
   long minTimeStamp = Long.MAX_VALUE;
 
   /**
+   * Constructor for filter that retains only the specified timestamps in the list.
+   * @param timestamps
+   */
+  public TimestampsFilter(List<Long> timestamps) {
+    this(timestamps, false);
+  }
+
+  /**
    * Constructor for filter that retains only those
    * cells whose timestamp (version) is in the specified
    * list of timestamps.
    *
-   * @param timestamps
+   * @param timestamps list of timestamps that are wanted.
+   * @param canHint should the filter provide a seek hint? This can skip
+   *                past delete tombstones, so it should only be used when that
+   *                is not an issue ( no deletes, or don't care if data
+   *                becomes visible)
    */
-  public TimestampsFilter(List<Long> timestamps) {
+  public TimestampsFilter(List<Long> timestamps, boolean canHint) {
     for (Long timestamp : timestamps) {
       Preconditions.checkArgument(timestamp >= 0, "must be positive %s", timestamp);
     }
+    this.canHint = canHint;
     this.timestamps = new TreeSet<Long>(timestamps);
     init();
   }
@@ -104,7 +119,41 @@ public class TimestampsFilter extends FilterBase {
       // to be lesser than all of the other values.
       return ReturnCode.NEXT_COL;
     }
-    return ReturnCode.SKIP;
+    return canHint ? ReturnCode.SEEK_NEXT_USING_HINT : ReturnCode.SKIP;
+  }
+
+
+  /**
+   * Pick the next cell that the scanner should seek to. Since this can skip any number of cells
+   * any of which can be a delete this can resurect old data.
+   *
+   * The method will only be used if canHint was set to true while creating the filter.
+   *
+   * @throws IOException This will never happen.
+   */
+  public Cell getNextCellHint(Cell currentCell) throws IOException {
+    if (!canHint) {
+      return null;
+    }
+
+    Long nextTimestampObject = timestamps.lower(currentCell.getTimestamp());
+
+    if (nextTimestampObject == null) {
+      // This should only happen if the current column's
+      // timestamp is below the last one in the list.
+      //
+      // It should never happen as the filterKeyValue should return NEXT_COL
+      // but it's always better to be extra safe and protect against future
+      // behavioral changes.
+
+      return CellUtil.createLastOnRowCol(currentCell);
+    }
+
+    // Since we know the nextTimestampObject isn't null here there must still be
+    // timestamps that can be included. Cast the Long to a long and return the
+    // a cell with the current row/cf/col and the next found timestamp.
+    long nextTimestamp = nextTimestampObject;
+    return CellUtil.createFirstOnRowColTS(currentCell, nextTimestamp);
   }
 
   public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
@@ -119,28 +168,30 @@ public class TimestampsFilter extends FilterBase {
   /**
    * @return The filter serialized using pb
    */
-  public byte [] toByteArray() {
+  public byte[] toByteArray() {
     FilterProtos.TimestampsFilter.Builder builder =
-      FilterProtos.TimestampsFilter.newBuilder();
+        FilterProtos.TimestampsFilter.newBuilder();
     builder.addAllTimestamps(this.timestamps);
+    builder.setCanHint(canHint);
     return builder.build().toByteArray();
   }
 
   /**
    * @param pbBytes A pb serialized {@link TimestampsFilter} instance
+   *
    * @return An instance of {@link TimestampsFilter} made from <code>bytes</code>
-   * @throws DeserializationException
    * @see #toByteArray
    */
-  public static TimestampsFilter parseFrom(final byte [] pbBytes)
-  throws DeserializationException {
+  public static TimestampsFilter parseFrom(final byte[] pbBytes)
+      throws DeserializationException {
     FilterProtos.TimestampsFilter proto;
     try {
       proto = FilterProtos.TimestampsFilter.parseFrom(pbBytes);
     } catch (InvalidProtocolBufferException e) {
       throw new DeserializationException(e);
     }
-    return new TimestampsFilter(proto.getTimestampsList());
+    return new TimestampsFilter(proto.getTimestampsList(),
+        proto.hasCanHint() && proto.getCanHint());
   }
 
   /**
@@ -176,7 +227,7 @@ public class TimestampsFilter extends FilterBase {
       }
     }
 
-    return String.format("%s (%d/%d): [%s]", this.getClass().getSimpleName(),
-        count, this.timestamps.size(), tsList.toString());
+    return String.format("%s (%d/%d): [%s] canHint: [%b]", this.getClass().getSimpleName(),
+        count, this.timestamps.size(), tsList.toString(), canHint);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/925c1859/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java
index 0a9922c..4deab19 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java
@@ -29034,7 +29034,7 @@ public final class ClientProtos {
     /**
      * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
      */
-    java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier>
+    java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier> 
         getRegionList();
     /**
      * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
@@ -29047,7 +29047,7 @@ public final class ClientProtos {
     /**
      * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
      */
-    java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>
+    java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
         getRegionOrBuilderList();
     /**
      * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
@@ -29059,7 +29059,7 @@ public final class ClientProtos {
     /**
      * <code>repeated .hbase.pb.RegionLoadStats stat = 2;</code>
      */
-    java.util.List<org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionLoadStats>
+    java.util.List<org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionLoadStats> 
         getStatList();
     /**
      * <code>repeated .hbase.pb.RegionLoadStats stat = 2;</code>
@@ -29072,7 +29072,7 @@ public final class ClientProtos {
     /**
      * <code>repeated .hbase.pb.RegionLoadStats stat = 2;</code>
      */
-    java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionLoadStatsOrBuilder>
+    java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionLoadStatsOrBuilder> 
         getStatOrBuilderList();
     /**
      * <code>repeated .hbase.pb.RegionLoadStats stat = 2;</code>
@@ -29204,7 +29204,7 @@ public final class ClientProtos {
     /**
      * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
      */
-    public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>
+    public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
         getRegionOrBuilderList() {
       return region_;
     }
@@ -29240,7 +29240,7 @@ public final class ClientProtos {
     /**
      * <code>repeated .hbase.pb.RegionLoadStats stat = 2;</code>
      */
-    public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionLoadStatsOrBuilder>
+    public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionLoadStatsOrBuilder> 
         getStatOrBuilderList() {
       return stat_;
     }
@@ -29558,7 +29558,7 @@ public final class ClientProtos {
               regionBuilder_ = null;
               region_ = other.region_;
               bitField0_ = (bitField0_ & ~0x00000001);
-              regionBuilder_ =
+              regionBuilder_ = 
                 com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
                    getRegionFieldBuilder() : null;
             } else {
@@ -29584,7 +29584,7 @@ public final class ClientProtos {
               statBuilder_ = null;
               stat_ = other.stat_;
               bitField0_ = (bitField0_ & ~0x00000002);
-              statBuilder_ =
+              statBuilder_ = 
                 com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
                    getStatFieldBuilder() : null;
             } else {
@@ -29599,7 +29599,7 @@ public final class ClientProtos {
       public final boolean isInitialized() {
         for (int i = 0; i < getRegionCount(); i++) {
           if (!getRegion(i).isInitialized()) {
-
+            
             return false;
           }
         }
@@ -29820,7 +29820,7 @@ public final class ClientProtos {
       /**
        * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
        */
-      public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>
+      public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
            getRegionOrBuilderList() {
         if (regionBuilder_ != null) {
           return regionBuilder_.getMessageOrBuilderList();
@@ -29846,12 +29846,12 @@ public final class ClientProtos {
       /**
        * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
        */
-      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Builder>
+      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Builder> 
            getRegionBuilderList() {
         return getRegionFieldBuilder().getBuilderList();
       }
       private com.google.protobuf.RepeatedFieldBuilder<
-          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
           getRegionFieldBuilder() {
         if (regionBuilder_ == null) {
           regionBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
@@ -30060,7 +30060,7 @@ public final class ClientProtos {
       /**
        * <code>repeated .hbase.pb.RegionLoadStats stat = 2;</code>
        */
-      public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionLoadStatsOrBuilder>
+      public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionLoadStatsOrBuilder> 
            getStatOrBuilderList() {
         if (statBuilder_ != null) {
           return statBuilder_.getMessageOrBuilderList();
@@ -30086,12 +30086,12 @@ public final class ClientProtos {
       /**
        * <code>repeated .hbase.pb.RegionLoadStats stat = 2;</code>
        */
-      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionLoadStats.Builder>
+      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionLoadStats.Builder> 
            getStatBuilderList() {
         return getStatFieldBuilder().getBuilderList();
       }
       private com.google.protobuf.RepeatedFieldBuilder<
-          org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionLoadStats, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionLoadStats.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionLoadStatsOrBuilder>
+          org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionLoadStats, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionLoadStats.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionLoadStatsOrBuilder> 
           getStatFieldBuilder() {
         if (statBuilder_ == null) {
           statBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
@@ -34220,7 +34220,7 @@ public final class ClientProtos {
         }
         if (hasRegionStatistics()) {
           if (!getRegionStatistics().isInitialized()) {
-
+            
             return false;
           }
         }
@@ -34639,7 +34639,7 @@ public final class ClientProtos {
        * <code>optional .hbase.pb.MultiRegionLoadStats regionStatistics = 3;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRegionLoadStats, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRegionLoadStats.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRegionLoadStatsOrBuilder>
+          org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRegionLoadStats, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRegionLoadStats.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRegionLoadStatsOrBuilder> 
           getRegionStatisticsFieldBuilder() {
         if (regionStatisticsBuilder_ == null) {
           regionStatisticsBuilder_ = new com.google.protobuf.SingleFieldBuilder<

http://git-wip-us.apache.org/repos/asf/hbase/blob/925c1859/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 e558371..5b995a4 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
@@ -13923,6 +13923,16 @@ public final class FilterProtos {
      * <code>repeated int64 timestamps = 1 [packed = true];</code>
      */
     long getTimestamps(int index);
+
+    // optional bool can_hint = 2;
+    /**
+     * <code>optional bool can_hint = 2;</code>
+     */
+    boolean hasCanHint();
+    /**
+     * <code>optional bool can_hint = 2;</code>
+     */
+    boolean getCanHint();
   }
   /**
    * Protobuf type {@code hbase.pb.TimestampsFilter}
@@ -13996,6 +14006,11 @@ public final class FilterProtos {
               input.popLimit(limit);
               break;
             }
+            case 16: {
+              bitField0_ |= 0x00000001;
+              canHint_ = input.readBool();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -14038,6 +14053,7 @@ public final class FilterProtos {
       return PARSER;
     }
 
+    private int bitField0_;
     // repeated int64 timestamps = 1 [packed = true];
     public static final int TIMESTAMPS_FIELD_NUMBER = 1;
     private java.util.List<java.lang.Long> timestamps_;
@@ -14062,8 +14078,25 @@ public final class FilterProtos {
     }
     private int timestampsMemoizedSerializedSize = -1;
 
+    // optional bool can_hint = 2;
+    public static final int CAN_HINT_FIELD_NUMBER = 2;
+    private boolean canHint_;
+    /**
+     * <code>optional bool can_hint = 2;</code>
+     */
+    public boolean hasCanHint() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional bool can_hint = 2;</code>
+     */
+    public boolean getCanHint() {
+      return canHint_;
+    }
+
     private void initFields() {
       timestamps_ = java.util.Collections.emptyList();
+      canHint_ = false;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -14084,6 +14117,9 @@ public final class FilterProtos {
       for (int i = 0; i < timestamps_.size(); i++) {
         output.writeInt64NoTag(timestamps_.get(i));
       }
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeBool(2, canHint_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -14107,6 +14143,10 @@ public final class FilterProtos {
         }
         timestampsMemoizedSerializedSize = dataSize;
       }
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(2, canHint_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -14132,6 +14172,11 @@ public final class FilterProtos {
       boolean result = true;
       result = result && getTimestampsList()
           .equals(other.getTimestampsList());
+      result = result && (hasCanHint() == other.hasCanHint());
+      if (hasCanHint()) {
+        result = result && (getCanHint()
+            == other.getCanHint());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -14149,6 +14194,10 @@ public final class FilterProtos {
         hash = (37 * hash) + TIMESTAMPS_FIELD_NUMBER;
         hash = (53 * hash) + getTimestampsList().hashCode();
       }
+      if (hasCanHint()) {
+        hash = (37 * hash) + CAN_HINT_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getCanHint());
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -14260,6 +14309,8 @@ public final class FilterProtos {
         super.clear();
         timestamps_ = java.util.Collections.emptyList();
         bitField0_ = (bitField0_ & ~0x00000001);
+        canHint_ = false;
+        bitField0_ = (bitField0_ & ~0x00000002);
         return this;
       }
 
@@ -14287,11 +14338,17 @@ public final class FilterProtos {
       public org.apache.hadoop.hbase.protobuf.generated.FilterProtos.TimestampsFilter buildPartial() {
         org.apache.hadoop.hbase.protobuf.generated.FilterProtos.TimestampsFilter result = new org.apache.hadoop.hbase.protobuf.generated.FilterProtos.TimestampsFilter(this);
         int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
         if (((bitField0_ & 0x00000001) == 0x00000001)) {
           timestamps_ = java.util.Collections.unmodifiableList(timestamps_);
           bitField0_ = (bitField0_ & ~0x00000001);
         }
         result.timestamps_ = timestamps_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.canHint_ = canHint_;
+        result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
       }
@@ -14317,6 +14374,9 @@ public final class FilterProtos {
           }
           onChanged();
         }
+        if (other.hasCanHint()) {
+          setCanHint(other.getCanHint());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -14410,6 +14470,39 @@ public final class FilterProtos {
         return this;
       }
 
+      // optional bool can_hint = 2;
+      private boolean canHint_ ;
+      /**
+       * <code>optional bool can_hint = 2;</code>
+       */
+      public boolean hasCanHint() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>optional bool can_hint = 2;</code>
+       */
+      public boolean getCanHint() {
+        return canHint_;
+      }
+      /**
+       * <code>optional bool can_hint = 2;</code>
+       */
+      public Builder setCanHint(boolean value) {
+        bitField0_ |= 0x00000002;
+        canHint_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bool can_hint = 2;</code>
+       */
+      public Builder clearCanHint() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        canHint_ = false;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:hbase.pb.TimestampsFilter)
     }
 
@@ -17503,18 +17596,18 @@ public final class FilterProtos {
       "ompareType\022(\n\ncomparator\030\004 \002(\0132\024.hbase.p" +
       "b.Comparator\022\031\n\021filter_if_missing\030\005 \001(\010\022" +
       "\033\n\023latest_version_only\030\006 \001(\010\".\n\nSkipFilt" +
-      "er\022 \n\006filter\030\001 \002(\0132\020.hbase.pb.Filter\"*\n\020" +
+      "er\022 \n\006filter\030\001 \002(\0132\020.hbase.pb.Filter\"<\n\020" +
       "TimestampsFilter\022\026\n\ntimestamps\030\001 \003(\003B\002\020\001" +
-      "\">\n\013ValueFilter\022/\n\016compare_filter\030\001 \002(\0132" +
-      "\027.hbase.pb.CompareFilter\"4\n\020WhileMatchFi" +
-      "lter\022 \n\006filter\030\001 \002(\0132\020.hbase.pb.Filter\"\021" +
-      "\n\017FilterAllFilter\"h\n\010RowRange\022\021\n\tstart_r",
-      "ow\030\001 \001(\014\022\033\n\023start_row_inclusive\030\002 \001(\010\022\020\n" +
-      "\010stop_row\030\003 \001(\014\022\032\n\022stop_row_inclusive\030\004 " +
-      "\001(\010\"A\n\023MultiRowRangeFilter\022*\n\016row_range_" +
-      "list\030\001 \003(\0132\022.hbase.pb.RowRangeBB\n*org.ap" +
-      "ache.hadoop.hbase.protobuf.generatedB\014Fi" +
-      "lterProtosH\001\210\001\001\240\001\001"
+      "\022\020\n\010can_hint\030\002 \001(\010\">\n\013ValueFilter\022/\n\016com" +
+      "pare_filter\030\001 \002(\0132\027.hbase.pb.CompareFilt" +
+      "er\"4\n\020WhileMatchFilter\022 \n\006filter\030\001 \002(\0132\020" +
+      ".hbase.pb.Filter\"\021\n\017FilterAllFilter\"h\n\010R",
+      "owRange\022\021\n\tstart_row\030\001 \001(\014\022\033\n\023start_row_" +
+      "inclusive\030\002 \001(\010\022\020\n\010stop_row\030\003 \001(\014\022\032\n\022sto" +
+      "p_row_inclusive\030\004 \001(\010\"A\n\023MultiRowRangeFi" +
+      "lter\022*\n\016row_range_list\030\001 \003(\0132\022.hbase.pb." +
+      "RowRangeBB\n*org.apache.hadoop.hbase.prot" +
+      "obuf.generatedB\014FilterProtosH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -17670,7 +17763,7 @@ public final class FilterProtos {
           internal_static_hbase_pb_TimestampsFilter_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_TimestampsFilter_descriptor,
-              new java.lang.String[] { "Timestamps", });
+              new java.lang.String[] { "Timestamps", "CanHint", });
           internal_static_hbase_pb_ValueFilter_descriptor =
             getDescriptor().getMessageTypes().get(25);
           internal_static_hbase_pb_ValueFilter_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/hbase/blob/925c1859/hbase-protocol/src/main/protobuf/Filter.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/Filter.proto b/hbase-protocol/src/main/protobuf/Filter.proto
index 67d5717..1fa6697 100644
--- a/hbase-protocol/src/main/protobuf/Filter.proto
+++ b/hbase-protocol/src/main/protobuf/Filter.proto
@@ -146,6 +146,7 @@ message SkipFilter {
 
 message TimestampsFilter {
   repeated int64 timestamps = 1 [packed=true];
+  optional bool can_hint = 2;
 }
 
 message ValueFilter {

http://git-wip-us.apache.org/repos/asf/hbase/blob/925c1859/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
index 66fb49c..8c080c1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
@@ -397,6 +397,7 @@ public class TestHFile  {
   /**
    * Make sure the ordinals for our compression algorithms do not change on us.
    */
+  @Test
   public void testCompressionOrdinance() {
     assertTrue(Compression.Algorithm.LZO.ordinal() == 0);
     assertTrue(Compression.Algorithm.GZ.ordinal() == 1);
@@ -406,6 +407,25 @@ public class TestHFile  {
   }
 
   @Test
+  public void testShortMidpointSameQual() {
+    Cell left = CellUtil.createCell(Bytes.toBytes("a"),
+        Bytes.toBytes("a"),
+        Bytes.toBytes("a"),
+        9,
+        KeyValue.Type.Maximum.getCode(),
+        HConstants.EMPTY_BYTE_ARRAY);
+    Cell right = CellUtil.createCell(Bytes.toBytes("a"),
+        Bytes.toBytes("a"),
+        Bytes.toBytes("a"),
+        11,
+        KeyValue.Type.Maximum.getCode(),
+        HConstants.EMPTY_BYTE_ARRAY);
+    Cell mid = HFileWriterImpl.getMidpoint(CellComparator.COMPARATOR, left, right);
+    assertTrue(CellComparator.COMPARATOR.compareKeyIgnoresMvcc(left, mid) <= 0);
+    assertTrue(CellComparator.COMPARATOR.compareKeyIgnoresMvcc(mid, right) == 0);
+  }
+
+  @Test
   public void testGetShortMidpoint() {
     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"));

http://git-wip-us.apache.org/repos/asf/hbase/blob/925c1859/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTimestampFilterSeekHint.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTimestampFilterSeekHint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTimestampFilterSeekHint.java
new file mode 100644
index 0000000..58f0c56
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTimestampFilterSeekHint.java
@@ -0,0 +1,106 @@
+/*
+ *
+ * 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 com.google.common.collect.ImmutableList;
+import org.apache.commons.lang.RandomStringUtils;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.filter.TimestampsFilter;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import static org.junit.Assert.assertTrue;
+
+@Category({RegionServerTests.class, LargeTests.class})
+public class TestTimestampFilterSeekHint {
+
+  private final static HBaseTestingUtility TEST_UTIL = HBaseTestingUtility.createLocalHTU();
+  private final static String RK = "myRK";
+  private final static byte[] RK_BYTES = Bytes.toBytes(RK);
+
+  private final static String FAMILY = "D";
+  private final static byte[] FAMILY_BYTES = Bytes.toBytes(FAMILY);
+
+  private final static String QUAL = "0";
+  private final static byte[] QUAL_BYTES = Bytes.toBytes(QUAL);
+
+  public static final int MAX_VERSIONS = 50000;
+  private HRegion region;
+  private int regionCount = 0;
+
+  @Test
+  public void testGetSeek() throws IOException {
+    StoreFileScanner.instrument();
+    prepareRegion();
+
+    Get g = new Get(RK_BYTES);
+    final TimestampsFilter timestampsFilter = new TimestampsFilter(ImmutableList.of(5L), true);
+    g.setFilter(timestampsFilter);
+    final long initialSeekCount = StoreFileScanner.getSeekCount();
+    region.get(g);
+    final long finalSeekCount = StoreFileScanner.getSeekCount();
+
+    /*
+      Make sure there's more than one.
+      Aka one seek to get to the row, and one to get to the time.
+    */
+    assertTrue(finalSeekCount >= initialSeekCount + 3 );
+  }
+
+  @Test
+  public void testGetDoesntSeekWithNoHint() throws IOException {
+    StoreFileScanner.instrument();
+    prepareRegion();
+
+    Get g = new Get(RK_BYTES);
+    g.setFilter(new TimestampsFilter(ImmutableList.of(5L)));
+    final long initialSeekCount = StoreFileScanner.getSeekCount();
+    region.get(g);
+    final long finalSeekCount = StoreFileScanner.getSeekCount();
+
+    assertTrue(finalSeekCount >= initialSeekCount );
+    assertTrue(finalSeekCount < initialSeekCount + 3);
+  }
+
+  @Before
+  public void prepareRegion() throws IOException {
+    region =
+        TEST_UTIL.createTestRegion("TestTimestampFilterSeekHint" + regionCount++,
+            new HColumnDescriptor(FAMILY)
+                .setBlocksize(1024)
+                .setMaxVersions(MAX_VERSIONS)
+        );
+
+    for (long i = 0; i <MAX_VERSIONS - 2; i++) {
+      Put p = new Put(RK_BYTES, i);
+      p.addColumn(FAMILY_BYTES, QUAL_BYTES, Bytes.toBytes(RandomStringUtils.randomAlphabetic(255)));
+      region.put(p);
+    }
+    region.flush(true);
+  }
+}


[15/50] [abbrv] hbase git commit: Revert "HBASE-15477 Purge 'next block header' from cached blocks" Overcommit. Revert.

Posted by sy...@apache.org.
Revert "HBASE-15477 Purge 'next block header' from cached blocks"
Overcommit. Revert.

This reverts commit ef94b552399d75c0c16aca0109a39b66f15af892.


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

Branch: refs/heads/hbase-12439
Commit: c49d0caf53b27c0c22481e074c807a5ad4753eda
Parents: 54a543d
Author: stack <st...@apache.org>
Authored: Tue Mar 22 18:38:29 2016 -0700
Committer: stack <st...@apache.org>
Committed: Tue Mar 22 18:38:29 2016 -0700

----------------------------------------------------------------------
 .../hbase/regionserver/KeyValueScanner.java.rej    | 17 -----------------
 1 file changed, 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c49d0caf/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java.rej
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java.rej b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java.rej
deleted file mode 100644
index 4e3ebd4..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java.rej
+++ /dev/null
@@ -1,17 +0,0 @@
-diff a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java	(rejected hunks)
-@@ -29,7 +30,7 @@ import org.apache.hadoop.hbase.client.Scan;
-  * Scanner that returns the next KeyValue.
-  */
- @InterfaceAudience.Private
--public interface KeyValueScanner extends Shipper {
-+public interface KeyValueScanner extends Shipper, Closeable {
-   /**
-    * The byte array represents for NO_NEXT_INDEXED_KEY;
-    * The actual value is irrelevant because this is always compared by reference.
-@@ -165,4 +167,4 @@ public interface KeyValueScanner extends Shipper {
-    * if known, or null otherwise
-    */
-   public Cell getNextIndexedKey();
--}
-+}
-\ No newline at end of file


[25/50] [abbrv] hbase git commit: HBASE-15520 Fix broken TestAsyncIPC

Posted by sy...@apache.org.
HBASE-15520 Fix broken TestAsyncIPC


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

Branch: refs/heads/hbase-12439
Commit: fd5c0934b60664ecdde21a994910953339c7060d
Parents: 925c185
Author: zhangduo <zh...@apache.org>
Authored: Thu Mar 24 10:24:34 2016 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Mar 24 10:46:44 2016 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/ipc/AsyncRpcChannel.java       | 271 +++++++++----------
 .../hbase/ipc/AsyncServerResponseHandler.java   | 122 +++++----
 .../hadoop/hbase/ipc/AbstractTestIPC.java       |   3 +-
 3 files changed, 188 insertions(+), 208 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/fd5c0934/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcChannel.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcChannel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcChannel.java
index 787aa47..53eb824 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcChannel.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcChannel.java
@@ -17,18 +17,6 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
-import io.netty.bootstrap.Bootstrap;
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.ByteBufOutputStream;
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelFuture;
-import io.netty.channel.ChannelFutureListener;
-import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
-import io.netty.util.Timeout;
-import io.netty.util.TimerTask;
-import io.netty.util.concurrent.GenericFutureListener;
-import io.netty.util.concurrent.Promise;
-
 import java.io.IOException;
 import java.net.ConnectException;
 import java.net.InetSocketAddress;
@@ -52,6 +40,7 @@ import org.apache.hadoop.hbase.client.MetricsConnection;
 import org.apache.hadoop.hbase.exceptions.ConnectionClosingException;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
+import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos.TokenIdentifier.Kind;
 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos;
 import org.apache.hadoop.hbase.protobuf.generated.TracingProtos;
 import org.apache.hadoop.hbase.security.AuthMethod;
@@ -75,6 +64,18 @@ import com.google.protobuf.Descriptors;
 import com.google.protobuf.Message;
 import com.google.protobuf.RpcCallback;
 
+import io.netty.bootstrap.Bootstrap;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufOutputStream;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelFutureListener;
+import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
+import io.netty.util.Timeout;
+import io.netty.util.TimerTask;
+import io.netty.util.concurrent.GenericFutureListener;
+import io.netty.util.concurrent.Promise;
+
 /**
  * Netty RPC channel
  */
@@ -84,12 +85,12 @@ public class AsyncRpcChannel {
 
   private static final int MAX_SASL_RETRIES = 5;
 
-  protected final static Map<AuthenticationProtos.TokenIdentifier.Kind, TokenSelector<? extends
-      TokenIdentifier>> tokenHandlers = new HashMap<>();
+  protected final static Map<Kind, TokenSelector<? extends TokenIdentifier>> TOKEN_HANDDLERS
+    = new HashMap<>();
 
   static {
-    tokenHandlers.put(AuthenticationProtos.TokenIdentifier.Kind.HBASE_AUTH_TOKEN,
-        new AuthenticationTokenSelector());
+    TOKEN_HANDDLERS.put(AuthenticationProtos.TokenIdentifier.Kind.HBASE_AUTH_TOKEN,
+      new AuthenticationTokenSelector());
   }
 
   final AsyncRpcClient client;
@@ -111,7 +112,6 @@ public class AsyncRpcChannel {
   private Token<? extends TokenIdentifier> token;
   private String serverPrincipal;
 
-
   // NOTE: closed and connected flags below are only changed when a lock on pendingCalls
   private final Map<Integer, AsyncCall> pendingCalls = new HashMap<Integer, AsyncCall>();
   private boolean connected = false;
@@ -128,15 +128,14 @@ public class AsyncRpcChannel {
 
   /**
    * Constructor for netty RPC channel
-   *
    * @param bootstrap to construct channel on
-   * @param client    to connect with
+   * @param client to connect with
    * @param ticket of user which uses connection
    * @param serviceName name of service to connect to
    * @param address to connect to
    */
-  public AsyncRpcChannel(Bootstrap bootstrap, final AsyncRpcClient client, User ticket, String
-      serviceName, InetSocketAddress address) {
+  public AsyncRpcChannel(Bootstrap bootstrap, final AsyncRpcClient client, User ticket,
+      String serviceName, InetSocketAddress address) {
     this.client = client;
 
     this.ticket = ticket;
@@ -145,16 +144,12 @@ public class AsyncRpcChannel {
 
     this.channel = connect(bootstrap).channel();
 
-    name = ("IPC Client (" + channel.hashCode() + ") to " +
-        address.toString() +
-        ((ticket == null) ?
-            " from unknown user" :
-            (" from " + ticket.getName())));
+    name = ("IPC Client (" + channel.hashCode() + ") to " + address.toString()
+        + ((ticket == null) ? " from unknown user" : (" from " + ticket.getName())));
   }
 
   /**
    * Connect to channel
-   *
    * @param bootstrap to connect to
    * @return future of connection
    */
@@ -209,12 +204,11 @@ public class AsyncRpcChannel {
 
   /**
    * Start HBase connection
-   *
    * @param ch channel to start connection on
    */
   private void startHBaseConnection(Channel ch) {
-    ch.pipeline()
-        .addLast("frameDecoder", new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 4));
+    ch.pipeline().addLast("frameDecoder",
+      new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 4));
     ch.pipeline().addLast(new AsyncServerResponseHandler(this));
     try {
       writeChannelHeader(ch).addListener(new GenericFutureListener<ChannelFuture>() {
@@ -248,7 +242,8 @@ public class AsyncRpcChannel {
   private SaslClientHandler getSaslHandler(final UserGroupInformation realTicket,
       final Bootstrap bootstrap) throws IOException {
     return new SaslClientHandler(realTicket, authMethod, token, serverPrincipal,
-        client.fallbackAllowed, client.conf.get("hbase.rpc.protection",
+        client.fallbackAllowed,
+        client.conf.get("hbase.rpc.protection",
           SaslUtil.QualityOfProtection.AUTHENTICATION.name().toLowerCase()),
         new SaslClientHandler.SaslExceptionHandler() {
           @Override
@@ -258,7 +253,7 @@ public class AsyncRpcChannel {
               handleSaslConnectionFailure(retryCount, cause, realTicket);
 
               retryOrClose(bootstrap, failureCounter++, random.nextInt(reloginMaxBackoff) + 1,
-                  cause);
+                cause);
             } catch (IOException | InterruptedException e) {
               close(e);
             }
@@ -273,13 +268,12 @@ public class AsyncRpcChannel {
 
   /**
    * Retry to connect or close
-   *
-   * @param bootstrap      to connect with
-   * @param failureCount   failure count
-   * @param e              exception of fail
+   * @param bootstrap to connect with
+   * @param failureCount failure count
+   * @param e exception of fail
    */
-  private void retryOrClose(final Bootstrap bootstrap, int failureCount,
-      long timeout, Throwable e) {
+  private void retryOrClose(final Bootstrap bootstrap, int failureCount, long timeout,
+      Throwable e) {
     if (failureCount < client.maxRetries) {
       client.newTimeout(new TimerTask() {
         @Override
@@ -303,9 +297,8 @@ public class AsyncRpcChannel {
   public Promise<Message> callMethod(final Descriptors.MethodDescriptor method,
       final PayloadCarryingRpcController controller, final Message request,
       final Message responsePrototype, MetricsConnection.CallStats callStats) {
-    final AsyncCall call =
-        new AsyncCall(channel.eventLoop(), client.callIdCnt.getAndIncrement(), method, request,
-            controller, responsePrototype, callStats);
+    final AsyncCall call = new AsyncCall(channel.eventLoop(), client.callIdCnt.getAndIncrement(),
+        method, request, controller, responsePrototype, callStats);
     controller.notifyOnCancel(new RpcCallback<Object>() {
       @Override
       public void run(Object parameter) {
@@ -331,9 +324,7 @@ public class AsyncRpcChannel {
       pendingCalls.put(call.id, call);
       // Add timeout for cleanup if none is present
       if (cleanupTimer == null && call.getRpcTimeout() > 0) {
-        cleanupTimer =
-            client.newTimeout(timeoutTask, call.getRpcTimeout(),
-              TimeUnit.MILLISECONDS);
+        cleanupTimer = client.newTimeout(timeoutTask, call.getRpcTimeout(), TimeUnit.MILLISECONDS);
       }
       if (!connected) {
         return call;
@@ -351,14 +342,13 @@ public class AsyncRpcChannel {
 
   /**
    * Write the channel header
-   *
    * @param channel to write to
    * @return future of write
    * @throws java.io.IOException on failure to write
    */
   private ChannelFuture writeChannelHeader(Channel channel) throws IOException {
-    RPCProtos.ConnectionHeader.Builder headerBuilder =
-        RPCProtos.ConnectionHeader.newBuilder().setServiceName(serviceName);
+    RPCProtos.ConnectionHeader.Builder headerBuilder = RPCProtos.ConnectionHeader.newBuilder()
+        .setServiceName(serviceName);
 
     RPCProtos.UserInformation userInfoPB = buildUserInfo(ticket.getUGI(), authMethod);
     if (userInfoPB != null) {
@@ -375,7 +365,6 @@ public class AsyncRpcChannel {
     headerBuilder.setVersionInfo(ProtobufUtil.getVersionInfo());
     RPCProtos.ConnectionHeader header = headerBuilder.build();
 
-
     int totalSize = IPCUtil.getTotalSizeWhenWrittenDelimited(header);
 
     ByteBuf b = channel.alloc().directBuffer(totalSize);
@@ -388,20 +377,19 @@ public class AsyncRpcChannel {
 
   /**
    * Write request to channel
-   *
-   * @param call    to write
+   * @param call to write
    */
   private void writeRequest(final AsyncCall call) {
     try {
       final RPCProtos.RequestHeader.Builder requestHeaderBuilder = RPCProtos.RequestHeader
           .newBuilder();
-      requestHeaderBuilder.setCallId(call.id)
-              .setMethodName(call.method.getName()).setRequestParam(call.param != null);
+      requestHeaderBuilder.setCallId(call.id).setMethodName(call.method.getName())
+          .setRequestParam(call.param != null);
 
       if (Trace.isTracing()) {
         Span s = Trace.currentSpan();
-        requestHeaderBuilder.setTraceInfo(TracingProtos.RPCTInfo.newBuilder().
-            setParentId(s.getSpanId()).setTraceId(s.getTraceId()));
+        requestHeaderBuilder.setTraceInfo(TracingProtos.RPCTInfo.newBuilder()
+            .setParentId(s.getSpanId()).setTraceId(s.getTraceId()));
       }
 
       ByteBuffer cellBlock = client.buildCellBlock(call.controller.cellScanner());
@@ -411,7 +399,7 @@ public class AsyncRpcChannel {
         cellBlockBuilder.setLength(cellBlock.limit());
         requestHeaderBuilder.setCellBlockMeta(cellBlockBuilder.build());
       }
-      // Only pass priority if there one.  Let zero be same as no priority.
+      // Only pass priority if there one. Let zero be same as no priority.
       if (call.controller.getPriority() != PayloadCarryingRpcController.PRIORITY_UNSET) {
         requestHeaderBuilder.setPriority(call.controller.getPriority());
       }
@@ -424,7 +412,7 @@ public class AsyncRpcChannel {
       }
 
       ByteBuf b = channel.alloc().directBuffer(4 + totalSize);
-      try(ByteBufOutputStream out = new ByteBufOutputStream(b)) {
+      try (ByteBufOutputStream out = new ByteBufOutputStream(b)) {
         call.callStats.setRequestSizeBytes(IPCUtil.write(out, rh, call.param, cellBlock));
       }
 
@@ -436,7 +424,6 @@ public class AsyncRpcChannel {
 
   /**
    * Set up server authorization
-   *
    * @throws java.io.IOException if auth setup failed
    */
   private void setupAuthorization() throws IOException {
@@ -447,10 +434,10 @@ public class AsyncRpcChannel {
     if (useSasl && securityInfo != null) {
       AuthenticationProtos.TokenIdentifier.Kind tokenKind = securityInfo.getTokenKind();
       if (tokenKind != null) {
-        TokenSelector<? extends TokenIdentifier> tokenSelector = tokenHandlers.get(tokenKind);
+        TokenSelector<? extends TokenIdentifier> tokenSelector = TOKEN_HANDDLERS.get(tokenKind);
         if (tokenSelector != null) {
-          token = tokenSelector
-              .selectToken(new Text(client.clusterId), ticket.getUGI().getTokens());
+          token = tokenSelector.selectToken(new Text(client.clusterId),
+            ticket.getUGI().getTokens());
         } else if (LOG.isDebugEnabled()) {
           LOG.debug("No token selector found for type " + tokenKind);
         }
@@ -460,7 +447,7 @@ public class AsyncRpcChannel {
         throw new IOException("Can't obtain server Kerberos config key from SecurityInfo");
       }
       this.serverPrincipal = SecurityUtil.getServerPrincipal(client.conf.get(serverKey),
-          address.getAddress().getCanonicalHostName().toLowerCase());
+        address.getAddress().getCanonicalHostName().toLowerCase());
       if (LOG.isDebugEnabled()) {
         LOG.debug("RPC Server Kerberos principal name for service=" + serviceName + " is "
             + serverPrincipal);
@@ -476,16 +463,15 @@ public class AsyncRpcChannel {
     }
 
     if (LOG.isDebugEnabled()) {
-      LOG.debug("Use " + authMethod + " authentication for service " + serviceName +
-          ", sasl=" + useSasl);
+      LOG.debug(
+        "Use " + authMethod + " authentication for service " + serviceName + ", sasl=" + useSasl);
     }
     reloginMaxBackoff = client.conf.getInt("hbase.security.relogin.maxbackoff", 5000);
   }
 
   /**
    * Build the user information
-   *
-   * @param ugi        User Group Information
+   * @param ugi User Group Information
    * @param authMethod Authorization method
    * @return UserInformation protobuf
    */
@@ -499,7 +485,7 @@ public class AsyncRpcChannel {
       // Send effective user for Kerberos auth
       userInfoPB.setEffectiveUser(ugi.getUserName());
     } else if (authMethod == AuthMethod.SIMPLE) {
-      //Send both effective user and real user for simple auth
+      // Send both effective user and real user for simple auth
       userInfoPB.setEffectiveUser(ugi.getUserName());
       if (ugi.getRealUser() != null) {
         userInfoPB.setRealUser(ugi.getRealUser().getUserName());
@@ -510,8 +496,7 @@ public class AsyncRpcChannel {
 
   /**
    * Create connection preamble
-   *
-   * @param byteBuf    to write to
+   * @param byteBuf to write to
    * @param authMethod to write
    */
   private void createPreamble(ByteBuf byteBuf, AuthMethod authMethod) {
@@ -520,53 +505,61 @@ public class AsyncRpcChannel {
     byteBuf.writeByte(authMethod.code);
   }
 
+  private void close0(Throwable e) {
+    List<AsyncCall> toCleanup;
+    synchronized (pendingCalls) {
+      if (closed) {
+        return;
+      }
+      closed = true;
+      toCleanup = new ArrayList<AsyncCall>(pendingCalls.values());
+      pendingCalls.clear();
+    }
+    IOException closeException = null;
+    if (e != null) {
+      if (e instanceof IOException) {
+        closeException = (IOException) e;
+      } else {
+        closeException = new IOException(e);
+      }
+    }
+    // log the info
+    if (LOG.isDebugEnabled() && closeException != null) {
+      LOG.debug(name + ": closing ipc connection to " + address, closeException);
+    }
+    if (cleanupTimer != null) {
+      cleanupTimer.cancel();
+      cleanupTimer = null;
+    }
+    for (AsyncCall call : toCleanup) {
+      call.setFailed(closeException != null ? closeException
+          : new ConnectionClosingException(
+              "Call id=" + call.id + " on server " + address + " aborted: connection is closing"));
+    }
+    channel.disconnect().addListener(ChannelFutureListener.CLOSE);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(name + ": closed");
+    }
+  }
+
   /**
    * Close connection
-   *
    * @param e exception on close
    */
   public void close(final Throwable e) {
     client.removeConnection(this);
 
     // Move closing from the requesting thread to the channel thread
-    channel.eventLoop().execute(new Runnable() {
-      @Override
-      public void run() {
-        List<AsyncCall> toCleanup;
-        synchronized (pendingCalls) {
-          if (closed) {
-            return;
-          }
-          closed = true;
-          toCleanup = new ArrayList<AsyncCall>(pendingCalls.values());
-          pendingCalls.clear();
-        }
-        IOException closeException = null;
-        if (e != null) {
-          if (e instanceof IOException) {
-            closeException = (IOException) e;
-          } else {
-            closeException = new IOException(e);
-          }
-        }
-        // log the info
-        if (LOG.isDebugEnabled() && closeException != null) {
-          LOG.debug(name + ": closing ipc connection to " + address, closeException);
-        }
-        if (cleanupTimer != null) {
-          cleanupTimer.cancel();
-          cleanupTimer = null;
-        }
-        for (AsyncCall call : toCleanup) {
-          call.setFailed(closeException != null ? closeException : new ConnectionClosingException(
-              "Call id=" + call.id + " on server " + address + " aborted: connection is closing"));
-        }
-        channel.disconnect().addListener(ChannelFutureListener.CLOSE);
-        if (LOG.isDebugEnabled()) {
-          LOG.debug(name + ": closed");
+    if (channel.eventLoop().inEventLoop()) {
+      close0(e);
+    } else {
+      channel.eventLoop().execute(new Runnable() {
+        @Override
+        public void run() {
+          close0(e);
         }
-      }
-    });
+      });
+    }
   }
 
   /**
@@ -592,9 +585,7 @@ public class AsyncRpcChannel {
         }
       }
       if (nextCleanupTaskDelay > 0) {
-        cleanupTimer =
-            client.newTimeout(timeoutTask, nextCleanupTaskDelay,
-              TimeUnit.MILLISECONDS);
+        cleanupTimer = client.newTimeout(timeoutTask, nextCleanupTaskDelay, TimeUnit.MILLISECONDS);
       } else {
         cleanupTimer = null;
       }
@@ -607,7 +598,6 @@ public class AsyncRpcChannel {
 
   /**
    * Check if the connection is alive
-   *
    * @return true if alive
    */
   public boolean isAlive() {
@@ -616,7 +606,6 @@ public class AsyncRpcChannel {
 
   /**
    * Check if user should authenticate over Kerberos
-   *
    * @return true if should be authenticated over Kerberos
    * @throws java.io.IOException on failure of check
    */
@@ -624,37 +613,31 @@ public class AsyncRpcChannel {
     UserGroupInformation loginUser = UserGroupInformation.getLoginUser();
     UserGroupInformation currentUser = UserGroupInformation.getCurrentUser();
     UserGroupInformation realUser = currentUser.getRealUser();
-    return authMethod == AuthMethod.KERBEROS &&
-        loginUser != null &&
-        //Make sure user logged in using Kerberos either keytab or TGT
-        loginUser.hasKerberosCredentials() &&
-        // relogin only in case it is the login user (e.g. JT)
-        // or superuser (like oozie).
-        (loginUser.equals(currentUser) || loginUser.equals(realUser));
+    return authMethod == AuthMethod.KERBEROS && loginUser != null &&
+      // Make sure user logged in using Kerberos either keytab or TGT
+      loginUser.hasKerberosCredentials() &&
+      // relogin only in case it is the login user (e.g. JT)
+      // or superuser (like oozie).
+      (loginUser.equals(currentUser) || loginUser.equals(realUser));
   }
 
   /**
-   * If multiple clients with the same principal try to connect
-   * to the same server at the same time, the server assumes a
-   * replay attack is in progress. This is a feature of kerberos.
-   * In order to work around this, what is done is that the client
-   * backs off randomly and tries to initiate the connection
-   * again.
-   * The other problem is to do with ticket expiry. To handle that,
-   * a relogin is attempted.
+   * If multiple clients with the same principal try to connect to the same server at the same time,
+   * the server assumes a replay attack is in progress. This is a feature of kerberos. In order to
+   * work around this, what is done is that the client backs off randomly and tries to initiate the
+   * connection again. The other problem is to do with ticket expiry. To handle that, a relogin is
+   * attempted.
    * <p>
-   * The retry logic is governed by the {@link #shouldAuthenticateOverKrb}
-   * method. In case when the user doesn't have valid credentials, we don't
-   * need to retry (from cache or ticket). In such cases, it is prudent to
-   * throw a runtime exception when we receive a SaslException from the
-   * underlying authentication implementation, so there is no retry from
-   * other high level (for eg, HCM or HBaseAdmin).
+   * The retry logic is governed by the {@link #shouldAuthenticateOverKrb} method. In case when the
+   * user doesn't have valid credentials, we don't need to retry (from cache or ticket). In such
+   * cases, it is prudent to throw a runtime exception when we receive a SaslException from the
+   * underlying authentication implementation, so there is no retry from other high level (for eg,
+   * HCM or HBaseAdmin).
    * </p>
-   *
    * @param currRetries retry count
-   * @param ex          exception describing fail
-   * @param user        which is trying to connect
-   * @throws java.io.IOException  if IO fail
+   * @param ex exception describing fail
+   * @param user which is trying to connect
+   * @throws java.io.IOException if IO fail
    * @throws InterruptedException if thread is interrupted
    */
   private void handleSaslConnectionFailure(final int currRetries, final Throwable ex,
@@ -665,7 +648,7 @@ public class AsyncRpcChannel {
         if (shouldAuthenticateOverKrb()) {
           if (currRetries < MAX_SASL_RETRIES) {
             LOG.debug("Exception encountered while connecting to the server : " + ex);
-            //try re-login
+            // try re-login
             if (UserGroupInformation.isLoginKeytabBased()) {
               UserGroupInformation.getLoginUser().reloginFromKeytab();
             } else {
@@ -675,23 +658,20 @@ public class AsyncRpcChannel {
             // Should reconnect
             return null;
           } else {
-            String msg = "Couldn't setup connection for " +
-                UserGroupInformation.getLoginUser().getUserName() +
-                " to " + serverPrincipal;
+            String msg = "Couldn't setup connection for "
+                + UserGroupInformation.getLoginUser().getUserName() + " to " + serverPrincipal;
             LOG.warn(msg);
             throw (IOException) new IOException(msg).initCause(ex);
           }
         } else {
-          LOG.warn("Exception encountered while connecting to " +
-              "the server : " + ex);
+          LOG.warn("Exception encountered while connecting to " + "the server : " + ex);
         }
         if (ex instanceof RemoteException) {
           throw (RemoteException) ex;
         }
         if (ex instanceof SaslException) {
-          String msg = "SASL authentication failed." +
-              " The most likely cause is missing or invalid credentials." +
-              " Consider 'kinit'.";
+          String msg = "SASL authentication failed."
+              + " The most likely cause is missing or invalid credentials." + " Consider 'kinit'.";
           LOG.fatal(msg, ex);
           throw new RuntimeException(msg, ex);
         }
@@ -718,7 +698,6 @@ public class AsyncRpcChannel {
     return false;
   }
 
-
   @Override
   public String toString() {
     return this.address.toString() + "/" + this.serviceName + "/" + this.ticket;

http://git-wip-us.apache.org/repos/asf/hbase/blob/fd5c0934/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncServerResponseHandler.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncServerResponseHandler.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncServerResponseHandler.java
index 8f6c85b..e0c7586 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncServerResponseHandler.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncServerResponseHandler.java
@@ -17,11 +17,6 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.ByteBufInputStream;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.ChannelInboundHandlerAdapter;
-
 import java.io.IOException;
 
 import org.apache.hadoop.hbase.CellScanner;
@@ -32,82 +27,89 @@ import org.apache.hadoop.ipc.RemoteException;
 
 import com.google.protobuf.Message;
 
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufInputStream;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.SimpleChannelInboundHandler;
+
 /**
  * Handles Hbase responses
  */
 @InterfaceAudience.Private
-public class AsyncServerResponseHandler extends ChannelInboundHandlerAdapter {
+public class AsyncServerResponseHandler extends SimpleChannelInboundHandler<ByteBuf> {
   private final AsyncRpcChannel channel;
 
   /**
    * Constructor
-   *
    * @param channel on which this response handler operates
    */
   public AsyncServerResponseHandler(AsyncRpcChannel channel) {
     this.channel = channel;
   }
 
-  @Override public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
-    ByteBuf inBuffer = (ByteBuf) msg;
+  @Override
+  protected void channelRead0(ChannelHandlerContext ctx, ByteBuf inBuffer) throws Exception {
     ByteBufInputStream in = new ByteBufInputStream(inBuffer);
     int totalSize = inBuffer.readableBytes();
-    try {
-      // Read the header
-      RPCProtos.ResponseHeader responseHeader = RPCProtos.ResponseHeader.parseDelimitedFrom(in);
-      int id = responseHeader.getCallId();
-      AsyncCall call = channel.removePendingCall(id);
-      if (call == null) {
-        // So we got a response for which we have no corresponding 'call' here on the client-side.
-        // We probably timed out waiting, cleaned up all references, and now the server decides
-        // to return a response.  There is nothing we can do w/ the response at this stage. Clean
-        // out the wire of the response so its out of the way and we can get other responses on
-        // this connection.
-        int readSoFar = IPCUtil.getTotalSizeWhenWrittenDelimited(responseHeader);
-        int whatIsLeftToRead = totalSize - readSoFar;
+    // Read the header
+    RPCProtos.ResponseHeader responseHeader = RPCProtos.ResponseHeader.parseDelimitedFrom(in);
+    int id = responseHeader.getCallId();
+    AsyncCall call = channel.removePendingCall(id);
+    if (call == null) {
+      // So we got a response for which we have no corresponding 'call' here on the client-side.
+      // We probably timed out waiting, cleaned up all references, and now the server decides
+      // to return a response. There is nothing we can do w/ the response at this stage. Clean
+      // out the wire of the response so its out of the way and we can get other responses on
+      // this connection.
+      int readSoFar = IPCUtil.getTotalSizeWhenWrittenDelimited(responseHeader);
+      int whatIsLeftToRead = totalSize - readSoFar;
 
-        // This is done through a Netty ByteBuf which has different behavior than InputStream.
-        // It does not return number of bytes read but will update pointer internally and throws an
-        // exception when too many bytes are to be skipped.
-        inBuffer.skipBytes(whatIsLeftToRead);
-        return;
-      }
+      // This is done through a Netty ByteBuf which has different behavior than InputStream.
+      // It does not return number of bytes read but will update pointer internally and throws an
+      // exception when too many bytes are to be skipped.
+      inBuffer.skipBytes(whatIsLeftToRead);
+      return;
+    }
 
-      if (responseHeader.hasException()) {
-        RPCProtos.ExceptionResponse exceptionResponse = responseHeader.getException();
-        RemoteException re = createRemoteException(exceptionResponse);
-        if (exceptionResponse.getExceptionClassName().
-            equals(FatalConnectionException.class.getName())) {
-          channel.close(re);
-        } else {
-          call.setFailed(re);
-        }
+    if (responseHeader.hasException()) {
+      RPCProtos.ExceptionResponse exceptionResponse = responseHeader.getException();
+      RemoteException re = createRemoteException(exceptionResponse);
+      if (exceptionResponse.getExceptionClassName()
+          .equals(FatalConnectionException.class.getName())) {
+        channel.close(re);
       } else {
-        Message value = null;
-        // Call may be null because it may have timedout and been cleaned up on this side already
-        if (call.responseDefaultType != null) {
-          Message.Builder builder = call.responseDefaultType.newBuilderForType();
-          ProtobufUtil.mergeDelimitedFrom(builder, in);
-          value = builder.build();
-        }
-        CellScanner cellBlockScanner = null;
-        if (responseHeader.hasCellBlockMeta()) {
-          int size = responseHeader.getCellBlockMeta().getLength();
-          byte[] cellBlock = new byte[size];
-          inBuffer.readBytes(cellBlock, 0, cellBlock.length);
-          cellBlockScanner = channel.client.createCellScanner(cellBlock);
-        }
-        call.setSuccess(value, cellBlockScanner);
-        call.callStats.setResponseSizeBytes(totalSize);
+        call.setFailed(re);
+      }
+    } else {
+      Message value = null;
+      // Call may be null because it may have timedout and been cleaned up on this side already
+      if (call.responseDefaultType != null) {
+        Message.Builder builder = call.responseDefaultType.newBuilderForType();
+        ProtobufUtil.mergeDelimitedFrom(builder, in);
+        value = builder.build();
+      }
+      CellScanner cellBlockScanner = null;
+      if (responseHeader.hasCellBlockMeta()) {
+        int size = responseHeader.getCellBlockMeta().getLength();
+        byte[] cellBlock = new byte[size];
+        inBuffer.readBytes(cellBlock, 0, cellBlock.length);
+        cellBlockScanner = channel.client.createCellScanner(cellBlock);
       }
-    } catch (IOException e) {
-      // Treat this as a fatal condition and close this connection
-      channel.close(e);
-    } finally {
-      inBuffer.release();
+      call.setSuccess(value, cellBlockScanner);
+      call.callStats.setResponseSizeBytes(totalSize);
     }
   }
 
+  @Override
+  public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
+    channel.close(cause);
+  }
+
+  @Override
+  public void channelInactive(ChannelHandlerContext ctx) throws Exception {
+    channel.close(new IOException("connection closed"));
+  }
+
   /**
    * @param e Proto exception
    * @return RemoteException made from passed <code>e</code>
@@ -118,7 +120,7 @@ public class AsyncServerResponseHandler extends ChannelInboundHandlerAdapter {
     return e.hasHostname() ?
         // If a hostname then add it to the RemoteWithExtrasException
         new RemoteWithExtrasException(innerExceptionClassName, e.getStackTrace(), e.getHostname(),
-            e.getPort(), doNotRetry) :
-        new RemoteWithExtrasException(innerExceptionClassName, e.getStackTrace(), doNotRetry);
+            e.getPort(), doNotRetry)
+        : new RemoteWithExtrasException(innerExceptionClassName, e.getStackTrace(), doNotRetry);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fd5c0934/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
index e8da9ee..69c8fe2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
@@ -57,7 +57,6 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.io.compress.GzipCodec;
 import org.apache.hadoop.util.StringUtils;
-import org.apache.http.ConnectionClosedException;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -320,7 +319,7 @@ public abstract class AbstractTestIPC {
           md.getOutputType().toProto(), User.getCurrent(), address,
           new MetricsConnection.CallStats());
         fail("RPC should have failed because it exceeds max request size");
-      } catch(ConnectionClosingException | ConnectionClosedException ex) {
+      } catch(IOException ex) {
         // pass
       }
     } finally {


[36/50] [abbrv] hbase git commit: HBASE-15494 Close obviated PRs on the GitHub mirror.

Posted by sy...@apache.org.
HBASE-15494 Close obviated PRs on the GitHub mirror.

  - closes #1 HBASE-1015 obviated by HBASE-14850
  - closes #3 obviated by HBASE-15059
  - closes #17 obviated by HBASE-15223

Signed-off-by: Elliott Clark <ec...@apache.org>


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

Branch: refs/heads/hbase-12439
Commit: 1a9837ab745da05f675490272a026912c1384c91
Parents: b865fb9
Author: Sean Busbey <bu...@apache.org>
Authored: Mon Mar 21 00:42:05 2016 -0500
Committer: Sean Busbey <bu...@apache.org>
Committed: Fri Mar 25 22:21:22 2016 -0500

----------------------------------------------------------------------

----------------------------------------------------------------------



[42/50] [abbrv] hbase git commit: HBASE-15191 CopyTable and VerifyReplication - Option to specify batch size, versions (Parth Shah)

Posted by sy...@apache.org.
HBASE-15191 CopyTable and VerifyReplication - Option to specify batch size, versions (Parth Shah)


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

Branch: refs/heads/hbase-12439
Commit: cbf9c1e116717972c243abedb53e7d26ef03cc85
Parents: 7793bc5
Author: tedyu <yu...@gmail.com>
Authored: Mon Mar 28 17:25:00 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Mon Mar 28 17:25:00 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/mapreduce/CopyTable.java       | 31 ++++++++++++++++----
 .../replication/VerifyReplication.java          |  9 ++++++
 2 files changed, 35 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/cbf9c1e1/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/CopyTable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/CopyTable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/CopyTable.java
index b7f4b66..dd8b891 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/CopyTable.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/CopyTable.java
@@ -54,7 +54,9 @@ public class CopyTable extends Configured implements Tool {
 
   final static String NAME = "copytable";
   long startTime = 0;
-  long endTime = 0;
+  long endTime = HConstants.LATEST_TIMESTAMP;
+  int batch = Integer.MAX_VALUE;
+  int cacheRow = -1;
   int versions = -1;
   String tableName = null;
   String startRow = null;
@@ -82,15 +84,22 @@ public class CopyTable extends Configured implements Tool {
     if (!doCommandLine(args)) {
       return null;
     }
-
+    
     Job job = Job.getInstance(getConf(), getConf().get(JOB_NAME_CONF_KEY, NAME + "_" + tableName));
     job.setJarByClass(CopyTable.class);
     Scan scan = new Scan();
+    
+    scan.setBatch(batch);
     scan.setCacheBlocks(false);
-    if (startTime != 0) {
-      scan.setTimeRange(startTime,
-          endTime == 0 ? HConstants.LATEST_TIMESTAMP : endTime);
+    
+    if (cacheRow > 0) {
+      scan.setCaching(cacheRow);
+    } else {
+      scan.setCaching(getConf().getInt(HConstants.HBASE_CLIENT_SCANNER_CACHING, 100));
     }
+    
+    scan.setTimeRange(startTime, endTime);
+    
     if (allCells) {
       scan.setRaw(true);
     }
@@ -252,6 +261,18 @@ public class CopyTable extends Configured implements Tool {
           endTime = Long.parseLong(cmd.substring(endTimeArgKey.length()));
           continue;
         }
+        
+        final String batchArgKey = "--batch=";
+        if (cmd.startsWith(batchArgKey)) {
+          batch = Integer.parseInt(cmd.substring(batchArgKey.length()));
+          continue;
+        }
+        
+        final String cacheRowArgKey = "--cacheRow=";
+        if (cmd.startsWith(cacheRowArgKey)) {
+          cacheRow = Integer.parseInt(cmd.substring(cacheRowArgKey.length()));
+          continue;
+        }
 
         final String versionsArgKey = "--versions=";
         if (cmd.startsWith(versionsArgKey)) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/cbf9c1e1/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
index e6b4802..a452036 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
@@ -72,6 +72,7 @@ public class VerifyReplication extends Configured implements Tool {
   private final static String PEER_CONFIG_PREFIX = NAME + ".peer.";
   static long startTime = 0;
   static long endTime = Long.MAX_VALUE;
+  static int batch = Integer.MAX_VALUE;
   static int versions = -1;
   static String tableName = null;
   static String families = null;
@@ -110,6 +111,8 @@ public class VerifyReplication extends Configured implements Tool {
       if (replicatedScanner == null) {
         Configuration conf = context.getConfiguration();
         final Scan scan = new Scan();
+        scan.setBatch(batch);
+        scan.setCacheBlocks(false);
         scan.setCaching(conf.getInt(TableInputFormat.SCAN_CACHEDROWS, 1));
         long startTime = conf.getLong(NAME + ".startTime", 0);
         long endTime = conf.getLong(NAME + ".endTime", Long.MAX_VALUE);
@@ -338,6 +341,12 @@ public class VerifyReplication extends Configured implements Tool {
           versions = Integer.parseInt(cmd.substring(versionsArgKey.length()));
           continue;
         }
+        
+        final String batchArgKey = "--batch=";
+        if (cmd.startsWith(batchArgKey)) {
+          batch = Integer.parseInt(cmd.substring(batchArgKey.length()));
+          continue;
+        }
 
         final String familiesArgKey = "--families=";
         if (cmd.startsWith(familiesArgKey)) {


[09/50] [abbrv] hbase git commit: HBASE-15360 addendum fix testCoDelScheduling

Posted by sy...@apache.org.
HBASE-15360 addendum fix testCoDelScheduling

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


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

Branch: refs/heads/hbase-12439
Commit: 23484785067bb4a31d5d6a180f578cee70ff9276
Parents: 000117a
Author: zhangduo <zh...@apache.org>
Authored: Tue Mar 22 16:14:20 2016 +0800
Committer: stack <st...@apache.org>
Committed: Tue Mar 22 13:52:26 2016 -0700

----------------------------------------------------------------------
 .../hbase/ipc/AdaptiveLifoCoDelCallQueue.java   |   5 +-
 .../hadoop/hbase/ipc/SimpleRpcScheduler.java    |   2 +-
 .../hbase/ipc/TestSimpleRpcScheduler.java       | 135 +++++++++++++------
 3 files changed, 96 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/23484785/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/AdaptiveLifoCoDelCallQueue.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/AdaptiveLifoCoDelCallQueue.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/AdaptiveLifoCoDelCallQueue.java
index 37e86be..266c6a2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/AdaptiveLifoCoDelCallQueue.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/AdaptiveLifoCoDelCallQueue.java
@@ -28,6 +28,7 @@ import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.locks.ReentrantLock;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 
 /**
  * Adaptive LIFO blocking queue utilizing CoDel algorithm to prevent queue overloading.
@@ -77,7 +78,7 @@ public class AdaptiveLifoCoDelCallQueue implements BlockingQueue<CallRunner> {
   private volatile long minDelay;
 
   // the moment when current interval ends
-  private volatile long intervalTime = System.currentTimeMillis();
+  private volatile long intervalTime = EnvironmentEdgeManager.currentTime();
 
   // switch to ensure only one threads does interval cutoffs
   private AtomicBoolean resetDelay = new AtomicBoolean(true);
@@ -147,7 +148,7 @@ public class AdaptiveLifoCoDelCallQueue implements BlockingQueue<CallRunner> {
    *   and internal queue state (deemed overloaded).
    */
   private boolean needToDrop(CallRunner callRunner) {
-    long now = System.currentTimeMillis();
+    long now = EnvironmentEdgeManager.currentTime();
     long callDelay = now - callRunner.getCall().timestamp;
 
     long localMinDelay = this.minDelay;

http://git-wip-us.apache.org/repos/asf/hbase/blob/23484785/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java
index 0cd34bb..431aeeb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java
@@ -203,7 +203,7 @@ public class SimpleRpcScheduler extends RpcScheduler implements ConfigurationObs
       } else if (callQueueType.equals(CALL_QUEUE_TYPE_CODEL_CONF_VALUE)) {
         Object[] callQueueInitArgs = {maxQueueLength, codelTargetDelay, codelInterval,
           codelLifoThreshold, numGeneralCallsDropped, numLifoModeSwitches};
-        callExecutor = new RWQueueRpcExecutor("B.default", handlerCount,
+        callExecutor = new RWQueueRpcExecutor("RW.default", handlerCount,
           numCallQueues, callqReadShare, callqScanShare,
           AdaptiveLifoCoDelCallQueue.class, callQueueInitArgs,
           AdaptiveLifoCoDelCallQueue.class, callQueueInitArgs);

http://git-wip-us.apache.org/repos/asf/hbase/blob/23484785/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java
index 6454537..97ef973 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java
@@ -17,28 +17,49 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Maps;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyObject;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.timeout;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.LinkedBlockingQueue;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import com.google.protobuf.Message;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CategoryBasedTimeout;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandlerImpl;
-import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.testclassification.RPCTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.ipc.RpcServer.Call;
+import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandlerImpl;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
+import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos;
 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.testclassification.RPCTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdge;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.Threads;
 import org.junit.Before;
 import org.junit.Rule;
@@ -48,25 +69,11 @@ import org.junit.rules.TestRule;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.CountDownLatch;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyObject;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.timeout;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Maps;
+import com.google.protobuf.Message;
 
 @Category({RPCTests.class, SmallTests.class})
 public class TestSimpleRpcScheduler {
@@ -218,7 +225,7 @@ public class TestSimpleRpcScheduler {
       scheduler.dispatch(smallCallTask);
 
       while (work.size() < 8) {
-        Threads.sleepWithoutInterrupt(100);
+        Thread.sleep(100);
       }
 
       int seqSum = 0;
@@ -298,7 +305,7 @@ public class TestSimpleRpcScheduler {
       scheduler.dispatch(scanCallTask);
 
       while (work.size() < 6) {
-        Threads.sleepWithoutInterrupt(100);
+        Thread.sleep(100);
       }
 
       for (int i = 0; i < work.size() - 2; i += 3) {
@@ -326,6 +333,13 @@ public class TestSimpleRpcScheduler {
     }).when(callTask).run();
   }
 
+  private static void waitUntilQueueEmpty(SimpleRpcScheduler scheduler)
+      throws InterruptedException {
+    while (scheduler.getGeneralQueueLength() > 0) {
+      Thread.sleep(100);
+    }
+  }
+
   @Test
   public void testSoftAndHardQueueLimits() throws Exception {
     Configuration schedConf = HBaseConfiguration.create();
@@ -354,9 +368,7 @@ public class TestSimpleRpcScheduler {
       schedConf.setInt("hbase.ipc.server.max.callqueue.length", 0);
       scheduler.onConfigurationChange(schedConf);
       assertFalse(scheduler.dispatch(putCallTask));
-      while (scheduler.getGeneralQueueLength() > 0) {
-        Threads.sleepWithoutInterrupt(100);
-      }
+      waitUntilQueueEmpty(scheduler);
       schedConf.setInt("hbase.ipc.server.max.callqueue.length", 1);
       scheduler.onConfigurationChange(schedConf);
       assertTrue(scheduler.dispatch(putCallTask));
@@ -365,8 +377,30 @@ public class TestSimpleRpcScheduler {
     }
   }
 
+  private static final class CoDelEnvironmentEdge implements EnvironmentEdge {
+
+    private final BlockingQueue<Long> timeQ = new LinkedBlockingQueue<>();
+
+    private long offset;
+
+    private final Set<String> threadNamePrefixs = new HashSet<>();
+
+    @Override
+    public long currentTime() {
+      for (String threadNamePrefix : threadNamePrefixs) {
+        if (Thread.currentThread().getName().startsWith(threadNamePrefix)) {
+          return timeQ.poll().longValue() + offset;
+        }
+      }
+      return System.currentTimeMillis();
+    }
+  }
+
   @Test
   public void testCoDelScheduling() throws Exception {
+    CoDelEnvironmentEdge envEdge = new CoDelEnvironmentEdge();
+    envEdge.threadNamePrefixs.add("RW.default");
+    envEdge.threadNamePrefixs.add("B.default");
     Configuration schedConf = HBaseConfiguration.create();
 
     schedConf.set(SimpleRpcScheduler.CALL_QUEUE_TYPE_CONF_KEY,
@@ -379,36 +413,51 @@ public class TestSimpleRpcScheduler {
       HConstants.QOS_THRESHOLD);
     try {
       scheduler.start();
-
+      EnvironmentEdgeManager.injectEdge(envEdge);
+      envEdge.offset = 5;
       // calls faster than min delay
       for (int i = 0; i < 100; i++) {
-        CallRunner cr = getMockedCallRunner();
+        long time = System.currentTimeMillis();
+        envEdge.timeQ.put(time);
+        CallRunner cr = getMockedCallRunner(time);
         Thread.sleep(5);
         scheduler.dispatch(cr);
       }
-      Thread.sleep(100); // make sure fast calls are handled
+      // make sure fast calls are handled
+      waitUntilQueueEmpty(scheduler);
+      Thread.sleep(100);
       assertEquals("None of these calls should have been discarded", 0,
         scheduler.getNumGeneralCallsDropped());
 
+      envEdge.offset = 6;
       // calls slower than min delay, but not individually slow enough to be dropped
       for (int i = 0; i < 20; i++) {
-        CallRunner cr = getMockedCallRunner();
+        long time = System.currentTimeMillis();
+        envEdge.timeQ.put(time);
+        CallRunner cr = getMockedCallRunner(time);
         Thread.sleep(6);
         scheduler.dispatch(cr);
       }
 
-      Thread.sleep(100); // make sure somewhat slow calls are handled
+      // make sure somewhat slow calls are handled
+      waitUntilQueueEmpty(scheduler);
+      Thread.sleep(100);
       assertEquals("None of these calls should have been discarded", 0,
         scheduler.getNumGeneralCallsDropped());
 
+      envEdge.offset = 12;
       // now slow calls and the ones to be dropped
       for (int i = 0; i < 20; i++) {
-        CallRunner cr = getMockedCallRunner();
+        long time = System.currentTimeMillis();
+        envEdge.timeQ.put(time);
+        CallRunner cr = getMockedCallRunner(time);
         Thread.sleep(12);
         scheduler.dispatch(cr);
       }
 
-      Thread.sleep(100); // make sure somewhat slow calls are handled
+      // make sure somewhat slow calls are handled
+      waitUntilQueueEmpty(scheduler);
+      Thread.sleep(100);
       assertTrue("There should have been at least 12 calls dropped",
         scheduler.getNumGeneralCallsDropped() > 12);
     } finally {
@@ -416,7 +465,7 @@ public class TestSimpleRpcScheduler {
     }
   }
 
-  private CallRunner getMockedCallRunner() throws IOException {
+  private CallRunner getMockedCallRunner(long timestamp) throws IOException {
     CallRunner putCallTask = mock(CallRunner.class);
     RpcServer.Call putCall = mock(RpcServer.Call.class);
     putCall.param = RequestConverter.buildMutateRequest(
@@ -424,7 +473,7 @@ public class TestSimpleRpcScheduler {
     RPCProtos.RequestHeader putHead = RPCProtos.RequestHeader.newBuilder().setMethodName("mutate").build();
     when(putCallTask.getCall()).thenReturn(putCall);
     when(putCall.getHeader()).thenReturn(putHead);
-    putCall.timestamp = System.currentTimeMillis();
+    putCall.timestamp = timestamp;
     return putCallTask;
   }
 }


[29/50] [abbrv] hbase git commit: HBASE-14256 Flush task message may be confusing when region is recovered (Gabor Liptak)

Posted by sy...@apache.org.
HBASE-14256 Flush task message may be confusing when region is recovered (Gabor Liptak)


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

Branch: refs/heads/hbase-12439
Commit: 7c93098210d5f17242857a9a0148b4b26ff7aaae
Parents: e71bc71
Author: Jerry He <je...@apache.org>
Authored: Thu Mar 24 10:01:53 2016 -0700
Committer: Jerry He <je...@apache.org>
Committed: Thu Mar 24 10:06:38 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/regionserver/HRegion.java    | 11 ++++++-----
 1 file changed, 6 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/7c930982/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 ab19d08..4da0f13 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
@@ -1217,11 +1217,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       // force a flush only if region replication is set up for this region. Otherwise no need.
       boolean forceFlush = getTableDesc().getRegionReplication() > 1;
 
-      // force a flush first
-      MonitoredTask status = TaskMonitor.get().createStatus(
-        "Flushing region " + this + " because recovery is finished");
+      MonitoredTask status = TaskMonitor.get().createStatus("Recovering region " + this);
+
       try {
+        // force a flush first
         if (forceFlush) {
+          status.setStatus("Flushing region " + this + " because recovery is finished");
           internalFlushcache(status);
         }
 
@@ -1237,13 +1238,13 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
           // We cannot rethrow this exception since we are being called from the zk thread. The
           // region has already opened. In this case we log the error, but continue
           LOG.warn(getRegionInfo().getEncodedName() + " : was not able to write region opening "
-              + "event to WAL, continueing", e);
+              + "event to WAL, continuing", e);
         }
       } catch (IOException ioe) {
         // Distributed log replay semantics does not necessarily require a flush, since the replayed
         // data is already written again in the WAL. So failed flush should be fine.
         LOG.warn(getRegionInfo().getEncodedName() + " : was not able to flush "
-            + "event to WAL, continueing", ioe);
+            + "event to WAL, continuing", ioe);
       } finally {
         status.cleanup();
       }


[20/50] [abbrv] hbase git commit: HBASE-15064 BufferUnderflowException after last Cell fetched from an HFile Block served from L2 offheap cache - Addendum.

Posted by sy...@apache.org.
HBASE-15064 BufferUnderflowException after last Cell fetched from an HFile Block served from L2 offheap cache - Addendum.


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

Branch: refs/heads/hbase-12439
Commit: cadfb21f4bb465d1e305db2a159b8574282c8150
Parents: 5fcadb8
Author: anoopsjohn <an...@gmail.com>
Authored: Wed Mar 23 15:03:50 2016 +0530
Committer: anoopsjohn <an...@gmail.com>
Committed: Wed Mar 23 15:03:50 2016 +0530

----------------------------------------------------------------------
 .../apache/hadoop/hbase/nio/MultiByteBuff.java  |  3 ++-
 .../hadoop/hbase/nio/TestMultiByteBuff.java     | 25 ++++++++++++++++++++
 2 files changed, 27 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/cadfb21f/hbase-common/src/main/java/org/apache/hadoop/hbase/nio/MultiByteBuff.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/nio/MultiByteBuff.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/nio/MultiByteBuff.java
index ab2b5ea..107bb3f 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/nio/MultiByteBuff.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/nio/MultiByteBuff.java
@@ -472,7 +472,8 @@ public class MultiByteBuff extends ByteBuff {
    */
   @Override
   public final boolean hasRemaining() {
-    return this.curItem.hasRemaining() || this.curItemIndex < this.items.length - 1;
+    return this.curItem.hasRemaining() || (this.curItemIndex < this.limitedItemIndex
+        && this.items[this.curItemIndex + 1].hasRemaining());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/cadfb21f/hbase-common/src/test/java/org/apache/hadoop/hbase/nio/TestMultiByteBuff.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/nio/TestMultiByteBuff.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/nio/TestMultiByteBuff.java
index 800c8e1..af4c464 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/nio/TestMultiByteBuff.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/nio/TestMultiByteBuff.java
@@ -378,4 +378,29 @@ public class TestMultiByteBuff {
     assertTrue(Bytes.equals(actual, 0, actual.length,
             b, 1, 3));
   }
+
+  @Test
+  public void testHasRemaining() {
+    ByteBuffer b1 = ByteBuffer.allocate(8);
+    ByteBuffer b2 = ByteBuffer.allocate(8);
+    ByteBuffer b3 = ByteBuffer.allocate(8);
+    MultiByteBuff mbb1 = new MultiByteBuff(b1, b2, b3);
+    assertTrue(mbb1.hasRemaining());
+    mbb1.limit(20); // Limit in mid of last of BB
+    mbb1.position(15);
+    mbb1.get();// We are at the end of second BB
+    assertTrue(mbb1.hasRemaining());
+    mbb1.position(20);
+    assertFalse(mbb1.hasRemaining());
+    mbb1.limit(12); // Limit in mid of second BB
+    mbb1.position(11);
+    assertTrue(mbb1.hasRemaining());
+    mbb1.get(); // Now we have reached the limit
+    assertFalse(mbb1.hasRemaining());
+    mbb1.limit(16);// Limit at begin of the last BB
+    mbb1.position(15);
+    assertTrue(mbb1.hasRemaining());
+    mbb1.get(); // Now we have reached the limit
+    assertFalse(mbb1.hasRemaining());
+  }
 }


[44/50] [abbrv] hbase git commit: HBASE-15295 MutateTableAccess.multiMutate() does not get high priority causing a deadlock

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableConfiguration.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableConfiguration.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableConfiguration.java
deleted file mode 100644
index 1113cfd..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableConfiguration.java
+++ /dev/null
@@ -1,144 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required by applicable
- * law or agreed to in writing, software distributed under the License is distributed on an "AS IS"
- * BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License
- * for the specific language governing permissions and limitations under the License.
- */
-
-package org.apache.hadoop.hbase.client;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-
-import com.google.common.annotations.VisibleForTesting;
-
-/**
- *
- * Configuration is a heavy weight registry that does a lot of string operations and regex matching.
- * Method calls into Configuration account for high CPU usage and have huge performance impact.
- * This class caches the value in the TableConfiguration object to improve performance.
- * see HBASE-12128
- *
- */
-@InterfaceAudience.Private
-public class TableConfiguration {
-
-  public static final String WRITE_BUFFER_SIZE_KEY = "hbase.client.write.buffer";
-  public static final long WRITE_BUFFER_SIZE_DEFAULT = 2097152;
-  public static final String MAX_KEYVALUE_SIZE_KEY = "hbase.client.keyvalue.maxsize";
-  public static final int MAX_KEYVALUE_SIZE_DEFAULT = -1;
-
-  private final long writeBufferSize;
-  private final int metaOperationTimeout;
-  private final int operationTimeout;
-  private final int scannerCaching;
-  private final long scannerMaxResultSize;
-  private final int primaryCallTimeoutMicroSecond;
-  private final int replicaCallTimeoutMicroSecondScan;
-  private final int retries;
-  private final int maxKeyValueSize;
-
-    // toggle for async/sync prefetch
-  private final boolean clientScannerAsyncPrefetch;
-
-    /**
-   * Constructor
-   * @param conf Configuration object
-   */
-  TableConfiguration(Configuration conf) {
-    this.writeBufferSize = conf.getLong(WRITE_BUFFER_SIZE_KEY, WRITE_BUFFER_SIZE_DEFAULT);
-
-    this.metaOperationTimeout = conf.getInt(
-      HConstants.HBASE_CLIENT_META_OPERATION_TIMEOUT,
-      HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
-
-    this.operationTimeout = conf.getInt(
-      HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
-
-    this.scannerCaching = conf.getInt(
-      HConstants.HBASE_CLIENT_SCANNER_CACHING, HConstants.DEFAULT_HBASE_CLIENT_SCANNER_CACHING);
-
-    this.scannerMaxResultSize =
-        conf.getLong(HConstants.HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE_KEY,
-          HConstants.DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE);
-
-    this.primaryCallTimeoutMicroSecond =
-        conf.getInt("hbase.client.primaryCallTimeout.get", 10000); // 10ms
-
-    this.replicaCallTimeoutMicroSecondScan =
-        conf.getInt("hbase.client.replicaCallTimeout.scan", 1000000); // 1000 ms
-
-    this.retries = conf.getInt(
-       HConstants.HBASE_CLIENT_RETRIES_NUMBER, HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
-
-    this.clientScannerAsyncPrefetch = conf.getBoolean(
-       Scan.HBASE_CLIENT_SCANNER_ASYNC_PREFETCH, Scan.DEFAULT_HBASE_CLIENT_SCANNER_ASYNC_PREFETCH);
-
-    this.maxKeyValueSize = conf.getInt(MAX_KEYVALUE_SIZE_KEY, MAX_KEYVALUE_SIZE_DEFAULT);
-  }
-
-  /**
-   * Constructor
-   * This is for internal testing purpose (using the default value).
-   * In real usage, we should read the configuration from the Configuration object.
-   */
-  @VisibleForTesting
-  protected TableConfiguration() {
-    this.writeBufferSize = WRITE_BUFFER_SIZE_DEFAULT;
-    this.metaOperationTimeout = HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT;
-    this.operationTimeout = HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT;
-    this.scannerCaching = HConstants.DEFAULT_HBASE_CLIENT_SCANNER_CACHING;
-    this.scannerMaxResultSize = HConstants.DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE;
-    this.primaryCallTimeoutMicroSecond = 10000;
-    this.replicaCallTimeoutMicroSecondScan = 1000000;
-    this.retries = HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER;
-    this.clientScannerAsyncPrefetch = Scan.DEFAULT_HBASE_CLIENT_SCANNER_ASYNC_PREFETCH;
-    this.maxKeyValueSize = MAX_KEYVALUE_SIZE_DEFAULT;
-  }
-
-  public long getWriteBufferSize() {
-    return writeBufferSize;
-  }
-
-  public int getMetaOperationTimeout() {
-    return metaOperationTimeout;
-  }
-
-  public int getOperationTimeout() {
-    return operationTimeout;
-  }
-
-  public int getScannerCaching() {
-    return scannerCaching;
-  }
-
-  public int getPrimaryCallTimeoutMicroSecond() {
-    return primaryCallTimeoutMicroSecond;
-  }
-
-  public int getReplicaCallTimeoutMicroSecondScan() {
-    return replicaCallTimeoutMicroSecondScan;
-  }
-
-  public int getRetriesNumber() {
-    return retries;
-  }
-
-  public int getMaxKeyValueSize() {
-    return maxKeyValueSize;
-  }
-
-  public long getScannerMaxResultSize() {
-    return scannerMaxResultSize;
-  }
-
-  public boolean isClientScannerAsyncPrefetch() {
-    return clientScannerAsyncPrefetch;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
index a53fb70..ec6332a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
@@ -318,7 +318,7 @@ public abstract class AbstractRpcClient implements RpcClient {
     public Message callBlockingMethod(Descriptors.MethodDescriptor md, RpcController controller,
         Message param, Message returnType) throws ServiceException {
       PayloadCarryingRpcController pcrc;
-      if (controller != null) {
+      if (controller != null && controller instanceof PayloadCarryingRpcController) {
         pcrc = (PayloadCarryingRpcController) controller;
         if (!pcrc.hasCallTimeout()) {
           pcrc.setCallTimeout(channelOperationTimeout);

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CoprocessorRpcChannel.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CoprocessorRpcChannel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CoprocessorRpcChannel.java
index e60fbd6..b1d54a4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CoprocessorRpcChannel.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CoprocessorRpcChannel.java
@@ -36,7 +36,7 @@ import com.google.protobuf.ServiceException;
 
 /**
  * Base class which provides clients with an RPC connection to
- * call coprocessor endpoint {@link com.google.protobuf.Service}s.  
+ * call coprocessor endpoint {@link com.google.protobuf.Service}s.
  * Note that clients should not use this class directly, except through
  * {@link org.apache.hadoop.hbase.client.HTableInterface#coprocessorService(byte[])}.
  */
@@ -53,7 +53,7 @@ public abstract class CoprocessorRpcChannel implements RpcChannel, BlockingRpcCh
                          RpcCallback<Message> callback) {
     Message response = null;
     try {
-      response = callExecService(method, request, responsePrototype);
+      response = callExecService(controller, method, request, responsePrototype);
     } catch (IOException ioe) {
       LOG.warn("Call failed on IOException", ioe);
       ResponseConverter.setControllerException(controller, ioe);
@@ -70,12 +70,13 @@ public abstract class CoprocessorRpcChannel implements RpcChannel, BlockingRpcCh
                                     Message request, Message responsePrototype)
       throws ServiceException {
     try {
-      return callExecService(method, request, responsePrototype);
+      return callExecService(controller, method, request, responsePrototype);
     } catch (IOException ioe) {
       throw new ServiceException("Error calling method "+method.getFullName(), ioe);
     }
   }
 
-  protected abstract Message callExecService(Descriptors.MethodDescriptor method,
-                                  Message request, Message responsePrototype) throws IOException;
+  protected abstract Message callExecService(RpcController controller,
+      Descriptors.MethodDescriptor method, Message request, Message responsePrototype)
+          throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MasterCoprocessorRpcChannel.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MasterCoprocessorRpcChannel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MasterCoprocessorRpcChannel.java
index 98a74ef..6e59972 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MasterCoprocessorRpcChannel.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MasterCoprocessorRpcChannel.java
@@ -24,7 +24,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
@@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.util.ByteStringer;
 
 import com.google.protobuf.Descriptors;
 import com.google.protobuf.Message;
+import com.google.protobuf.RpcController;
 
 /**
  * Provides clients with an RPC connection to call coprocessor endpoint {@link com.google.protobuf.Service}s
@@ -45,18 +46,18 @@ import com.google.protobuf.Message;
 public class MasterCoprocessorRpcChannel extends CoprocessorRpcChannel{
   private static final Log LOG = LogFactory.getLog(MasterCoprocessorRpcChannel.class);
 
-  private final HConnection connection;
+  private final ClusterConnection connection;
 
-  public MasterCoprocessorRpcChannel(HConnection conn) {
+  public MasterCoprocessorRpcChannel(ClusterConnection conn) {
     this.connection = conn;
   }
 
   @Override
-  protected Message callExecService(Descriptors.MethodDescriptor method,
+  protected Message callExecService(RpcController controller, Descriptors.MethodDescriptor method,
                                   Message request, Message responsePrototype)
       throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Call: "+method.getName()+", "+request.toString());
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Call: "+method.getName()+", "+request.toString());
     }
 
     final ClientProtos.CoprocessorServiceCall call =
@@ -65,7 +66,10 @@ public class MasterCoprocessorRpcChannel extends CoprocessorRpcChannel{
             .setServiceName(method.getService().getFullName())
             .setMethodName(method.getName())
             .setRequest(request.toByteString()).build();
-    CoprocessorServiceResponse result = ProtobufUtil.execService(connection.getMaster(), call);
+
+    // TODO: Are we retrying here? Does not seem so. We should use RetryingRpcCaller
+    CoprocessorServiceResponse result = ProtobufUtil.execService(controller,
+      connection.getMaster(), call);
     Message response = null;
     if (result.getValue().hasValue()) {
       Message.Builder builder = responsePrototype.newBuilderForType();

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java
index 3fcfceb..321dd62 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java
@@ -22,10 +22,9 @@ import java.io.IOException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.RegionServerCallable;
 import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@@ -36,6 +35,7 @@ import org.apache.hadoop.hbase.util.Bytes;
 
 import com.google.protobuf.Descriptors;
 import com.google.protobuf.Message;
+import com.google.protobuf.RpcController;
 
 /**
  * Provides clients with an RPC connection to call coprocessor endpoint {@link com.google.protobuf.Service}s
@@ -49,28 +49,28 @@ import com.google.protobuf.Message;
 public class RegionCoprocessorRpcChannel extends CoprocessorRpcChannel{
   private static final Log LOG = LogFactory.getLog(RegionCoprocessorRpcChannel.class);
 
-  private final HConnection connection;
+  private final ClusterConnection connection;
   private final TableName table;
   private final byte[] row;
   private byte[] lastRegion;
   private int operationTimeout;
 
-  private RpcRetryingCallerFactory rpcFactory;
+  private RpcRetryingCallerFactory rpcCallerFactory;
+  private RpcControllerFactory rpcControllerFactory;
 
-  public RegionCoprocessorRpcChannel(HConnection conn, TableName table, byte[] row) {
+  public RegionCoprocessorRpcChannel(ClusterConnection conn, TableName table, byte[] row) {
     this.connection = conn;
     this.table = table;
     this.row = row;
-    this.rpcFactory = RpcRetryingCallerFactory.instantiate(conn.getConfiguration(), null);
-    this.operationTimeout = conn.getConfiguration().getInt(
-        HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,
-        HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
+    this.rpcCallerFactory = conn.getRpcRetryingCallerFactory();
+    this.rpcControllerFactory = conn.getRpcControllerFactory();
+    this.operationTimeout = conn.getConnectionConfiguration().getOperationTimeout();
   }
 
   @Override
-  protected Message callExecService(Descriptors.MethodDescriptor method,
-                                  Message request, Message responsePrototype)
-      throws IOException {
+  protected Message callExecService(RpcController controller,
+      Descriptors.MethodDescriptor method, Message request, Message responsePrototype)
+          throws IOException {
     if (LOG.isTraceEnabled()) {
       LOG.trace("Call: "+method.getName()+", "+request.toString());
     }
@@ -79,6 +79,9 @@ public class RegionCoprocessorRpcChannel extends CoprocessorRpcChannel{
       throw new IllegalArgumentException("Missing row property for remote region location");
     }
 
+    final RpcController rpcController = controller == null
+        ? rpcControllerFactory.newController() : controller;
+
     final ClientProtos.CoprocessorServiceCall call =
         ClientProtos.CoprocessorServiceCall.newBuilder()
             .setRow(ByteStringer.wrap(row))
@@ -87,12 +90,19 @@ public class RegionCoprocessorRpcChannel extends CoprocessorRpcChannel{
             .setRequest(request.toByteString()).build();
     RegionServerCallable<CoprocessorServiceResponse> callable =
         new RegionServerCallable<CoprocessorServiceResponse>(connection, table, row) {
-          public CoprocessorServiceResponse call(int callTimeout) throws Exception {
-            byte[] regionName = getLocation().getRegionInfo().getRegionName();
-            return ProtobufUtil.execService(getStub(), call, regionName);
-          }
-        };
-    CoprocessorServiceResponse result = rpcFactory.<CoprocessorServiceResponse> newCaller()
+      @Override
+      public CoprocessorServiceResponse call(int callTimeout) throws Exception {
+        if (rpcController instanceof PayloadCarryingRpcController) {
+          ((PayloadCarryingRpcController) rpcController).setPriority(tableName);
+        }
+        if (rpcController instanceof TimeLimitedRpcController) {
+          ((TimeLimitedRpcController) rpcController).setCallTimeout(callTimeout);
+        }
+        byte[] regionName = getLocation().getRegionInfo().getRegionName();
+        return ProtobufUtil.execService(rpcController, getStub(), call, regionName);
+      }
+    };
+    CoprocessorServiceResponse result = rpcCallerFactory.<CoprocessorServiceResponse> newCaller()
         .callWithRetries(callable, operationTimeout);
     Message response = null;
     if (result.getValue().hasValue()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionServerCoprocessorRpcChannel.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionServerCoprocessorRpcChannel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionServerCoprocessorRpcChannel.java
index 3f0a5d9..24d2de4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionServerCoprocessorRpcChannel.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionServerCoprocessorRpcChannel.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.hbase.util.ByteStringer;
 
 import com.google.protobuf.Descriptors;
 import com.google.protobuf.Message;
+import com.google.protobuf.RpcController;
 
 /**
  * Provides clients with an RPC connection to call coprocessor endpoint
@@ -47,8 +48,9 @@ public class RegionServerCoprocessorRpcChannel extends CoprocessorRpcChannel {
   }
 
   @Override
-  protected Message callExecService(Descriptors.MethodDescriptor method, Message request,
-      Message responsePrototype) throws IOException {
+  protected Message callExecService(RpcController controller,
+      Descriptors.MethodDescriptor method, Message request, Message responsePrototype)
+          throws IOException {
     if (LOG.isTraceEnabled()) {
       LOG.trace("Call: " + method.getName() + ", " + request.toString());
     }
@@ -57,8 +59,10 @@ public class RegionServerCoprocessorRpcChannel extends CoprocessorRpcChannel {
             .setRow(ByteStringer.wrap(HConstants.EMPTY_BYTE_ARRAY))
             .setServiceName(method.getService().getFullName()).setMethodName(method.getName())
             .setRequest(request.toByteString()).build();
+
+    // TODO: Are we retrying here? Does not seem so. We should use RetryingRpcCaller
     CoprocessorServiceResponse result =
-        ProtobufUtil.execRegionServerService(connection.getClient(serverName), call);
+        ProtobufUtil.execRegionServerService(controller, connection.getClient(serverName), call);
     Message response = null;
     if (result.getValue().hasValue()) {
       Message.Builder builder = responsePrototype.newBuilderForType();

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/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 2636777..f9fa21c 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
@@ -173,6 +173,19 @@ import org.apache.hadoop.security.token.Token;
 
 import static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier
     .RegionSpecifierType.REGION_NAME;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.ListMultimap;
+import com.google.common.collect.Lists;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.CodedInputStream;
+import com.google.protobuf.InvalidProtocolBufferException;
+import com.google.protobuf.Message;
+import com.google.protobuf.Parser;
+import com.google.protobuf.RpcChannel;
+import com.google.protobuf.RpcController;
+import com.google.protobuf.Service;
+import com.google.protobuf.ServiceException;
+import com.google.protobuf.TextFormat;
 
 /**
  * Protobufs utility.
@@ -1575,21 +1588,22 @@ public final class ProtobufUtil {
     }
   }
 
-  public static CoprocessorServiceResponse execService(final ClientService.BlockingInterface client,
-      final CoprocessorServiceCall call, final byte[] regionName) throws IOException {
+  public static CoprocessorServiceResponse execService(final RpcController controller,
+      final ClientService.BlockingInterface client, final CoprocessorServiceCall call,
+      final byte[] regionName) throws IOException {
     CoprocessorServiceRequest request = CoprocessorServiceRequest.newBuilder()
         .setCall(call).setRegion(
             RequestConverter.buildRegionSpecifier(REGION_NAME, regionName)).build();
     try {
       CoprocessorServiceResponse response =
-          client.execService(null, request);
+          client.execService(controller, request);
       return response;
     } catch (ServiceException se) {
       throw getRemoteException(se);
     }
   }
 
-  public static CoprocessorServiceResponse execService(
+  public static CoprocessorServiceResponse execService(final RpcController controller,
     final MasterService.BlockingInterface client, final CoprocessorServiceCall call)
   throws IOException {
     CoprocessorServiceRequest request = CoprocessorServiceRequest.newBuilder()
@@ -1597,7 +1611,7 @@ public final class ProtobufUtil {
            RequestConverter.buildRegionSpecifier(REGION_NAME, HConstants.EMPTY_BYTE_ARRAY)).build();
     try {
       CoprocessorServiceResponse response =
-          client.execMasterService(null, request);
+          client.execMasterService(controller, request);
       return response;
     } catch (ServiceException se) {
       throw getRemoteException(se);
@@ -1612,7 +1626,8 @@ public final class ProtobufUtil {
    * @throws IOException
    */
   public static CoprocessorServiceResponse execRegionServerService(
-      final ClientService.BlockingInterface client, final CoprocessorServiceCall call)
+      final RpcController controller, final ClientService.BlockingInterface client,
+      final CoprocessorServiceCall call)
       throws IOException {
     CoprocessorServiceRequest request =
         CoprocessorServiceRequest
@@ -1622,7 +1637,7 @@ public final class ProtobufUtil {
               RequestConverter.buildRegionSpecifier(REGION_NAME, HConstants.EMPTY_BYTE_ARRAY))
             .build();
     try {
-      CoprocessorServiceResponse response = client.execRegionServerService(null, request);
+      CoprocessorServiceResponse response = client.execRegionServerService(controller, request);
       return response;
     } catch (ServiceException se) {
       throw getRemoteException(se);
@@ -1648,13 +1663,13 @@ public final class ProtobufUtil {
    * @return the retrieved region info
    * @throws IOException
    */
-  public static HRegionInfo getRegionInfo(final AdminService.BlockingInterface admin,
-      final byte[] regionName) throws IOException {
+  public static HRegionInfo getRegionInfo(final RpcController controller,
+      final AdminService.BlockingInterface admin, final byte[] regionName) throws IOException {
     try {
       GetRegionInfoRequest request =
         RequestConverter.buildGetRegionInfoRequest(regionName);
       GetRegionInfoResponse response =
-        admin.getRegionInfo(null, request);
+        admin.getRegionInfo(controller, request);
       return HRegionInfo.convert(response.getRegionInfo());
     } catch (ServiceException se) {
       throw getRemoteException(se);
@@ -1669,12 +1684,13 @@ public final class ProtobufUtil {
    * @param regionName
    * @throws IOException
    */
-  public static void closeRegion(final AdminService.BlockingInterface admin,
-      final ServerName server, final byte[] regionName) throws IOException {
+  public static void closeRegion(final RpcController controller,
+      final AdminService.BlockingInterface admin, final ServerName server, final byte[] regionName)
+          throws IOException {
     CloseRegionRequest closeRegionRequest =
       RequestConverter.buildCloseRegionRequest(server, regionName);
     try {
-      admin.closeRegion(null, closeRegionRequest);
+      admin.closeRegion(controller, closeRegionRequest);
     } catch (ServiceException se) {
       throw getRemoteException(se);
     }
@@ -1689,14 +1705,15 @@ public final class ProtobufUtil {
    * @return true if the region is closed
    * @throws IOException
    */
-  public static boolean closeRegion(final AdminService.BlockingInterface admin,
+  public static boolean closeRegion(final RpcController controller,
+      final AdminService.BlockingInterface admin,
       final ServerName server, final byte[] regionName,
       final ServerName destinationServer) throws IOException {
     CloseRegionRequest closeRegionRequest =
       RequestConverter.buildCloseRegionRequest(server,
         regionName, destinationServer);
     try {
-      CloseRegionResponse response = admin.closeRegion(null, closeRegionRequest);
+      CloseRegionResponse response = admin.closeRegion(controller, closeRegionRequest);
       return ResponseConverter.isClosed(response);
     } catch (ServiceException se) {
       throw getRemoteException(se);
@@ -1711,14 +1728,14 @@ public final class ProtobufUtil {
    * @param regionInfo
    *
    */
-  public static void warmupRegion(final AdminService.BlockingInterface admin,
-      final HRegionInfo regionInfo) throws IOException  {
+  public static void warmupRegion(final RpcController controller,
+      final AdminService.BlockingInterface admin, final HRegionInfo regionInfo) throws IOException {
 
     try {
       WarmupRegionRequest warmupRegionRequest =
            RequestConverter.buildWarmupRegionRequest(regionInfo);
 
-      admin.warmupRegion(null, warmupRegionRequest);
+      admin.warmupRegion(controller, warmupRegionRequest);
     } catch (ServiceException e) {
       throw getRemoteException(e);
     }
@@ -1730,18 +1747,18 @@ public final class ProtobufUtil {
    * @param region
    * @throws IOException
    */
-  public static void openRegion(final AdminService.BlockingInterface admin,
-      ServerName server, final HRegionInfo region) throws IOException {
+  public static void openRegion(final RpcController controller,
+      final AdminService.BlockingInterface admin, ServerName server, final HRegionInfo region)
+          throws IOException {
     OpenRegionRequest request =
       RequestConverter.buildOpenRegionRequest(server, region, null, null);
     try {
-      admin.openRegion(null, request);
+      admin.openRegion(controller, request);
     } catch (ServiceException se) {
       throw ProtobufUtil.getRemoteException(se);
     }
   }
 
-
   /**
    * A helper to get the all the online regions on a region
    * server using admin protocol.
@@ -1751,11 +1768,22 @@ public final class ProtobufUtil {
    * @throws IOException
    */
   public static List<HRegionInfo> getOnlineRegions(final AdminService.BlockingInterface admin)
+      throws IOException {
+    return getOnlineRegions(null, admin);
+  }
+
+  /**
+   * A helper to get the all the online regions on a region
+   * server using admin protocol.
+   * @return a list of online region info
+   */
+  public static List<HRegionInfo> getOnlineRegions(final RpcController controller,
+      final AdminService.BlockingInterface admin)
   throws IOException {
     GetOnlineRegionRequest request = RequestConverter.buildGetOnlineRegionRequest();
     GetOnlineRegionResponse response = null;
     try {
-      response = admin.getOnlineRegion(null, request);
+      response = admin.getOnlineRegion(controller, request);
     } catch (ServiceException se) {
       throw getRemoteException(se);
     }
@@ -1779,16 +1807,14 @@ public final class ProtobufUtil {
 
   /**
    * A helper to get the info of a region server using admin protocol.
-   *
-   * @param admin
    * @return the server name
-   * @throws IOException
    */
-  public static ServerInfo getServerInfo(final AdminService.BlockingInterface admin)
+  public static ServerInfo getServerInfo(final RpcController controller,
+      final AdminService.BlockingInterface admin)
   throws IOException {
     GetServerInfoRequest request = RequestConverter.buildGetServerInfoRequest();
     try {
-      GetServerInfoResponse response = admin.getServerInfo(null, request);
+      GetServerInfoResponse response = admin.getServerInfo(controller, request);
       return response.getServerInfo();
     } catch (ServiceException se) {
       throw getRemoteException(se);
@@ -1799,19 +1825,27 @@ public final class ProtobufUtil {
    * A helper to get the list of files of a column family
    * on a given region using admin protocol.
    *
-   * @param admin
-   * @param regionName
-   * @param family
    * @return the list of store files
-   * @throws IOException
    */
   public static List<String> getStoreFiles(final AdminService.BlockingInterface admin,
       final byte[] regionName, final byte[] family)
   throws IOException {
+    return getStoreFiles(null, admin, regionName, family);
+  }
+
+  /**
+   * A helper to get the list of files of a column family
+   * on a given region using admin protocol.
+   *
+   * @return the list of store files
+   */
+  public static List<String> getStoreFiles(final RpcController controller,
+      final AdminService.BlockingInterface admin, final byte[] regionName, final byte[] family)
+  throws IOException {
     GetStoreFileRequest request =
       RequestConverter.buildGetStoreFileRequest(regionName, family);
     try {
-      GetStoreFileResponse response = admin.getStoreFile(null, request);
+      GetStoreFileResponse response = admin.getStoreFile(controller, request);
       return response.getStoreFileList();
     } catch (ServiceException se) {
       throw ProtobufUtil.getRemoteException(se);
@@ -1826,12 +1860,13 @@ public final class ProtobufUtil {
    * @param splitPoint
    * @throws IOException
    */
-  public static void split(final AdminService.BlockingInterface admin,
-      final HRegionInfo hri, byte[] splitPoint) throws IOException {
+  public static void split(final RpcController controller,
+      final AdminService.BlockingInterface admin, final HRegionInfo hri, byte[] splitPoint)
+          throws IOException {
     SplitRegionRequest request =
       RequestConverter.buildSplitRegionRequest(hri.getRegionName(), splitPoint);
     try {
-      admin.splitRegion(null, request);
+      admin.splitRegion(controller, request);
     } catch (ServiceException se) {
       throw ProtobufUtil.getRemoteException(se);
     }
@@ -1848,7 +1883,8 @@ public final class ProtobufUtil {
    * @param user effective user
    * @throws IOException
    */
-  public static void mergeRegions(final AdminService.BlockingInterface admin,
+  public static void mergeRegions(final RpcController controller,
+      final AdminService.BlockingInterface admin,
       final HRegionInfo region_a, final HRegionInfo region_b,
       final boolean forcible, final User user) throws IOException {
     final MergeRegionsRequest request = RequestConverter.buildMergeRegionsRequest(
@@ -1858,7 +1894,7 @@ public final class ProtobufUtil {
         user.getUGI().doAs(new PrivilegedExceptionAction<Void>() {
           @Override
           public Void run() throws Exception {
-            admin.mergeRegions(null, request);
+            admin.mergeRegions(controller, request);
             return null;
           }
         });
@@ -1869,7 +1905,7 @@ public final class ProtobufUtil {
       }
     } else {
       try {
-        admin.mergeRegions(null, request);
+        admin.mergeRegions(controller, request);
       } catch (ServiceException se) {
         throw ProtobufUtil.getRemoteException(se);
       }
@@ -2144,8 +2180,9 @@ public final class ProtobufUtil {
    * @param actions the permissions to be granted
    * @throws ServiceException
    */
-  public static void grant(AccessControlService.BlockingInterface protocol,
-      String userShortName, Permission.Action... actions) throws ServiceException {
+  public static void grant(RpcController controller,
+      AccessControlService.BlockingInterface protocol, String userShortName,
+      Permission.Action... actions) throws ServiceException {
     List<AccessControlProtos.Permission.Action> permActions =
         Lists.newArrayListWithCapacity(actions.length);
     for (Permission.Action a : actions) {
@@ -2154,7 +2191,7 @@ public final class ProtobufUtil {
     AccessControlProtos.GrantRequest request = RequestConverter.
       buildGrantRequest(userShortName, permActions.toArray(
         new AccessControlProtos.Permission.Action[actions.length]));
-    protocol.grant(null, request);
+    protocol.grant(controller, request);
   }
 
   /**
@@ -2171,9 +2208,9 @@ public final class ProtobufUtil {
    * @param actions the permissions to be granted
    * @throws ServiceException
    */
-  public static void grant(AccessControlService.BlockingInterface protocol,
-      String userShortName, TableName tableName, byte[] f, byte[] q,
-      Permission.Action... actions) throws ServiceException {
+  public static void grant(RpcController controller,
+      AccessControlService.BlockingInterface protocol, String userShortName, TableName tableName,
+      byte[] f, byte[] q, Permission.Action... actions) throws ServiceException {
     List<AccessControlProtos.Permission.Action> permActions =
         Lists.newArrayListWithCapacity(actions.length);
     for (Permission.Action a : actions) {
@@ -2182,7 +2219,7 @@ public final class ProtobufUtil {
     AccessControlProtos.GrantRequest request = RequestConverter.
       buildGrantRequest(userShortName, tableName, f, q, permActions.toArray(
         new AccessControlProtos.Permission.Action[actions.length]));
-    protocol.grant(null, request);
+    protocol.grant(controller, request);
   }
 
   /**
@@ -2195,8 +2232,8 @@ public final class ProtobufUtil {
    * @param actions the permissions to be granted
    * @throws ServiceException
    */
-  public static void grant(AccessControlService.BlockingInterface protocol,
-      String userShortName, String namespace,
+  public static void grant(RpcController controller,
+      AccessControlService.BlockingInterface protocol, String userShortName, String namespace,
       Permission.Action... actions) throws ServiceException {
     List<AccessControlProtos.Permission.Action> permActions =
         Lists.newArrayListWithCapacity(actions.length);
@@ -2206,7 +2243,7 @@ public final class ProtobufUtil {
     AccessControlProtos.GrantRequest request = RequestConverter.
       buildGrantRequest(userShortName, namespace, permActions.toArray(
         new AccessControlProtos.Permission.Action[actions.length]));
-    protocol.grant(null, request);
+    protocol.grant(controller, request);
   }
 
   /**
@@ -2219,8 +2256,9 @@ public final class ProtobufUtil {
    * @param actions the permissions to be revoked
    * @throws ServiceException
    */
-  public static void revoke(AccessControlService.BlockingInterface protocol,
-      String userShortName, Permission.Action... actions) throws ServiceException {
+  public static void revoke(RpcController controller,
+      AccessControlService.BlockingInterface protocol, String userShortName,
+      Permission.Action... actions) throws ServiceException {
     List<AccessControlProtos.Permission.Action> permActions =
         Lists.newArrayListWithCapacity(actions.length);
     for (Permission.Action a : actions) {
@@ -2229,7 +2267,7 @@ public final class ProtobufUtil {
     AccessControlProtos.RevokeRequest request = RequestConverter.
       buildRevokeRequest(userShortName, permActions.toArray(
         new AccessControlProtos.Permission.Action[actions.length]));
-    protocol.revoke(null, request);
+    protocol.revoke(controller, request);
   }
 
   /**
@@ -2246,9 +2284,9 @@ public final class ProtobufUtil {
    * @param actions the permissions to be revoked
    * @throws ServiceException
    */
-  public static void revoke(AccessControlService.BlockingInterface protocol,
-      String userShortName, TableName tableName, byte[] f, byte[] q,
-      Permission.Action... actions) throws ServiceException {
+  public static void revoke(RpcController controller,
+      AccessControlService.BlockingInterface protocol, String userShortName, TableName tableName,
+      byte[] f, byte[] q, Permission.Action... actions) throws ServiceException {
     List<AccessControlProtos.Permission.Action> permActions =
         Lists.newArrayListWithCapacity(actions.length);
     for (Permission.Action a : actions) {
@@ -2257,7 +2295,7 @@ public final class ProtobufUtil {
     AccessControlProtos.RevokeRequest request = RequestConverter.
       buildRevokeRequest(userShortName, tableName, f, q, permActions.toArray(
         new AccessControlProtos.Permission.Action[actions.length]));
-    protocol.revoke(null, request);
+    protocol.revoke(controller, request);
   }
 
   /**
@@ -2271,8 +2309,8 @@ public final class ProtobufUtil {
    * @param actions the permissions to be revoked
    * @throws ServiceException
    */
-  public static void revoke(AccessControlService.BlockingInterface protocol,
-      String userShortName, String namespace,
+  public static void revoke(RpcController controller,
+      AccessControlService.BlockingInterface protocol, String userShortName, String namespace,
       Permission.Action... actions) throws ServiceException {
     List<AccessControlProtos.Permission.Action> permActions =
         Lists.newArrayListWithCapacity(actions.length);
@@ -2282,7 +2320,7 @@ public final class ProtobufUtil {
     AccessControlProtos.RevokeRequest request = RequestConverter.
       buildRevokeRequest(userShortName, namespace, permActions.toArray(
         new AccessControlProtos.Permission.Action[actions.length]));
-    protocol.revoke(null, request);
+    protocol.revoke(controller, request);
   }
 
   /**
@@ -2293,14 +2331,14 @@ public final class ProtobufUtil {
    * @param protocol the AccessControlService protocol proxy
    * @throws ServiceException
    */
-  public static List<UserPermission> getUserPermissions(
+  public static List<UserPermission> getUserPermissions(RpcController controller,
       AccessControlService.BlockingInterface protocol) throws ServiceException {
     AccessControlProtos.GetUserPermissionsRequest.Builder builder =
       AccessControlProtos.GetUserPermissionsRequest.newBuilder();
     builder.setType(AccessControlProtos.Permission.Type.Global);
     AccessControlProtos.GetUserPermissionsRequest request = builder.build();
     AccessControlProtos.GetUserPermissionsResponse response =
-      protocol.getUserPermissions(null, request);
+      protocol.getUserPermissions(controller, request);
     List<UserPermission> perms = new ArrayList<UserPermission>(response.getUserPermissionCount());
     for (AccessControlProtos.UserPermission perm: response.getUserPermissionList()) {
       perms.add(ProtobufUtil.toUserPermission(perm));
@@ -2317,7 +2355,7 @@ public final class ProtobufUtil {
    * @param t optional table name
    * @throws ServiceException
    */
-  public static List<UserPermission> getUserPermissions(
+  public static List<UserPermission> getUserPermissions(RpcController controller,
       AccessControlService.BlockingInterface protocol,
       TableName t) throws ServiceException {
     AccessControlProtos.GetUserPermissionsRequest.Builder builder =
@@ -2328,7 +2366,7 @@ public final class ProtobufUtil {
     builder.setType(AccessControlProtos.Permission.Type.Table);
     AccessControlProtos.GetUserPermissionsRequest request = builder.build();
     AccessControlProtos.GetUserPermissionsResponse response =
-      protocol.getUserPermissions(null, request);
+      protocol.getUserPermissions(controller, request);
     List<UserPermission> perms = new ArrayList<UserPermission>(response.getUserPermissionCount());
     for (AccessControlProtos.UserPermission perm: response.getUserPermissionList()) {
       perms.add(ProtobufUtil.toUserPermission(perm));
@@ -2345,7 +2383,7 @@ public final class ProtobufUtil {
    * @param namespace name of the namespace
    * @throws ServiceException
    */
-  public static List<UserPermission> getUserPermissions(
+  public static List<UserPermission> getUserPermissions(RpcController controller,
       AccessControlService.BlockingInterface protocol,
       byte[] namespace) throws ServiceException {
     AccessControlProtos.GetUserPermissionsRequest.Builder builder =
@@ -2356,7 +2394,7 @@ public final class ProtobufUtil {
     builder.setType(AccessControlProtos.Permission.Type.Namespace);
     AccessControlProtos.GetUserPermissionsRequest request = builder.build();
     AccessControlProtos.GetUserPermissionsResponse response =
-      protocol.getUserPermissions(null, request);
+      protocol.getUserPermissions(controller, request);
     List<UserPermission> perms = new ArrayList<UserPermission>(response.getUserPermissionCount());
     for (AccessControlProtos.UserPermission perm: response.getUserPermissionList()) {
       perms.add(ProtobufUtil.toUserPermission(perm));

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java
index c50abc1..25ac01f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java
@@ -31,10 +31,12 @@ 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.Connection;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.security.SecurityCapability;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
+import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
 import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService.BlockingInterface;
@@ -92,9 +94,12 @@ public class AccessControlClient {
   public static void grant(Connection connection, final TableName tableName,
       final String userName, final byte[] family, final byte[] qual,
       final Permission.Action... actions) throws Throwable {
+    PayloadCarryingRpcController controller
+      = ((ClusterConnection) connection).getRpcControllerFactory().newController();
+    controller.setPriority(tableName);
     try (Table table = connection.getTable(ACL_TABLE_NAME)) {
-      ProtobufUtil.grant(getAccessControlServiceStub(table), userName, tableName, family, qual,
-          actions);
+      ProtobufUtil.grant(controller, getAccessControlServiceStub(table), userName, tableName,
+        family, qual, actions);
     }
   }
 
@@ -108,8 +113,12 @@ public class AccessControlClient {
    */
   public static void grant(Connection connection, final String namespace,
       final String userName, final Permission.Action... actions) throws Throwable {
+    PayloadCarryingRpcController controller
+      = ((ClusterConnection) connection).getRpcControllerFactory().newController();
+
     try (Table table = connection.getTable(ACL_TABLE_NAME)) {
-      ProtobufUtil.grant(getAccessControlServiceStub(table), userName, namespace, actions);
+      ProtobufUtil.grant(controller, getAccessControlServiceStub(table), userName, namespace,
+        actions);
     }
   }
 
@@ -119,8 +128,10 @@ public class AccessControlClient {
    */
   public static void grant(Connection connection, final String userName,
        final Permission.Action... actions) throws Throwable {
+    PayloadCarryingRpcController controller
+      = ((ClusterConnection) connection).getRpcControllerFactory().newController();
     try (Table table = connection.getTable(ACL_TABLE_NAME)) {
-      ProtobufUtil.grant(getAccessControlServiceStub(table), userName, actions);
+      ProtobufUtil.grant(controller, getAccessControlServiceStub(table), userName, actions);
     }
   }
 
@@ -144,9 +155,12 @@ public class AccessControlClient {
   public static void revoke(Connection connection, final TableName tableName,
       final String username, final byte[] family, final byte[] qualifier,
       final Permission.Action... actions) throws Throwable {
+    PayloadCarryingRpcController controller
+      = ((ClusterConnection) connection).getRpcControllerFactory().newController();
+    controller.setPriority(tableName);
     try (Table table = connection.getTable(ACL_TABLE_NAME)) {
-      ProtobufUtil.revoke(getAccessControlServiceStub(table), username, tableName, family,
-          qualifier, actions);
+      ProtobufUtil.revoke(controller, getAccessControlServiceStub(table), username, tableName,
+        family, qualifier, actions);
     }
   }
 
@@ -160,8 +174,11 @@ public class AccessControlClient {
    */
   public static void revoke(Connection connection, final String namespace,
       final String userName, final Permission.Action... actions) throws Throwable {
+    PayloadCarryingRpcController controller
+      = ((ClusterConnection) connection).getRpcControllerFactory().newController();
     try (Table table = connection.getTable(ACL_TABLE_NAME)) {
-      ProtobufUtil.revoke(getAccessControlServiceStub(table), userName, namespace, actions);
+      ProtobufUtil.revoke(controller, getAccessControlServiceStub(table), userName, namespace,
+        actions);
     }
   }
 
@@ -171,10 +188,11 @@ public class AccessControlClient {
    */
   public static void revoke(Connection connection, final String userName,
       final Permission.Action... actions) throws Throwable {
+    PayloadCarryingRpcController controller
+      = ((ClusterConnection) connection).getRpcControllerFactory().newController();
     try (Table table = connection.getTable(ACL_TABLE_NAME)) {
-      ProtobufUtil.revoke(getAccessControlServiceStub(table), userName, actions);
+      ProtobufUtil.revoke(controller, getAccessControlServiceStub(table), userName, actions);
     }
-
   }
 
   /**
@@ -188,6 +206,8 @@ public class AccessControlClient {
    */
   public static List<UserPermission> getUserPermissions(Connection connection, String tableRegex)
       throws Throwable {
+    PayloadCarryingRpcController controller
+      = ((ClusterConnection) connection).getRpcControllerFactory().newController();
     List<UserPermission> permList = new ArrayList<UserPermission>();
     try (Table table = connection.getTable(ACL_TABLE_NAME)) {
       try (Admin admin = connection.getAdmin()) {
@@ -196,14 +216,16 @@ public class AccessControlClient {
             AccessControlProtos.AccessControlService.newBlockingStub(service);
         HTableDescriptor[] htds = null;
         if (tableRegex == null || tableRegex.isEmpty()) {
-          permList = ProtobufUtil.getUserPermissions(protocol);
+          permList = ProtobufUtil.getUserPermissions(controller, protocol);
         } else if (tableRegex.charAt(0) == '@') {
           String namespace = tableRegex.substring(1);
-          permList = ProtobufUtil.getUserPermissions(protocol, Bytes.toBytes(namespace));
+          permList = ProtobufUtil.getUserPermissions(controller, protocol,
+            Bytes.toBytes(namespace));
         } else {
           htds = admin.listTables(Pattern.compile(tableRegex), true);
           for (HTableDescriptor hd : htds) {
-            permList.addAll(ProtobufUtil.getUserPermissions(protocol, hd.getTableName()));
+            permList.addAll(ProtobufUtil.getUserPermissions(controller, protocol,
+              hd.getTableName()));
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
index 0b844a2..0b53f95 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
@@ -36,11 +36,14 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
 import org.apache.hadoop.hbase.ServerName;
 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.HConnection;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.client.RetriesExhaustedException;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.ipc.FailedServerException;
+import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
 import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
 import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@@ -287,7 +290,7 @@ public class MetaTableLocator {
     } catch (RegionServerStoppedException e) {
       // Pass -- server name sends us to a server that is dying or already dead.
     }
-    return (service != null) && verifyRegionLocation(service,
+    return (service != null) && verifyRegionLocation(hConnection, service,
             getMetaRegionLocation(zkw, replicaId), RegionReplicaUtil.getRegionInfoForReplica(
                 HRegionInfo.FIRST_META_REGIONINFO, replicaId).getRegionName());
   }
@@ -307,17 +310,22 @@ public class MetaTableLocator {
   // rather than have to pass it in.  Its made awkward by the fact that the
   // HRI is likely a proxy against remote server so the getServerName needs
   // to be fixed to go to a local method or to a cache before we can do this.
-  private boolean verifyRegionLocation(AdminService.BlockingInterface hostingServer,
-      final ServerName address, final byte [] regionName)
+  private boolean verifyRegionLocation(final Connection connection,
+      AdminService.BlockingInterface hostingServer, final ServerName address,
+      final byte [] regionName)
   throws IOException {
     if (hostingServer == null) {
       LOG.info("Passed hostingServer is null");
       return false;
     }
     Throwable t;
+    PayloadCarryingRpcController controller = null;
+    if (connection instanceof ClusterConnection) {
+      controller = ((ClusterConnection) connection).getRpcControllerFactory().newController();
+    }
     try {
       // Try and get regioninfo from the hosting server.
-      return ProtobufUtil.getRegionInfo(hostingServer, regionName) != null;
+      return ProtobufUtil.getRegionInfo(controller, hostingServer, regionName) != null;
     } catch (ConnectException e) {
       t = e;
     } catch (RetriesExhaustedException e) {
@@ -594,7 +602,7 @@ public class MetaTableLocator {
     ServerName sn = null;
     while (true) {
       sn = getMetaRegionLocation(zkw, replicaId);
-      if (sn != null || (System.currentTimeMillis() - startTime) 
+      if (sn != null || (System.currentTimeMillis() - startTime)
           > timeout - HConstants.SOCKET_RETRY_WAIT_MS) {
         break;
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java
index 1b039bd..4d55c33 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java
@@ -28,6 +28,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
@@ -78,26 +80,34 @@ public class TestSnapshotFromAdmin {
     // setup the conf to match the expected properties
     conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, numRetries);
     conf.setLong("hbase.client.pause", pauseTime);
+
     // mock the master admin to our mock
     MasterKeepAliveConnection mockMaster = Mockito.mock(MasterKeepAliveConnection.class);
     Mockito.when(mockConnection.getConfiguration()).thenReturn(conf);
     Mockito.when(mockConnection.getKeepAliveMasterService()).thenReturn(mockMaster);
+    // we need a real retrying caller
+    RpcRetryingCallerFactory callerFactory = new RpcRetryingCallerFactory(conf);
+    RpcControllerFactory controllerFactory = Mockito.mock(RpcControllerFactory.class);
+    Mockito.when(controllerFactory.newController()).thenReturn(
+      Mockito.mock(PayloadCarryingRpcController.class));
+    Mockito.when(mockConnection.getRpcRetryingCallerFactory()).thenReturn(callerFactory);
+    Mockito.when(mockConnection.getRpcControllerFactory()).thenReturn(controllerFactory);
     // set the max wait time for the snapshot to complete
     SnapshotResponse response = SnapshotResponse.newBuilder()
         .setExpectedTimeout(maxWaitTime)
         .build();
     Mockito
-        .when(
-          mockMaster.snapshot((RpcController) Mockito.isNull(),
-            Mockito.any(SnapshotRequest.class))).thenReturn(response);
+    .when(
+      mockMaster.snapshot((RpcController) Mockito.any(),
+        Mockito.any(SnapshotRequest.class))).thenReturn(response);
     // setup the response
     IsSnapshotDoneResponse.Builder builder = IsSnapshotDoneResponse.newBuilder();
     builder.setDone(false);
     // first five times, we return false, last we get success
     Mockito.when(
-      mockMaster.isSnapshotDone((RpcController) Mockito.isNull(),
+      mockMaster.isSnapshotDone((RpcController) Mockito.any(),
         Mockito.any(IsSnapshotDoneRequest.class))).thenReturn(builder.build(), builder.build(),
-      builder.build(), builder.build(), builder.build(), builder.setDone(true).build());
+          builder.build(), builder.build(), builder.build(), builder.setDone(true).build());
 
     // setup the admin and run the test
     Admin admin = new HBaseAdmin(mockConnection);
@@ -123,6 +133,13 @@ public class TestSnapshotFromAdmin {
         .mock(ConnectionImplementation.class);
     Configuration conf = HBaseConfiguration.create();
     Mockito.when(mockConnection.getConfiguration()).thenReturn(conf);
+    // we need a real retrying caller
+    RpcRetryingCallerFactory callerFactory = new RpcRetryingCallerFactory(conf);
+    RpcControllerFactory controllerFactory = Mockito.mock(RpcControllerFactory.class);
+    Mockito.when(controllerFactory.newController()).thenReturn(
+      Mockito.mock(PayloadCarryingRpcController.class));
+    Mockito.when(mockConnection.getRpcRetryingCallerFactory()).thenReturn(callerFactory);
+    Mockito.when(mockConnection.getRpcControllerFactory()).thenReturn(controllerFactory);
     Admin admin = new HBaseAdmin(mockConnection);
     SnapshotDescription.Builder builder = SnapshotDescription.newBuilder();
     // check that invalid snapshot names fail
@@ -142,11 +159,11 @@ public class TestSnapshotFromAdmin {
     Mockito.when(mockConnection.getKeepAliveMasterService()).thenReturn(master);
     SnapshotResponse response = SnapshotResponse.newBuilder().setExpectedTimeout(0).build();
     Mockito.when(
-      master.snapshot((RpcController) Mockito.isNull(), Mockito.any(SnapshotRequest.class)))
+      master.snapshot((RpcController) Mockito.any(), Mockito.any(SnapshotRequest.class)))
         .thenReturn(response);
     IsSnapshotDoneResponse doneResponse = IsSnapshotDoneResponse.newBuilder().setDone(true).build();
     Mockito.when(
-      master.isSnapshotDone((RpcController) Mockito.isNull(),
+      master.isSnapshotDone((RpcController) Mockito.any(),
           Mockito.any(IsSnapshotDoneRequest.class))).thenReturn(doneResponse);
 
       // make sure that we can use valid names

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java
index 6f3baa0..31ca996 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java
@@ -279,7 +279,7 @@ public class DistributedHBaseCluster extends HBaseCluster {
 
     AdminProtos.AdminService.BlockingInterface client =
         ((ClusterConnection)this.connection).getAdmin(regionLoc.getServerName());
-    ServerInfo info = ProtobufUtil.getServerInfo(client);
+    ServerInfo info = ProtobufUtil.getServerInfo(null, client);
     return ProtobufUtil.toServerName(info.getServerName());
   }
 
@@ -433,7 +433,7 @@ public class DistributedHBaseCluster extends HBaseCluster {
     Set<ServerName> toKill = new TreeSet<ServerName>(new ServerNameIgnoreStartCodeComparator());
     toStart.addAll(initial.getServers());
     toKill.addAll(current.getServers());
-    
+
     ServerName master = initial.getMaster();
 
     for (ServerName server : current.getServers()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon
index 158a239..b98f50d 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon
@@ -42,7 +42,7 @@ org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
   <%java return; %>
 </%if>
 <%java>
-  ServerInfo serverInfo = ProtobufUtil.getServerInfo(regionServer.getRSRpcServices());
+  ServerInfo serverInfo = ProtobufUtil.getServerInfo(null, regionServer.getRSRpcServices());
   ServerName serverName = ProtobufUtil.toServerName(serverInfo.getServerName());
   List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(regionServer.getRSRpcServices());
   MasterAddressTracker masterAddressTracker = regionServer.getMasterAddressTracker();

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
index 64a75b9..a9cf0f1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
@@ -22,7 +22,9 @@ import java.nio.channels.ClosedChannelException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.CellScanner;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.ipc.RpcServer.Call;
 import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
 import org.apache.hadoop.hbase.util.Pair;
@@ -38,7 +40,8 @@ import com.google.protobuf.Message;
  * {@link RpcScheduler}.  Call {@link #run()} to actually execute the contained
  * RpcServer.Call
  */
-@InterfaceAudience.Private
+@InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
+@InterfaceStability.Evolving
 public class CallRunner {
   private static final Log LOG = LogFactory.getLog(CallRunner.class);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcScheduler.java
index 2414e3d..91c152b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcScheduler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcScheduler.java
@@ -37,7 +37,7 @@ public abstract class RpcScheduler {
       "hbase.ipc.server.priority.max.callqueue.length";
 
   /** Exposes runtime information of a {@code RpcServer} that a {@code RpcScheduler} may need. */
-  static abstract class Context {
+  public static abstract class Context {
     public abstract InetSocketAddress getListenerAddress();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/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 2c6084a..f0aed2e 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
@@ -289,7 +289,9 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
    * Datastructure that holds all necessary to a method invocation and then afterward, carries
    * the result.
    */
-  class Call implements RpcCallContext {
+  @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
+  @InterfaceStability.Evolving
+  public class Call implements RpcCallContext {
     protected int id;                             // the client's call id
     protected BlockingService service;
     protected MethodDescriptor md;
@@ -369,6 +371,14 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
       return this.header;
     }
 
+    public boolean hasPriority() {
+      return this.header.hasPriority();
+    }
+
+    public int getPriority() {
+      return this.header.getPriority();
+    }
+
     /*
      * Short string representation without param info because param itself could be huge depends on
      * the payload of a command

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java
index 196320d..d31711e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java
@@ -695,7 +695,7 @@ public class RegionPlacementMaintainer {
           UpdateFavoredNodesResponse updateFavoredNodesResponse =
               currentRegionServer.updateFavoredNodes(null, request);
           LOG.info("Region server " +
-              ProtobufUtil.getServerInfo(currentRegionServer).getServerName() +
+              ProtobufUtil.getServerInfo(null, currentRegionServer).getServerName() +
               " has updated " + updateFavoredNodesResponse.getResponse() + " / " +
               singleServerPlan.getAssignmentMap().size() +
               " regions with the assignment plan");

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
index 341d51c..dabef71 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
@@ -50,6 +50,8 @@ import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.RetriesExhaustedException;
+import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer;
 import org.apache.hadoop.hbase.master.procedure.ServerCrashProcedure;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
@@ -158,6 +160,7 @@ public class ServerManager {
   private final long warningSkew;
 
   private final RetryCounterFactory pingRetryCounterFactory;
+  private final RpcControllerFactory rpcControllerFactory;
 
   /**
    * Set of region servers which are dead but not processed immediately. If one
@@ -222,6 +225,9 @@ public class ServerManager {
     int pingSleepInterval = Math.max(1, master.getConfiguration().getInt(
       "hbase.master.ping.server.retry.sleep.interval", 100));
     this.pingRetryCounterFactory = new RetryCounterFactory(pingMaxAttempts, pingSleepInterval);
+    this.rpcControllerFactory = this.connection == null
+        ? null
+        : connection.getRpcControllerFactory();
   }
 
   /**
@@ -784,6 +790,10 @@ public class ServerManager {
     }
   }
 
+  private PayloadCarryingRpcController newRpcController() {
+    return rpcControllerFactory == null ? null : rpcControllerFactory.newController();
+  }
+
   /**
    * Sends an CLOSE RPC to the specified server to close the specified region.
    * <p>
@@ -804,8 +814,8 @@ public class ServerManager {
         region.getRegionNameAsString() +
         " failed because no RPC connection found to this server");
     }
-    return ProtobufUtil.closeRegion(admin, server, region.getRegionName(),
-      dest);
+    PayloadCarryingRpcController controller = newRpcController();
+    return ProtobufUtil.closeRegion(controller, admin, server, region.getRegionName(), dest);
   }
 
   public boolean sendRegionClose(ServerName server,
@@ -826,7 +836,8 @@ public class ServerManager {
     if (server == null) return;
     try {
       AdminService.BlockingInterface admin = getRsAdmin(server);
-      ProtobufUtil.warmupRegion(admin, region);
+      PayloadCarryingRpcController controller = newRpcController();
+      ProtobufUtil.warmupRegion(controller, admin, region);
     } catch (IOException e) {
       LOG.error("Received exception in RPC for warmup server:" +
         server + "region: " + region +
@@ -838,11 +849,12 @@ public class ServerManager {
    * Contacts a region server and waits up to timeout ms
    * to close the region.  This bypasses the active hmaster.
    */
-  public static void closeRegionSilentlyAndWait(ClusterConnection connection, 
+  public static void closeRegionSilentlyAndWait(ClusterConnection connection,
     ServerName server, HRegionInfo region, long timeout) throws IOException, InterruptedException {
     AdminService.BlockingInterface rs = connection.getAdmin(server);
+    PayloadCarryingRpcController controller = connection.getRpcControllerFactory().newController();
     try {
-      ProtobufUtil.closeRegion(rs, server, region.getRegionName());
+      ProtobufUtil.closeRegion(controller, rs, server, region.getRegionName());
     } catch (IOException e) {
       LOG.warn("Exception when closing region: " + region.getRegionNameAsString(), e);
     }
@@ -850,12 +862,13 @@ public class ServerManager {
     while (System.currentTimeMillis() < expiration) {
       try {
         HRegionInfo rsRegion =
-          ProtobufUtil.getRegionInfo(rs, region.getRegionName());
+          ProtobufUtil.getRegionInfo(controller, rs, region.getRegionName());
         if (rsRegion == null) return;
       } catch (IOException ioe) {
         if (ioe instanceof NotServingRegionException) // no need to retry again
           return;
-        LOG.warn("Exception when retrieving regioninfo from: " + region.getRegionNameAsString(), ioe);
+        LOG.warn("Exception when retrieving regioninfo from: "
+          + region.getRegionNameAsString(), ioe);
       }
       Thread.sleep(1000);
     }
@@ -890,7 +903,8 @@ public class ServerManager {
           + region_b.getRegionNameAsString()
           + " failed because no RPC connection found to this server");
     }
-    ProtobufUtil.mergeRegions(admin, region_a, region_b, forcible, user);
+    PayloadCarryingRpcController controller = newRpcController();
+    ProtobufUtil.mergeRegions(controller, admin, region_a, region_b, forcible, user);
   }
 
   /**
@@ -899,12 +913,14 @@ public class ServerManager {
   public boolean isServerReachable(ServerName server) {
     if (server == null) throw new NullPointerException("Passed server is null");
 
+
     RetryCounter retryCounter = pingRetryCounterFactory.create();
     while (retryCounter.shouldRetry()) {
       try {
+        PayloadCarryingRpcController controller = newRpcController();
         AdminService.BlockingInterface admin = getRsAdmin(server);
         if (admin != null) {
-          ServerInfo info = ProtobufUtil.getServerInfo(admin);
+          ServerInfo info = ProtobufUtil.getServerInfo(controller, admin);
           return info != null && info.hasServerName()
             && server.getStartcode() == info.getServerName().getStartCode();
         }

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
index 3bd3c3f..002bdb2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
@@ -693,8 +693,8 @@ public class MiniHBaseCluster extends HBaseCluster {
     int count = 0;
     for (JVMClusterUtil.RegionServerThread rst: getRegionServerThreads()) {
       HRegionServer hrs = rst.getRegionServer();
-      Region metaRegion = hrs.getOnlineRegion(regionName);
-      if (metaRegion != null) {
+      Region region = hrs.getOnlineRegion(regionName);
+      if (region != null) {
         index = count;
         break;
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java
index e460535..8ac89da 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java
@@ -55,11 +55,11 @@ public class TestGlobalMemStoreSize {
 
   private HBaseTestingUtility TEST_UTIL;
   private MiniHBaseCluster cluster;
-  
+
   /**
    * Test the global mem store size in the region server is equal to sum of each
    * region's mem store size
-   * @throws Exception 
+   * @throws Exception
    */
   @Test
   public void testGlobalMemStore() throws Exception {
@@ -87,8 +87,8 @@ public class TestGlobalMemStoreSize {
     for (HRegionServer server : getOnlineRegionServers()) {
       long globalMemStoreSize = 0;
       for (HRegionInfo regionInfo :
-          ProtobufUtil.getOnlineRegions(server.getRSRpcServices())) {
-        globalMemStoreSize += 
+          ProtobufUtil.getOnlineRegions(null, server.getRSRpcServices())) {
+        globalMemStoreSize +=
           server.getFromOnlineRegions(regionInfo.getEncodedName()).
           getMemstoreSize();
       }
@@ -103,7 +103,7 @@ public class TestGlobalMemStoreSize {
         ", size=" + server.getRegionServerAccounting().getGlobalMemstoreSize());
 
       for (HRegionInfo regionInfo :
-          ProtobufUtil.getOnlineRegions(server.getRSRpcServices())) {
+          ProtobufUtil.getOnlineRegions(null, server.getRSRpcServices())) {
         Region r = server.getFromOnlineRegions(regionInfo.getEncodedName());
         flush(r, server);
       }
@@ -119,7 +119,7 @@ public class TestGlobalMemStoreSize {
         // If size > 0, see if its because the meta region got edits while
         // our test was running....
         for (HRegionInfo regionInfo :
-            ProtobufUtil.getOnlineRegions(server.getRSRpcServices())) {
+            ProtobufUtil.getOnlineRegions(null, server.getRSRpcServices())) {
           Region r = server.getFromOnlineRegions(regionInfo.getEncodedName());
           long l = r.getMemstoreSize();
           if (l > 0) {
@@ -154,7 +154,7 @@ public class TestGlobalMemStoreSize {
 
   private List<HRegionServer> getOnlineRegionServers() {
     List<HRegionServer> list = new ArrayList<HRegionServer>();
-    for (JVMClusterUtil.RegionServerThread rst : 
+    for (JVMClusterUtil.RegionServerThread rst :
           cluster.getRegionServerThreads()) {
       if (rst.getRegionServer().isOnline()) {
         list.add(rst.getRegionServer());

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/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 7e934c0..8b84452 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
@@ -44,6 +44,13 @@ import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.ipc.CallRunner;
+import org.apache.hadoop.hbase.ipc.DelegatingRpcScheduler;
+import org.apache.hadoop.hbase.ipc.PriorityFunction;
+import org.apache.hadoop.hbase.ipc.RpcScheduler;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.regionserver.RSRpcServices;
+import org.apache.hadoop.hbase.regionserver.SimpleRpcSchedulerFactory;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -618,5 +625,77 @@ public class TestMetaTableAccessor {
       meta.close();
     }
   }
+
+  public static class SpyingRpcSchedulerFactory extends SimpleRpcSchedulerFactory {
+    @Override
+    public RpcScheduler create(Configuration conf, PriorityFunction priority, Abortable server) {
+      final RpcScheduler delegate = super.create(conf, priority, server);
+      return new SpyingRpcScheduler(delegate);
+    }
+  }
+
+  public static class SpyingRpcScheduler extends DelegatingRpcScheduler {
+    long numPriorityCalls = 0;
+
+    public SpyingRpcScheduler(RpcScheduler delegate) {
+      super(delegate);
+    }
+
+    @Override
+    public boolean dispatch(CallRunner task) throws IOException, InterruptedException {
+      int priority = task.getCall().getPriority();
+
+      if (priority > HConstants.QOS_THRESHOLD) {
+        numPriorityCalls++;
+      }
+      return super.dispatch(task);
+    }
+  }
+
+  @Test
+  public void testMetaUpdatesGoToPriorityQueue() throws Exception {
+    // This test has to be end-to-end, and do the verification from the server side
+    Configuration c = UTIL.getConfiguration();
+
+    c.set(RSRpcServices.REGION_SERVER_RPC_SCHEDULER_FACTORY_CLASS,
+      SpyingRpcSchedulerFactory.class.getName());
+
+    // restart so that new config takes place
+    afterClass();
+    beforeClass();
+
+    TableName tableName = TableName.valueOf("foo");
+    try (Admin admin = connection.getAdmin();
+        RegionLocator rl = connection.getRegionLocator(tableName)) {
+
+      // create a table and prepare for a manual split
+      UTIL.createTable(tableName, "cf1");
+
+      HRegionLocation loc = rl.getAllRegionLocations().get(0);
+      HRegionInfo parent = loc.getRegionInfo();
+      long rid = 1000;
+      byte[] splitKey = Bytes.toBytes("a");
+      HRegionInfo splitA = new HRegionInfo(parent.getTable(), parent.getStartKey(),
+        splitKey, false, rid);
+      HRegionInfo splitB = new HRegionInfo(parent.getTable(), splitKey,
+        parent.getEndKey(), false, rid);
+
+      // find the meta server
+      MiniHBaseCluster cluster = UTIL.getMiniHBaseCluster();
+      int rsIndex = cluster.getServerWithMeta();
+      HRegionServer rs;
+      if (rsIndex >= 0) {
+        rs = cluster.getRegionServer(rsIndex);
+      } else {
+        // it is in master
+        rs = cluster.getMaster();
+      }
+      SpyingRpcScheduler scheduler = (SpyingRpcScheduler) rs.getRpcServer().getScheduler();
+      long prevCalls = scheduler.numPriorityCalls;
+      MetaTableAccessor.splitRegion(connection, parent, splitA, splitB, loc.getServerName(), 1);
+
+      assertTrue(prevCalls < scheduler.numPriorityCalls);
+    }
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableLocator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableLocator.java
index 9943749..ba6e1d4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableLocator.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableLocator.java
@@ -31,6 +31,8 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
+import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
 import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
@@ -180,7 +182,7 @@ public class TestMetaTableLocator {
     // Mock an ClientProtocol.
     final ClientProtos.ClientService.BlockingInterface implementation =
       Mockito.mock(ClientProtos.ClientService.BlockingInterface.class);
- 
+
     ClusterConnection connection = mockConnection(null, implementation);
 
     // If a 'get' is called on mocked interface, throw connection refused.
@@ -250,6 +252,10 @@ public class TestMetaTableLocator {
       (GetRegionInfoRequest)Mockito.any())).thenThrow(connectException);
     Mockito.when(connection.getAdmin(Mockito.any(ServerName.class))).
       thenReturn(implementation);
+        RpcControllerFactory controllerFactory = Mockito.mock(RpcControllerFactory.class);
+        Mockito.when(controllerFactory.newController()).thenReturn(
+          Mockito.mock(PayloadCarryingRpcController.class));
+        Mockito.when(connection.getRpcControllerFactory()).thenReturn(controllerFactory);
 
     ServerName sn = ServerName.valueOf("example.com", 1234, System.currentTimeMillis());
     MetaTableLocator.setMetaLocation(this.watcher,

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
index 24c0c32..dc1ecf1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
@@ -54,6 +54,11 @@ public class HConnectionTestingUtility {
   throws ZooKeeperConnectionException {
     ConnectionImplementation connection = Mockito.mock(ConnectionImplementation.class);
     Mockito.when(connection.getConfiguration()).thenReturn(conf);
+    Mockito.when(connection.getRpcControllerFactory()).thenReturn(
+      Mockito.mock(RpcControllerFactory.class));
+    // we need a real retrying caller
+    RpcRetryingCallerFactory callerFactory = new RpcRetryingCallerFactory(conf);
+    Mockito.when(connection.getRpcRetryingCallerFactory()).thenReturn(callerFactory);
     return connection;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/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 df8f4f6..10dbed0 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
@@ -1254,8 +1254,8 @@ public class TestAdmin1 {
     try {
       AdminService.BlockingInterface admin = TEST_UTIL.getHBaseAdmin().getConnection()
           .getAdmin(regions.get(1).getSecond());
-      ProtobufUtil.mergeRegions(admin, regions.get(1).getFirst(), regions.get(2).getFirst(), true,
-        null);
+      ProtobufUtil.mergeRegions(null, admin, regions.get(1).getFirst(), regions.get(2).getFirst(),
+        true, null);
     } catch (MergeRegionException mm) {
       gotException = true;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
index 3b88184..520f210 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
@@ -604,7 +604,7 @@ public class TestFromClientSide {
   public void testMaxKeyValueSize() throws Exception {
     TableName TABLE = TableName.valueOf("testMaxKeyValueSize");
     Configuration conf = TEST_UTIL.getConfiguration();
-    String oldMaxSize = conf.get(TableConfiguration.MAX_KEYVALUE_SIZE_KEY);
+    String oldMaxSize = conf.get(ConnectionConfiguration.MAX_KEYVALUE_SIZE_KEY);
     Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
     byte[] value = new byte[4 * 1024 * 1024];
     Put put = new Put(ROW);
@@ -612,7 +612,7 @@ public class TestFromClientSide {
     ht.put(put);
     try {
       TEST_UTIL.getConfiguration().setInt(
-          TableConfiguration.MAX_KEYVALUE_SIZE_KEY, 2 * 1024 * 1024);
+          ConnectionConfiguration.MAX_KEYVALUE_SIZE_KEY, 2 * 1024 * 1024);
       // Create new table so we pick up the change in Configuration.
       try (Connection connection =
           ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())) {
@@ -624,7 +624,7 @@ public class TestFromClientSide {
       }
       fail("Inserting a too large KeyValue worked, should throw exception");
     } catch(Exception e) {}
-    conf.set(TableConfiguration.MAX_KEYVALUE_SIZE_KEY, oldMaxSize);
+    conf.set(ConnectionConfiguration.MAX_KEYVALUE_SIZE_KEY, oldMaxSize);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
index 5995191..ddd5fa3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
@@ -270,20 +270,20 @@ public class TestFromClientSide3 {
       // create an empty Put
       Put put1 = new Put(ROW);
       actions.add(put1);
-      
+
       Put put2 = new Put(ANOTHERROW);
       put2.addColumn(FAMILY, QUALIFIER, VALUE);
       actions.add(put2);
-      
+
       table.batch(actions, results);
       fail("Empty Put should have failed the batch call");
     } catch (IllegalArgumentException iae) {
-      
+
     } finally {
       table.close();
     }
   }
-  
+
   @Test
   public void testHTableExistsMethodSingleRegionSingleGet() throws Exception {
       // Test with a single region table.
@@ -401,7 +401,7 @@ public class TestFromClientSide3 {
   @Test
   public void testHTableExistsMethodMultipleRegionsMultipleGets() throws Exception {
     Table table = TEST_UTIL.createTable(
-      TableName.valueOf("testHTableExistsMethodMultipleRegionsMultipleGets"), 
+      TableName.valueOf("testHTableExistsMethodMultipleRegionsMultipleGets"),
       new byte[][] { FAMILY }, 1, new byte[] { 0x00 }, new byte[] { (byte) 0xff }, 255);
     Put put = new Put(ROW);
     put.addColumn(FAMILY, QUALIFIER, VALUE);


[26/50] [abbrv] hbase git commit: HBASE-15524 Fix NPE in client-side metrics

Posted by sy...@apache.org.
HBASE-15524 Fix NPE in client-side metrics


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

Branch: refs/heads/hbase-12439
Commit: db3ba652f88083b0b1c57b4857f11fce7ae5b131
Parents: fd5c093
Author: Mikhail Antonov <an...@apache.org>
Authored: Wed Mar 23 21:23:54 2016 -0700
Committer: Mikhail Antonov <an...@apache.org>
Committed: Wed Mar 23 21:23:54 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/client/AsyncProcess.java       | 26 +++++++++++++++++---
 .../hadoop/hbase/client/MetricsConnection.java  |  4 ++-
 2 files changed, 25 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/db3ba652/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
index cb45cf7..142e2a0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
@@ -1190,9 +1190,15 @@ class AsyncProcess {
         byte[] row = e.getValue().iterator().next().getAction().getRow();
         // Do not use the exception for updating cache because it might be coming from
         // any of the regions in the MultiAction.
-        if (tableName != null) {
-          connection.updateCachedLocations(tableName, regionName, row,
+        try {
+          if (tableName != null) {
+            connection.updateCachedLocations(tableName, regionName, row,
               ClientExceptionsUtil.isMetaClearingException(t) ? null : t, server);
+          }
+        } catch (Throwable ex) {
+          // That should never happen, but if it did, we want to make sure
+          // we still process errors
+          LOG.error("Couldn't update cached region locations: " + ex);
         }
         for (Action<Row> action : e.getValue()) {
           Retry retry = manageError(
@@ -1317,8 +1323,14 @@ class AsyncProcess {
             // Register corresponding failures once per server/once per region.
             if (!regionFailureRegistered) {
               regionFailureRegistered = true;
-              connection.updateCachedLocations(
+              try {
+                connection.updateCachedLocations(
                   tableName, regionName, row.getRow(), result, server);
+              } catch (Throwable ex) {
+                // That should never happen, but if it did, we want to make sure
+                // we still process errors
+                LOG.error("Couldn't update cached region locations: " + ex);
+              }
             }
             if (failureCount == 0) {
               errorsByServer.reportServerError(server);
@@ -1372,8 +1384,14 @@ class AsyncProcess {
           // for every possible exception that comes through, however.
           connection.clearCaches(server);
         } else {
-          connection.updateCachedLocations(
+          try {
+            connection.updateCachedLocations(
               tableName, region, actions.get(0).getAction().getRow(), throwable, server);
+          } catch (Throwable ex) {
+            // That should never happen, but if it did, we want to make sure
+            // we still process errors
+            LOG.error("Couldn't update cached region locations: " + ex);
+          }
         }
         failureCount += actions.size();
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/db3ba652/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetricsConnection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetricsConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetricsConnection.java
index 4467417..53a3326 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetricsConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetricsConnection.java
@@ -63,6 +63,7 @@ public class MetricsConnection implements StatisticTrackable {
   private static final String MEMLOAD_BASE = "memstoreLoad_";
   private static final String HEAP_BASE = "heapOccupancy_";
   private static final String CACHE_BASE = "cacheDroppingExceptions_";
+  private static final String UNKNOWN_EXCEPTION = "UnknownException";
   private static final String CLIENT_SVC = ClientService.getDescriptor().getName();
 
   /** A container class for collecting details about the RPC call as it percolates. */
@@ -464,7 +465,8 @@ public class MetricsConnection implements StatisticTrackable {
   }
 
   public void incrCacheDroppingExceptions(Object exception) {
-    getMetric(CACHE_BASE + exception.getClass().getSimpleName(),
+    getMetric(CACHE_BASE +
+      (exception == null? UNKNOWN_EXCEPTION : exception.getClass().getSimpleName()),
       cacheDroppingExceptions, counterFactory).inc();
   }
 }


[07/50] [abbrv] hbase git commit: HBASE-15477 Purge 'next block header' from cached blocks

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/000117ad/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileScanner.java
index c67bdd4..e0f3d74 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileScanner.java
@@ -99,18 +99,21 @@ public interface HFileScanner extends Shipper, Closeable {
    * @throws IOException
    */
   boolean seekTo() throws IOException;
+
   /**
    * Scans to the next entry in the file.
    * @return Returns false if you are at the end otherwise true if more in file.
    * @throws IOException
    */
   boolean next() throws IOException;
+
   /**
    * Gets the current key in the form of a cell. You must call
    * {@link #seekTo(Cell)} before this method.
    * @return gets the current key as a Cell.
    */
   Cell getKey();
+
   /**
    * Gets a buffer view to the current value.  You must call
    * {@link #seekTo(Cell)} before this method.
@@ -119,26 +122,35 @@ public interface HFileScanner extends Shipper, Closeable {
    * the position is 0, the start of the buffer view.
    */
   ByteBuffer getValue();
+
   /**
    * @return Instance of {@link org.apache.hadoop.hbase.Cell}.
    */
   Cell getCell();
+
   /**
    * Convenience method to get a copy of the key as a string - interpreting the
    * bytes as UTF8. You must call {@link #seekTo(Cell)} before this method.
    * @return key as a string
+   * @deprecated Since hbase-2.0.0
    */
+  @Deprecated
   String getKeyString();
+
   /**
    * Convenience method to get a copy of the value as a string - interpreting
    * the bytes as UTF8. You must call {@link #seekTo(Cell)} before this method.
    * @return value as a string
+   * @deprecated Since hbase-2.0.0
    */
+  @Deprecated
   String getValueString();
+
   /**
    * @return Reader that underlies this Scanner instance.
    */
   HFile.Reader getReader();
+
   /**
    * @return True is scanner has had one of the seek calls invoked; i.e.
    * {@link #seekBefore(Cell)} or {@link #seekTo()} or {@link #seekTo(Cell)}.

http://git-wip-us.apache.org/repos/asf/hbase/blob/000117ad/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
index 66aced0..69c42c3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
@@ -1317,25 +1317,22 @@ public class BucketCache implements BlockCache, HeapSize {
         final AtomicLong realCacheSize) throws CacheFullException, IOException,
         BucketAllocatorException {
       int len = data.getSerializedLength();
-      // This cacheable thing can't be serialized...
+      // This cacheable thing can't be serialized
       if (len == 0) return null;
       long offset = bucketAllocator.allocateBlock(len);
       BucketEntry bucketEntry = new BucketEntry(offset, len, accessCounter, inMemory);
       bucketEntry.setDeserialiserReference(data.getDeserializer(), deserialiserMap);
       try {
         if (data instanceof HFileBlock) {
-          HFileBlock block = (HFileBlock) data;
-          ByteBuff sliceBuf = block.getBufferReadOnlyWithHeader();
-          sliceBuf.rewind();
-          assert len == sliceBuf.limit() + HFileBlock.EXTRA_SERIALIZATION_SPACE ||
-            len == sliceBuf.limit() + block.headerSize() + HFileBlock.EXTRA_SERIALIZATION_SPACE;
-          ByteBuffer extraInfoBuffer = ByteBuffer.allocate(HFileBlock.EXTRA_SERIALIZATION_SPACE);
-          block.serializeExtraInfo(extraInfoBuffer);
+          // If an instance of HFileBlock, save on some allocations.
+          HFileBlock block = (HFileBlock)data;
+          ByteBuff sliceBuf = block.getBufferReadOnly();
+          ByteBuffer metadata = block.getMetaData();
           if (LOG.isTraceEnabled()) {
             LOG.trace("Write offset=" + offset + ", len=" + len);
           }
           ioEngine.write(sliceBuf, offset);
-          ioEngine.write(extraInfoBuffer, offset + len - HFileBlock.EXTRA_SERIALIZATION_SPACE);
+          ioEngine.write(metadata, offset + len - metadata.limit());
         } else {
           ByteBuffer bb = ByteBuffer.allocate(len);
           data.serialize(bb);

http://git-wip-us.apache.org/repos/asf/hbase/blob/000117ad/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java
index e26022e..ed86a83 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java
@@ -18,6 +18,7 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
+import java.io.Closeable;
 import java.io.IOException;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -32,7 +33,7 @@ import org.apache.hadoop.hbase.client.Scan;
 // TODO: Change name from KeyValueScanner to CellScanner only we already have a simple CellScanner
 // so this should be something else altogether, a decoration on our base CellScanner. TODO.
 // This class shows in CPs so do it all in one swell swoop. HBase-2.0.0.
-public interface KeyValueScanner extends Shipper {
+public interface KeyValueScanner extends Shipper, Closeable {
   /**
    * The byte array represents for NO_NEXT_INDEXED_KEY;
    * The actual value is irrelevant because this is always compared by reference.
@@ -74,6 +75,7 @@ public interface KeyValueScanner extends Shipper {
    * The default implementation for this would be to return 0. A file having
    * lower sequence id will be considered to be the older one.
    */
+  // TODO: Implement SequenceId Interface instead.
   long getSequenceID();
 
   /**
@@ -137,11 +139,11 @@ public interface KeyValueScanner extends Shipper {
    * peek KeyValue of scanner has the same row with specified Cell,
    * otherwise seek the scanner at the first Cell of the row which is the
    * previous row of specified KeyValue
-   * 
+   *
    * @param key seek KeyValue
    * @return true if the scanner is at the valid KeyValue, false if such
    *         KeyValue does not exist
-   * 
+   *
    */
   public boolean backwardSeek(Cell key) throws IOException;
 
@@ -156,7 +158,7 @@ public interface KeyValueScanner extends Shipper {
 
   /**
    * Seek the scanner at the first KeyValue of last row
-   * 
+   *
    * @return true if scanner has values left, false if the underlying data is
    *         empty
    * @throws IOException
@@ -169,4 +171,4 @@ public interface KeyValueScanner extends Shipper {
    * see HFileWriterImpl#getMidpoint, or null if not known.
    */
   public Cell getNextIndexedKey();
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/000117ad/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
index 61eb9b8..b6164b2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
@@ -1271,7 +1271,7 @@ public class StoreFile {
     }
 
     /**
-     * Warning: Do not write further code which depends on this call. Instead
+     * @deprecated Do not write further code which depends on this call. Instead
      * use getStoreFileScanner() which uses the StoreFileScanner class/interface
      * which is the preferred way to scan a store with higher level concepts.
      *
@@ -1285,7 +1285,7 @@ public class StoreFile {
     }
 
     /**
-     * Warning: Do not write further code which depends on this call. Instead
+     * @deprecated Do not write further code which depends on this call. Instead
      * use getStoreFileScanner() which uses the StoreFileScanner class/interface
      * which is the preferred way to scan a store with higher level concepts.
      *

http://git-wip-us.apache.org/repos/asf/hbase/blob/000117ad/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java
index 69671e2..040685d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java
@@ -40,7 +40,6 @@ import org.apache.hadoop.hbase.io.HeapSize;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache;
 import org.apache.hadoop.hbase.nio.ByteBuff;
-import org.apache.hadoop.hbase.nio.SingleByteBuff;
 import org.apache.hadoop.hbase.util.ChecksumType;
 
 public class CacheTestUtils {
@@ -66,6 +65,7 @@ public class CacheTestUtils {
     /*Post eviction, heapsize should be the same */
     assertEquals(heapSize, ((HeapSize) toBeTested).heapSize());
   }
+
   public static void testCacheMultiThreaded(final BlockCache toBeTested,
       final int blockSize, final int numThreads, final int numQueries,
       final double passingScore) throws Exception {
@@ -339,25 +339,16 @@ public class CacheTestUtils {
   }
 
 
-  private static HFileBlockPair[] generateHFileBlocks(int blockSize,
-      int numBlocks) {
+  private static HFileBlockPair[] generateHFileBlocks(int blockSize, int numBlocks) {
     HFileBlockPair[] returnedBlocks = new HFileBlockPair[numBlocks];
     Random rand = new Random();
     HashSet<String> usedStrings = new HashSet<String>();
     for (int i = 0; i < numBlocks; i++) {
-
-      // The buffer serialized size needs to match the size of BlockSize. So we
-      // declare our data size to be smaller than it by the serialization space
-      // required.
-
-      SingleByteBuff cachedBuffer = new SingleByteBuff(ByteBuffer.allocate(blockSize
-          - HFileBlock.EXTRA_SERIALIZATION_SPACE));
+      ByteBuffer cachedBuffer = ByteBuffer.allocate(blockSize);
       rand.nextBytes(cachedBuffer.array());
       cachedBuffer.rewind();
-      int onDiskSizeWithoutHeader = blockSize
-          - HFileBlock.EXTRA_SERIALIZATION_SPACE;
-      int uncompressedSizeWithoutHeader = blockSize
-          - HFileBlock.EXTRA_SERIALIZATION_SPACE;
+      int onDiskSizeWithoutHeader = blockSize;
+      int uncompressedSizeWithoutHeader = blockSize;
       long prevBlockOffset = rand.nextLong();
       BlockType.DATA.write(cachedBuffer);
       cachedBuffer.putInt(onDiskSizeWithoutHeader);
@@ -376,7 +367,7 @@ public class CacheTestUtils {
           onDiskSizeWithoutHeader, uncompressedSizeWithoutHeader,
           prevBlockOffset, cachedBuffer, HFileBlock.DONT_FILL_HEADER,
           blockSize,
-          onDiskSizeWithoutHeader + HConstants.HFILEBLOCK_HEADER_SIZE, meta);
+          onDiskSizeWithoutHeader + HConstants.HFILEBLOCK_HEADER_SIZE, -1, meta);
 
       String strKey;
       /* No conflicting keys */
@@ -395,4 +386,4 @@ public class CacheTestUtils {
     BlockCacheKey blockName;
     HFileBlock block;
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/000117ad/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
index 5158e35..a9d8258 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
@@ -259,7 +259,6 @@ public class TestCacheOnWrite {
     assertTrue(testDescription, scanner.seekTo());
 
     long offset = 0;
-    HFileBlock prevBlock = null;
     EnumMap<BlockType, Integer> blockCountByType =
         new EnumMap<BlockType, Integer>(BlockType.class);
 
@@ -267,14 +266,10 @@ public class TestCacheOnWrite {
     List<Long> cachedBlocksOffset = new ArrayList<Long>();
     Map<Long, HFileBlock> cachedBlocks = new HashMap<Long, HFileBlock>();
     while (offset < reader.getTrailer().getLoadOnOpenDataOffset()) {
-      long onDiskSize = -1;
-      if (prevBlock != null) {
-         onDiskSize = prevBlock.getNextBlockOnDiskSizeWithHeader();
-      }
       // Flags: don't cache the block, use pread, this is not a compaction.
       // Also, pass null for expected block type to avoid checking it.
-      HFileBlock block = reader.readBlock(offset, onDiskSize, false, true,
-        false, true, null, encodingInCache);
+      HFileBlock block = reader.readBlock(offset, -1, false, true, false, true, null,
+          encodingInCache);
       BlockCacheKey blockCacheKey = new BlockCacheKey(reader.getName(),
           offset);
       HFileBlock fromCache = (HFileBlock) blockCache.getBlock(blockCacheKey, true, false, true);
@@ -307,7 +302,6 @@ public class TestCacheOnWrite {
         assertEquals(
           block.getUncompressedSizeWithoutHeader(), fromCache.getUncompressedSizeWithoutHeader());
       }
-      prevBlock = block;
       offset += block.getOnDiskSizeWithHeader();
       BlockType bt = block.getBlockType();
       Integer count = blockCountByType.get(bt);

http://git-wip-us.apache.org/repos/asf/hbase/blob/000117ad/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestChecksum.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestChecksum.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestChecksum.java
index 91ab8c0..d91a811 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestChecksum.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestChecksum.java
@@ -94,7 +94,7 @@ public class TestChecksum {
     meta = new HFileContextBuilder().withHBaseCheckSum(true).build();
     HFileBlock.FSReader hbr = new HFileBlock.FSReaderImpl(
         is, totalSize, (HFileSystem) fs, path, meta);
-    HFileBlock b = hbr.readBlockData(0, -1, -1, false);
+    HFileBlock b = hbr.readBlockData(0, -1, false);
     assertEquals(b.getChecksumType(), ChecksumType.getDefaultChecksumType().getCode());
   }
 
@@ -108,12 +108,14 @@ public class TestChecksum {
       ChecksumType cktype = itr.next();
       Path path = new Path(TEST_UTIL.getDataTestDir(), "checksum" + cktype.getName());
       FSDataOutputStream os = fs.create(path);
-      HFileContext meta = new HFileContextBuilder()
-          .withChecksumType(cktype).build();
+      HFileContext meta = new HFileContextBuilder().
+          withChecksumType(cktype).
+          build();
       HFileBlock.Writer hbw = new HFileBlock.Writer(null, meta);
       DataOutputStream dos = hbw.startWriting(BlockType.DATA);
-      for (int i = 0; i < 1000; ++i)
+      for (int i = 0; i < 1000; ++i) {
         dos.writeInt(i);
+      }
       hbw.writeHeaderAndData(os);
       int totalSize = hbw.getOnDiskSizeWithHeader();
       os.close();
@@ -125,7 +127,7 @@ public class TestChecksum {
       meta = new HFileContextBuilder().withHBaseCheckSum(true).build();
       HFileBlock.FSReader hbr = new HFileBlock.FSReaderImpl(
           is, totalSize, (HFileSystem) fs, path, meta);
-      HFileBlock b = hbr.readBlockData(0, -1, -1, false);
+      HFileBlock b = hbr.readBlockData(0, -1, false);
       ByteBuff data = b.getBufferWithoutHeader();
       for (int i = 0; i < 1000; i++) {
         assertEquals(i, data.getInt());
@@ -188,7 +190,7 @@ public class TestChecksum {
               .withHBaseCheckSum(true)
               .build();
         HFileBlock.FSReader hbr = new FSReaderImplTest(is, totalSize, fs, path, meta);
-        HFileBlock b = hbr.readBlockData(0, -1, -1, pread);
+        HFileBlock b = hbr.readBlockData(0, -1, pread);
         b.sanityCheck();
         assertEquals(4936, b.getUncompressedSizeWithoutHeader());
         assertEquals(algo == GZ ? 2173 : 4936, 
@@ -209,17 +211,17 @@ public class TestChecksum {
         // requests. Verify that this is correct.
         for (int i = 0; i < 
              HFileBlock.CHECKSUM_VERIFICATION_NUM_IO_THRESHOLD + 1; i++) {
-          b = hbr.readBlockData(0, -1, -1, pread);
+          b = hbr.readBlockData(0, -1, pread);
           assertEquals(0, HFile.getChecksumFailuresCount());
         }
         // The next read should have hbase checksum verification reanabled,
         // we verify this by assertng that there was a hbase-checksum failure.
-        b = hbr.readBlockData(0, -1, -1, pread);
+        b = hbr.readBlockData(0, -1, pread);
         assertEquals(1, HFile.getChecksumFailuresCount());
 
         // Since the above encountered a checksum failure, we switch
         // back to not checking hbase checksums.
-        b = hbr.readBlockData(0, -1, -1, pread);
+        b = hbr.readBlockData(0, -1, pread);
         assertEquals(0, HFile.getChecksumFailuresCount());
         is.close();
 
@@ -230,7 +232,7 @@ public class TestChecksum {
         assertEquals(false, newfs.useHBaseChecksum());
         is = new FSDataInputStreamWrapper(newfs, path);
         hbr = new FSReaderImplTest(is, totalSize, newfs, path, meta);
-        b = hbr.readBlockData(0, -1, -1, pread);
+        b = hbr.readBlockData(0, -1, pread);
         is.close();
         b.sanityCheck();
         b = b.unpack(meta, hbr);
@@ -314,7 +316,7 @@ public class TestChecksum {
                .build();
         HFileBlock.FSReader hbr = new HFileBlock.FSReaderImpl(new FSDataInputStreamWrapper(
             is, nochecksum), totalSize, hfs, path, meta);
-        HFileBlock b = hbr.readBlockData(0, -1, -1, pread);
+        HFileBlock b = hbr.readBlockData(0, -1, pread);
         is.close();
         b.sanityCheck();
         assertEquals(dataSize, b.getUncompressedSizeWithoutHeader());
@@ -354,5 +356,4 @@ public class TestChecksum {
       return false;  // checksum validation failure
     }
   }
-}
-
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/000117ad/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java
index 6748efc..eb87a0c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java
@@ -320,7 +320,7 @@ public class TestHFileBlock {
         .withIncludesTags(includesTag)
         .withCompression(algo).build();
         HFileBlock.FSReader hbr = new HFileBlock.FSReaderImpl(is, totalSize, meta);
-        HFileBlock b = hbr.readBlockData(0, -1, -1, pread);
+        HFileBlock b = hbr.readBlockData(0, -1, pread);
         is.close();
         assertEquals(0, HFile.getChecksumFailuresCount());
 
@@ -334,17 +334,15 @@ public class TestHFileBlock {
           is = fs.open(path);
           hbr = new HFileBlock.FSReaderImpl(is, totalSize, meta);
           b = hbr.readBlockData(0, 2173 + HConstants.HFILEBLOCK_HEADER_SIZE +
-                                b.totalChecksumBytes(), -1, pread);
+                                b.totalChecksumBytes(), pread);
           assertEquals(expected, b);
           int wrongCompressedSize = 2172;
           try {
             b = hbr.readBlockData(0, wrongCompressedSize
-                + HConstants.HFILEBLOCK_HEADER_SIZE, -1, pread);
+                + HConstants.HFILEBLOCK_HEADER_SIZE, pread);
             fail("Exception expected");
           } catch (IOException ex) {
-            String expectedPrefix = "On-disk size without header provided is "
-                + wrongCompressedSize + ", but block header contains "
-                + b.getOnDiskSizeWithoutHeader() + ".";
+            String expectedPrefix = "Passed in onDiskSizeWithHeader=";
             assertTrue("Invalid exception message: '" + ex.getMessage()
                 + "'.\nMessage is expected to start with: '" + expectedPrefix
                 + "'", ex.getMessage().startsWith(expectedPrefix));
@@ -424,7 +422,7 @@ public class TestHFileBlock {
           HFileBlock blockFromHFile, blockUnpacked;
           int pos = 0;
           for (int blockId = 0; blockId < numBlocks; ++blockId) {
-            blockFromHFile = hbr.readBlockData(pos, -1, -1, pread);
+            blockFromHFile = hbr.readBlockData(pos, -1, pread);
             assertEquals(0, HFile.getChecksumFailuresCount());
             blockFromHFile.sanityCheck();
             pos += blockFromHFile.getOnDiskSizeWithHeader();
@@ -560,7 +558,7 @@ public class TestHFileBlock {
             if (detailedLogging) {
               LOG.info("Reading block #" + i + " at offset " + curOffset);
             }
-            HFileBlock b = hbr.readBlockData(curOffset, -1, -1, pread);
+            HFileBlock b = hbr.readBlockData(curOffset, -1, pread);
             if (detailedLogging) {
               LOG.info("Block #" + i + ": " + b);
             }
@@ -574,8 +572,7 @@ public class TestHFileBlock {
 
             // Now re-load this block knowing the on-disk size. This tests a
             // different branch in the loader.
-            HFileBlock b2 = hbr.readBlockData(curOffset,
-                b.getOnDiskSizeWithHeader(), -1, pread);
+            HFileBlock b2 = hbr.readBlockData(curOffset, b.getOnDiskSizeWithHeader(), pread);
             b2.sanityCheck();
 
             assertEquals(b.getBlockType(), b2.getBlockType());
@@ -601,7 +598,7 @@ public class TestHFileBlock {
               b = b.unpack(meta, hbr);
               // b's buffer has header + data + checksum while
               // expectedContents have header + data only
-              ByteBuff bufRead = b.getBufferWithHeader();
+              ByteBuff bufRead = b.getBufferReadOnly();
               ByteBuffer bufExpected = expectedContents.get(i);
               boolean bytesAreCorrect = Bytes.compareTo(bufRead.array(),
                   bufRead.arrayOffset(),
@@ -684,7 +681,7 @@ public class TestHFileBlock {
         HFileBlock b;
         try {
           long onDiskSizeArg = withOnDiskSize ? expectedSize : -1;
-          b = hbr.readBlockData(offset, onDiskSizeArg, -1, pread);
+          b = hbr.readBlockData(offset, onDiskSizeArg, pread);
         } catch (IOException ex) {
           LOG.error("Error in client " + clientId + " trying to read block at "
               + offset + ", pread=" + pread + ", withOnDiskSize=" +
@@ -719,8 +716,7 @@ public class TestHFileBlock {
   protected void testConcurrentReadingInternals() throws IOException,
       InterruptedException, ExecutionException {
     for (Compression.Algorithm compressAlgo : COMPRESSION_ALGORITHMS) {
-      Path path =
-          new Path(TEST_UTIL.getDataTestDir(), "concurrent_reading");
+      Path path = new Path(TEST_UTIL.getDataTestDir(), "concurrent_reading");
       Random rand = defaultRandom();
       List<Long> offsets = new ArrayList<Long>();
       List<BlockType> types = new ArrayList<BlockType>();
@@ -843,8 +839,7 @@ public class TestHFileBlock {
                           .withBytesPerCheckSum(HFile.DEFAULT_BYTES_PER_CHECKSUM)
                           .withChecksumType(ChecksumType.NULL).build();
       HFileBlock block = new HFileBlock(BlockType.DATA, size, size, -1, buf,
-          HFileBlock.FILL_HEADER, -1,
-          0, meta);
+          HFileBlock.FILL_HEADER, -1, 0, -1, meta);
       long byteBufferExpectedSize = ClassSize.align(ClassSize.estimateBase(
           new MultiByteBuff(buf).getClass(), true)
           + HConstants.HFILEBLOCK_HEADER_SIZE + size);

http://git-wip-us.apache.org/repos/asf/hbase/blob/000117ad/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockCompatibility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockCompatibility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockCompatibility.java
deleted file mode 100644
index 16607b9..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockCompatibility.java
+++ /dev/null
@@ -1,750 +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 static org.apache.hadoop.hbase.io.compress.Compression.Algorithm.GZ;
-import static org.apache.hadoop.hbase.io.compress.Compression.Algorithm.NONE;
-import static org.junit.Assert.*;
-
-import java.io.ByteArrayOutputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HConstants;
-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.FSDataInputStreamWrapper;
-import org.apache.hadoop.hbase.io.compress.Compression;
-import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
-import org.apache.hadoop.hbase.io.encoding.HFileBlockDefaultEncodingContext;
-import org.apache.hadoop.hbase.io.encoding.HFileBlockEncodingContext;
-import org.apache.hadoop.hbase.io.hfile.HFileBlock.BlockWritable;
-import org.apache.hadoop.hbase.nio.ByteBuff;
-import org.apache.hadoop.hbase.nio.SingleByteBuff;
-import org.apache.hadoop.hbase.testclassification.IOTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.ChecksumType;
-import org.apache.hadoop.io.WritableUtils;
-import org.apache.hadoop.io.compress.Compressor;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-
-import com.google.common.base.Preconditions;
-
-/**
- * This class has unit tests to prove that older versions of
- * HFiles (without checksums) are compatible with current readers.
- */
-@Category({IOTests.class, SmallTests.class})
-@RunWith(Parameterized.class)
-public class TestHFileBlockCompatibility {
-
-  private static final Log LOG = LogFactory.getLog(TestHFileBlockCompatibility.class);
-  private static final Compression.Algorithm[] COMPRESSION_ALGORITHMS = {
-      NONE, GZ };
-
-  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private HFileSystem fs;
-
-  private final boolean includesMemstoreTS;
-  private final boolean includesTag;
-
-  public TestHFileBlockCompatibility(boolean includesMemstoreTS, boolean includesTag) {
-    this.includesMemstoreTS = includesMemstoreTS;
-    this.includesTag = includesTag;
-  }
-
-  @Parameters
-  public static Collection<Object[]> parameters() {
-    return HBaseTestingUtility.MEMSTORETS_TAGS_PARAMETRIZED;
-  }
-
-  @Before
-  public void setUp() throws IOException {
-    fs = (HFileSystem)HFileSystem.get(TEST_UTIL.getConfiguration());
-  }
-
-  public byte[] createTestV1Block(Compression.Algorithm algo)
-      throws IOException {
-    Compressor compressor = algo.getCompressor();
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    OutputStream os = algo.createCompressionStream(baos, compressor, 0);
-    DataOutputStream dos = new DataOutputStream(os);
-    BlockType.META.write(dos); // Let's make this a meta block.
-    TestHFileBlock.writeTestBlockContents(dos);
-    dos.flush();
-    algo.returnCompressor(compressor);
-    return baos.toByteArray();
-  }
-
-  private Writer createTestV2Block(Compression.Algorithm algo)
-      throws IOException {
-    final BlockType blockType = BlockType.DATA;
-    Writer hbw = new Writer(algo, null,
-        includesMemstoreTS, includesTag);
-    DataOutputStream dos = hbw.startWriting(blockType);
-    TestHFileBlock.writeTestBlockContents(dos);
-    // make sure the block is ready by calling hbw.getHeaderAndData()
-    hbw.getHeaderAndData();
-    assertEquals(1000 * 4, hbw.getUncompressedSizeWithoutHeader());
-    hbw.releaseCompressor();
-    return hbw;
-  }
-
- private String createTestBlockStr(Compression.Algorithm algo,
-      int correctLength) throws IOException {
-    Writer hbw = createTestV2Block(algo);
-    byte[] testV2Block = hbw.getHeaderAndData();
-    int osOffset = HConstants.HFILEBLOCK_HEADER_SIZE_NO_CHECKSUM + 9;
-    if (testV2Block.length == correctLength) {
-      // Force-set the "OS" field of the gzip header to 3 (Unix) to avoid
-      // variations across operating systems.
-      // See http://www.gzip.org/zlib/rfc-gzip.html for gzip format.
-      testV2Block[osOffset] = 3;
-    }
-    return Bytes.toStringBinary(testV2Block);
-  }
-
-  @Test
-  public void testNoCompression() throws IOException {
-    assertEquals(4000, createTestV2Block(NONE).getBlockForCaching().
-        getUncompressedSizeWithoutHeader());
-  }
-
-  @Test
-  public void testGzipCompression() throws IOException {
-    final String correctTestBlockStr =
-        "DATABLK*\\x00\\x00\\x00:\\x00\\x00\\x0F\\xA0\\xFF\\xFF\\xFF\\xFF"
-            + "\\xFF\\xFF\\xFF\\xFF"
-            // gzip-compressed block: http://www.gzip.org/zlib/rfc-gzip.html
-            + "\\x1F\\x8B"  // gzip magic signature
-            + "\\x08"  // Compression method: 8 = "deflate"
-            + "\\x00"  // Flags
-            + "\\x00\\x00\\x00\\x00"  // mtime
-            + "\\x00"  // XFL (extra flags)
-            // OS (0 = FAT filesystems, 3 = Unix). However, this field
-            // sometimes gets set to 0 on Linux and Mac, so we reset it to 3.
-            + "\\x03"
-            + "\\xED\\xC3\\xC1\\x11\\x00 \\x08\\xC00DD\\xDD\\x7Fa"
-            + "\\xD6\\xE8\\xA3\\xB9K\\x84`\\x96Q\\xD3\\xA8\\xDB\\xA8e\\xD4c"
-            + "\\xD46\\xEA5\\xEA3\\xEA7\\xE7\\x00LI\\x5Cs\\xA0\\x0F\\x00\\x00";
-    final int correctGzipBlockLength = 82;
-
-    String returnedStr = createTestBlockStr(GZ, correctGzipBlockLength);
-    assertEquals(correctTestBlockStr, returnedStr);
-  }
-
-  @Test
-  public void testReaderV2() throws IOException {
-    if(includesTag) {
-      TEST_UTIL.getConfiguration().setInt("hfile.format.version", 3);
-    }
-    for (Compression.Algorithm algo : COMPRESSION_ALGORITHMS) {
-      for (boolean pread : new boolean[] { false, true }) {
-          LOG.info("testReaderV2: Compression algorithm: " + algo +
-                   ", pread=" + pread);
-        Path path = new Path(TEST_UTIL.getDataTestDir(), "blocks_v2_"
-            + algo);
-        FSDataOutputStream os = fs.create(path);
-        Writer hbw = new Writer(algo, null,
-            includesMemstoreTS, includesTag);
-        long totalSize = 0;
-        for (int blockId = 0; blockId < 2; ++blockId) {
-          DataOutputStream dos = hbw.startWriting(BlockType.DATA);
-          for (int i = 0; i < 1234; ++i)
-            dos.writeInt(i);
-          hbw.writeHeaderAndData(os);
-          totalSize += hbw.getOnDiskSizeWithHeader();
-        }
-        os.close();
-
-        FSDataInputStream is = fs.open(path);
-        HFileContext meta = new HFileContextBuilder()
-                           .withHBaseCheckSum(false)
-                           .withIncludesMvcc(includesMemstoreTS)
-                           .withIncludesTags(includesTag)
-                           .withCompression(algo)
-                           .build();
-        HFileBlock.FSReader hbr =
-          new HFileBlock.FSReaderImpl(new FSDataInputStreamWrapper(is), totalSize, fs, path, meta);
-        HFileBlock b = hbr.readBlockData(0, -1, -1, pread);
-        is.close();
-
-        b.sanityCheck();
-        assertEquals(4936, b.getUncompressedSizeWithoutHeader());
-        assertEquals(algo == GZ ? 2173 : 4936,
-                     b.getOnDiskSizeWithoutHeader() - b.totalChecksumBytes());
-        HFileBlock expected = b;
-
-        if (algo == GZ) {
-          is = fs.open(path);
-          hbr = new HFileBlock.FSReaderImpl(new FSDataInputStreamWrapper(is), totalSize, fs, path,
-              meta);
-          b = hbr.readBlockData(0, 2173 + HConstants.HFILEBLOCK_HEADER_SIZE_NO_CHECKSUM +
-                                b.totalChecksumBytes(), -1, pread);
-          assertEquals(expected, b);
-          int wrongCompressedSize = 2172;
-          try {
-            b = hbr.readBlockData(0, wrongCompressedSize
-                + HConstants.HFILEBLOCK_HEADER_SIZE_NO_CHECKSUM, -1, pread);
-            fail("Exception expected");
-          } catch (IOException ex) {
-            String expectedPrefix = "On-disk size without header provided is "
-                + wrongCompressedSize + ", but block header contains "
-                + b.getOnDiskSizeWithoutHeader() + ".";
-            assertTrue("Invalid exception message: '" + ex.getMessage()
-                + "'.\nMessage is expected to start with: '" + expectedPrefix
-                + "'", ex.getMessage().startsWith(expectedPrefix));
-          }
-          is.close();
-        }
-      }
-    }
-  }
-
-  /**
-   * Test encoding/decoding data blocks.
-   * @throws IOException a bug or a problem with temporary files.
-   */
-  @Test
-  public void testDataBlockEncoding() throws IOException {
-    if(includesTag) {
-      TEST_UTIL.getConfiguration().setInt("hfile.format.version", 3);
-    }
-    final int numBlocks = 5;
-    for (Compression.Algorithm algo : COMPRESSION_ALGORITHMS) {
-      for (boolean pread : new boolean[] { false, true }) {
-        for (DataBlockEncoding encoding : DataBlockEncoding.values()) {
-          LOG.info("testDataBlockEncoding algo " + algo +
-                   " pread = " + pread +
-                   " encoding " + encoding);
-          Path path = new Path(TEST_UTIL.getDataTestDir(), "blocks_v2_"
-              + algo + "_" + encoding.toString());
-          FSDataOutputStream os = fs.create(path);
-          HFileDataBlockEncoder dataBlockEncoder = (encoding != DataBlockEncoding.NONE) ?
-              new HFileDataBlockEncoderImpl(encoding) : NoOpDataBlockEncoder.INSTANCE;
-          TestHFileBlockCompatibility.Writer hbw =
-              new TestHFileBlockCompatibility.Writer(algo,
-                  dataBlockEncoder, includesMemstoreTS, includesTag);
-          long totalSize = 0;
-          final List<Integer> encodedSizes = new ArrayList<Integer>();
-          final List<ByteBuffer> encodedBlocks = new ArrayList<ByteBuffer>();
-          for (int blockId = 0; blockId < numBlocks; ++blockId) {
-            hbw.startWriting(BlockType.DATA);
-            TestHFileBlock.writeTestKeyValues(hbw, blockId, pread, includesTag);
-            hbw.writeHeaderAndData(os);
-            int headerLen = HConstants.HFILEBLOCK_HEADER_SIZE_NO_CHECKSUM;
-            byte[] encodedResultWithHeader = hbw.getUncompressedDataWithHeader();
-            final int encodedSize = encodedResultWithHeader.length - headerLen;
-            if (encoding != DataBlockEncoding.NONE) {
-              // We need to account for the two-byte encoding algorithm ID that
-              // comes after the 24-byte block header but before encoded KVs.
-              headerLen += DataBlockEncoding.ID_SIZE;
-            }
-            byte[] encodedDataSection =
-                new byte[encodedResultWithHeader.length - headerLen];
-            System.arraycopy(encodedResultWithHeader, headerLen,
-                encodedDataSection, 0, encodedDataSection.length);
-            final ByteBuffer encodedBuf =
-                ByteBuffer.wrap(encodedDataSection);
-            encodedSizes.add(encodedSize);
-            encodedBlocks.add(encodedBuf);
-            totalSize += hbw.getOnDiskSizeWithHeader();
-          }
-          os.close();
-
-          FSDataInputStream is = fs.open(path);
-          HFileContext meta = new HFileContextBuilder()
-                              .withHBaseCheckSum(false)
-                              .withIncludesMvcc(includesMemstoreTS)
-                              .withIncludesTags(includesTag)
-                              .withCompression(algo)
-                              .build();
-          HFileBlock.FSReaderImpl hbr = new HFileBlock.FSReaderImpl(new FSDataInputStreamWrapper(is),
-              totalSize, fs, path, meta);
-          hbr.setDataBlockEncoder(dataBlockEncoder);
-          hbr.setIncludesMemstoreTS(includesMemstoreTS);
-
-          HFileBlock b;
-          int pos = 0;
-          for (int blockId = 0; blockId < numBlocks; ++blockId) {
-            b = hbr.readBlockData(pos, -1, -1, pread);
-            b.sanityCheck();
-            if (meta.isCompressedOrEncrypted()) {
-              assertFalse(b.isUnpacked());
-              b = b.unpack(meta, hbr);
-            }
-            pos += b.getOnDiskSizeWithHeader();
-
-            assertEquals((int) encodedSizes.get(blockId),
-                b.getUncompressedSizeWithoutHeader());
-            ByteBuff actualBuffer = b.getBufferWithoutHeader();
-            if (encoding != DataBlockEncoding.NONE) {
-              // We expect a two-byte big-endian encoding id.
-              assertEquals(0, actualBuffer.get(0));
-              assertEquals(encoding.getId(), actualBuffer.get(1));
-              actualBuffer.position(2);
-              actualBuffer = actualBuffer.slice();
-            }
-
-            ByteBuffer expectedBuffer = encodedBlocks.get(blockId);
-            expectedBuffer.rewind();
-
-            // test if content matches, produce nice message
-            TestHFileBlock.assertBuffersEqual(new SingleByteBuff(expectedBuffer), actualBuffer,
-              algo, encoding, pread);
-          }
-          is.close();
-        }
-      }
-    }
-  }
-  /**
-   * This is the version of the HFileBlock.Writer that is used to
-   * create V2 blocks with minor version 0. These blocks do not
-   * have hbase-level checksums. The code is here to test
-   * backward compatibility. The reason we do not inherit from
-   * HFileBlock.Writer is because we never ever want to change the code
-   * in this class but the code in HFileBlock.Writer will continually
-   * evolve.
-   */
-  public static final class Writer extends HFileBlock.Writer {
-
-    // These constants are as they were in minorVersion 0.
-    private static final int HEADER_SIZE = HConstants.HFILEBLOCK_HEADER_SIZE_NO_CHECKSUM;
-    private static final boolean DONT_FILL_HEADER = HFileBlock.DONT_FILL_HEADER;
-    private static final byte[] DUMMY_HEADER = HFileBlock.DUMMY_HEADER_NO_CHECKSUM;
-
-    private enum State {
-      INIT,
-      WRITING,
-      BLOCK_READY
-    };
-
-    /** Writer state. Used to ensure the correct usage protocol. */
-    private State state = State.INIT;
-
-    /** Compression algorithm for all blocks this instance writes. */
-    private final Compression.Algorithm compressAlgo;
-
-    /** Data block encoder used for data blocks */
-    private final HFileDataBlockEncoder dataBlockEncoder;
-
-    private HFileBlockEncodingContext dataBlockEncodingCtx;
-    /** block encoding context for non-data blocks */
-    private HFileBlockDefaultEncodingContext defaultBlockEncodingCtx;
-
-    /**
-     * The stream we use to accumulate data in uncompressed format for each
-     * block. We reset this stream at the end of each block and reuse it. The
-     * header is written as the first {@link #HEADER_SIZE} bytes into this
-     * stream.
-     */
-    private ByteArrayOutputStream baosInMemory;
-
-    /** Compressor, which is also reused between consecutive blocks. */
-    private Compressor compressor;
-
-    /**
-     * Current block type. Set in {@link #startWriting(BlockType)}. Could be
-     * changed in {@link #encodeDataBlockForDisk()} from {@link BlockType#DATA}
-     * to {@link BlockType#ENCODED_DATA}.
-     */
-    private BlockType blockType;
-
-    /**
-     * A stream that we write uncompressed bytes to, which compresses them and
-     * writes them to {@link #baosInMemory}.
-     */
-    private DataOutputStream userDataStream;
-
-    /**
-     * Bytes to be written to the file system, including the header. Compressed
-     * if compression is turned on.
-     */
-    private byte[] onDiskBytesWithHeader;
-
-    /**
-     * Valid in the READY state. Contains the header and the uncompressed (but
-     * potentially encoded, if this is a data block) bytes, so the length is
-     * {@link #uncompressedSizeWithoutHeader} + {@link org.apache.hadoop.hbase.HConstants#HFILEBLOCK_HEADER_SIZE}.
-     */
-    private byte[] uncompressedBytesWithHeader;
-
-    /**
-     * Current block's start offset in the {@link HFile}. Set in
-     * {@link #writeHeaderAndData(FSDataOutputStream)}.
-     */
-    private long startOffset;
-
-    /**
-     * Offset of previous block by block type. Updated when the next block is
-     * started.
-     */
-    private long[] prevOffsetByType;
-
-    /** The offset of the previous block of the same type */
-    private long prevOffset;
-
-    private int unencodedDataSizeWritten;
-
-    public Writer(Compression.Algorithm compressionAlgorithm,
-        HFileDataBlockEncoder dataBlockEncoder, boolean includesMemstoreTS, boolean includesTag) {
-      this(dataBlockEncoder, new HFileContextBuilder().withHBaseCheckSum(false)
-          .withIncludesMvcc(includesMemstoreTS).withIncludesTags(includesTag)
-          .withCompression(compressionAlgorithm).build());
-    }
-
-    public Writer(HFileDataBlockEncoder dataBlockEncoder, HFileContext meta) {
-      super(dataBlockEncoder, meta);
-      compressAlgo = meta.getCompression() == null ? NONE : meta.getCompression();
-      this.dataBlockEncoder = dataBlockEncoder != null ? dataBlockEncoder
-          : NoOpDataBlockEncoder.INSTANCE;
-      defaultBlockEncodingCtx = new HFileBlockDefaultEncodingContext(null, DUMMY_HEADER, meta);
-      dataBlockEncodingCtx = this.dataBlockEncoder.newDataBlockEncodingContext(DUMMY_HEADER, meta);
-      baosInMemory = new ByteArrayOutputStream();
-
-      prevOffsetByType = new long[BlockType.values().length];
-      for (int i = 0; i < prevOffsetByType.length; ++i)
-        prevOffsetByType[i] = -1;
-    }
-
-    /**
-     * Starts writing into the block. The previous block's data is discarded.
-     *
-     * @return the stream the user can write their data into
-     * @throws IOException
-     */
-    public DataOutputStream startWriting(BlockType newBlockType)
-        throws IOException {
-      if (state == State.BLOCK_READY && startOffset != -1) {
-        // We had a previous block that was written to a stream at a specific
-        // offset. Save that offset as the last offset of a block of that type.
-        prevOffsetByType[blockType.getId()] = startOffset;
-      }
-
-      startOffset = -1;
-      blockType = newBlockType;
-
-      baosInMemory.reset();
-      baosInMemory.write(DUMMY_HEADER);
-
-      state = State.WRITING;
-
-      // We will compress it later in finishBlock()
-      userDataStream = new DataOutputStream(baosInMemory);
-      if (newBlockType == BlockType.DATA) {
-        this.dataBlockEncoder.startBlockEncoding(dataBlockEncodingCtx, userDataStream);
-      }
-      this.unencodedDataSizeWritten = 0;
-      return userDataStream;
-    }
-
-    @Override
-    public void write(Cell c) throws IOException {
-      KeyValue kv = KeyValueUtil.ensureKeyValue(c);
-      expectState(State.WRITING);
-      this.dataBlockEncoder.encode(kv, dataBlockEncodingCtx, this.userDataStream);
-      this.unencodedDataSizeWritten += kv.getLength();
-      if (dataBlockEncodingCtx.getHFileContext().isIncludesMvcc()) {
-        this.unencodedDataSizeWritten += WritableUtils.getVIntSize(kv.getSequenceId());
-      }
-    }
-
-    /**
-     * Returns the stream for the user to write to. The block writer takes care
-     * of handling compression and buffering for caching on write. Can only be
-     * called in the "writing" state.
-     *
-     * @return the data output stream for the user to write to
-     */
-    DataOutputStream getUserDataStream() {
-      expectState(State.WRITING);
-      return userDataStream;
-    }
-
-    /**
-     * Transitions the block writer from the "writing" state to the "block
-     * ready" state.  Does nothing if a block is already finished.
-     */
-    void ensureBlockReady() throws IOException {
-      Preconditions.checkState(state != State.INIT,
-          "Unexpected state: " + state);
-
-      if (state == State.BLOCK_READY)
-        return;
-
-      // This will set state to BLOCK_READY.
-      finishBlock();
-    }
-
-    /**
-     * An internal method that flushes the compressing stream (if using
-     * compression), serializes the header, and takes care of the separate
-     * uncompressed stream for caching on write, if applicable. Sets block
-     * write state to "block ready".
-     */
-    void finishBlock() throws IOException {
-      if (blockType == BlockType.DATA) {
-        this.dataBlockEncoder.endBlockEncoding(dataBlockEncodingCtx, userDataStream,
-            baosInMemory.toByteArray(), blockType);
-        blockType = dataBlockEncodingCtx.getBlockType();
-      }
-      userDataStream.flush();
-      // This does an array copy, so it is safe to cache this byte array.
-      uncompressedBytesWithHeader = baosInMemory.toByteArray();
-      prevOffset = prevOffsetByType[blockType.getId()];
-
-      // We need to set state before we can package the block up for
-      // cache-on-write. In a way, the block is ready, but not yet encoded or
-      // compressed.
-      state = State.BLOCK_READY;
-      if (blockType == BlockType.DATA || blockType == BlockType.ENCODED_DATA) {
-        onDiskBytesWithHeader = dataBlockEncodingCtx
-            .compressAndEncrypt(uncompressedBytesWithHeader);
-      } else {
-        onDiskBytesWithHeader = defaultBlockEncodingCtx
-            .compressAndEncrypt(uncompressedBytesWithHeader);
-      }
-
-      // put the header for on disk bytes
-      putHeader(onDiskBytesWithHeader, 0,
-          onDiskBytesWithHeader.length,
-          uncompressedBytesWithHeader.length);
-      //set the header for the uncompressed bytes (for cache-on-write)
-      putHeader(uncompressedBytesWithHeader, 0,
-          onDiskBytesWithHeader.length,
-        uncompressedBytesWithHeader.length);
-    }
-
-    /**
-     * Put the header into the given byte array at the given offset.
-     * @param onDiskSize size of the block on disk
-     * @param uncompressedSize size of the block after decompression (but
-     *          before optional data block decoding)
-     */
-    private void putHeader(byte[] dest, int offset, int onDiskSize,
-        int uncompressedSize) {
-      offset = blockType.put(dest, offset);
-      offset = Bytes.putInt(dest, offset, onDiskSize - HEADER_SIZE);
-      offset = Bytes.putInt(dest, offset, uncompressedSize - HEADER_SIZE);
-      Bytes.putLong(dest, offset, prevOffset);
-    }
-
-    /**
-     * Similar to {@link #writeHeaderAndData(FSDataOutputStream)}, but records
-     * the offset of this block so that it can be referenced in the next block
-     * of the same type.
-     *
-     * @param out
-     * @throws IOException
-     */
-    public void writeHeaderAndData(FSDataOutputStream out) throws IOException {
-      long offset = out.getPos();
-      if (startOffset != -1 && offset != startOffset) {
-        throw new IOException("A " + blockType + " block written to a "
-            + "stream twice, first at offset " + startOffset + ", then at "
-            + offset);
-      }
-      startOffset = offset;
-
-      writeHeaderAndData((DataOutputStream) out);
-    }
-
-    /**
-     * Writes the header and the compressed data of this block (or uncompressed
-     * data when not using compression) into the given stream. Can be called in
-     * the "writing" state or in the "block ready" state. If called in the
-     * "writing" state, transitions the writer to the "block ready" state.
-     *
-     * @param out the output stream to write the
-     * @throws IOException
-     */
-    private void writeHeaderAndData(DataOutputStream out) throws IOException {
-      ensureBlockReady();
-      out.write(onDiskBytesWithHeader);
-    }
-
-    /**
-     * Returns the header or the compressed data (or uncompressed data when not
-     * using compression) as a byte array. Can be called in the "writing" state
-     * or in the "block ready" state. If called in the "writing" state,
-     * transitions the writer to the "block ready" state.
-     *
-     * @return header and data as they would be stored on disk in a byte array
-     * @throws IOException
-     */
-    public byte[] getHeaderAndData() throws IOException {
-      ensureBlockReady();
-      return onDiskBytesWithHeader;
-    }
-
-    /**
-     * Releases the compressor this writer uses to compress blocks into the
-     * compressor pool. Needs to be called before the writer is discarded.
-     */
-    public void releaseCompressor() {
-      if (compressor != null) {
-        compressAlgo.returnCompressor(compressor);
-        compressor = null;
-      }
-    }
-
-    /**
-     * Returns the on-disk size of the data portion of the block. This is the
-     * compressed size if compression is enabled. Can only be called in the
-     * "block ready" state. Header is not compressed, and its size is not
-     * included in the return value.
-     *
-     * @return the on-disk size of the block, not including the header.
-     */
-    public int getOnDiskSizeWithoutHeader() {
-      expectState(State.BLOCK_READY);
-      return onDiskBytesWithHeader.length - HEADER_SIZE;
-    }
-
-    /**
-     * Returns the on-disk size of the block. Can only be called in the
-     * "block ready" state.
-     *
-     * @return the on-disk size of the block ready to be written, including the
-     *         header size
-     */
-    public int getOnDiskSizeWithHeader() {
-      expectState(State.BLOCK_READY);
-      return onDiskBytesWithHeader.length;
-    }
-
-    /**
-     * The uncompressed size of the block data. Does not include header size.
-     */
-    public int getUncompressedSizeWithoutHeader() {
-      expectState(State.BLOCK_READY);
-      return uncompressedBytesWithHeader.length - HEADER_SIZE;
-    }
-
-    /**
-     * The uncompressed size of the block data, including header size.
-     */
-    public int getUncompressedSizeWithHeader() {
-      expectState(State.BLOCK_READY);
-      return uncompressedBytesWithHeader.length;
-    }
-
-    /** @return true if a block is being written  */
-    public boolean isWriting() {
-      return state == State.WRITING;
-    }
-
-    /**
-     * Returns the number of bytes written into the current block so far, or
-     * zero if not writing the block at the moment. Note that this will return
-     * zero in the "block ready" state as well.
-     *
-     * @return the number of bytes written
-     */
-    public int blockSizeWritten() {
-      if (state != State.WRITING)
-        return 0;
-      return this.unencodedDataSizeWritten;
-    }
-
-    /**
-     * Returns the header followed by the uncompressed data, even if using
-     * compression. This is needed for storing uncompressed blocks in the block
-     * cache. Can be called in the "writing" state or the "block ready" state.
-     *
-     * @return uncompressed block bytes for caching on write
-     */
-    private byte[] getUncompressedDataWithHeader() {
-      expectState(State.BLOCK_READY);
-
-      return uncompressedBytesWithHeader;
-    }
-
-    private void expectState(State expectedState) {
-      if (state != expectedState) {
-        throw new IllegalStateException("Expected state: " + expectedState +
-            ", actual state: " + state);
-      }
-    }
-
-    /**
-     * Similar to {@link #getUncompressedBufferWithHeader()} but returns a byte
-     * buffer.
-     *
-     * @return uncompressed block for caching on write in the form of a buffer
-     */
-    public ByteBuffer getUncompressedBufferWithHeader() {
-      byte[] b = getUncompressedDataWithHeader();
-      return ByteBuffer.wrap(b, 0, b.length);
-    }
-
-    /**
-     * Takes the given {@link BlockWritable} instance, creates a new block of
-     * its appropriate type, writes the writable into this block, and flushes
-     * the block into the output stream. The writer is instructed not to buffer
-     * uncompressed bytes for cache-on-write.
-     *
-     * @param bw the block-writable object to write as a block
-     * @param out the file system output stream
-     * @throws IOException
-     */
-    public void writeBlock(BlockWritable bw, FSDataOutputStream out)
-        throws IOException {
-      bw.writeToBlock(startWriting(bw.getBlockType()));
-      writeHeaderAndData(out);
-    }
-
-    /**
-     * Creates a new HFileBlock.
-     */
-    public HFileBlock getBlockForCaching() {
-      HFileContext meta = new HFileContextBuilder()
-             .withHBaseCheckSum(false)
-             .withChecksumType(ChecksumType.NULL)
-             .withBytesPerCheckSum(0)
-             .build();
-      return new HFileBlock(blockType, getOnDiskSizeWithoutHeader(),
-          getUncompressedSizeWithoutHeader(), prevOffset,
-          getUncompressedBufferWithHeader(), DONT_FILL_HEADER, startOffset,
-          getOnDiskSizeWithoutHeader(), meta);
-    }
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/hbase/blob/000117ad/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 687d3cd..470d483 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
@@ -185,8 +185,7 @@ public class TestHFileBlockIndex {
       }
 
       missCount += 1;
-      prevBlock = realReader.readBlockData(offset, onDiskSize,
-          -1, pread);
+      prevBlock = realReader.readBlockData(offset, onDiskSize, pread);
       prevOffset = offset;
       prevOnDiskSize = onDiskSize;
       prevPread = pread;

http://git-wip-us.apache.org/repos/asf/hbase/blob/000117ad/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileDataBlockEncoder.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileDataBlockEncoder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileDataBlockEncoder.java
index 6f434bb..387514e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileDataBlockEncoder.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileDataBlockEncoder.java
@@ -92,8 +92,7 @@ public class TestHFileDataBlockEncoder {
 
     if (blockEncoder.getDataBlockEncoding() ==
         DataBlockEncoding.NONE) {
-      assertEquals(block.getBufferWithHeader(),
-          returnedBlock.getBufferWithHeader());
+      assertEquals(block.getBufferReadOnly(), returnedBlock.getBufferReadOnly());
     } else {
       if (BlockType.ENCODED_DATA != returnedBlock.getBlockType()) {
         System.out.println(blockEncoder);
@@ -127,7 +126,7 @@ public class TestHFileDataBlockEncoder {
                         .build();
     HFileBlock block = new HFileBlock(BlockType.DATA, size, size, -1, buf,
         HFileBlock.FILL_HEADER, 0,
-        0, hfileContext);
+        0, -1, hfileContext);
     HFileBlock cacheBlock = createBlockOnDisk(kvs, block, useTags);
     assertEquals(headerSize, cacheBlock.getDummyHeaderForVersion().length);
   }
@@ -198,7 +197,7 @@ public class TestHFileDataBlockEncoder {
                         .build();
     HFileBlock b = new HFileBlock(BlockType.DATA, size, size, -1, buf,
         HFileBlock.FILL_HEADER, 0, 
-         0, meta);
+         0, -1, meta);
     return b;
   }
 
@@ -220,7 +219,8 @@ public class TestHFileDataBlockEncoder {
     byte[] encodedBytes = baos.toByteArray();
     size = encodedBytes.length - block.getDummyHeaderForVersion().length;
     return new HFileBlock(context.getBlockType(), size, size, -1, ByteBuffer.wrap(encodedBytes),
-        HFileBlock.FILL_HEADER, 0, block.getOnDiskDataSizeWithHeader(), block.getHFileContext());
+        HFileBlock.FILL_HEADER, 0, block.getOnDiskDataSizeWithHeader(), -1,
+        block.getHFileContext());
   }
 
   private void writeBlock(List<Cell> kvs, HFileContext fileContext, boolean useTags)

http://git-wip-us.apache.org/repos/asf/hbase/blob/000117ad/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileEncryption.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileEncryption.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileEncryption.java
index ba3a344..3264558 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileEncryption.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileEncryption.java
@@ -99,7 +99,7 @@ public class TestHFileEncryption {
 
   private long readAndVerifyBlock(long pos, HFileContext ctx, HFileBlock.FSReaderImpl hbr, int size)
       throws IOException {
-    HFileBlock b = hbr.readBlockData(pos, -1, -1, false);
+    HFileBlock b = hbr.readBlockData(pos, -1, false);
     assertEquals(0, HFile.getChecksumFailuresCount());
     b.sanityCheck();
     assertFalse(b.isUnpacked());

http://git-wip-us.apache.org/repos/asf/hbase/blob/000117ad/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java
index c7eb11b..983ec2f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java
@@ -218,7 +218,7 @@ public class TestHFileWriterV3 {
     fsdis.seek(0);
     long curBlockPos = 0;
     while (curBlockPos <= trailer.getLastDataBlockOffset()) {
-      HFileBlock block = blockReader.readBlockData(curBlockPos, -1, -1, false)
+      HFileBlock block = blockReader.readBlockData(curBlockPos, -1, false)
         .unpack(context, blockReader);
       assertEquals(BlockType.DATA, block.getBlockType());
       ByteBuff buf = block.getBufferWithoutHeader();
@@ -279,13 +279,14 @@ public class TestHFileWriterV3 {
     while (fsdis.getPos() < trailer.getLoadOnOpenDataOffset()) {
       LOG.info("Current offset: " + fsdis.getPos() + ", scanning until " +
           trailer.getLoadOnOpenDataOffset());
-      HFileBlock block = blockReader.readBlockData(curBlockPos, -1, -1, false)
+      HFileBlock block = blockReader.readBlockData(curBlockPos, -1, false)
         .unpack(context, blockReader);
       assertEquals(BlockType.META, block.getBlockType());
       Text t = new Text();
       ByteBuff buf = block.getBufferWithoutHeader();
       if (Writables.getWritable(buf.array(), buf.arrayOffset(), buf.limit(), t) == null) {
-        throw new IOException("Failed to deserialize block " + this + " into a " + t.getClass().getSimpleName());
+        throw new IOException("Failed to deserialize block " + this +
+            " into a " + t.getClass().getSimpleName());
       }
       Text expectedText =
           (metaCounter == 0 ? new Text("Paris") : metaCounter == 1 ? new Text(

http://git-wip-us.apache.org/repos/asf/hbase/blob/000117ad/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java
index 69a77bf..d20ba2b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java
@@ -78,14 +78,8 @@ public class TestPrefetch {
     // Check that all of the data blocks were preloaded
     BlockCache blockCache = cacheConf.getBlockCache();
     long offset = 0;
-    HFileBlock prevBlock = null;
     while (offset < reader.getTrailer().getLoadOnOpenDataOffset()) {
-      long onDiskSize = -1;
-      if (prevBlock != null) {
-         onDiskSize = prevBlock.getNextBlockOnDiskSizeWithHeader();
-      }
-      HFileBlock block = reader.readBlock(offset, onDiskSize, false, true, false, true, null,
-        null);
+      HFileBlock block = reader.readBlock(offset, -1, false, true, false, true, null, null);
       BlockCacheKey blockCacheKey = new BlockCacheKey(reader.getName(), offset);
       boolean isCached = blockCache.getBlock(blockCacheKey, true, false, true) != null;
       if (block.getBlockType() == BlockType.DATA ||
@@ -93,7 +87,6 @@ public class TestPrefetch {
           block.getBlockType() == BlockType.INTERMEDIATE_INDEX) {
         assertTrue(isCached);
       }
-      prevBlock = block;
       offset += block.getOnDiskSizeWithHeader();
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/000117ad/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
index 0916fe6..2357bef 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
@@ -227,15 +227,10 @@ public class TestCacheOnWriteInSchema {
       assertTrue(testDescription, scanner.seekTo());
       // Cribbed from io.hfile.TestCacheOnWrite
       long offset = 0;
-      HFileBlock prevBlock = null;
       while (offset < reader.getTrailer().getLoadOnOpenDataOffset()) {
-        long onDiskSize = -1;
-        if (prevBlock != null) {
-          onDiskSize = prevBlock.getNextBlockOnDiskSizeWithHeader();
-        }
         // Flags: don't cache the block, use pread, this is not a compaction.
         // Also, pass null for expected block type to avoid checking it.
-        HFileBlock block = reader.readBlock(offset, onDiskSize, false, true,
+        HFileBlock block = reader.readBlock(offset, -1, false, true,
           false, true, null, DataBlockEncoding.NONE);
         BlockCacheKey blockCacheKey = new BlockCacheKey(reader.getName(),
           offset);
@@ -249,7 +244,6 @@ public class TestCacheOnWriteInSchema {
             "block: " + block + "\n" +
             "blockCacheKey: " + blockCacheKey);
         }
-        prevBlock = block;
         offset += block.getOnDiskSizeWithHeader();
       }
     } finally {


[49/50] [abbrv] hbase git commit: HBASE-15295 MutateTableAccess.multiMutate() does not get high priority causing a deadlock - ADDENDUM for failing test

Posted by sy...@apache.org.
HBASE-15295 MutateTableAccess.multiMutate() does not get high priority causing a deadlock - ADDENDUM for failing test


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

Branch: refs/heads/hbase-12439
Commit: afdfd1bd9c938fa4b5c2aa9346e559167d550785
Parents: 7f39baf
Author: Enis Soztutar <en...@apache.org>
Authored: Tue Mar 29 15:02:18 2016 -0700
Committer: Enis Soztutar <en...@apache.org>
Committed: Tue Mar 29 15:02:18 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/client/HConnectionTestingUtility.java   | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/afdfd1bd/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
index dc1ecf1..24ef5b2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
@@ -125,6 +125,7 @@ public class HConnectionTestingUtility {
     Mockito.when(c.getNewRpcRetryingCallerFactory(conf)).thenReturn(
         RpcRetryingCallerFactory.instantiate(conf,
             RetryingCallerInterceptorFactory.NO_OP_INTERCEPTOR, null));
+    Mockito.when(c.getRpcControllerFactory()).thenReturn(Mockito.mock(RpcControllerFactory.class));
     HTableInterface t = Mockito.mock(HTableInterface.class);
     Mockito.when(c.getTable((TableName)Mockito.any())).thenReturn(t);
     ResultScanner rs = Mockito.mock(ResultScanner.class);


[37/50] [abbrv] hbase git commit: HBASE-15265 Implement an asynchronous FSHLog

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogWriter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogWriter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogWriter.java
index 39743d5..41a790e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogWriter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogWriter.java
@@ -27,12 +27,15 @@ import java.util.TreeMap;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.io.util.LRUDictionary;
 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.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.wal.DefaultWALProvider;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.SequenceFile.CompressionType;
@@ -51,7 +54,7 @@ import org.apache.hadoop.io.compress.DefaultCodec;
  * Hadoop serialization).
  */
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
-public class SequenceFileLogWriter extends WriterBase {
+public class SequenceFileLogWriter implements DefaultWALProvider.Writer {
   private static final Log LOG = LogFactory.getLog(SequenceFileLogWriter.class);
   // The sequence file we delegate to.
   private SequenceFile.Writer writer;
@@ -59,6 +62,8 @@ public class SequenceFileLogWriter extends WriterBase {
   // in the SequenceFile.Writer 'writer' instance above.
   private FSDataOutputStream writer_out;
 
+  private CompressionContext compressionContext;
+
   // Legacy stuff from pre-PB WAL metadata.
   private static final Text WAL_VERSION_KEY = new Text("version");
   private static final Text WAL_COMPRESSION_TYPE_KEY = new Text("compression.type");
@@ -88,10 +93,23 @@ public class SequenceFileLogWriter extends WriterBase {
     return new Metadata(metaMap);
   }
 
+  private boolean initializeCompressionContext(Configuration conf, Path path) throws IOException {
+    boolean doCompress = conf.getBoolean(HConstants.ENABLE_WAL_COMPRESSION, false);
+    if (doCompress) {
+      try {
+        this.compressionContext = new CompressionContext(LRUDictionary.class,
+            FSUtils.isRecoveredEdits(path), conf.getBoolean(
+                CompressionContext.ENABLE_WAL_TAGS_COMPRESSION, true));
+      } catch (Exception e) {
+        throw new IOException("Failed to initiate CompressionContext", e);
+      }
+    }
+    return doCompress;
+  }
+
   @Override
   public void init(FileSystem fs, Path path, Configuration conf, boolean overwritable)
   throws IOException {
-    super.init(fs, path, conf, overwritable);
     boolean compress = initializeCompressionContext(conf, path);
     // Create a SF.Writer instance.
     try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncLogRollPeriod.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncLogRollPeriod.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncLogRollPeriod.java
new file mode 100644
index 0000000..bedb915
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncLogRollPeriod.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver.wal;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.junit.BeforeClass;
+import org.junit.experimental.categories.Category;
+
+@Category({ RegionServerTests.class, MediumTests.class })
+public class TestAsyncLogRollPeriod extends TestLogRollPeriod {
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    Configuration conf = TestLogRollPeriod.TEST_UTIL.getConfiguration();
+    conf.set(WALFactory.WAL_PROVIDER, "asyncfs");
+    TestLogRollPeriod.setUpBeforeClass();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncLogRolling.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncLogRolling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncLogRolling.java
new file mode 100644
index 0000000..fabf6d2
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncLogRolling.java
@@ -0,0 +1,65 @@
+/**
+ * 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.wal;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.VerySlowRegionServerTests;
+import org.apache.hadoop.hbase.wal.AsyncFSWALProvider;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ VerySlowRegionServerTests.class, LargeTests.class })
+public class TestAsyncLogRolling extends AbstractTestLogRolling {
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    Configuration conf = TestAsyncLogRolling.TEST_UTIL.getConfiguration();
+    conf.setInt(AsyncFSWAL.ASYNC_WAL_CREATE_MAX_RETRIES, 100);
+    conf.set(WALFactory.WAL_PROVIDER, "asyncfs");
+    AbstractTestLogRolling.setUpBeforeClass();
+  }
+
+  @Test(timeout = 180000)
+  public void testLogRollOnDatanodeDeath() throws IOException, InterruptedException {
+    dfsCluster.startDataNodes(TEST_UTIL.getConfiguration(), 3, true, null, null);
+    tableName = getName();
+    Table table = createTestTable(tableName);
+    TEST_UTIL.waitUntilAllRegionsAssigned(table.getName());
+    doPut(table, 1);
+    server = TEST_UTIL.getRSForFirstRegionInTable(table.getName());
+    HRegionInfo hri = server.getOnlineRegions(table.getName()).get(0).getRegionInfo();
+    AsyncFSWAL wal = (AsyncFSWAL) server.getWAL(hri);
+    int numRolledLogFiles = AsyncFSWALProvider.getNumRolledLogFiles(wal);
+    DatanodeInfo[] dnInfos = wal.getPipeline();
+    DataNodeProperties dnProp = TEST_UTIL.getDFSCluster().stopDataNode(dnInfos[0].getName());
+    TEST_UTIL.getDFSCluster().restartDataNode(dnProp);
+    doPut(table, 2);
+    assertEquals(numRolledLogFiles + 1, AsyncFSWALProvider.getNumRolledLogFiles(wal));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncProtobufLog.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncProtobufLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncProtobufLog.java
new file mode 100644
index 0000000..7d6c6d9
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncProtobufLog.java
@@ -0,0 +1,82 @@
+/**
+ * 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.wal;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.concurrent.ExecutionException;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.wal.AsyncFSWALProvider;
+import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.util.FanOutOneBlockAsyncDFSOutputFlushHandler;
+import org.apache.hadoop.hbase.wal.WALProvider;
+import org.apache.hadoop.hbase.wal.WALProvider.AsyncWriter;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.experimental.categories.Category;
+
+import com.google.common.base.Throwables;
+
+import io.netty.channel.EventLoopGroup;
+import io.netty.channel.nio.NioEventLoopGroup;
+
+@Category({ RegionServerTests.class, MediumTests.class })
+public class TestAsyncProtobufLog extends AbstractTestProtobufLog<WALProvider.AsyncWriter> {
+
+  private static EventLoopGroup EVENT_LOOP_GROUP;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    EVENT_LOOP_GROUP = new NioEventLoopGroup();
+    AbstractTestProtobufLog.setUpBeforeClass();
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    AbstractTestProtobufLog.tearDownAfterClass();
+    EVENT_LOOP_GROUP.shutdownGracefully().syncUninterruptibly();
+  }
+
+  @Override
+  protected AsyncWriter createWriter(Path path) throws IOException {
+    return AsyncFSWALProvider.createAsyncWriter(TEST_UTIL.getConfiguration(), fs, path, false,
+      EVENT_LOOP_GROUP.next());
+  }
+
+  @Override
+  protected void append(AsyncWriter writer, Entry entry) throws IOException {
+    writer.append(entry);
+  }
+
+  @Override
+  protected void sync(AsyncWriter writer) throws IOException {
+    FanOutOneBlockAsyncDFSOutputFlushHandler handler = new FanOutOneBlockAsyncDFSOutputFlushHandler();
+    writer.sync(handler, null);
+    try {
+      handler.get();
+    } catch (InterruptedException e) {
+      throw new InterruptedIOException();
+    } catch (ExecutionException e) {
+      Throwables.propagateIfPossible(e.getCause(), IOException.class);
+      throw new IOException(e.getCause());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncWALReplay.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncWALReplay.java
new file mode 100644
index 0000000..ca415fd
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncWALReplay.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver.wal;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.junit.BeforeClass;
+import org.junit.experimental.categories.Category;
+
+@Category({ RegionServerTests.class, MediumTests.class })
+public class TestAsyncWALReplay extends TestWALReplay {
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    Configuration conf = TestWALReplay.TEST_UTIL.getConfiguration();
+    conf.set(WALFactory.WAL_PROVIDER, "asyncfs");
+    TestWALReplay.setUpBeforeClass();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncWALReplayCompressed.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncWALReplayCompressed.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncWALReplayCompressed.java
new file mode 100644
index 0000000..3b8869b
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncWALReplayCompressed.java
@@ -0,0 +1,38 @@
+/**
+ * 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.wal;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.junit.BeforeClass;
+import org.junit.experimental.categories.Category;
+
+@Category({ RegionServerTests.class, MediumTests.class })
+public class TestAsyncWALReplayCompressed extends TestWALReplay {
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    Configuration conf = TestWALReplay.TEST_UTIL.getConfiguration();
+    conf.set(WALFactory.WAL_PROVIDER, "asyncfs");
+    conf.setBoolean(HConstants.ENABLE_WAL_COMPRESSION, true);
+    TestWALReplay.setUpBeforeClass();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
index 0662716..5783106 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
 
 import java.io.IOException;
+import java.util.Arrays;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -30,28 +31,35 @@ 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.testclassification.MediumTests;
-import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Increment;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.wal.DefaultWALProvider;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALFactory;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+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.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameter;
+import org.junit.runners.Parameterized.Parameters;
 
 /**
  * Tests for WAL write durability
  */
+@RunWith(Parameterized.class)
 @Category({RegionServerTests.class, MediumTests.class})
 public class TestDurability {
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
@@ -64,6 +72,13 @@ public class TestDurability {
   private static byte[] ROW = Bytes.toBytes("row");
   private static byte[] COL = Bytes.toBytes("col");
 
+  @Parameter
+  public String walProvider;
+
+  @Parameters(name = "{index}: provider={0}")
+  public static Iterable<Object[]> data() {
+    return Arrays.asList(new Object[] { "defaultProvider" }, new Object[] { "asyncfs" });
+  }
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
@@ -81,6 +96,16 @@ public class TestDurability {
     TEST_UTIL.shutdownMiniCluster();
   }
 
+  @Before
+  public void setUp() {
+    CONF.set(WALFactory.WAL_PROVIDER, walProvider);
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    FS.delete(DIR, true);
+  }
+
   @Test
   public void testDurability() throws Exception {
     final WALFactory wals = new WALFactory(CONF, null, "TestDurability");

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java
index c05e7f0..b7c1c73 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java
@@ -65,11 +65,11 @@ import org.junit.experimental.categories.Category;
  */
 @Category({RegionServerTests.class, MediumTests.class})
 public class TestLogRollAbort {
-  private static final Log LOG = LogFactory.getLog(TestLogRolling.class);
+  private static final Log LOG = LogFactory.getLog(AbstractTestLogRolling.class);
   private static MiniDFSCluster dfsCluster;
   private static Admin admin;
   private static MiniHBaseCluster cluster;
-  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 
   /* For the split-then-roll test */
   private static final Path HBASEDIR = new Path("/hbase");
@@ -212,7 +212,7 @@ public class TestLogRollAbort {
       }
       // Send the data to HDFS datanodes and close the HDFS writer
       log.sync();
-      ((FSHLog) log).replaceWriter(((FSHLog)log).getOldPath(), null, null, null);
+      ((AbstractFSWAL<?>) log).replaceWriter(((FSHLog)log).getOldPath(), null, null);
 
       /* code taken from MasterFileSystem.getLogDirs(), which is called from MasterFileSystem.splitLog()
        * handles RS shutdowns (as observed by the splitting process)

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollPeriod.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollPeriod.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollPeriod.java
index 1bf686f..1141871 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollPeriod.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollPeriod.java
@@ -19,21 +19,21 @@ package org.apache.hadoop.hbase.regionserver.wal;
 
 import static org.junit.Assert.assertFalse;
 
-import java.util.List;
 import java.util.ArrayList;
+import java.util.List;
 
 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.TableName;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.wal.WAL;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -44,9 +44,9 @@ import org.junit.experimental.categories.Category;
  */
 @Category({RegionServerTests.class, MediumTests.class})
 public class TestLogRollPeriod {
-  private static final Log LOG = LogFactory.getLog(TestLogRolling.class);
+  private static final Log LOG = LogFactory.getLog(AbstractTestLogRolling.class);
 
-  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 
   private final static long LOG_ROLL_PERIOD = 4000;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java
index 3ab49c0..37b23e0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java
@@ -1,5 +1,4 @@
 /**
- *
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -24,7 +23,6 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
 import java.io.EOFException;
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
@@ -33,275 +31,70 @@ import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.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.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.MiniHBaseCluster;
-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.Get;
 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.client.Table;
 import org.apache.hadoop.hbase.fs.HFileSystem;
-import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.Region;
-import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.VerySlowRegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.JVMClusterUtil;
-import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.wal.DefaultWALProvider;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALFactory;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
 import org.junit.BeforeClass;
-import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.junit.rules.TestName;
 
-/**
- * Test log deletion as logs are rolled.
- */
-@Category({VerySlowRegionServerTests.class, LargeTests.class})
-public class TestLogRolling  {
+@Category({ VerySlowRegionServerTests.class, LargeTests.class })
+public class TestLogRolling extends AbstractTestLogRolling {
+
   private static final Log LOG = LogFactory.getLog(TestLogRolling.class);
-  private HRegionServer server;
-  private String tableName;
-  private byte[] value;
-  private FileSystem fs;
-  private MiniDFSCluster dfsCluster;
-  private Admin admin;
-  private MiniHBaseCluster cluster;
-  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  @Rule public final TestName name = new TestName();
-
-  public TestLogRolling()  {
-    this.server = null;
-    this.tableName = null;
-
-    String className = this.getClass().getName();
-    StringBuilder v = new StringBuilder(className);
-    while (v.length() < 1000) {
-      v.append(className);
-    }
-    this.value = Bytes.toBytes(v.toString());
-  }
 
-  // Need to override this setup so we can edit the config before it gets sent
-  // to the HDFS & HBase cluster startup.
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     // TODO: testLogRollOnDatanodeDeath fails if short circuit reads are on under the hadoop2
     // profile. See HBASE-9337 for related issues.
     System.setProperty("hbase.tests.use.shortcircuit.reads", "false");
 
-    /**** configuration for testLogRolling ****/
-    // Force a region split after every 768KB
-    TEST_UTIL.getConfiguration().setLong(HConstants.HREGION_MAX_FILESIZE, 768L * 1024L);
-
-    // We roll the log after every 32 writes
-    TEST_UTIL.getConfiguration().setInt("hbase.regionserver.maxlogentries", 32);
-
-    TEST_UTIL.getConfiguration().setInt("hbase.regionserver.logroll.errors.tolerated", 2);
-    TEST_UTIL.getConfiguration().setInt("hbase.rpc.timeout", 10 * 1000);
-
-    // For less frequently updated regions flush after every 2 flushes
-    TEST_UTIL.getConfiguration().setInt("hbase.hregion.memstore.optionalflushcount", 2);
-
-    // We flush the cache after every 8192 bytes
-    TEST_UTIL.getConfiguration().setInt(
-        HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 8192);
-
-    // Increase the amount of time between client retries
-    TEST_UTIL.getConfiguration().setLong("hbase.client.pause", 10 * 1000);
-
-    // Reduce thread wake frequency so that other threads can get
-    // a chance to run.
-    TEST_UTIL.getConfiguration().setInt(HConstants.THREAD_WAKE_FREQUENCY, 2 * 1000);
-
-   /**** configuration for testLogRollOnDatanodeDeath ****/
-   // make sure log.hflush() calls syncFs() to open a pipeline
+    /**** configuration for testLogRollOnDatanodeDeath ****/
+    // make sure log.hflush() calls syncFs() to open a pipeline
     TEST_UTIL.getConfiguration().setBoolean("dfs.support.append", true);
-   // lower the namenode & datanode heartbeat so the namenode
-   // quickly detects datanode failures
+    // lower the namenode & datanode heartbeat so the namenode
+    // quickly detects datanode failures
     TEST_UTIL.getConfiguration().setInt("dfs.namenode.heartbeat.recheck-interval", 5000);
     TEST_UTIL.getConfiguration().setInt("dfs.heartbeat.interval", 1);
     // the namenode might still try to choose the recently-dead datanode
     // for a pipeline, so try to a new pipeline multiple times
-     TEST_UTIL.getConfiguration().setInt("dfs.client.block.write.retries", 30);
+    TEST_UTIL.getConfiguration().setInt("dfs.client.block.write.retries", 30);
     TEST_UTIL.getConfiguration().setInt("hbase.regionserver.hlog.tolerable.lowreplication", 2);
     TEST_UTIL.getConfiguration().setInt("hbase.regionserver.hlog.lowreplication.rolllimit", 3);
-  }
-
-  @Before
-  public void setUp() throws Exception {
-    TEST_UTIL.startMiniCluster(1, 1, 2);
-
-    cluster = TEST_UTIL.getHBaseCluster();
-    dfsCluster = TEST_UTIL.getDFSCluster();
-    fs = TEST_UTIL.getTestFileSystem();
-    admin = TEST_UTIL.getHBaseAdmin();
-
-    // disable region rebalancing (interferes with log watching)
-    cluster.getMaster().balanceSwitch(false);
-  }
-
-  @After
-  public void tearDown() throws Exception  {
-    TEST_UTIL.shutdownMiniCluster();
-  }
-
-  private void startAndWriteData() throws IOException, InterruptedException {
-    // When the hbase:meta table can be opened, the region servers are running
-    TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME);
-    this.server = cluster.getRegionServerThreads().get(0).getRegionServer();
-
-    Table table = createTestTable(this.tableName);
-
-    server = TEST_UTIL.getRSForFirstRegionInTable(table.getName());
-    for (int i = 1; i <= 256; i++) {    // 256 writes should cause 8 log rolls
-      doPut(table, i);
-      if (i % 32 == 0) {
-        // After every 32 writes sleep to let the log roller run
-        try {
-          Thread.sleep(2000);
-        } catch (InterruptedException e) {
-          // continue
-        }
-      }
-    }
-  }
-
-  /**
-   * Tests that log rolling doesn't hang when no data is written.
-   */
-  @Test(timeout=120000)
-  public void testLogRollOnNothingWritten() throws Exception {
-    final Configuration conf = TEST_UTIL.getConfiguration();
-    final WALFactory wals = new WALFactory(conf, null,
-        ServerName.valueOf("test.com",8080, 1).toString());
-    final WAL newLog = wals.getWAL(new byte[]{}, null);
-    try {
-      // Now roll the log before we write anything.
-      newLog.rollWriter(true);
-    } finally {
-      wals.close();
-    }
-  }
-
-  /**
-   * Tests that logs are deleted
-   * @throws IOException
-   * @throws org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException
-   */
-  @Test
-  public void testLogRolling() throws Exception {
-    this.tableName = getName();
-    // TODO: Why does this write data take for ever?
-    startAndWriteData();
-    HRegionInfo region =
-        server.getOnlineRegions(TableName.valueOf(tableName)).get(0).getRegionInfo();
-    final WAL log = server.getWAL(region);
-    LOG.info("after writing there are " + DefaultWALProvider.getNumRolledLogFiles(log) +
-        " log files");
-
-      // flush all regions
-      for (Region r: server.getOnlineRegionsLocalContext()) {
-        r.flush(true);
-      }
-
-      // Now roll the log
-      log.rollWriter();
-
-    int count = DefaultWALProvider.getNumRolledLogFiles(log);
-    LOG.info("after flushing all regions and rolling logs there are " + count + " log files");
-      assertTrue(("actual count: " + count), count <= 2);
-  }
-
-  private String getName() {
-    return "TestLogRolling-" + name.getMethodName();
-  }
-
-  void writeData(Table table, int rownum) throws IOException {
-    doPut(table, rownum);
-
-    // sleep to let the log roller run (if it needs to)
-    try {
-      Thread.sleep(2000);
-    } catch (InterruptedException e) {
-      // continue
-    }
-  }
-
-  void validateData(Table table, int rownum) throws IOException {
-    String row = "row" + String.format("%1$04d", rownum);
-    Get get = new Get(Bytes.toBytes(row));
-    get.addFamily(HConstants.CATALOG_FAMILY);
-    Result result = table.get(get);
-    assertTrue(result.size() == 1);
-    assertTrue(Bytes.equals(value,
-                result.getValue(HConstants.CATALOG_FAMILY, null)));
-    LOG.info("Validated row " + row);
-  }
-
-  void batchWriteAndWait(Table table, final FSHLog log, int start, boolean expect, int timeout)
-      throws IOException {
-    for (int i = 0; i < 10; i++) {
-      Put put = new Put(Bytes.toBytes("row"
-          + String.format("%1$04d", (start + i))));
-      put.addColumn(HConstants.CATALOG_FAMILY, null, value);
-      table.put(put);
-    }
-    Put tmpPut = new Put(Bytes.toBytes("tmprow"));
-    tmpPut.addColumn(HConstants.CATALOG_FAMILY, null, value);
-    long startTime = System.currentTimeMillis();
-    long remaining = timeout;
-    while (remaining > 0) {
-      if (log.isLowReplicationRollEnabled() == expect) {
-        break;
-      } else {
-        // Trigger calling FSHlog#checkLowReplication()
-        table.put(tmpPut);
-        try {
-          Thread.sleep(200);
-        } catch (InterruptedException e) {
-          // continue
-        }
-        remaining = timeout - (System.currentTimeMillis() - startTime);
-      }
-    }
+    AbstractTestLogRolling.setUpBeforeClass();
   }
 
   /**
-   * Tests that logs are rolled upon detecting datanode death
-   * Requires an HDFS jar with HDFS-826 & syncFs() support (HDFS-200)
+   * Tests that logs are rolled upon detecting datanode death Requires an HDFS jar with HDFS-826 &
+   * syncFs() support (HDFS-200)
    */
   @Test
   public void testLogRollOnDatanodeDeath() throws Exception {
     TEST_UTIL.ensureSomeRegionServersAvailable(2);
     assertTrue("This test requires WAL file replication set to 2.",
       fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()) == 2);
-    LOG.info("Replication=" +
-      fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()));
+    LOG.info("Replication=" + fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()));
 
     this.server = cluster.getRegionServer(0);
 
@@ -328,29 +121,28 @@ public class TestLogRolling  {
     });
 
     // don't run this test without append support (HDFS-200 & HDFS-142)
-    assertTrue("Need append support for this test", FSUtils
-        .isAppendSupported(TEST_UTIL.getConfiguration()));
+    assertTrue("Need append support for this test",
+      FSUtils.isAppendSupported(TEST_UTIL.getConfiguration()));
 
     // add up the datanode count, to ensure proper replication when we kill 1
     // This function is synchronous; when it returns, the dfs cluster is active
     // We start 3 servers and then stop 2 to avoid a directory naming conflict
-    //  when we stop/start a namenode later, as mentioned in HBASE-5163
+    // when we stop/start a namenode later, as mentioned in HBASE-5163
     List<DataNode> existingNodes = dfsCluster.getDataNodes();
     int numDataNodes = 3;
-    dfsCluster.startDataNodes(TEST_UTIL.getConfiguration(), numDataNodes, true,
-        null, null);
+    dfsCluster.startDataNodes(TEST_UTIL.getConfiguration(), numDataNodes, true, null, null);
     List<DataNode> allNodes = dfsCluster.getDataNodes();
-    for (int i = allNodes.size()-1; i >= 0; i--) {
+    for (int i = allNodes.size() - 1; i >= 0; i--) {
       if (existingNodes.contains(allNodes.get(i))) {
-        dfsCluster.stopDataNode( i );
+        dfsCluster.stopDataNode(i);
       }
     }
 
-    assertTrue("DataNodes " + dfsCluster.getDataNodes().size() +
-        " default replication " +
-        fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()),
-    dfsCluster.getDataNodes().size() >=
-      fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()) + 1);
+    assertTrue(
+      "DataNodes " + dfsCluster.getDataNodes().size() + " default replication "
+          + fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()),
+      dfsCluster.getDataNodes()
+          .size() >= fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()) + 1);
 
     writeData(table, 2);
 
@@ -358,13 +150,12 @@ public class TestLogRolling  {
     LOG.info("log.getCurrentFileName(): " + log.getCurrentFileName());
     long oldFilenum = DefaultWALProvider.extractFileNumFromWAL(log);
     assertTrue("Log should have a timestamp older than now",
-        curTime > oldFilenum && oldFilenum != -1);
+      curTime > oldFilenum && oldFilenum != -1);
 
     assertTrue("The log shouldn't have rolled yet",
-        oldFilenum == DefaultWALProvider.extractFileNumFromWAL(log));
-    final DatanodeInfo[] pipeline = log.getPipeLine();
-    assertTrue(pipeline.length ==
-        fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()));
+      oldFilenum == DefaultWALProvider.extractFileNumFromWAL(log));
+    final DatanodeInfo[] pipeline = log.getPipeline();
+    assertTrue(pipeline.length == fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()));
 
     // kill a datanode in the pipeline to force a log roll on the next sync()
     // This function is synchronous, when it returns the node is killed.
@@ -375,41 +166,38 @@ public class TestLogRolling  {
     long newFilenum = DefaultWALProvider.extractFileNumFromWAL(log);
 
     assertTrue("Missing datanode should've triggered a log roll",
-        newFilenum > oldFilenum && newFilenum > curTime);
+      newFilenum > oldFilenum && newFilenum > curTime);
 
     assertTrue("The log rolling hook should have been called with the low replication flag",
-        lowReplicationHookCalled.get());
+      lowReplicationHookCalled.get());
 
     // write some more log data (this should use a new hdfs_out)
     writeData(table, 3);
     assertTrue("The log should not roll again.",
-        DefaultWALProvider.extractFileNumFromWAL(log) == newFilenum);
+      DefaultWALProvider.extractFileNumFromWAL(log) == newFilenum);
     // kill another datanode in the pipeline, so the replicas will be lower than
     // the configured value 2.
     assertTrue(dfsCluster.stopDataNode(pipeline[1].getName()) != null);
 
     batchWriteAndWait(table, log, 3, false, 14000);
     int replication = log.getLogReplication();
-    assertTrue("LowReplication Roller should've been disabled, current replication="
-            + replication, !log.isLowReplicationRollEnabled());
+    assertTrue("LowReplication Roller should've been disabled, current replication=" + replication,
+      !log.isLowReplicationRollEnabled());
 
-    dfsCluster
-        .startDataNodes(TEST_UTIL.getConfiguration(), 1, true, null, null);
+    dfsCluster.startDataNodes(TEST_UTIL.getConfiguration(), 1, true, null, null);
 
     // Force roll writer. The new log file will have the default replications,
     // and the LowReplication Roller will be enabled.
     log.rollWriter(true);
     batchWriteAndWait(table, log, 13, true, 10000);
     replication = log.getLogReplication();
-    assertTrue("New log file should have the default replication instead of " +
-      replication,
+    assertTrue("New log file should have the default replication instead of " + replication,
       replication == fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()));
     assertTrue("LowReplication Roller should've been enabled", log.isLowReplicationRollEnabled());
   }
 
   /**
-   * Test that WAL is rolled when all data nodes in the pipeline have been
-   * restarted.
+   * Test that WAL is rolled when all data nodes in the pipeline have been restarted.
    * @throws Exception
    */
   @Test
@@ -417,8 +205,7 @@ public class TestLogRolling  {
     LOG.info("Starting testLogRollOnPipelineRestart");
     assertTrue("This test requires WAL file replication.",
       fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()) > 1);
-    LOG.info("Replication=" +
-      fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()));
+    LOG.info("Replication=" + fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()));
     // When the hbase:meta table can be opened, the region servers are running
     Table t = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME);
     try {
@@ -441,10 +228,11 @@ public class TestLogRolling  {
       log.registerWALActionsListener(new WALActionsListener.Base() {
 
         @Override
-        public void preLogRoll(Path oldFile, Path newFile)  {
-          LOG.debug("preLogRoll: oldFile="+oldFile+" newFile="+newFile);
+        public void preLogRoll(Path oldFile, Path newFile) {
+          LOG.debug("preLogRoll: oldFile=" + oldFile + " newFile=" + newFile);
           preLogRolledCalled.add(new Integer(1));
         }
+
         @Override
         public void postLogRoll(Path oldFile, Path newFile) {
           paths.add(newFile);
@@ -452,8 +240,8 @@ public class TestLogRolling  {
       });
 
       // don't run this test without append support (HDFS-200 & HDFS-142)
-      assertTrue("Need append support for this test", FSUtils
-          .isAppendSupported(TEST_UTIL.getConfiguration()));
+      assertTrue("Need append support for this test",
+        FSUtils.isAppendSupported(TEST_UTIL.getConfiguration()));
 
       writeData(table, 1002);
 
@@ -461,10 +249,10 @@ public class TestLogRolling  {
       LOG.info("log.getCurrentFileName()): " + DefaultWALProvider.getCurrentFileName(log));
       long oldFilenum = DefaultWALProvider.extractFileNumFromWAL(log);
       assertTrue("Log should have a timestamp older than now",
-          curTime > oldFilenum && oldFilenum != -1);
+        curTime > oldFilenum && oldFilenum != -1);
 
-      assertTrue("The log shouldn't have rolled yet", oldFilenum ==
-          DefaultWALProvider.extractFileNumFromWAL(log));
+      assertTrue("The log shouldn't have rolled yet",
+        oldFilenum == DefaultWALProvider.extractFileNumFromWAL(log));
 
       // roll all datanodes in the pipeline
       dfsCluster.restartDataNodes();
@@ -478,7 +266,7 @@ public class TestLogRolling  {
       long newFilenum = DefaultWALProvider.extractFileNumFromWAL(log);
 
       assertTrue("Missing datanode should've triggered a log roll",
-          newFilenum > oldFilenum && newFilenum > curTime);
+        newFilenum > oldFilenum && newFilenum > curTime);
       validateData(table, 1003);
 
       writeData(table, 1004);
@@ -496,30 +284,30 @@ public class TestLogRolling  {
       // force a log roll to read back and verify previously written logs
       log.rollWriter(true);
       assertTrue("preLogRolledCalled has size of " + preLogRolledCalled.size(),
-          preLogRolledCalled.size() >= 1);
+        preLogRolledCalled.size() >= 1);
 
       // read back the data written
       Set<String> loggedRows = new HashSet<String>();
       FSUtils fsUtils = FSUtils.getInstance(fs, TEST_UTIL.getConfiguration());
       for (Path p : paths) {
         LOG.debug("recovering lease for " + p);
-        fsUtils.recoverFileLease(((HFileSystem)fs).getBackingFs(), p,
-          TEST_UTIL.getConfiguration(), null);
+        fsUtils.recoverFileLease(((HFileSystem) fs).getBackingFs(), p, TEST_UTIL.getConfiguration(),
+          null);
 
-        LOG.debug("Reading WAL "+FSUtils.getPath(p));
+        LOG.debug("Reading WAL " + FSUtils.getPath(p));
         WAL.Reader reader = null;
         try {
           reader = WALFactory.createReader(fs, p, TEST_UTIL.getConfiguration());
           WAL.Entry entry;
           while ((entry = reader.next()) != null) {
-            LOG.debug("#"+entry.getKey().getLogSeqNum()+": "+entry.getEdit().getCells());
+            LOG.debug("#" + entry.getKey().getSequenceId() + ": " + entry.getEdit().getCells());
             for (Cell cell : entry.getEdit().getCells()) {
-              loggedRows.add(Bytes.toStringBinary(cell.getRowArray(), cell.getRowOffset(),
-                cell.getRowLength()));
+              loggedRows.add(
+                Bytes.toStringBinary(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()));
             }
           }
         } catch (EOFException e) {
-          LOG.debug("EOF reading file "+FSUtils.getPath(p));
+          LOG.debug("EOF reading file " + FSUtils.getPath(p));
         } finally {
           if (reader != null) reader.close();
         }
@@ -532,7 +320,7 @@ public class TestLogRolling  {
       assertTrue(loggedRows.contains("row1005"));
 
       // flush all regions
-      for (Region r: server.getOnlineRegionsLocalContext()) {
+      for (Region r : server.getOnlineRegionsLocalContext()) {
         try {
           r.flush(true);
         } catch (Exception e) {
@@ -547,19 +335,19 @@ public class TestLogRolling  {
 
       ResultScanner scanner = table.getScanner(new Scan());
       try {
-        for (int i=2; i<=5; i++) {
+        for (int i = 2; i <= 5; i++) {
           Result r = scanner.next();
           assertNotNull(r);
           assertFalse(r.isEmpty());
-          assertEquals("row100"+i, Bytes.toString(r.getRow()));
+          assertEquals("row100" + i, Bytes.toString(r.getRow()));
         }
       } finally {
         scanner.close();
       }
 
       // verify that no region servers aborted
-      for (JVMClusterUtil.RegionServerThread rsThread:
-        TEST_UTIL.getHBaseCluster().getRegionServerThreads()) {
+      for (JVMClusterUtil.RegionServerThread rsThread : TEST_UTIL.getHBaseCluster()
+          .getRegionServerThreads()) {
         assertFalse(rsThread.getRegionServer().isAborted());
       }
     } finally {
@@ -567,80 +355,4 @@ public class TestLogRolling  {
     }
   }
 
-  /**
-   * Tests that logs are deleted when some region has a compaction
-   * record in WAL and no other records. See HBASE-8597.
-   */
-  @Test
-  public void testCompactionRecordDoesntBlockRolling() throws Exception {
-    Table table = null;
-
-    // When the hbase:meta table can be opened, the region servers are running
-    Table t = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME);
-    try {
-      table = createTestTable(getName());
-
-      server = TEST_UTIL.getRSForFirstRegionInTable(table.getName());
-      Region region = server.getOnlineRegions(table.getName()).get(0);
-      final WAL log = server.getWAL(region.getRegionInfo());
-      Store s = region.getStore(HConstants.CATALOG_FAMILY);
-
-      //have to flush namespace to ensure it doesn't affect wall tests
-      admin.flush(TableName.NAMESPACE_TABLE_NAME);
-
-      // Put some stuff into table, to make sure we have some files to compact.
-      for (int i = 1; i <= 2; ++i) {
-        doPut(table, i);
-        admin.flush(table.getName());
-      }
-      doPut(table, 3); // don't flush yet, or compaction might trigger before we roll WAL
-      assertEquals("Should have no WAL after initial writes", 0,
-          DefaultWALProvider.getNumRolledLogFiles(log));
-      assertEquals(2, s.getStorefilesCount());
-
-      // Roll the log and compact table, to have compaction record in the 2nd WAL.
-      log.rollWriter();
-      assertEquals("Should have WAL; one table is not flushed", 1,
-          DefaultWALProvider.getNumRolledLogFiles(log));
-      admin.flush(table.getName());
-      region.compact(false);
-      // Wait for compaction in case if flush triggered it before us.
-      Assert.assertNotNull(s);
-      for (int waitTime = 3000; s.getStorefilesCount() > 1 && waitTime > 0; waitTime -= 200) {
-        Threads.sleepWithoutInterrupt(200);
-      }
-      assertEquals("Compaction didn't happen", 1, s.getStorefilesCount());
-
-      // Write some value to the table so the WAL cannot be deleted until table is flushed.
-      doPut(table, 0); // Now 2nd WAL will have both compaction and put record for table.
-      log.rollWriter(); // 1st WAL deleted, 2nd not deleted yet.
-      assertEquals("Should have WAL; one table is not flushed", 1,
-          DefaultWALProvider.getNumRolledLogFiles(log));
-
-      // Flush table to make latest WAL obsolete; write another record, and roll again.
-      admin.flush(table.getName());
-      doPut(table, 1);
-      log.rollWriter(); // Now 2nd WAL is deleted and 3rd is added.
-      assertEquals("Should have 1 WALs at the end", 1,
-          DefaultWALProvider.getNumRolledLogFiles(log));
-    } finally {
-      if (t != null) t.close();
-      if (table != null) table.close();
-    }
-  }
-
-  private void doPut(Table table, int i) throws IOException {
-    Put put = new Put(Bytes.toBytes("row" + String.format("%1$04d", i)));
-    put.addColumn(HConstants.CATALOG_FAMILY, null, value);
-    table.put(put);
-  }
-
-  private Table createTestTable(String tableName) throws IOException {
-    // Create the test table and open it
-    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
-    desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
-    admin.createTable(desc);
-    return TEST_UTIL.getConnection().getTable(desc.getTableName());
-  }
 }
-

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestProtobufLog.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestProtobufLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestProtobufLog.java
index e6237f8..61ee589 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestProtobufLog.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestProtobufLog.java
@@ -1,5 +1,4 @@
 /**
- *
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -18,190 +17,32 @@
  */
 package org.apache.hadoop.hbase.regionserver.wal;
 
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
 import java.io.IOException;
 
-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.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
-import org.apache.hadoop.hbase.coprocessor.SampleRegionWALObserver;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.wal.WAL;
-import org.apache.hadoop.hbase.wal.WALFactory;
-import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.hadoop.hbase.wal.DefaultWALProvider;
 import org.apache.hadoop.hbase.wal.WALProvider;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
+import org.apache.hadoop.hbase.wal.WALProvider.Writer;
 import org.junit.experimental.categories.Category;
-import org.junit.rules.TestName;
-
-/**
- * WAL tests that can be reused across providers.
- */
-@Category({RegionServerTests.class, MediumTests.class})
-public class TestProtobufLog {
-  protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-
-  protected FileSystem fs;
-  protected Path dir;
-  protected WALFactory wals;
-
-  @Rule
-  public final TestName currentTest = new TestName();
 
-  @Before
-  public void setUp() throws Exception {
-    fs = TEST_UTIL.getDFSCluster().getFileSystem();
-    dir = new Path(TEST_UTIL.createRootDir(), currentTest.getMethodName());
-    wals = new WALFactory(TEST_UTIL.getConfiguration(), null, currentTest.getMethodName());
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    wals.close();
-    FileStatus[] entries = fs.listStatus(new Path("/"));
-    for (FileStatus dir : entries) {
-      fs.delete(dir.getPath(), true);
-    }
-  }
-
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    // Make block sizes small.
-    TEST_UTIL.getConfiguration().setInt("dfs.blocksize", 1024 * 1024);
-    // needed for testAppendClose()
-    TEST_UTIL.getConfiguration().setBoolean("dfs.support.broken.append", true);
-    TEST_UTIL.getConfiguration().setBoolean("dfs.support.append", true);
-    // quicker heartbeat interval for faster DN death notification
-    TEST_UTIL.getConfiguration().setInt("dfs.namenode.heartbeat.recheck-interval", 5000);
-    TEST_UTIL.getConfiguration().setInt("dfs.heartbeat.interval", 1);
-    TEST_UTIL.getConfiguration().setInt("dfs.client.socket-timeout", 5000);
+@Category({ RegionServerTests.class, MediumTests.class })
+public class TestProtobufLog extends AbstractTestProtobufLog<WALProvider.Writer> {
 
-    // faster failover with cluster.shutdown();fs.close() idiom
-    TEST_UTIL.getConfiguration()
-        .setInt("hbase.ipc.client.connect.max.retries", 1);
-    TEST_UTIL.getConfiguration().setInt(
-        "dfs.client.block.recovery.retries", 1);
-    TEST_UTIL.getConfiguration().setInt(
-      "hbase.ipc.client.connection.maxidletime", 500);
-    TEST_UTIL.getConfiguration().set(CoprocessorHost.WAL_COPROCESSOR_CONF_KEY,
-        SampleRegionWALObserver.class.getName());
-    TEST_UTIL.startMiniDFSCluster(3);
+  @Override
+  protected Writer createWriter(Path path) throws IOException {
+    return DefaultWALProvider.createWriter(TEST_UTIL.getConfiguration(), fs, path, false);
   }
 
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    TEST_UTIL.shutdownMiniCluster();
+  @Override
+  protected void append(Writer writer, Entry entry) throws IOException {
+    writer.append(entry);
   }
 
-  /**
-   * Reads the WAL with and without WALTrailer.
-   * @throws IOException
-   */
-  @Test
-  public void testWALTrailer() throws IOException {
-    // read With trailer.
-    doRead(true);
-    // read without trailer
-    doRead(false);
-  }
-
-  /**
-   * Appends entries in the WAL and reads it.
-   * @param withTrailer If 'withTrailer' is true, it calls a close on the WALwriter before reading
-   *          so that a trailer is appended to the WAL. Otherwise, it starts reading after the sync
-   *          call. This means that reader is not aware of the trailer. In this scenario, if the
-   *          reader tries to read the trailer in its next() call, it returns false from
-   *          ProtoBufLogReader.
-   * @throws IOException
-   */
-  private void doRead(boolean withTrailer) throws IOException {
-    final int columnCount = 5;
-    final int recordCount = 5;
-    final TableName tableName =
-        TableName.valueOf("tablename");
-    final byte[] row = Bytes.toBytes("row");
-    long timestamp = System.currentTimeMillis();
-    Path path = new Path(dir, "tempwal");
-    // delete the log if already exists, for test only
-    fs.delete(path, true);
-    WALProvider.Writer writer = null;
-    ProtobufLogReader reader = null;
-    try {
-      HRegionInfo hri = new HRegionInfo(tableName,
-          HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
-      HTableDescriptor htd = new HTableDescriptor(tableName);
-      fs.mkdirs(dir);
-      // Write log in pb format.
-      writer = wals.createWALWriter(fs, path);
-      for (int i = 0; i < recordCount; ++i) {
-        WALKey key = new WALKey(
-            hri.getEncodedNameAsBytes(), tableName, i, timestamp, HConstants.DEFAULT_CLUSTER_ID);
-        WALEdit edit = new WALEdit();
-        for (int j = 0; j < columnCount; ++j) {
-          if (i == 0) {
-            htd.addFamily(new HColumnDescriptor("column" + j));
-          }
-          String value = i + "" + j;
-          edit.add(new KeyValue(row, row, row, timestamp, Bytes.toBytes(value)));
-        }
-        writer.append(new WAL.Entry(key, edit));
-      }
-      writer.sync();
-      if (withTrailer) writer.close();
-
-      // Now read the log using standard means.
-      reader = (ProtobufLogReader) wals.createReader(fs, path);
-      if (withTrailer) {
-        assertNotNull(reader.trailer);
-      } else {
-        assertNull(reader.trailer);
-      }
-      for (int i = 0; i < recordCount; ++i) {
-        WAL.Entry entry = reader.next();
-        assertNotNull(entry);
-        assertEquals(columnCount, entry.getEdit().size());
-        assertArrayEquals(hri.getEncodedNameAsBytes(), entry.getKey().getEncodedRegionName());
-        assertEquals(tableName, entry.getKey().getTablename());
-        int idx = 0;
-        for (Cell val : entry.getEdit().getCells()) {
-          assertTrue(Bytes.equals(row, 0, row.length, val.getRowArray(), val.getRowOffset(),
-            val.getRowLength()));
-          String value = i + "" + idx;
-          assertArrayEquals(Bytes.toBytes(value), CellUtil.cloneValue(val));
-          idx++;
-        }
-      }
-      WAL.Entry entry = reader.next();
-      assertNull(entry);
-    } finally {
-      if (writer != null) {
-        writer.close();
-      }
-      if (reader != null) {
-        reader.close();
-      }
-    }
+  @Override
+  protected void sync(Writer writer) throws IOException {
+    writer.sync();
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplayCompressed.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplayCompressed.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplayCompressed.java
index 4987fd4..b225554 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplayCompressed.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplayCompressed.java
@@ -36,5 +36,4 @@ public class TestWALReplayCompressed extends TestWALReplay {
     Configuration conf = TestWALReplay.TEST_UTIL.getConfiguration();
     conf.setBoolean(HConstants.ENABLE_WAL_COMPRESSION, true);
   }
-
 }