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/02/13 21:10:12 UTC

[01/22] hbase git commit: HBASE-15231 Make TableState.State private (Misty Stanley-Jones)

Repository: hbase
Updated Branches:
  refs/heads/hbase-12439 3aff98c75 -> 3897c4e10


HBASE-15231 Make TableState.State private (Misty Stanley-Jones)


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

Branch: refs/heads/hbase-12439
Commit: 7bb68b9031591cf378954a0eb8f71a8b9be01f9c
Parents: 3aff98c
Author: tedyu <yu...@gmail.com>
Authored: Mon Feb 8 15:21:18 2016 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Mon Feb 8 15:21:18 2016 -0800

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/7bb68b90/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableState.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableState.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableState.java
index c69cdfc..5d4ac8e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableState.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableState.java
@@ -30,7 +30,7 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 @InterfaceAudience.Private
 public class TableState {
 
-  @InterfaceAudience.Public
+  @InterfaceAudience.Private
   @InterfaceStability.Evolving
   public static enum State {
     ENABLED,


[07/22] hbase git commit: HBASE-15239 Remove unused LoadBalancer.immediateAssignment()

Posted by sy...@apache.org.
HBASE-15239 Remove unused LoadBalancer.immediateAssignment()


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

Branch: refs/heads/hbase-12439
Commit: d53318163be54ff8b0eff44402fdf5b16a233100
Parents: df829ea
Author: Matteo Bertozzi <ma...@cloudera.com>
Authored: Wed Feb 10 09:16:42 2016 -0800
Committer: Matteo Bertozzi <ma...@cloudera.com>
Committed: Wed Feb 10 09:16:42 2016 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/master/LoadBalancer.java       | 15 --------
 .../hbase/master/balancer/BaseLoadBalancer.java | 39 ++------------------
 .../master/balancer/SimpleLoadBalancer.java     |  6 +--
 .../master/balancer/TestBaseLoadBalancer.java   | 32 ----------------
 4 files changed, 4 insertions(+), 88 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d5331816/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java
index 15dedc6..6a618e1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java
@@ -40,10 +40,6 @@ import org.apache.hadoop.hbase.TableName;
  * <p>Cluster-wide load balancing will occur only when there are no regions in
  * transition and according to a fixed period of a time using {@link #balanceCluster(Map)}.
  *
- * <p>Inline region placement with {@link #immediateAssignment} can be used when
- * the Master needs to handle closed regions that it currently does not have
- * a destination set for.  This can happen during master failover.
- *
  * <p>On cluster startup, bulk assignment can be used to determine
  * locations for all Regions in a cluster.
  *
@@ -106,17 +102,6 @@ public interface LoadBalancer extends Configurable, Stoppable, ConfigurationObse
   ) throws HBaseIOException;
 
   /**
-   * Sync assign a region
-   * @param regions
-   * @param servers
-    * @return Map regioninfos to servernames
-   */
-  Map<HRegionInfo, ServerName> immediateAssignment(
-    List<HRegionInfo> regions,
-    List<ServerName> servers
-  ) throws HBaseIOException;
-
-  /**
    * Get a random region server from the list
    * @param regionInfo Region for which this selection is being done.
    * @param servers

http://git-wip-us.apache.org/repos/asf/hbase/blob/d5331816/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 44e1f79..bde5c61 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
@@ -81,16 +81,16 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
       return UNKNOWN_RACK;
     }
   }
-  
+
   /**
    * The constructor that uses the basic MetricsBalancer
    */
   protected BaseLoadBalancer() {
     metricsBalancer = new MetricsBalancer();
   }
-  
+
   /**
-   * This Constructor accepts an instance of MetricsBalancer, 
+   * This Constructor accepts an instance of MetricsBalancer,
    * which will be used instead of creating a new one
    */
   protected BaseLoadBalancer(MetricsBalancer metricsBalancer) {
@@ -1279,39 +1279,6 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
   }
 
   /**
-   * Generates an immediate assignment plan to be used by a new master for
-   * regions in transition that do not have an already known destination.
-   *
-   * Takes a list of regions that need immediate assignment and a list of all
-   * available servers. Returns a map of regions to the server they should be
-   * assigned to.
-   *
-   * This method will return quickly and does not do any intelligent balancing.
-   * The goal is to make a fast decision not the best decision possible.
-   *
-   * Currently this is random.
-   *
-   * @param regions
-   * @param servers
-   * @return map of regions to the server it should be assigned to
-   */
-  @Override
-  public Map<HRegionInfo, ServerName> immediateAssignment(List<HRegionInfo> regions,
-      List<ServerName> servers) {
-    metricsBalancer.incrMiscInvocations();
-    if (servers == null || servers.isEmpty()) {
-      LOG.warn("Wanted to do random assignment but no servers to assign to");
-      return null;
-    }
-
-    Map<HRegionInfo, ServerName> assignments = new TreeMap<HRegionInfo, ServerName>();
-    for (HRegionInfo region : regions) {
-      assignments.put(region, randomAssignment(region, servers));
-    }
-    return assignments;
-  }
-
-  /**
    * Used to assign a single region to a random server.
    */
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/d5331816/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/SimpleLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/SimpleLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/SimpleLoadBalancer.java
index 4325585..fdcedf1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/SimpleLoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/SimpleLoadBalancer.java
@@ -46,14 +46,10 @@ import com.google.common.collect.MinMaxPriorityQueue;
  * <p>Cluster-wide load balancing will occur only when there are no regions in
  * transition and according to a fixed period of a time using {@link #balanceCluster(Map)}.
  *
- * <p>Inline region placement with {@link #immediateAssignment} can be used when
- * the Master needs to handle closed regions that it currently does not have
- * a destination set for.  This can happen during master failover.
- *
  * <p>On cluster startup, bulk assignment can be used to determine
  * locations for all Regions in a cluster.
  *
- * <p>This classes produces plans for the 
+ * <p>This classes produces plans for the
  * {@link org.apache.hadoop.hbase.master.AssignmentManager} to execute.
  */
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)

http://git-wip-us.apache.org/repos/asf/hbase/blob/d5331816/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBaseLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBaseLoadBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBaseLoadBalancer.java
index 205fbea..45d9fe5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBaseLoadBalancer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBaseLoadBalancer.java
@@ -118,38 +118,6 @@ public class TestBaseLoadBalancer extends BalancerTestBase {
   }
 
   /**
-   * Tests immediate assignment.
-   *
-   * Invariant is that all regions have an assignment.
-   *
-   * @throws Exception
-   */
-  @Test (timeout=30000)
-  public void testImmediateAssignment() throws Exception {
-    List<ServerName> tmp = getListOfServerNames(randomServers(1, 0));
-    tmp.add(master);
-    ServerName sn = loadBalancer.randomAssignment(HRegionInfo.FIRST_META_REGIONINFO, tmp);
-    assertEquals(master, sn);
-    HRegionInfo hri = randomRegions(1, -1).get(0);
-    sn = loadBalancer.randomAssignment(hri, tmp);
-    assertNotEquals(master, sn);
-    tmp = new ArrayList<ServerName>();
-    tmp.add(master);
-    sn = loadBalancer.randomAssignment(hri, tmp);
-    assertNull("Should not assign user regions on master", sn);
-    for (int[] mock : regionsAndServersMocks) {
-      LOG.debug("testImmediateAssignment with " + mock[0] + " regions and " + mock[1] + " servers");
-      List<HRegionInfo> regions = randomRegions(mock[0]);
-      List<ServerAndLoad> servers = randomServers(mock[1], 0);
-      List<ServerName> list = getListOfServerNames(servers);
-      Map<HRegionInfo, ServerName> assignments = loadBalancer.immediateAssignment(regions, list);
-      assertImmediateAssignment(regions, list, assignments);
-      returnRegions(regions);
-      returnServers(list);
-    }
-  }
-
-  /**
    * All regions have an assignment.
    * @param regions
    * @param servers


[21/22] hbase git commit: HBASE-13839 Fix AssgnmentManagerTmpl.jamon issues (coloring, content etc.)

Posted by sy...@apache.org.
HBASE-13839 Fix AssgnmentManagerTmpl.jamon issues (coloring, content etc.)


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

Branch: refs/heads/hbase-12439
Commit: 1419628881698be9e7dec92424d8a5719d35c1cf
Parents: c8d1331
Author: stack <st...@apache.org>
Authored: Fri Feb 12 15:06:38 2016 -0800
Committer: stack <st...@apache.org>
Committed: Fri Feb 12 15:06:38 2016 -0800

----------------------------------------------------------------------
 .../master/AssignmentManagerStatusTmpl.jamon    | 141 +++++++++++--------
 .../hadoop/hbase/master/RegionStates.java       |  26 ++++
 2 files changed, 110 insertions(+), 57 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/14196288/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/AssignmentManagerStatusTmpl.jamon
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/AssignmentManagerStatusTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/AssignmentManagerStatusTmpl.jamon
index 2bf034a..42334ff 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/AssignmentManagerStatusTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/AssignmentManagerStatusTmpl.jamon
@@ -25,84 +25,111 @@ org.apache.hadoop.hbase.HBaseConfiguration;
 org.apache.hadoop.hbase.HConstants;
 java.util.Iterator;
 java.util.Map;
+java.util.List;
+java.util.ArrayList;
+java.util.Map.Entry;
+java.util.Arrays;
 </%import>
 <%args>
 AssignmentManager assignmentManager;
 int limit = 100;
 </%args>
+
+<%java Map<String, RegionState> rit = assignmentManager
+  .getRegionStates().getRegionsInTransitionOrderedByTimestamp(); %>
+
+<%if !rit.isEmpty() %>
 <%java>
-Map<String, RegionState> rit = assignmentManager
-  .getRegionStates().getRegionsInTransition();
+List<String> ritsOverThreshold = new ArrayList<>();
+List<String> ritsTwiceThreshold = new ArrayList<>();
 // process the map to find region in transition details
 Configuration conf = HBaseConfiguration.create();
 int ritThreshold = conf.getInt(HConstants.METRICS_RIT_STUCK_WARNING_THRESHOLD, 60000);
 int numOfRITOverThreshold = 0;
-long maxRITTime = Long.MIN_VALUE;
 long currentTime = System.currentTimeMillis();
-String regionIDForOldestRIT = ""; // avoiding null
 for (Map.Entry<String, RegionState> e : rit.entrySet()) {
   long ritTime = currentTime - e.getValue().getStamp();
-  if(ritTime > ritThreshold) {
+  if(ritTime > (ritThreshold * 2)) {
      numOfRITOverThreshold++;
-   }
-   if(maxRITTime < ritTime) {
-     maxRITTime = ritTime;
-     regionIDForOldestRIT = e.getKey();
-   }
-}
-
-int totalRITs = rit.size();
-int toRemove = rit.size() - limit;
-int removed = 0;
-if (toRemove > 0) {
-  // getRegionsInTransition returned a copy, so we can mutate it
-  for (Iterator<Map.Entry<String, RegionState>> it = rit.entrySet().iterator();
-       it.hasNext() && toRemove > 0;
-       ) {
-    Map.Entry<String, RegionState> e = it.next();
-    if (HRegionInfo.FIRST_META_REGIONINFO.getEncodedName().equals(
-          e.getKey()) ||
-         regionIDForOldestRIT.equals(e.getKey())) {
-      // don't remove the meta & the oldest rit regions, they're too interesting!
-      continue;
-    } 
-    it.remove();
-    toRemove--;
-    removed++;
+     ritsTwiceThreshold.add(e.getKey());
+  } else if (ritTime > ritThreshold) {
+     numOfRITOverThreshold++;
+     ritsOverThreshold.add(e.getKey());
   }
 }
 
+int numOfRITs = rit.size();
+int ritsPerPage = Math.min(5, numOfRITs);
+int numOfPages = (int) Math.ceil(numOfRITs * 1.0 / ritsPerPage);
 </%java>
-
-
-<%if !rit.isEmpty() %>
     <section>
     <h2>Regions in Transition</h2>
-    <table class="table table-striped">
-            <tr><th>Region</th><th>State</th><th>RIT time (ms)</th></tr>
-            <%for Map.Entry<String, RegionState> entry : rit.entrySet() %>
-            <%if regionIDForOldestRIT.equals(entry.getKey()) %>
-                    <tr BGCOLOR="#FE2E2E" >
-            <%else>
-                    <tr>
-            </%if>
-            <td><% entry.getKey() %></td><td>
-            <% HRegionInfo.getDescriptiveNameFromRegionStateForDisplay(
-            entry.getValue(), conf) %></td>
-    <td><% (currentTime - entry.getValue().getStamp()) %> </td></tr>
-            </%for>
-            <%if numOfRITOverThreshold > 0 %>
-                    <tr BGCOLOR="#D7DF01" >
+     <p><% numOfRITs %> region(s) in transition.  
+     <%if !ritsTwiceThreshold.isEmpty()  %>
+         <span class="label label-danger" style="font-size:100%;font-weight:normal">
+     <%elseif !ritsOverThreshold.isEmpty() %>
+         <span class="label label-warning" style="font-size:100%;font-weight:normal">
+     <%else>
+         <span>
+     </%if>
+         <% numOfRITOverThreshold %> region(s) in transition for 
+             more than <% ritThreshold %> milliseconds.
+         </span>
+     </p>
+     <div class="tabbable">
+         <div class="tab-content">
+         <%java int recordItr = 0; %>
+         <%for Map.Entry<String, RegionState> entry : rit.entrySet() %>
+             <%if (recordItr % ritsPerPage) == 0 %>
+                 <%if recordItr == 0 %>
+             <div class="tab-pane active" id="tab_rits<% (recordItr / ritsPerPage) + 1 %>">
+                 <%else>
+             <div class="tab-pane" id="tab_rits<% (recordItr / ritsPerPage) + 1 %>">
+                 </%if>
+                 <table class="table table-striped" style="margin-bottom:0px;"><tr><th>Region</th>
+                     <th>State</th><th>RIT time (ms)</th></tr>
+             </%if>
+ 
+             <%if ritsOverThreshold.contains(entry.getKey()) %>
+                     <tr class="alert alert-warning" role="alert">
+             <%elseif ritsTwiceThreshold.contains(entry.getKey()) %>
+                     <tr class="alert alert-danger" role="alert">
             <%else>
                     <tr>
             </%if>
-            <td>Total number of Regions in Transition for more than <% ritThreshold %> milliseconds</td><td> <% numOfRITOverThreshold %></td><td></td>
-            </tr>
-    <tr> <td> Total number of Regions in Transition</td><td><% totalRITs %> </td><td></td>
-    </table>
-    <%if removed > 0 %>
-    (<% removed %> more regions in transition not shown)
-    </%if>
-    </section>
-</%if>
+                         <td><% entry.getKey() %></td><td>
+                         <% HRegionInfo.getDescriptiveNameFromRegionStateForDisplay(
+                         entry.getValue(), conf) %></td>
+                         <td><% (currentTime - entry.getValue().getStamp()) %> </td>
+                     </tr>
+                     <%java recordItr++; %>
+             <%if (recordItr % ritsPerPage) == 0 %>
+                 </table>
+             </div>
+         </%if>
+         </%for>
+  
+         <%if (recordItr % ritsPerPage) != 0 %>
+             <%for ; (recordItr % ritsPerPage) != 0 ; recordItr++ %>
+             <tr><td colspan="3" style="height:61px"></td></tr>
+             </%for>
+                   </table>
+             </div>
+         </%if>
+         </div>
+         <nav>
+             <ul class="nav nav-pills pagination">
+             <%for int i = 1 ; i <= numOfPages; i++ %>
+                 <%if i == 1 %>
+                 <li class="active">
+                 <%else>
+                 <li>
+                 </%if>
+                 <a href="#tab_rits<% i %>"><% i %></a></li>
+             </%for>
+             </ul>
+         </nav>
+     </div>
+   </section>
+ </%if>
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/14196288/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java
index 3743616..be9758a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java
@@ -23,9 +23,13 @@ import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.Collections;
+import java.util.Comparator;
 import java.util.TreeMap;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -206,6 +210,28 @@ public class RegionStates {
     return (Map<String, RegionState>)regionsInTransition.clone();
   }
 
+  @SuppressWarnings("unchecked")
+  public synchronized Map<String, RegionState> getRegionsInTransitionOrderedByTimestamp() {
+    Map<String, RegionState> rit = (Map<String, RegionState>)regionsInTransition.clone();
+    List<Map.Entry<String, RegionState>> list = new LinkedList<>(rit.entrySet());
+
+    // Compare the RITs' timestamps for ordering.
+    Comparator<Map.Entry<String, RegionState>> c =
+        new Comparator<Map.Entry<String, RegionState>>() {
+      @Override
+      public int compare(Map.Entry<String, RegionState> o1, Map.Entry<String, RegionState> o2) {
+        return ((Long)o1.getValue().getStamp()).compareTo((Long)o2.getValue().getStamp());
+      }
+    };
+
+    Collections.sort(list, c);
+    Map<String, RegionState> result = new LinkedHashMap<>();
+    for (Map.Entry<String, RegionState> entry : list) {
+      result.put(entry.getKey(), entry.getValue());
+    }
+    return result;
+  }
+
   /**
    * @return True if specified region in transition.
    */


[14/22] hbase git commit: HBASE-14919 Refactoring for in-memory flush and compaction

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/25dfc112/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreScanner.java
new file mode 100644
index 0000000..dfcec25
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreScanner.java
@@ -0,0 +1,348 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.regionserver;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.htrace.Trace;
+
+/**
+ * This is the scanner for any MemStore implementation, derived from MemStore.
+ * The MemStoreScanner combines SegmentScanner from different Segments and
+ * uses the key-value heap and the reversed key-value heap for the aggregated key-values set.
+ * It is assumed that only traversing forward or backward is used (without zigzagging in between)
+ */
+@InterfaceAudience.Private
+public class MemStoreScanner extends NonLazyKeyValueScanner {
+  /**
+   * Types of cell MemStoreScanner
+   */
+  static public enum Type {
+    UNDEFINED,
+    COMPACT_FORWARD,
+    USER_SCAN_FORWARD,
+    USER_SCAN_BACKWARD
+  }
+
+  // heap of scanners used for traversing forward
+  private KeyValueHeap forwardHeap;
+  // reversed scanners heap for traversing backward
+  private ReversedKeyValueHeap backwardHeap;
+
+  // The type of the scan is defined by constructor
+  // or according to the first usage
+  private Type type = Type.UNDEFINED;
+
+  private long readPoint;
+  // remember the initial version of the scanners list
+  List<SegmentScanner> scanners;
+  // pointer back to the relevant MemStore
+  // is needed for shouldSeek() method
+  private AbstractMemStore backwardReferenceToMemStore;
+
+  /**
+   * Constructor.
+   * If UNDEFINED type for MemStoreScanner is provided, the forward heap is used as default!
+   * After constructor only one heap is going to be initialized for entire lifespan
+   * of the MemStoreScanner. A specific scanner can only be one directional!
+   *
+   * @param ms        Pointer back to the MemStore
+   * @param readPoint Read point below which we can safely remove duplicate KVs
+   * @param type      The scan type COMPACT_FORWARD should be used for compaction
+   */
+  public MemStoreScanner(AbstractMemStore ms, long readPoint, Type type) throws IOException {
+    this(ms, ms.getListOfScanners(readPoint), readPoint, type);
+  }
+
+  /* Constructor used only when the scan usage is unknown
+  and need to be defined according to the first move */
+  public MemStoreScanner(AbstractMemStore ms, long readPt) throws IOException {
+    this(ms, readPt, Type.UNDEFINED);
+  }
+
+  public MemStoreScanner(AbstractMemStore ms, List<SegmentScanner> scanners, long readPoint,
+      Type type) throws IOException {
+    super();
+    this.readPoint = readPoint;
+    this.type = type;
+    switch (type) {
+      case UNDEFINED:
+      case USER_SCAN_FORWARD:
+      case COMPACT_FORWARD:
+        this.forwardHeap = new KeyValueHeap(scanners, ms.getComparator());
+        break;
+      case USER_SCAN_BACKWARD:
+        this.backwardHeap = new ReversedKeyValueHeap(scanners, ms.getComparator());
+        break;
+      default:
+        throw new IllegalArgumentException("Unknown scanner type in MemStoreScanner");
+    }
+    this.backwardReferenceToMemStore = ms;
+    this.scanners = scanners;
+    if (Trace.isTracing() && Trace.currentSpan() != null) {
+      Trace.currentSpan().addTimelineAnnotation("Creating MemStoreScanner");
+    }
+  }
+
+  /**
+   * Returns the cell from the top-most scanner without advancing the iterator.
+   * The backward traversal is assumed, only if specified explicitly
+   */
+  @Override
+  public synchronized Cell peek() {
+    if (type == Type.USER_SCAN_BACKWARD) {
+      return backwardHeap.peek();
+    }
+    return forwardHeap.peek();
+  }
+
+  /**
+   * Gets the next cell from the top-most scanner. Assumed forward scanning.
+   */
+  @Override
+  public synchronized Cell next() throws IOException {
+    KeyValueHeap heap = (Type.USER_SCAN_BACKWARD == type) ? backwardHeap : forwardHeap;
+
+    // loop over till the next suitable value
+    // take next value from the heap
+    for (Cell currentCell = heap.next();
+         currentCell != null;
+         currentCell = heap.next()) {
+
+      // all the logic of presenting cells is inside the internal SegmentScanners
+      // located inside the heap
+
+      return currentCell;
+    }
+    return null;
+  }
+
+  /**
+   * Set the scanner at the seek key. Assumed forward scanning.
+   * Must be called only once: there is no thread safety between the scanner
+   * and the memStore.
+   *
+   * @param cell seek value
+   * @return false if the key is null or if there is no data
+   */
+  @Override
+  public synchronized boolean seek(Cell cell) throws IOException {
+    assertForward();
+
+    if (cell == null) {
+      close();
+      return false;
+    }
+
+    return forwardHeap.seek(cell);
+  }
+
+  /**
+   * Move forward on the sub-lists set previously by seek. Assumed forward scanning.
+   *
+   * @param cell seek value (should be non-null)
+   * @return true if there is at least one KV to read, false otherwise
+   */
+  @Override
+  public synchronized boolean reseek(Cell cell) throws IOException {
+    /*
+    * See HBASE-4195 & HBASE-3855 & HBASE-6591 for the background on this implementation.
+    * This code is executed concurrently with flush and puts, without locks.
+    * Two points must be known when working on this code:
+    * 1) It's not possible to use the 'kvTail' and 'snapshot'
+    *  variables, as they are modified during a flush.
+    * 2) The ideal implementation for performance would use the sub skip list
+    *  implicitly pointed by the iterators 'kvsetIt' and
+    *  'snapshotIt'. Unfortunately the Java API does not offer a method to
+    *  get it. So we remember the last keys we iterated to and restore
+    *  the reseeked set to at least that point.
+    *
+    *  TODO: The above comment copied from the original MemStoreScanner
+    */
+    assertForward();
+    return forwardHeap.reseek(cell);
+  }
+
+  /**
+   * MemStoreScanner returns max value as sequence id because it will
+   * always have the latest data among all files.
+   */
+  @Override
+  public synchronized long getSequenceID() {
+    return Long.MAX_VALUE;
+  }
+
+  @Override
+  public synchronized void close() {
+
+    if (forwardHeap != null) {
+      assert ((type == Type.USER_SCAN_FORWARD) ||
+          (type == Type.COMPACT_FORWARD) || (type == Type.UNDEFINED));
+      forwardHeap.close();
+      forwardHeap = null;
+      if (backwardHeap != null) {
+        backwardHeap.close();
+        backwardHeap = null;
+      }
+    } else if (backwardHeap != null) {
+      assert (type == Type.USER_SCAN_BACKWARD);
+      backwardHeap.close();
+      backwardHeap = null;
+    }
+  }
+
+  /**
+   * Set the scanner at the seek key. Assumed backward scanning.
+   *
+   * @param cell seek value
+   * @return false if the key is null or if there is no data
+   */
+  @Override
+  public synchronized boolean backwardSeek(Cell cell) throws IOException {
+    initBackwardHeapIfNeeded(cell, false);
+    return backwardHeap.backwardSeek(cell);
+  }
+
+  /**
+   * Assumed backward scanning.
+   *
+   * @param cell seek value
+   * @return false if the key is null or if there is no data
+   */
+  @Override
+  public synchronized boolean seekToPreviousRow(Cell cell) throws IOException {
+    initBackwardHeapIfNeeded(cell, false);
+    if (backwardHeap.peek() == null) {
+      restartBackwardHeap(cell);
+    }
+    return backwardHeap.seekToPreviousRow(cell);
+  }
+
+  @Override
+  public synchronized boolean seekToLastRow() throws IOException {
+    // TODO: it looks like this is how it should be, however ReversedKeyValueHeap class doesn't
+    // implement seekToLastRow() method :(
+    // however seekToLastRow() was implemented in internal MemStoreScanner
+    // so I wonder whether we need to come with our own workaround, or to update
+    // ReversedKeyValueHeap
+    return initBackwardHeapIfNeeded(KeyValue.LOWESTKEY, true);
+  }
+
+  /**
+   * Check if this memstore may contain the required keys
+   * @return False if the key definitely does not exist in this Memstore
+   */
+  @Override
+  public synchronized boolean shouldUseScanner(Scan scan, Store store, long oldestUnexpiredTS) {
+
+    if (type == Type.COMPACT_FORWARD) {
+      return true;
+    }
+
+    for (SegmentScanner sc : scanners) {
+      if (sc.shouldSeek(scan, oldestUnexpiredTS)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  // debug method
+  @Override
+  public String toString() {
+    StringBuffer buf = new StringBuffer();
+    int i = 1;
+    for (SegmentScanner scanner : scanners) {
+      buf.append("scanner (" + i + ") " + scanner.toString() + " ||| ");
+      i++;
+    }
+    return buf.toString();
+  }
+  /****************** Private methods ******************/
+  /**
+   * Restructure the ended backward heap after rerunning a seekToPreviousRow()
+   * on each scanner
+   * @return false if given Cell does not exist in any scanner
+   */
+  private boolean restartBackwardHeap(Cell cell) throws IOException {
+    boolean res = false;
+    for (SegmentScanner scan : scanners) {
+      res |= scan.seekToPreviousRow(cell);
+    }
+    this.backwardHeap =
+        new ReversedKeyValueHeap(scanners, backwardReferenceToMemStore.getComparator());
+    return res;
+  }
+
+  /**
+   * Checks whether the type of the scan suits the assumption of moving backward
+   */
+  private boolean initBackwardHeapIfNeeded(Cell cell, boolean toLast) throws IOException {
+    boolean res = false;
+    if (toLast && (type != Type.UNDEFINED)) {
+      throw new IllegalStateException(
+          "Wrong usage of initBackwardHeapIfNeeded in parameters. The type is:" + type.toString());
+    }
+    if (type == Type.UNDEFINED) {
+      // In case we started from peek, release the forward heap
+      // and build backward. Set the correct type. Thus this turn
+      // can happen only once
+      if ((backwardHeap == null) && (forwardHeap != null)) {
+        forwardHeap.close();
+        forwardHeap = null;
+        // before building the heap seek for the relevant key on the scanners,
+        // for the heap to be built from the scanners correctly
+        for (SegmentScanner scan : scanners) {
+          if (toLast) {
+            res |= scan.seekToLastRow();
+          } else {
+            res |= scan.backwardSeek(cell);
+          }
+        }
+        this.backwardHeap =
+            new ReversedKeyValueHeap(scanners, backwardReferenceToMemStore.getComparator());
+        type = Type.USER_SCAN_BACKWARD;
+      }
+    }
+
+    if (type == Type.USER_SCAN_FORWARD) {
+      throw new IllegalStateException("Traversing backward with forward scan");
+    }
+    return res;
+  }
+
+  /**
+   * Checks whether the type of the scan suits the assumption of moving forward
+   */
+  private void assertForward() throws IllegalStateException {
+    if (type == Type.UNDEFINED) {
+      type = Type.USER_SCAN_FORWARD;
+    }
+
+    if (type == Type.USER_SCAN_BACKWARD) {
+      throw new IllegalStateException("Traversing forward with backward scan");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/25dfc112/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreSnapshot.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreSnapshot.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreSnapshot.java
index be853c5..28ab693 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreSnapshot.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreSnapshot.java
@@ -34,14 +34,13 @@ public class MemStoreSnapshot {
   private final KeyValueScanner scanner;
   private final boolean tagsPresent;
 
-  public MemStoreSnapshot(long id, int cellsCount, long size, TimeRangeTracker timeRangeTracker,
-      KeyValueScanner scanner, boolean tagsPresent) {
+  public MemStoreSnapshot(long id, ImmutableSegment snapshot) {
     this.id = id;
-    this.cellsCount = cellsCount;
-    this.size = size;
-    this.timeRangeTracker = timeRangeTracker;
-    this.scanner = scanner;
-    this.tagsPresent = tagsPresent;
+    this.cellsCount = snapshot.getCellsCount();
+    this.size = snapshot.getSize();
+    this.timeRangeTracker = snapshot.getTimeRangeTracker();
+    this.scanner = snapshot.getKeyValueScanner();
+    this.tagsPresent = snapshot.isTagsPresent();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/25dfc112/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MutableCellSetSegment.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MutableCellSetSegment.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MutableCellSetSegment.java
new file mode 100644
index 0000000..743416c
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MutableCellSetSegment.java
@@ -0,0 +1,153 @@
+/**
+ *
+ * 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.util.Iterator;
+import java.util.SortedSet;
+
+import org.apache.commons.logging.Log;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * This mutable store segment encapsulates a mutable cell set and its respective memory allocation
+ * buffers (MSLAB).
+ */
+@InterfaceAudience.Private
+final class MutableCellSetSegment extends MutableSegment {
+
+  private volatile CellSet cellSet;
+  private final CellComparator comparator;
+
+  // Instantiate objects only using factory
+  MutableCellSetSegment(CellSet cellSet, MemStoreLAB memStoreLAB, long size,
+      CellComparator comparator) {
+    super(memStoreLAB, size);
+    this.cellSet = cellSet;
+    this.comparator = comparator;
+  }
+
+  @Override
+  public SegmentScanner getSegmentScanner(long readPoint) {
+    return new MutableCellSetSegmentScanner(this, readPoint);
+  }
+
+  @Override
+  public boolean isEmpty() {
+    return getCellSet().isEmpty();
+  }
+
+  @Override
+  public int getCellsCount() {
+    return getCellSet().size();
+  }
+
+  @Override
+  public long add(Cell cell) {
+    boolean succ = getCellSet().add(cell);
+    long s = AbstractMemStore.heapSizeChange(cell, succ);
+    updateMetaInfo(cell, s);
+    // In no tags case this NoTagsKeyValue.getTagsLength() is a cheap call.
+    // When we use ACL CP or Visibility CP which deals with Tags during
+    // mutation, the TagRewriteCell.getTagsLength() is a cheaper call. We do not
+    // parse the byte[] to identify the tags length.
+    if(cell.getTagsLength() > 0) {
+      tagsPresent = true;
+    }
+    return s;
+  }
+
+  @Override
+  public long rollback(Cell cell) {
+    Cell found = get(cell);
+    if (found != null && found.getSequenceId() == cell.getSequenceId()) {
+      long sz = AbstractMemStore.heapSizeChange(cell, true);
+      remove(cell);
+      incSize(-sz);
+      return sz;
+    }
+    return 0;
+  }
+
+  @Override
+  public Cell getFirstAfter(Cell cell) {
+    SortedSet<Cell> snTailSet = tailSet(cell);
+    if (!snTailSet.isEmpty()) {
+      return snTailSet.first();
+    }
+    return null;
+  }
+
+  @Override
+  public void dump(Log log) {
+    for (Cell cell: getCellSet()) {
+      log.debug(cell);
+    }
+  }
+
+  @Override
+  public SortedSet<Cell> tailSet(Cell firstCell) {
+    return getCellSet().tailSet(firstCell);
+  }
+  @Override
+  public CellSet getCellSet() {
+    return cellSet;
+  }
+  @Override
+  public CellComparator getComparator() {
+    return comparator;
+  }
+
+  //*** Methods for MemStoreSegmentsScanner
+  public Cell last() {
+    return getCellSet().last();
+  }
+
+  public Iterator<Cell> iterator() {
+    return getCellSet().iterator();
+  }
+
+  public SortedSet<Cell> headSet(Cell firstKeyOnRow) {
+    return getCellSet().headSet(firstKeyOnRow);
+  }
+
+  public int compare(Cell left, Cell right) {
+    return getComparator().compare(left, right);
+  }
+
+  public int compareRows(Cell left, Cell right) {
+    return getComparator().compareRows(left, right);
+  }
+
+  private Cell get(Cell cell) {
+    return getCellSet().get(cell);
+  }
+
+  private boolean remove(Cell e) {
+    return getCellSet().remove(e);
+  }
+
+  // methods for tests
+  @Override
+  Cell first() {
+    return this.getCellSet().first();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/25dfc112/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MutableCellSetSegmentScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MutableCellSetSegmentScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MutableCellSetSegmentScanner.java
new file mode 100644
index 0000000..17791ff
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MutableCellSetSegmentScanner.java
@@ -0,0 +1,258 @@
+/**
+ * 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.Iterator;
+import java.util.SortedSet;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * A scanner of a single cells segment {@link MutableCellSetSegment}.
+ */
+@InterfaceAudience.Private
+class MutableCellSetSegmentScanner extends SegmentScanner {
+
+  // the observed structure
+  private final MutableCellSetSegment segment;
+  // the highest relevant MVCC
+  private long readPoint;
+  // the current iterator that can be reinitialized by
+  // seek(), backwardSeek(), or reseek()
+  private Iterator<Cell> iter;
+  // the pre-calculated cell to be returned by peek()
+  private Cell current = null;
+  // or next()
+  // A flag represents whether could stop skipping KeyValues for MVCC
+  // if have encountered the next row. Only used for reversed scan
+  private boolean stopSkippingKVsIfNextRow = false;
+  // last iterated KVs by seek (to restore the iterator state after reseek)
+  private Cell last = null;
+
+  public MutableCellSetSegmentScanner(MutableCellSetSegment segment, long readPoint) {
+    super();
+    this.segment = segment;
+    this.readPoint = readPoint;
+    iter = segment.iterator();
+    // the initialization of the current is required for working with heap of SegmentScanners
+    current = getNext();
+    //increase the reference count so the underlying structure will not be de-allocated
+    this.segment.incScannerCount();
+  }
+
+  /**
+   * Look at the next Cell in this scanner, but do not iterate the scanner
+   * @return the currently observed Cell
+   */
+  @Override
+  public Cell peek() {          // sanity check, the current should be always valid
+    if (current!=null && current.getSequenceId() > readPoint) {
+      throw new RuntimeException("current is invalid: read point is "+readPoint+", " +
+          "while current sequence id is " +current.getSequenceId());
+    }
+
+    return current;
+  }
+
+  /**
+   * Return the next Cell in this scanner, iterating the scanner
+   * @return the next Cell or null if end of scanner
+   */
+  @Override
+  public Cell next() throws IOException {
+    Cell oldCurrent = current;
+    current = getNext();                  // update the currently observed Cell
+    return oldCurrent;
+  }
+
+  /**
+   * Seek the scanner at or after the specified Cell.
+   * @param cell seek value
+   * @return true if scanner has values left, false if end of scanner
+   */
+  @Override
+  public boolean seek(Cell cell) throws IOException {
+    if(cell == null) {
+      close();
+      return false;
+    }
+    // restart the iterator from new key
+    iter = segment.tailSet(cell).iterator();
+    // last is going to be reinitialized in the next getNext() call
+    last = null;
+    current = getNext();
+    return (current != null);
+  }
+
+  /**
+   * Reseek the scanner at or after the specified KeyValue.
+   * This method is guaranteed to seek at or after the required key only if the
+   * key comes after the current position of the scanner. Should not be used
+   * to seek to a key which may come before the current position.
+   *
+   * @param cell seek value (should be non-null)
+   * @return true if scanner has values left, false if end of scanner
+   */
+  @Override
+  public boolean reseek(Cell cell) throws IOException {
+
+    /*
+    See HBASE-4195 & HBASE-3855 & HBASE-6591 for the background on this implementation.
+    This code is executed concurrently with flush and puts, without locks.
+    The ideal implementation for performance would use the sub skip list implicitly
+    pointed by the iterator. Unfortunately the Java API does not offer a method to
+    get it. So we remember the last keys we iterated to and restore
+    the reseeked set to at least that point.
+    */
+    iter = segment.tailSet(getHighest(cell, last)).iterator();
+    current = getNext();
+    return (current != null);
+  }
+
+  /**
+   * Seek the scanner at or before the row of specified Cell, it firstly
+   * tries to seek the scanner at or after the specified Cell, return if
+   * 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 Cell
+   * @return true if the scanner is at the valid KeyValue, false if such Cell does not exist
+   */
+  @Override
+  public boolean backwardSeek(Cell key) throws IOException {
+    seek(key);    // seek forward then go backward
+    if (peek() == null || segment.compareRows(peek(), key) > 0) {
+      return seekToPreviousRow(key);
+    }
+    return true;
+  }
+
+  /**
+   * Seek the scanner at the first Cell of the row which is the previous row
+   * of specified key
+   *
+   * @param cell seek value
+   * @return true if the scanner at the first valid Cell of previous row,
+   *     false if not existing such Cell
+   */
+  @Override
+  public boolean seekToPreviousRow(Cell cell) throws IOException {
+    boolean keepSeeking = false;
+    Cell key = cell;
+
+    do {
+      Cell firstKeyOnRow = CellUtil.createFirstOnRow(key);
+      SortedSet<Cell> cellHead = segment.headSet(firstKeyOnRow);
+      Cell lastCellBeforeRow = cellHead.isEmpty() ? null : cellHead.last();
+      if (lastCellBeforeRow == null) {
+        current = null;
+        return false;
+      }
+      Cell firstKeyOnPreviousRow = CellUtil.createFirstOnRow(lastCellBeforeRow);
+      this.stopSkippingKVsIfNextRow = true;
+      seek(firstKeyOnPreviousRow);
+      this.stopSkippingKVsIfNextRow = false;
+      if (peek() == null
+          || segment.getComparator().compareRows(peek(), firstKeyOnPreviousRow) > 0) {
+        keepSeeking = true;
+        key = firstKeyOnPreviousRow;
+        continue;
+      } else {
+        keepSeeking = false;
+      }
+    } while (keepSeeking);
+    return true;
+  }
+
+  /**
+   * Seek the scanner at the first KeyValue of last row
+   *
+   * @return true if scanner has values left, false if the underlying data is empty
+   */
+  @Override
+  public boolean seekToLastRow() throws IOException {
+    Cell higherCell = segment.isEmpty() ? null : segment.last();
+    if (higherCell == null) {
+      return false;
+    }
+
+    Cell firstCellOnLastRow = CellUtil.createFirstOnRow(higherCell);
+
+    if (seek(firstCellOnLastRow)) {
+      return true;
+    } else {
+      return seekToPreviousRow(higherCell);
+    }
+  }
+
+  @Override protected Segment getSegment() {
+    return segment;
+  }
+
+  /********************* Private Methods **********************/
+
+  /**
+   * Private internal method for iterating over the segment,
+   * skipping the cells with irrelevant MVCC
+   */
+  private Cell getNext() {
+    Cell startKV = current;
+    Cell next = null;
+
+    try {
+      while (iter.hasNext()) {
+        next = iter.next();
+        if (next.getSequenceId() <= this.readPoint) {
+          return next;                    // skip irrelevant versions
+        }
+        if (stopSkippingKVsIfNextRow &&   // for backwardSeek() stay in the
+            startKV != null &&        // boundaries of a single row
+            segment.compareRows(next, startKV) > 0) {
+          return null;
+        }
+      } // end of while
+
+      return null; // nothing found
+    } finally {
+      if (next != null) {
+        // in all cases, remember the last KV we iterated to, needed for reseek()
+        last = next;
+      }
+    }
+  }
+
+  /**
+   * Private internal method that returns the higher of the two key values, or null
+   * if they are both null
+   */
+  private Cell getHighest(Cell first, Cell second) {
+    if (first == null && second == null) {
+      return null;
+    }
+    if (first != null && second != null) {
+      int compare = segment.compare(first, second);
+      return (compare > 0 ? first : second);
+    }
+    return (first != null ? first : second);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/25dfc112/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MutableSegment.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MutableSegment.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MutableSegment.java
new file mode 100644
index 0000000..fcaddb0
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MutableSegment.java
@@ -0,0 +1,57 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver;
+
+import java.util.SortedSet;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * An abstraction of a mutable segment in memstore, specifically the active segment.
+ */
+@InterfaceAudience.Private
+public abstract class MutableSegment extends Segment {
+
+  protected MutableSegment(MemStoreLAB memStoreLAB, long size) {
+    super(memStoreLAB, size);
+  }
+
+  /**
+   * Returns a subset of the segment cell set, which starts with the given cell
+   * @param firstCell a cell in the segment
+   * @return a subset of the segment cell set, which starts with the given cell
+   */
+  public abstract SortedSet<Cell> tailSet(Cell firstCell);
+
+  /**
+   * Returns the Cell comparator used by this segment
+   * @return the Cell comparator used by this segment
+   */
+  public abstract CellComparator getComparator();
+
+  //methods for test
+
+  /**
+   * Returns the first cell in the segment
+   * @return the first cell in the segment
+   */
+  abstract Cell first();
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/25dfc112/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Segment.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Segment.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Segment.java
new file mode 100644
index 0000000..7891809
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Segment.java
@@ -0,0 +1,218 @@
+/**
+ *
+ * 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.util.concurrent.atomic.AtomicLong;
+
+import org.apache.commons.logging.Log;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.util.ByteRange;
+
+/**
+ * This is an abstraction of a segment maintained in a memstore, e.g., the active
+ * cell set or its snapshot.
+ *
+ * This abstraction facilitates the management of the compaction pipeline and the shifts of these
+ * segments from active set to snapshot set in the default implementation.
+ */
+@InterfaceAudience.Private
+public abstract class Segment {
+
+  private volatile MemStoreLAB memStoreLAB;
+  private final AtomicLong size;
+  private final TimeRangeTracker timeRangeTracker;
+  protected volatile boolean tagsPresent;
+
+  protected Segment(MemStoreLAB memStoreLAB, long size) {
+    this.memStoreLAB = memStoreLAB;
+    this.size = new AtomicLong(size);
+    this.timeRangeTracker = new TimeRangeTracker();
+    this.tagsPresent = false;
+  }
+
+  protected Segment(Segment segment) {
+    this.memStoreLAB = segment.getMemStoreLAB();
+    this.size = new AtomicLong(segment.getSize());
+    this.timeRangeTracker = segment.getTimeRangeTracker();
+    this.tagsPresent = segment.isTagsPresent();
+  }
+
+  /**
+   * Creates the scanner that is able to scan the concrete segment
+   * @return a scanner for the given read point
+   */
+  public abstract SegmentScanner getSegmentScanner(long readPoint);
+
+  /**
+   * Returns whether the segment has any cells
+   * @return whether the segment has any cells
+   */
+  public abstract boolean isEmpty();
+
+  /**
+   * Returns number of cells in segment
+   * @return number of cells in segment
+   */
+  public abstract int getCellsCount();
+
+  /**
+   * Adds the given cell into the segment
+   * @return the change in the heap size
+   */
+  public abstract long add(Cell cell);
+
+  /**
+   * Removes the given cell from the segment
+   * @return the change in the heap size
+   */
+  public abstract long rollback(Cell cell);
+
+  /**
+   * Returns the first cell in the segment that has equal or greater key than the given cell
+   * @return the first cell in the segment that has equal or greater key than the given cell
+   */
+  public abstract Cell getFirstAfter(Cell cell);
+
+  /**
+   * Returns a set of all cells in the segment
+   * @return a set of all cells in the segment
+   */
+  public abstract CellSet getCellSet();
+
+  /**
+   * Closing a segment before it is being discarded
+   */
+  public void close() {
+    MemStoreLAB mslab = getMemStoreLAB();
+    if(mslab != null) {
+      mslab.close();
+    }
+    // do not set MSLab to null as scanners may still be reading the data here and need to decrease
+    // the counter when they finish
+  }
+
+  /**
+   * If the segment has a memory allocator the cell is being cloned to this space, and returned;
+   * otherwise the given cell is returned
+   * @return either the given cell or its clone
+   */
+  public Cell maybeCloneWithAllocator(Cell cell) {
+    if (getMemStoreLAB() == null) {
+      return cell;
+    }
+
+    int len = KeyValueUtil.length(cell);
+    ByteRange alloc = getMemStoreLAB().allocateBytes(len);
+    if (alloc == null) {
+      // The allocation was too large, allocator decided
+      // not to do anything with it.
+      return cell;
+    }
+    assert alloc.getBytes() != null;
+    KeyValueUtil.appendToByteArray(cell, alloc.getBytes(), alloc.getOffset());
+    KeyValue newKv = new KeyValue(alloc.getBytes(), alloc.getOffset(), len);
+    newKv.setSequenceId(cell.getSequenceId());
+    return newKv;
+  }
+
+  public boolean shouldSeek(Scan scan, long oldestUnexpiredTS) {
+    return (getTimeRangeTracker().includesTimeRange(scan.getTimeRange())
+        && (getTimeRangeTracker().getMaximumTimestamp() >=
+        oldestUnexpiredTS));
+  }
+
+  public long getMinTimestamp() {
+    return getTimeRangeTracker().getMinimumTimestamp();
+  }
+
+  public boolean isTagsPresent() {
+    return tagsPresent;
+  }
+
+  public void incScannerCount() {
+    if(getMemStoreLAB() != null) {
+      getMemStoreLAB().incScannerCount();
+    }
+  }
+
+  public void decScannerCount() {
+    if(getMemStoreLAB() != null) {
+      getMemStoreLAB().decScannerCount();
+    }
+  }
+
+  /**
+   * Setting the heap size of the segment - used to account for different class overheads
+   * @return this object
+   */
+
+  public Segment setSize(long size) {
+    this.size.set(size);
+    return this;
+  }
+
+  /**
+   * Returns the heap size of the segment
+   * @return the heap size of the segment
+   */
+  public long getSize() {
+    return size.get();
+  }
+
+  /**
+   * Increases the heap size counter of the segment by the given delta
+   */
+  public void incSize(long delta) {
+    size.addAndGet(delta);
+  }
+
+  public TimeRangeTracker getTimeRangeTracker() {
+    return timeRangeTracker;
+  }
+
+  protected void updateMetaInfo(Cell toAdd, long s) {
+    getTimeRangeTracker().includeTimestamp(toAdd);
+    size.addAndGet(s);
+  }
+
+  private MemStoreLAB getMemStoreLAB() {
+    return memStoreLAB;
+  }
+
+  // Debug methods
+  /**
+   * Dumps all cells of the segment into the given log
+   */
+  public abstract void dump(Log log);
+
+  @Override
+  public String toString() {
+    String res = "Store segment of type "+this.getClass().getName()+"; ";
+    res += "isEmpty "+(isEmpty()?"yes":"no")+"; ";
+    res += "cellCount "+getCellsCount()+"; ";
+    res += "size "+getSize()+"; ";
+    res += "Min ts "+getMinTimestamp()+"; ";
+    return res;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/25dfc112/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SegmentFactory.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SegmentFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SegmentFactory.java
new file mode 100644
index 0000000..ccb11df
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SegmentFactory.java
@@ -0,0 +1,89 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.util.ReflectionUtils;
+
+/**
+ * A singleton store segment factory.
+ * Generate concrete store segments.
+ */
+@InterfaceAudience.Private
+public final class SegmentFactory {
+
+  static final String USEMSLAB_KEY = "hbase.hregion.memstore.mslab.enabled";
+  static final boolean USEMSLAB_DEFAULT = true;
+  static final String MSLAB_CLASS_NAME = "hbase.regionserver.mslab.class";
+
+  private SegmentFactory() {}
+  private static SegmentFactory instance = new SegmentFactory();
+  public static SegmentFactory instance() {
+    return instance;
+  }
+
+  public ImmutableSegment createImmutableSegment(final Configuration conf,
+      final CellComparator comparator, long size) {
+    MemStoreLAB memStoreLAB = getMemStoreLAB(conf);
+    MutableSegment segment = generateMutableSegment(conf, comparator, memStoreLAB, size);
+    return createImmutableSegment(conf, segment);
+  }
+
+  public ImmutableSegment createImmutableSegment(CellComparator comparator,
+      long size) {
+    MutableSegment segment = generateMutableSegment(null, comparator, null, size);
+    return createImmutableSegment(null, segment);
+  }
+
+  public ImmutableSegment createImmutableSegment(final Configuration conf, MutableSegment segment) {
+    return generateImmutableSegment(conf, segment);
+  }
+  public MutableSegment createMutableSegment(final Configuration conf,
+      CellComparator comparator, long size) {
+    MemStoreLAB memStoreLAB = getMemStoreLAB(conf);
+    return generateMutableSegment(conf, comparator, memStoreLAB, size);
+  }
+
+  //****** private methods to instantiate concrete store segments **********//
+
+  private ImmutableSegment generateImmutableSegment(final Configuration conf,
+      MutableSegment segment) {
+    // TBD use configuration to set type of segment
+    return new ImmutableSegmentAdapter(segment);
+  }
+  private MutableSegment generateMutableSegment(
+      final Configuration conf, CellComparator comparator, MemStoreLAB memStoreLAB, long size) {
+    // TBD use configuration to set type of segment
+    CellSet set = new CellSet(comparator);
+    return new MutableCellSetSegment(set, memStoreLAB, size, comparator);
+  }
+
+  private MemStoreLAB getMemStoreLAB(Configuration conf) {
+    MemStoreLAB memStoreLAB = null;
+    if (conf.getBoolean(USEMSLAB_KEY, USEMSLAB_DEFAULT)) {
+      String className = conf.get(MSLAB_CLASS_NAME, HeapMemStoreLAB.class.getName());
+      memStoreLAB = ReflectionUtils.instantiateWithCustomCtor(className,
+          new Class[] { Configuration.class }, new Object[] { conf });
+    }
+    return memStoreLAB;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/25dfc112/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SegmentScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SegmentScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SegmentScanner.java
new file mode 100644
index 0000000..8852d5c
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SegmentScanner.java
@@ -0,0 +1,152 @@
+/**
+ *
+ * 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 org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Scan;
+
+/**
+ * An abstraction for store segment scanner.
+ */
+@InterfaceAudience.Private
+public abstract class SegmentScanner implements KeyValueScanner {
+
+  private long sequenceID = Long.MAX_VALUE;
+
+  protected abstract Segment getSegment();
+
+  /**
+   * Get the sequence id associated with this KeyValueScanner. This is required
+   * for comparing multiple files (or memstore segments) scanners to find out
+   * which one has the latest data.
+   *
+   */
+  @Override
+  public long getSequenceID() {
+    return sequenceID;
+  }
+
+  /**
+   * Close the KeyValue scanner.
+   */
+  @Override
+  public void close() {
+    getSegment().decScannerCount();
+  }
+
+  /**
+   * This functionality should be resolved in the higher level which is
+   * MemStoreScanner, currently returns true as default. Doesn't throw
+   * IllegalStateException in order not to change the signature of the
+   * overridden method
+   */
+  @Override
+  public boolean shouldUseScanner(Scan scan, Store store, long oldestUnexpiredTS) {
+    return true;
+  }
+  /**
+   * This scanner is working solely on the in-memory MemStore therefore this
+   * interface is not relevant.
+   */
+  @Override
+  public boolean requestSeek(Cell c, boolean forward, boolean useBloom)
+      throws IOException {
+
+    throw new IllegalStateException(
+        "requestSeek cannot be called on MutableCellSetSegmentScanner");
+  }
+
+  /**
+   * This scanner is working solely on the in-memory MemStore and doesn't work on
+   * store files, MutableCellSetSegmentScanner always does the seek,
+   * therefore always returning true.
+   */
+  @Override
+  public boolean realSeekDone() {
+    return true;
+  }
+
+  /**
+   * This function should be never called on scanners that always do real seek operations (i.e. most
+   * of the scanners and also this one). The easiest way to achieve this is to call
+   * {@link #realSeekDone()} first.
+   */
+  @Override
+  public void enforceSeek() throws IOException {
+    throw new IllegalStateException(
+        "enforceSeek cannot be called on MutableCellSetSegmentScanner");
+  }
+
+  /**
+   * @return true if this is a file scanner. Otherwise a memory scanner is assumed.
+   */
+  @Override
+  public boolean isFileScanner() {
+    return false;
+  }
+
+  /**
+   * @return the next key in the index (the key to seek to the next block)
+   *     if known, or null otherwise
+   *     Not relevant for in-memory scanner
+   */
+  @Override
+  public Cell getNextIndexedKey() {
+    return null;
+  }
+
+  /**
+   * Called after a batch of rows scanned (RPC) and set to be returned to client. Any in between
+   * cleanup can be done here. Nothing to be done for MutableCellSetSegmentScanner.
+   */
+  @Override
+  public void shipped() throws IOException {
+    // do nothing
+  }
+
+  /**
+   * Set the sequence id of the scanner.
+   * This is used to determine an order between memory segment scanners.
+   * @param x a unique sequence id
+   */
+  public void setSequenceID(long x) {
+    sequenceID = x;
+  }
+
+  /**
+   * Returns whether the given scan should seek in this segment
+   * @return whether the given scan should seek in this segment
+   */
+  public boolean shouldSeek(Scan scan, long oldestUnexpiredTS) {
+    return getSegment().shouldSeek(scan,oldestUnexpiredTS);
+  }
+
+  //debug method
+  @Override
+  public String toString() {
+    String res = "Store segment scanner of type "+this.getClass().getName()+"; ";
+    res += "sequence id "+getSequenceID()+"; ";
+    res += getSegment().toString();
+    return res;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/25dfc112/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 34ba1fa..f4f25dd 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
@@ -21,8 +21,8 @@ package org.apache.hadoop.hbase.regionserver;
 import java.io.IOException;
 import java.util.List;
 
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 
 /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/25dfc112/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java
index 4f30960..5c79d72 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java
@@ -19,22 +19,6 @@
 
 package org.apache.hadoop.hbase.io;
 
-import java.io.IOException;
-import java.lang.management.ManagementFactory;
-import java.lang.management.RuntimeMXBean;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Map;
-import java.util.TreeMap;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentSkipListMap;
-import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.concurrent.CopyOnWriteArraySet;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.KeyValue;
@@ -42,9 +26,9 @@ import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.io.hfile.BlockCacheKey;
-import org.apache.hadoop.hbase.io.hfile.LruCachedBlock;
 import org.apache.hadoop.hbase.io.hfile.LruBlockCache;
-import org.apache.hadoop.hbase.regionserver.CellSkipListSet;
+import org.apache.hadoop.hbase.io.hfile.LruCachedBlock;
+import org.apache.hadoop.hbase.regionserver.CellSet;
 import org.apache.hadoop.hbase.regionserver.DefaultMemStore;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HStore;
@@ -56,6 +40,22 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import java.io.IOException;
+import java.lang.management.ManagementFactory;
+import java.lang.management.RuntimeMXBean;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.CopyOnWriteArraySet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
 import static org.junit.Assert.assertEquals;
 
 /**
@@ -237,8 +237,8 @@ public class TestHeapSize  {
       assertEquals(expected, actual);
     }
 
-    // CellSkipListSet
-    cl = CellSkipListSet.class;
+    // CellSet
+    cl = CellSet.class;
     expected = ClassSize.estimateBase(cl, false);
     actual = ClassSize.CELL_SKIPLIST_SET;
     if (expected != actual) {
@@ -305,15 +305,16 @@ public class TestHeapSize  {
     // DefaultMemStore Deep Overhead
     actual = DefaultMemStore.DEEP_OVERHEAD;
     expected = ClassSize.estimateBase(cl, false);
-    expected += ClassSize.estimateBase(AtomicLong.class, false);
-    expected += (2 * ClassSize.estimateBase(CellSkipListSet.class, false));
+    expected += (2 * ClassSize.estimateBase(AtomicLong.class, false));
+    expected += (2 * ClassSize.estimateBase(CellSet.class, false));
     expected += (2 * ClassSize.estimateBase(ConcurrentSkipListMap.class, false));
     expected += (2 * ClassSize.estimateBase(TimeRangeTracker.class, false));
     if(expected != actual) {
       ClassSize.estimateBase(cl, true);
       ClassSize.estimateBase(AtomicLong.class, true);
-      ClassSize.estimateBase(CellSkipListSet.class, true);
-      ClassSize.estimateBase(CellSkipListSet.class, true);
+      ClassSize.estimateBase(AtomicLong.class, true);
+      ClassSize.estimateBase(CellSet.class, true);
+      ClassSize.estimateBase(CellSet.class, true);
       ClassSize.estimateBase(ConcurrentSkipListMap.class, true);
       ClassSize.estimateBase(ConcurrentSkipListMap.class, true);
       ClassSize.estimateBase(TimeRangeTracker.class, true);

http://git-wip-us.apache.org/repos/asf/hbase/blob/25dfc112/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellSkipListSet.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellSkipListSet.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellSkipListSet.java
index 684839d..e0cc39f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellSkipListSet.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellSkipListSet.java
@@ -18,11 +18,7 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
-import java.util.Iterator;
-import java.util.SortedSet;
-
 import junit.framework.TestCase;
-
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
@@ -32,10 +28,13 @@ import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.experimental.categories.Category;
 
+import java.util.Iterator;
+import java.util.SortedSet;
+
 @Category({RegionServerTests.class, SmallTests.class})
 public class TestCellSkipListSet extends TestCase {
-  private final CellSkipListSet csls =
-    new CellSkipListSet(CellComparator.COMPARATOR);
+  private final CellSet csls =
+    new CellSet(CellComparator.COMPARATOR);
 
   protected void setUp() throws Exception {
     super.setUp();
@@ -163,4 +162,4 @@ public class TestCellSkipListSet extends TestCase {
     assertTrue(Bytes.equals(head.first().getValueArray(), head.first().getValueOffset(),
       head.first().getValueLength(), value2, 0, value2.length));
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/25dfc112/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
index ec70740..5e6007d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
@@ -18,17 +18,10 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
-import java.io.IOException;
-import java.lang.management.ManagementFactory;
-import java.lang.management.MemoryMXBean;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
-
+import com.google.common.base.Joiner;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
 import junit.framework.TestCase;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -57,12 +50,14 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.wal.WALFactory;
 import org.junit.experimental.categories.Category;
 
-import com.google.common.base.Joiner;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
-
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
+import java.io.IOException;
+import java.lang.management.ManagementFactory;
+import java.lang.management.MemoryMXBean;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
 
 /** memstore test case */
 @Category({RegionServerTests.class, MediumTests.class})
@@ -89,11 +84,9 @@ public class TestDefaultMemStore extends TestCase {
     byte [] other = Bytes.toBytes("somethingelse");
     KeyValue samekey = new KeyValue(bytes, bytes, bytes, other);
     this.memstore.add(samekey);
-    Cell found = this.memstore.cellSet.first();
-    assertEquals(1, this.memstore.cellSet.size());
-    assertTrue(
-      Bytes.toString(found.getValueArray(), found.getValueOffset(), found.getValueLength()),
-      CellUtil.matchingValue(samekey, found));
+    Cell found = this.memstore.getActive().first();
+    assertEquals(1, this.memstore.getActive().getCellsCount());
+    assertTrue(Bytes.toString(found.getValueArray()), CellUtil.matchingValue(samekey, found));
   }
 
   /**
@@ -108,7 +101,7 @@ public class TestDefaultMemStore extends TestCase {
     Configuration conf = HBaseConfiguration.create();
     ScanInfo scanInfo =
         new ScanInfo(conf, null, 0, 1, HConstants.LATEST_TIMESTAMP, KeepDeletedCells.FALSE, 0,
-            this.memstore.comparator);
+            this.memstore.getComparator());
     ScanType scanType = ScanType.USER_SCAN;
     StoreScanner s = new StoreScanner(scan, scanInfo, scanType, null, memstorescanners);
     int count = 0;
@@ -476,7 +469,7 @@ public class TestDefaultMemStore extends TestCase {
     for (int i = 0; i < snapshotCount; i++) {
       addRows(this.memstore);
       runSnapshot(this.memstore);
-      assertEquals("History not being cleared", 0, this.memstore.snapshot.size());
+      assertEquals("History not being cleared", 0, this.memstore.getSnapshot().getCellsCount());
     }
   }
 
@@ -497,7 +490,7 @@ public class TestDefaultMemStore extends TestCase {
     m.add(key2);
 
     assertTrue("Expected memstore to hold 3 values, actually has " +
-        m.cellSet.size(), m.cellSet.size() == 3);
+        m.getActive().getCellsCount(), m.getActive().getCellsCount() == 3);
   }
 
   //////////////////////////////////////////////////////////////////////////////
@@ -529,7 +522,7 @@ public class TestDefaultMemStore extends TestCase {
     Configuration conf = HBaseConfiguration.create();
     for (int startRowId = 0; startRowId < ROW_COUNT; startRowId++) {
       ScanInfo scanInfo = new ScanInfo(conf, FAMILY, 0, 1, Integer.MAX_VALUE,
-        KeepDeletedCells.FALSE, 0, this.memstore.comparator);
+        KeepDeletedCells.FALSE, 0, this.memstore.getComparator());
       ScanType scanType = ScanType.USER_SCAN;
       InternalScanner scanner = new StoreScanner(new Scan(
           Bytes.toBytes(startRowId)), scanInfo, scanType, null,
@@ -570,12 +563,12 @@ public class TestDefaultMemStore extends TestCase {
     memstore.add(new KeyValue(row, fam ,qf3, val));
     //Creating a snapshot
     memstore.snapshot();
-    assertEquals(3, memstore.snapshot.size());
+    assertEquals(3, memstore.getSnapshot().getCellsCount());
     //Adding value to "new" memstore
-    assertEquals(0, memstore.cellSet.size());
+    assertEquals(0, memstore.getActive().getCellsCount());
     memstore.add(new KeyValue(row, fam ,qf4, val));
     memstore.add(new KeyValue(row, fam ,qf5, val));
-    assertEquals(2, memstore.cellSet.size());
+    assertEquals(2, memstore.getActive().getCellsCount());
   }
 
   //////////////////////////////////////////////////////////////////////////////
@@ -597,7 +590,7 @@ public class TestDefaultMemStore extends TestCase {
     memstore.add(put2);
     memstore.add(put3);
 
-    assertEquals(3, memstore.cellSet.size());
+    assertEquals(3, memstore.getActive().getCellsCount());
 
     KeyValue del2 = new KeyValue(row, fam, qf1, ts2, KeyValue.Type.Delete, val);
     memstore.delete(del2);
@@ -608,9 +601,9 @@ public class TestDefaultMemStore extends TestCase {
     expected.add(put2);
     expected.add(put1);
 
-    assertEquals(4, memstore.cellSet.size());
+    assertEquals(4, memstore.getActive().getCellsCount());
     int i = 0;
-    for(Cell cell : memstore.cellSet) {
+    for(Cell cell : memstore.getActive().getCellSet()) {
       assertEquals(expected.get(i++), cell);
     }
   }
@@ -631,7 +624,7 @@ public class TestDefaultMemStore extends TestCase {
     memstore.add(put2);
     memstore.add(put3);
 
-    assertEquals(3, memstore.cellSet.size());
+    assertEquals(3, memstore.getActive().getCellsCount());
 
     KeyValue del2 =
       new KeyValue(row, fam, qf1, ts2, KeyValue.Type.DeleteColumn, val);
@@ -644,9 +637,9 @@ public class TestDefaultMemStore extends TestCase {
     expected.add(put1);
 
 
-    assertEquals(4, memstore.cellSet.size());
+    assertEquals(4, memstore.getActive().getCellsCount());
     int i = 0;
-    for (Cell cell: memstore.cellSet) {
+    for (Cell cell: memstore.getActive().getCellSet()) {
       assertEquals(expected.get(i++), cell);
     }
   }
@@ -684,9 +677,9 @@ public class TestDefaultMemStore extends TestCase {
 
 
 
-    assertEquals(5, memstore.cellSet.size());
+    assertEquals(5, memstore.getActive().getCellsCount());
     int i = 0;
-    for (Cell cell: memstore.cellSet) {
+    for (Cell cell: memstore.getActive().getCellSet()) {
       assertEquals(expected.get(i++), cell);
     }
   }
@@ -700,8 +693,8 @@ public class TestDefaultMemStore extends TestCase {
     memstore.add(new KeyValue(row, fam, qf, ts, val));
     KeyValue delete = new KeyValue(row, fam, qf, ts, KeyValue.Type.Delete, val);
     memstore.delete(delete);
-    assertEquals(2, memstore.cellSet.size());
-    assertEquals(delete, memstore.cellSet.first());
+    assertEquals(2, memstore.getActive().getCellsCount());
+    assertEquals(delete, memstore.getActive().first());
   }
 
   public void testRetainsDeleteVersion() throws IOException {
@@ -713,8 +706,8 @@ public class TestDefaultMemStore extends TestCase {
         "row1", "fam", "a", 100, KeyValue.Type.Delete, "dont-care");
     memstore.delete(delete);
 
-    assertEquals(2, memstore.cellSet.size());
-    assertEquals(delete, memstore.cellSet.first());
+    assertEquals(2, memstore.getActive().getCellsCount());
+    assertEquals(delete, memstore.getActive().first());
   }
   public void testRetainsDeleteColumn() throws IOException {
     // add a put to memstore
@@ -725,8 +718,8 @@ public class TestDefaultMemStore extends TestCase {
         KeyValue.Type.DeleteColumn, "dont-care");
     memstore.delete(delete);
 
-    assertEquals(2, memstore.cellSet.size());
-    assertEquals(delete, memstore.cellSet.first());
+    assertEquals(2, memstore.getActive().getCellsCount());
+    assertEquals(delete, memstore.getActive().first());
   }
   public void testRetainsDeleteFamily() throws IOException {
     // add a put to memstore
@@ -737,43 +730,8 @@ public class TestDefaultMemStore extends TestCase {
         KeyValue.Type.DeleteFamily, "dont-care");
     memstore.delete(delete);
 
-    assertEquals(2, memstore.cellSet.size());
-    assertEquals(delete, memstore.cellSet.first());
-  }
-
-  ////////////////////////////////////
-  //Test for timestamps
-  ////////////////////////////////////
-
-  /**
-   * Test to ensure correctness when using Memstore with multiple timestamps
-   */
-  public void testMultipleTimestamps() throws Exception {
-    long[] timestamps = new long[] {20,10,5,1};
-    Scan scan = new Scan();
-
-    for (long timestamp: timestamps)
-      addRows(memstore,timestamp);
-
-    byte[] fam = Bytes.toBytes("fam");
-    HColumnDescriptor hcd = mock(HColumnDescriptor.class);
-    when(hcd.getName()).thenReturn(fam);
-    Store store = mock(Store.class);
-    when(store.getFamily()).thenReturn(hcd);
-    scan.setColumnFamilyTimeRange(fam, 0, 2);
-    assertTrue(memstore.shouldSeek(scan, store, Long.MIN_VALUE));
-
-    scan.setColumnFamilyTimeRange(fam, 20, 82);
-    assertTrue(memstore.shouldSeek(scan, store, Long.MIN_VALUE));
-
-    scan.setColumnFamilyTimeRange(fam, 10, 20);
-    assertTrue(memstore.shouldSeek(scan, store, Long.MIN_VALUE));
-
-    scan.setColumnFamilyTimeRange(fam, 8, 12);
-    assertTrue(memstore.shouldSeek(scan, store, Long.MIN_VALUE));
-
-    scan.setColumnFamilyTimeRange(fam, 28, 42);
-    assertTrue(!memstore.shouldSeek(scan, store, Long.MIN_VALUE));
+    assertEquals(2, memstore.getActive().getCellsCount());
+    assertEquals(delete, memstore.getActive().first());
   }
 
   ////////////////////////////////////
@@ -795,7 +753,7 @@ public class TestDefaultMemStore extends TestCase {
    */
   public void testUpsertMSLAB() throws Exception {
     Configuration conf = HBaseConfiguration.create();
-    conf.setBoolean(DefaultMemStore.USEMSLAB_KEY, true);
+    conf.setBoolean(SegmentFactory.USEMSLAB_KEY, true);
     memstore = new DefaultMemStore(conf, CellComparator.COMPARATOR);
 
     int ROW_SIZE = 2048;
@@ -838,7 +796,7 @@ public class TestDefaultMemStore extends TestCase {
   public void testUpsertMemstoreSize() throws Exception {
     Configuration conf = HBaseConfiguration.create();
     memstore = new DefaultMemStore(conf, CellComparator.COMPARATOR);
-    long oldSize = memstore.size.get();
+    long oldSize = memstore.size();
 
     List<Cell> l = new ArrayList<Cell>();
     KeyValue kv1 = KeyValueTestUtil.create("r", "f", "q", 100, "v");
@@ -849,18 +807,18 @@ public class TestDefaultMemStore extends TestCase {
     l.add(kv1); l.add(kv2); l.add(kv3);
 
     this.memstore.upsert(l, 2);// readpoint is 2
-    long newSize = this.memstore.size.get();
+    long newSize = this.memstore.size();
     assert(newSize > oldSize);
     //The kv1 should be removed.
-    assert(memstore.cellSet.size() == 2);
+    assert(memstore.getActive().getCellsCount() == 2);
 
     KeyValue kv4 = KeyValueTestUtil.create("r", "f", "q", 104, "v");
     kv4.setSequenceId(1);
     l.clear(); l.add(kv4);
     this.memstore.upsert(l, 3);
-    assertEquals(newSize, this.memstore.size.get());
+    assertEquals(newSize, this.memstore.size());
     //The kv2 should be removed.
-    assert(memstore.cellSet.size() == 2);
+    assert(memstore.getActive().getCellsCount() == 2);
     //this.memstore = null;
   }
 
@@ -1021,10 +979,11 @@ public class TestDefaultMemStore extends TestCase {
 
   private long runSnapshot(final DefaultMemStore hmc) throws UnexpectedStateException {
     // Save off old state.
-    int oldHistorySize = hmc.snapshot.size();
+    int oldHistorySize = hmc.getSnapshot().getCellsCount();
     MemStoreSnapshot snapshot = hmc.snapshot();
     // Make some assertions about what just happened.
-    assertTrue("History size has not increased", oldHistorySize < hmc.snapshot.size());
+    assertTrue("History size has not increased", oldHistorySize < hmc.getSnapshot().getCellsCount
+        ());
     long t = memstore.timeOfOldestEdit();
     assertTrue("Time of oldest edit is not Long.MAX_VALUE", t == Long.MAX_VALUE);
     hmc.clearSnapshot(snapshot.getId());

http://git-wip-us.apache.org/repos/asf/hbase/blob/25dfc112/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
index 385048c..b237490 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
@@ -18,20 +18,6 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
-import java.io.IOException;
-import java.security.Key;
-import java.security.SecureRandom;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Date;
-import java.util.Iterator;
-import java.util.List;
-import java.util.NavigableSet;
-import java.util.concurrent.ConcurrentSkipListSet;
-
-import javax.crypto.spec.SecretKeySpec;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -74,6 +60,19 @@ import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 import org.mockito.Mockito;
 
+import javax.crypto.spec.SecretKeySpec;
+import java.io.IOException;
+import java.security.Key;
+import java.security.SecureRandom;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NavigableSet;
+import java.util.concurrent.ConcurrentSkipListSet;
+
 @Category(MediumTests.class)
 public class TestHMobStore {
   public static final Log LOG = LogFactory.getLog(TestHMobStore.class);
@@ -468,7 +467,7 @@ public class TestHMobStore {
     this.store.snapshot();
     flushStore(store, id++);
     Assert.assertEquals(storeFilesSize, this.store.getStorefiles().size());
-    Assert.assertEquals(0, ((DefaultMemStore)this.store.memstore).cellSet.size());
+    Assert.assertEquals(0, ((AbstractMemStore)this.store.memstore).getActive().getCellsCount());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/25dfc112/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
index 7add8a9..a5574d3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
@@ -18,54 +18,10 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
-
-import static org.apache.hadoop.hbase.HBaseTestingUtility.COLUMNS;
-import static org.apache.hadoop.hbase.HBaseTestingUtility.FIRST_CHAR;
-import static org.apache.hadoop.hbase.HBaseTestingUtility.LAST_CHAR;
-import static org.apache.hadoop.hbase.HBaseTestingUtility.START_KEY;
-import static org.apache.hadoop.hbase.HBaseTestingUtility.fam1;
-import static org.apache.hadoop.hbase.HBaseTestingUtility.fam2;
-import static org.apache.hadoop.hbase.HBaseTestingUtility.fam3;
-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.junit.Assert.fail;
-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.never;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.security.PrivilegedExceptionAction;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.NavigableMap;
-import java.util.TreeMap;
-import java.util.UUID;
-import java.util.concurrent.Callable;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.protobuf.ByteString;
 import org.apache.commons.lang.RandomStringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -176,10 +132,52 @@ import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.protobuf.ByteString;
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hbase.HBaseTestingUtility.COLUMNS;
+import static org.apache.hadoop.hbase.HBaseTestingUtility.FIRST_CHAR;
+import static org.apache.hadoop.hbase.HBaseTestingUtility.LAST_CHAR;
+import static org.apache.hadoop.hbase.HBaseTestingUtility.START_KEY;
+import static org.apache.hadoop.hbase.HBaseTestingUtility.fam1;
+import static org.apache.hadoop.hbase.HBaseTestingUtility.fam2;
+import static org.apache.hadoop.hbase.HBaseTestingUtility.fam3;
+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.junit.Assert.fail;
+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.never;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
 
 /**
  * Basic stand-alone testing of HRegion.  No clusters!
@@ -302,8 +300,6 @@ public class TestHRegion {
     HBaseTestingUtility.closeRegionAndWAL(region);
   }
 
-
-
   /*
    * This test is for verifying memstore snapshot size is correctly updated in case of rollback
    * See HBASE-10845
@@ -332,7 +328,7 @@ public class TestHRegion {
     Path rootDir = new Path(dir + "testMemstoreSnapshotSize");
     MyFaultyFSLog faultyLog = new MyFaultyFSLog(fs, rootDir, "testMemstoreSnapshotSize", CONF);
     HRegion region = initHRegion(tableName, null, null, name.getMethodName(),
-        CONF, false, Durability.SYNC_WAL, faultyLog, COLUMN_FAMILY_BYTES);
+      CONF, false, Durability.SYNC_WAL, faultyLog, COLUMN_FAMILY_BYTES);
 
     Store store = region.getStore(COLUMN_FAMILY_BYTES);
     // Get some random bytes.
@@ -1289,7 +1285,8 @@ public class TestHRegion {
     private final AtomicInteger count;
     private Exception e;
 
-    GetTillDoneOrException(final int i, final byte[] r, final AtomicBoolean d, final AtomicInteger c) {
+    GetTillDoneOrException(final int i, final byte[] r, final AtomicBoolean d,
+        final AtomicInteger c) {
       super("getter." + i);
       this.g = new Get(r);
       this.done = d;
@@ -2452,10 +2449,10 @@ public class TestHRegion {
       // This is kinda hacky, but better than nothing...
       long now = System.currentTimeMillis();
       DefaultMemStore memstore = (DefaultMemStore) ((HStore) region.getStore(fam1)).memstore;
-      Cell firstCell = memstore.cellSet.first();
+      Cell firstCell = memstore.getActive().first();
       assertTrue(firstCell.getTimestamp() <= now);
       now = firstCell.getTimestamp();
-      for (Cell cell : memstore.cellSet) {
+      for (Cell cell : memstore.getActive().getCellSet()) {
         assertTrue(cell.getTimestamp() <= now);
         now = cell.getTimestamp();
       }
@@ -2782,7 +2779,8 @@ public class TestHRegion {
       } catch (NotServingRegionException e) {
         // this is the correct exception that is expected
       } catch (IOException e) {
-        fail("Got wrong type of exception - should be a NotServingRegionException, but was an IOException: "
+        fail("Got wrong type of exception - should be a NotServingRegionException, " +
+            "but was an IOException: "
             + e.getMessage());
       }
     } finally {
@@ -2980,7 +2978,8 @@ public class TestHRegion {
   }
 
   @Test
-  public void testScanner_ExplicitColumns_FromMemStoreAndFiles_EnforceVersions() throws IOException {
+  public void testScanner_ExplicitColumns_FromMemStoreAndFiles_EnforceVersions() throws
+      IOException {
     byte[] row1 = Bytes.toBytes("row1");
     byte[] fam1 = Bytes.toBytes("fam1");
     byte[][] families = { fam1 };
@@ -4978,7 +4977,8 @@ public class TestHRegion {
       // move the file of the primary region to the archive, simulating a compaction
       Collection<StoreFile> storeFiles = primaryRegion.getStore(families[0]).getStorefiles();
       primaryRegion.getRegionFileSystem().removeStoreFiles(Bytes.toString(families[0]), storeFiles);
-      Collection<StoreFileInfo> storeFileInfos = primaryRegion.getRegionFileSystem().getStoreFiles(families[0]);
+      Collection<StoreFileInfo> storeFileInfos = primaryRegion.getRegionFileSystem()
+          .getStoreFiles(families[0]);
       Assert.assertTrue(storeFileInfos == null || storeFileInfos.size() == 0);
 
       verifyData(secondaryRegion, 0, 1000, cq, families);
@@ -4992,7 +4992,8 @@ public class TestHRegion {
     }
   }
 
-  private void putData(int startRow, int numRows, byte[] qf, byte[]... families) throws IOException {
+  private void putData(int startRow, int numRows, byte[] qf, byte[]... families) throws
+      IOException {
     putData(this.region, startRow, numRows, qf, families);
   }
 
@@ -5085,7 +5086,6 @@ public class TestHRegion {
 
   /**
    * Test that we get the expected flush results back
-   * @throws IOException
    */
   @Test
   public void testFlushResult() throws IOException {
@@ -5138,11 +5138,6 @@ public class TestHRegion {
   }
 
   /**
-   * @param tableName
-   * @param callingMethod
-   * @param conf
-   * @param families
-   * @throws IOException
    * @return A region on which you must call
    *         {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} when done.
    */
@@ -5152,12 +5147,6 @@ public class TestHRegion {
   }
 
   /**
-   * @param tableName
-   * @param callingMethod
-   * @param conf
-   * @param isReadOnly
-   * @param families
-   * @throws IOException
    * @return A region on which you must call
    *         {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} when done.
    */
@@ -5177,14 +5166,6 @@ public class TestHRegion {
   }
 
   /**
-   * @param tableName
-   * @param startKey
-   * @param stopKey
-   * @param callingMethod
-   * @param conf
-   * @param isReadOnly
-   * @param families
-   * @throws IOException
    * @return A region on which you must call
    *         {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} when done.
    */
@@ -5676,7 +5657,8 @@ public class TestHRegion {
       currRow.clear();
       hasNext = scanner.next(currRow);
       assertEquals(2, currRow.size());
-      assertTrue(Bytes.equals(currRow.get(0).getRowArray(), currRow.get(0).getRowOffset(), currRow.get(0).getRowLength(), row4, 0,
+      assertTrue(Bytes.equals(currRow.get(0).getRowArray(), currRow.get(0).getRowOffset(),
+          currRow.get(0).getRowLength(), row4, 0,
         row4.length));
       assertTrue(hasNext);
       // 2. scan out "row3" (2 kv)
@@ -6088,7 +6070,7 @@ public class TestHRegion {
   public void testOpenRegionWrittenToWALForLogReplay() throws Exception {
     // similar to the above test but with distributed log replay
     final ServerName serverName = ServerName.valueOf("testOpenRegionWrittenToWALForLogReplay",
-      100, 42);
+        100, 42);
     final RegionServerServices rss = spy(TEST_UTIL.createMockRegionServerService(serverName));
 
     HTableDescriptor htd

http://git-wip-us.apache.org/repos/asf/hbase/blob/25dfc112/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java
index 80333e8..b5e9798 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java
@@ -18,12 +18,6 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.util.List;
-import java.util.Random;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
@@ -36,6 +30,13 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import java.io.IOException;
+import java.util.List;
+import java.util.Random;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
 /**
  * Test the {@link MemStoreChunkPool} class
  */
@@ -47,7 +48,7 @@ public class TestMemStoreChunkPool {
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
-    conf.setBoolean(DefaultMemStore.USEMSLAB_KEY, true);
+    conf.setBoolean(SegmentFactory.USEMSLAB_KEY, true);
     conf.setFloat(MemStoreChunkPool.CHUNK_POOL_MAXSIZE_KEY, 0.2f);
     chunkPoolDisabledBeforeTest = MemStoreChunkPool.chunkPoolDisabled;
     MemStoreChunkPool.chunkPoolDisabled = false;
@@ -116,13 +117,13 @@ public class TestMemStoreChunkPool {
 
     // Creating a snapshot
     MemStoreSnapshot snapshot = memstore.snapshot();
-    assertEquals(3, memstore.snapshot.size());
+    assertEquals(3, memstore.getSnapshot().getCellsCount());
 
     // Adding value to "new" memstore
-    assertEquals(0, memstore.cellSet.size());
+    assertEquals(0, memstore.getActive().getCellsCount());
     memstore.add(new KeyValue(row, fam, qf4, val));
     memstore.add(new KeyValue(row, fam, qf5, val));
-    assertEquals(2, memstore.cellSet.size());
+    assertEquals(2, memstore.getActive().getCellsCount());
     memstore.clearSnapshot(snapshot.getId());
 
     int chunkCount = chunkPool.getPoolSize();
@@ -132,7 +133,7 @@ public class TestMemStoreChunkPool {
 
   @Test
   public void testPuttingBackChunksWithOpeningScanner()
-      throws UnexpectedStateException {
+      throws IOException {
     byte[] row = Bytes.toBytes("testrow");
     byte[] fam = Bytes.toBytes("testfamily");
     byte[] qf1 = Bytes.toBytes("testqualifier1");
@@ -153,13 +154,13 @@ public class TestMemStoreChunkPool {
 
     // Creating a snapshot
     MemStoreSnapshot snapshot = memstore.snapshot();
-    assertEquals(3, memstore.snapshot.size());
+    assertEquals(3, memstore.getSnapshot().getCellsCount());
 
     // Adding value to "new" memstore
-    assertEquals(0, memstore.cellSet.size());
+    assertEquals(0, memstore.getActive().getCellsCount());
     memstore.add(new KeyValue(row, fam, qf4, val));
     memstore.add(new KeyValue(row, fam, qf5, val));
-    assertEquals(2, memstore.cellSet.size());
+    assertEquals(2, memstore.getActive().getCellsCount());
 
     // opening scanner before clear the snapshot
     List<KeyValueScanner> scanners = memstore.getScanners(0);

http://git-wip-us.apache.org/repos/asf/hbase/blob/25dfc112/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
index 354ea2d..0a67ff8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
@@ -27,6 +27,7 @@ import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 
+import com.google.common.collect.Lists;
 import java.io.IOException;
 import java.lang.ref.SoftReference;
 import java.security.PrivilegedExceptionAction;
@@ -92,8 +93,6 @@ import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 import org.mockito.Mockito;
 
-import com.google.common.collect.Lists;
-
 /**
  * Test class for the Store
  */
@@ -555,7 +554,7 @@ public class TestStore {
     this.store.snapshot();
     flushStore(store, id++);
     Assert.assertEquals(storeFilessize, this.store.getStorefiles().size());
-    Assert.assertEquals(0, ((DefaultMemStore)this.store.memstore).cellSet.size());
+    Assert.assertEquals(0, ((AbstractMemStore)this.store.memstore).getActive().getCellsCount());
   }
 
   private void assertCheck() {
@@ -600,7 +599,7 @@ public class TestStore {
     flushStore(store, id++);
     Assert.assertEquals(1, this.store.getStorefiles().size());
     // from the one we inserted up there, and a new one
-    Assert.assertEquals(2, ((DefaultMemStore)this.store.memstore).cellSet.size());
+    Assert.assertEquals(2, ((AbstractMemStore)this.store.memstore).getActive().getCellsCount());
 
     // how many key/values for this row are there?
     Get get = new Get(row);
@@ -669,7 +668,7 @@ public class TestStore {
     }
 
     long computedSize=0;
-    for (Cell cell : ((DefaultMemStore)this.store.memstore).cellSet) {
+    for (Cell cell : ((AbstractMemStore)this.store.memstore).getActive().getCellSet()) {
       long kvsize = DefaultMemStore.heapSizeChange(cell, true);
       //System.out.println(kv + " size= " + kvsize + " kvsize= " + kv.heapSize());
       computedSize += kvsize;
@@ -701,7 +700,7 @@ public class TestStore {
     // then flush.
     flushStore(store, id++);
     Assert.assertEquals(1, this.store.getStorefiles().size());
-    Assert.assertEquals(1, ((DefaultMemStore)this.store.memstore).cellSet.size());
+    Assert.assertEquals(1, ((AbstractMemStore)this.store.memstore).getActive().getCellsCount());
 
     // now increment again:
     newValue += 1;


[08/22] hbase git commit: HBASE-15229 Canary Tools should not call System.Exit on error (Vishal Khandelwal)

Posted by sy...@apache.org.
HBASE-15229 Canary Tools should not call System.Exit on error (Vishal Khandelwal)


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

Branch: refs/heads/hbase-12439
Commit: 2963d59ed21de246390426cbdb57c160a4f65303
Parents: d533181
Author: Andrew Purtell <ap...@apache.org>
Authored: Wed Feb 10 10:19:49 2016 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Feb 10 10:21:00 2016 -0800

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/hbase/tool/Canary.java | 13 ++++++-------
 1 file changed, 6 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2963d59e/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
index b2cca32..9248c71 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
@@ -607,9 +607,9 @@ public final class Canary implements Tool {
             if (this.failOnError && monitor.hasError()) {
               monitorThread.interrupt();
               if (monitor.initialized) {
-                System.exit(monitor.errorCode);
+                return monitor.errorCode;
               } else {
-                System.exit(INIT_ERROR_EXIT_CODE);
+                return INIT_ERROR_EXIT_CODE;
               }
             }
             currentTimeLength = System.currentTimeMillis() - startTime;
@@ -618,17 +618,16 @@ public final class Canary implements Tool {
                   + ") after timeout limit:" + this.timeout
                   + " will be killed itself !!");
               if (monitor.initialized) {
-                System.exit(TIMEOUT_ERROR_EXIT_CODE);
+                return TIMEOUT_ERROR_EXIT_CODE;
               } else {
-                System.exit(INIT_ERROR_EXIT_CODE);
+                return INIT_ERROR_EXIT_CODE;
               }
-              break;
             }
           }
 
           if (this.failOnError && monitor.finalCheckForErrors()) {
             monitorThread.interrupt();
-            System.exit(monitor.errorCode);
+            return monitor.errorCode;
           }
         } finally {
           if (monitor != null) monitor.close();
@@ -641,7 +640,7 @@ public final class Canary implements Tool {
     if (choreService != null) {
       choreService.shutdown();
     }
-    return(monitor.errorCode);
+    return monitor.errorCode;
   }
 
   private void printUsageAndExit() {


[12/22] hbase git commit: HBASE-15198 RPC client not using Codec and CellBlock for puts by default.

Posted by sy...@apache.org.
HBASE-15198 RPC client not using Codec and CellBlock for puts by default.


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

Branch: refs/heads/hbase-12439
Commit: 29a192ef3cbe3b9cc12a6ee38f39e1199ac9790f
Parents: fec9733
Author: anoopsjohn <an...@gmail.com>
Authored: Thu Feb 11 09:51:11 2016 +0530
Committer: anoopsjohn <an...@gmail.com>
Committed: Thu Feb 11 09:51:11 2016 +0530

----------------------------------------------------------------------
 .../hadoop/hbase/client/ClusterConnection.java  |  5 ++++
 .../hbase/client/ConnectionImplementation.java  |  5 ++++
 .../hbase/client/MultiServerCallable.java       |  8 ++-----
 .../hadoop/hbase/ipc/AbstractRpcClient.java     |  5 ++++
 .../org/apache/hadoop/hbase/ipc/RpcClient.java  |  6 +++++
 .../hadoop/hbase/protobuf/ProtobufUtil.java     |  4 ----
 .../hadoop/hbase/protobuf/RequestConverter.java | 10 +++++++-
 .../hbase/client/TestFromClientSide3.java       |  6 +++++
 .../security/access/TestAccessController.java   | 24 --------------------
 9 files changed, 38 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/29a192ef/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 741989f..45589be 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
@@ -303,4 +303,9 @@ public interface ClusterConnection extends HConnection {
    */
   public MetricsConnection getConnectionMetrics();
 
+  /**
+   * @return true when this connection uses a {@link org.apache.hadoop.hbase.codec.Codec} and so
+   *         supports cell blocks.
+   */
+  boolean hasCellBlockSupport();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/29a192ef/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 dc59e6e..dfa9937 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
@@ -2255,4 +2255,9 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
     return RpcRetryingCallerFactory
         .instantiate(conf, this.interceptor, this.getStatisticsTracker());
   }
+
+  @Override
+  public boolean hasCellBlockSupport() {
+    return this.rpcClient.hasCellBlockSupport();
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/29a192ef/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java
index 72ae829..85b401e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java
@@ -22,7 +22,6 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CellScannable;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
@@ -152,11 +151,8 @@ class MultiServerCallable<R> extends RegionServerCallable<MultiResponse> impleme
     // This is not exact -- the configuration could have changed on us after connection was set up
     // but it will do for now.
     HConnection connection = getConnection();
-    if (connection == null) return true; // Default is to do cellblocks.
-    Configuration configuration = connection.getConfiguration();
-    if (configuration == null) return true;
-    String codec = configuration.get(HConstants.RPC_CODEC_CONF_KEY, "");
-    return codec != null && codec.length() > 0;
+    if (!(connection instanceof ClusterConnection)) return true; // Default is to do cellblocks.
+    return ((ClusterConnection) connection).hasCellBlockSupport();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/29a192ef/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 e33ef3a..a53fb70 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
@@ -149,6 +149,11 @@ public abstract class AbstractRpcClient implements RpcClient {
     }
   }
 
+  @Override
+  public boolean hasCellBlockSupport() {
+    return this.codec != null;
+  }
+
   /**
    * Encapsulate the ugly casting and RuntimeException conversion in private method.
    * @param conf configuration

http://git-wip-us.apache.org/repos/asf/hbase/blob/29a192ef/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java
index cf689f5..540e224 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java
@@ -83,4 +83,10 @@ import java.io.IOException;
    * using this client.
    */
   @Override public void close();
+
+  /**
+   * @return true when this client uses a {@link org.apache.hadoop.hbase.codec.Codec} and so
+   *         supports cell blocks.
+   */
+  boolean hasCellBlockSupport();
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/29a192ef/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 e9a1223..261a9aa 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
@@ -1188,10 +1188,6 @@ public final class ProtobufUtil {
         valueBuilder.setValue(ByteStringer.wrap(
             cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()));
         valueBuilder.setTimestamp(cell.getTimestamp());
-        if(cell.getTagsLength() > 0) {
-          valueBuilder.setTags(ByteStringer.wrap(cell.getTagsArray(), cell.getTagsOffset(),
-              cell.getTagsLength()));
-        }
         if (type == MutationType.DELETE || (type == MutationType.PUT && CellUtil.isDelete(cell))) {
           KeyValue.Type keyValueType = KeyValue.Type.codeToType(cell.getTypeByte());
           valueBuilder.setDeleteType(toDeleteType(keyValueType));

http://git-wip-us.apache.org/repos/asf/hbase/blob/29a192ef/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
index 9d659fc..572d92c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
@@ -648,8 +648,16 @@ public final class RequestConverter {
         cells.add(i);
         builder.addAction(actionBuilder.setMutation(ProtobufUtil.toMutationNoData(
           MutationType.INCREMENT, i, mutationBuilder, action.getNonce())));
+      } else if (row instanceof RegionCoprocessorServiceExec) {
+        RegionCoprocessorServiceExec exec = (RegionCoprocessorServiceExec) row;
+        builder.addAction(actionBuilder.setServiceCall(
+            ClientProtos.CoprocessorServiceCall.newBuilder()
+              .setRow(ByteStringer.wrap(exec.getRow()))
+              .setServiceName(exec.getMethod().getService().getFullName())
+              .setMethodName(exec.getMethod().getName())
+              .setRequest(exec.getRequest().toByteString())));
       } else if (row instanceof RowMutations) {
-        continue; // ignore RowMutations
+        throw new UnsupportedOperationException("No RowMutations in multi calls; use mutateRow");
       } else {
         throw new DoNotRetryIOException("Multi doesn't support " + row.getClass().getName());
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/29a192ef/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 7194c57..5995191 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
@@ -475,4 +475,10 @@ public class TestFromClientSide3 {
     assertTrue(Arrays.equals(res.getValue(FAMILY, COL_QUAL), VAL_BYTES));
     table.close();
   }
+
+  @Test
+  public void testConnectionDefaultUsesCodec() throws Exception {
+    ClusterConnection con = (ClusterConnection) TEST_UTIL.getConnection();
+    assertTrue(con.hasCellBlockSupport());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/29a192ef/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 9f20c11..081663d 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
@@ -2509,30 +2509,6 @@ public class TestAccessController extends SecureTestUtil {
   }
 
   @Test (timeout=180000)
-  public void testReservedCellTags() throws Exception {
-    AccessTestAction putWithReservedTag = new AccessTestAction() {
-      @Override
-      public Object run() throws Exception {
-        try(Connection conn = ConnectionFactory.createConnection(conf);
-            Table t = conn.getTable(TEST_TABLE);) {
-          KeyValue kv = new KeyValue(TEST_ROW, TEST_FAMILY, TEST_QUALIFIER,
-            HConstants.LATEST_TIMESTAMP, HConstants.EMPTY_BYTE_ARRAY,
-            new Tag[] { new ArrayBackedTag(AccessControlLists.ACL_TAG_TYPE,
-              ProtobufUtil.toUsersAndPermissions(USER_OWNER.getShortName(),
-                new Permission(Permission.Action.READ)).toByteArray()) });
-          t.put(new Put(TEST_ROW).add(kv));
-        }
-        return null;
-      }
-    };
-
-    // Current user is superuser
-    verifyAllowed(putWithReservedTag, User.getCurrent());
-    // No other user should be allowed
-    verifyDenied(putWithReservedTag, USER_OWNER, USER_ADMIN, USER_CREATE, USER_RW, USER_RO);
-  }
-
-  @Test (timeout=180000)
   public void testSetQuota() throws Exception {
     AccessTestAction setUserQuotaAction = new AccessTestAction() {
       @Override


[05/22] hbase git commit: HBASE-15244 More doc around native lib setup and check and crc

Posted by sy...@apache.org.
HBASE-15244 More doc around native lib setup and check and crc


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

Branch: refs/heads/hbase-12439
Commit: 703e975d668bee5d8897355d43e16e3c4f9f6a23
Parents: bab812d
Author: stack <st...@apache.org>
Authored: Tue Feb 9 23:17:37 2016 -0800
Committer: stack <st...@apache.org>
Committed: Tue Feb 9 23:17:48 2016 -0800

----------------------------------------------------------------------
 src/main/asciidoc/_chapters/compression.adoc | 45 ++++++++++++++++++++---
 1 file changed, 40 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/703e975d/src/main/asciidoc/_chapters/compression.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/compression.adoc b/src/main/asciidoc/_chapters/compression.adoc
index 462bce3..80e2cb0 100644
--- a/src/main/asciidoc/_chapters/compression.adoc
+++ b/src/main/asciidoc/_chapters/compression.adoc
@@ -142,14 +142,23 @@ In general, you need to weigh your options between smaller size and faster compr
 [[hadoop.native.lib]]
 === Making use of Hadoop Native Libraries in HBase
 
-The Hadoop shared library has a bunch of facility including compression libraries and fast crc'ing. To make this facility available to HBase, do the following. HBase/Hadoop will fall back to use alternatives if it cannot find the native library versions -- or fail outright if you asking for an explicit compressor and there is no alternative available.
+The Hadoop shared library has a bunch of facility including compression libraries and fast crc'ing -- hardware crc'ing if your chipset supports it.
+To make this facility available to HBase, do the following. HBase/Hadoop will fall back to use alternatives if it cannot find the native library
+versions -- or fail outright if you asking for an explicit compressor and there is no alternative available.
 
-If you see the following in your HBase logs, you know that HBase was unable to locate the Hadoop native libraries:
+First make sure of your Hadoop. Fix this message if you are seeing it starting Hadoop processes:
+----
+16/02/09 22:40:24 WARN util.NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
+----
+It means is not properly pointing at its native libraries or the native libs were compiled for another platform.
+Fix this first.
+
+Then if you see the following in your HBase logs, you know that HBase was unable to locate the Hadoop native libraries:
 [source]
 ----
 2014-08-07 09:26:20,139 WARN  [main] util.NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
 ----
-If the libraries loaded successfully, the WARN message does not show.
+If the libraries loaded successfully, the WARN message does not show. Usually this means you are good to go but read on.
 
 Let's presume your Hadoop shipped with a native library that suits the platform you are running HBase on.
 To check if the Hadoop native library is available to HBase, run the following tool (available in  Hadoop 2.1 and greater):
@@ -167,8 +176,13 @@ bzip2:  false
 ----
 Above shows that the native hadoop library is not available in HBase context.
 
+The above NativeLibraryChecker tool may come back saying all is hunky-dory
+-- i.e. all libs show 'true', that they are available -- but follow the below
+presecription anyways to ensure the native libs are available in HBase context,
+when it goes to use them.
+
 To fix the above, either copy the Hadoop native libraries local or symlink to them if the Hadoop and HBase stalls are adjacent in the filesystem.
-You could also point at their location by setting the `LD_LIBRARY_PATH` environment variable.
+You could also point at their location by setting the `LD_LIBRARY_PATH` environment variable in your hbase-env.sh.
 
 Where the JVM looks to find native libraries is "system dependent" (See `java.lang.System#loadLibrary(name)`). On linux, by default, is going to look in _lib/native/PLATFORM_ where `PLATFORM`      is the label for the platform your HBase is installed on.
 On a local linux machine, it seems to be the concatenation of the java properties `os.name` and `os.arch` followed by whether 32 or 64 bit.
@@ -183,8 +197,29 @@ For example:
 ----
 So in this case, the PLATFORM string is `Linux-amd64-64`.
 Copying the Hadoop native libraries or symlinking at _lib/native/Linux-amd64-64_     will ensure they are found.
-Check with the Hadoop _NativeLibraryChecker_.
+Rolling restart after you have made this change.
 
+Here is an example of how you would set up the symlinks.
+Let the hadoop and hbase installs be in your home directory. Assume your hadoop native libs
+are at ~/hadoop/lib/native. Assume you are on a Linux-amd64-64 platform. In this case,
+you would do the following to link the hadoop native lib so hbase could find them.
+----
+...
+$ mkdir -p ~/hbaseLinux-amd64-64 -> /home/stack/hadoop/lib/native/lib/native/
+$ cd ~/hbase/lib/native/
+$ ln -s ~/hadoop/lib/native Linux-amd64-64
+$ ls -la
+# Linux-amd64-64 -> /home/USER/hadoop/lib/native
+...
+----
+
+If you see PureJavaCrc32C in a stack track or if you see something like the below in a perf trace, then native is not working; you are using the java CRC functions rather than native:
+----
+  5.02%  perf-53601.map      [.] Lorg/apache/hadoop/util/PureJavaCrc32C;.update
+----
+See link:https://issues.apache.org/jira/browse/HBASE-11927[HBASE-11927 Use Native Hadoop Library for HFile checksum (And flip default from CRC32 to CRC32C)],
+for more on native checksumming support. See in particular the release note for how to check if your hardware to see if your processor has support for hardware CRCs.
+Or checkout the Apache link:https://blogs.apache.org/hbase/entry/saving_cpu_using_native_hadoop[Checksums in HBase] blog post.
 
 Here is example of how to point at the Hadoop libs with `LD_LIBRARY_PATH`      environment variable:
 [source]


[18/22] hbase git commit: HBASE-15252 Data loss when replaying wal if HDFS timeout

Posted by sy...@apache.org.
HBASE-15252 Data loss when replaying wal if HDFS timeout


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

Branch: refs/heads/hbase-12439
Commit: 85e1d9a109341c5f4aabb0e82c96ab52e99a6d72
Parents: 12982d1
Author: zhangduo <zh...@apache.org>
Authored: Fri Feb 12 08:17:10 2016 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Fri Feb 12 16:16:37 2016 +0800

----------------------------------------------------------------------
 .../regionserver/wal/ProtobufLogReader.java     |   3 +-
 .../hbase/regionserver/wal/TestWALReplay.java   | 113 ++++++++++++++++++-
 2 files changed, 112 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/85e1d9a1/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogReader.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogReader.java
index dc5c9cc..bb25aa1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogReader.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogReader.java
@@ -45,6 +45,7 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
 
 import com.google.protobuf.CodedInputStream;
+import com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * A Protobuf based WAL has the following structure:
@@ -332,7 +333,7 @@ public class ProtobufLogReader extends ReaderBase {
           }
           ProtobufUtil.mergeFrom(builder, new LimitInputStream(this.inputStream, size),
             (int)size);
-        } catch (IOException ipbe) {
+        } catch (InvalidProtocolBufferException ipbe) {
           throw (EOFException) new EOFException("Invalid PB, EOF? Ignoring; originalPosition=" +
             originalPosition + ", currentPosition=" + this.inputStream.getPos() +
             ", messageSize=" + size + ", currentAvailable=" + available).initCause(ipbe);

http://git-wip-us.apache.org/repos/asf/hbase/blob/85e1d9a1/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
index f004aeb..40e5baa 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
@@ -22,9 +22,15 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.when;
 
+import java.io.FilterInputStream;
 import java.io.IOException;
+import java.lang.reflect.Field;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -37,6 +43,7 @@ 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.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -91,6 +98,7 @@ 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.WALSplitter;
+import org.apache.hadoop.hdfs.DFSInputStream;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -100,6 +108,8 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
 
 /**
  * Test replay of edits out of a WAL split.
@@ -499,7 +509,7 @@ public class TestWALReplay {
     boolean first = true;
     for (HColumnDescriptor hcd: htd.getFamilies()) {
       addRegionEdits(rowName, hcd.getName(), countPerFamily, this.ee, region, "x");
-      if (first ) {
+      if (first) {
         // If first, so we have at least one family w/ different seqid to rest.
         region.flush(true);
         first = false;
@@ -819,9 +829,9 @@ public class TestWALReplay {
     final Configuration newConf = HBaseConfiguration.create(this.conf);
     User user = HBaseTestingUtility.getDifferentUser(newConf,
       ".replay.wal.secondtime");
-    user.runAs(new PrivilegedExceptionAction() {
+    user.runAs(new PrivilegedExceptionAction<Void>() {
       @Override
-      public Object run() throws Exception {
+      public Void run() throws Exception {
         runWALSplit(newConf);
         FileSystem newFS = FileSystem.get(newConf);
         // 100k seems to make for about 4 flushes during HRegion#initialize.
@@ -927,6 +937,103 @@ public class TestWALReplay {
         lastestSeqNumber, editCount);
   }
 
+  /**
+   * testcase for https://issues.apache.org/jira/browse/HBASE-15252
+   */
+  @Test
+  public void testDatalossWhenInputError() throws IOException, InstantiationException,
+      IllegalAccessException {
+    final TableName tableName = TableName.valueOf("testDatalossWhenInputError");
+    final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
+    final Path basedir = FSUtils.getTableDir(this.hbaseRootDir, tableName);
+    deleteDir(basedir);
+    final byte[] rowName = tableName.getName();
+    final int countPerFamily = 10;
+    final HTableDescriptor htd = createBasic1FamilyHTD(tableName);
+    HRegion region1 = HBaseTestingUtility.createRegionAndWAL(hri, hbaseRootDir, this.conf, htd);
+    Path regionDir = region1.getRegionFileSystem().getRegionDir();
+    HBaseTestingUtility.closeRegionAndWAL(region1);
+
+    WAL wal = createWAL(this.conf);
+    HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, htd, wal);
+    for (HColumnDescriptor hcd : htd.getFamilies()) {
+      addRegionEdits(rowName, hcd.getName(), countPerFamily, this.ee, region, "x");
+    }
+    // Now assert edits made it in.
+    final Get g = new Get(rowName);
+    Result result = region.get(g);
+    assertEquals(countPerFamily * htd.getFamilies().size(), result.size());
+    // Now close the region (without flush), split the log, reopen the region and assert that
+    // replay of log has the correct effect.
+    region.close(true);
+    wal.shutdown();
+
+    runWALSplit(this.conf);
+
+    // here we let the DFSInputStream throw an IOException just after the WALHeader.
+    Path editFile = WALSplitter.getSplitEditFilesSorted(this.fs, regionDir).first();
+    FSDataInputStream stream = fs.open(editFile);
+    stream.seek(ProtobufLogReader.PB_WAL_MAGIC.length);
+    Class<? extends DefaultWALProvider.Reader> logReaderClass =
+        conf.getClass("hbase.regionserver.hlog.reader.impl", ProtobufLogReader.class,
+          DefaultWALProvider.Reader.class);
+    DefaultWALProvider.Reader reader = logReaderClass.newInstance();
+    reader.init(this.fs, editFile, conf, stream);
+    final long headerLength = stream.getPos();
+    reader.close();
+    FileSystem spyFs = spy(this.fs);
+    doAnswer(new Answer<FSDataInputStream>() {
+
+      @Override
+      public FSDataInputStream answer(InvocationOnMock invocation) throws Throwable {
+        FSDataInputStream stream = (FSDataInputStream) invocation.callRealMethod();
+        Field field = FilterInputStream.class.getDeclaredField("in");
+        field.setAccessible(true);
+        final DFSInputStream in = (DFSInputStream) field.get(stream);
+        DFSInputStream spyIn = spy(in);
+        doAnswer(new Answer<Integer>() {
+
+          private long pos;
+
+          @Override
+          public Integer answer(InvocationOnMock invocation) throws Throwable {
+            if (pos >= headerLength) {
+              throw new IOException("read over limit");
+            }
+            int b = (Integer) invocation.callRealMethod();
+            if (b > 0) {
+              pos += b;
+            }
+            return b;
+          }
+        }).when(spyIn).read(any(byte[].class), any(int.class), any(int.class));
+        doAnswer(new Answer<Void>() {
+
+          @Override
+          public Void answer(InvocationOnMock invocation) throws Throwable {
+            invocation.callRealMethod();
+            in.close();
+            return null;
+          }
+        }).when(spyIn).close();
+        field.set(stream, spyIn);
+        return stream;
+      }
+    }).when(spyFs).open(eq(editFile));
+
+    WAL wal2 = createWAL(this.conf);
+    HRegion region2;
+    try {
+      // log replay should fail due to the IOException, otherwise we may lose data.
+      region2 = HRegion.openHRegion(conf, spyFs, hbaseRootDir, hri, htd, wal2);
+      assertEquals(result.size(), region2.get(g).size());
+    } catch (IOException e) {
+      assertEquals("read over limit", e.getMessage());
+    }
+    region2 = HRegion.openHRegion(conf, fs, hbaseRootDir, hri, htd, wal2);
+    assertEquals(result.size(), region2.get(g).size());
+  }
+
   static class MockWAL extends FSHLog {
     boolean doCompleteCacheFlush = false;
 


[02/22] hbase git commit: HBASE-11792 Organize Performance Evaluation usage output

Posted by sy...@apache.org.
HBASE-11792 Organize Performance Evaluation usage output


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

Branch: refs/heads/hbase-12439
Commit: 7cab24729d4585689af745df179d6ae92b2a6248
Parents: 7bb68b9
Author: Misty Stanley-Jones <ms...@cloudera.com>
Authored: Thu Dec 17 13:15:52 2015 -0800
Committer: Misty Stanley-Jones <ms...@cloudera.com>
Committed: Tue Feb 9 11:16:22 2016 -0800

----------------------------------------------------------------------
 .../hbase/rest/PerformanceEvaluation.java       | 10 ++--
 .../hadoop/hbase/PerformanceEvaluation.java     | 50 ++++++++++----------
 2 files changed, 33 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/7cab2472/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java
index dcd5b0a..e207735 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java
@@ -1366,10 +1366,12 @@ public class PerformanceEvaluation extends Configured implements Tool {
     System.err.println("  [--compress=TYPE] [--blockEncoding=TYPE] " +
       "[-D<property=value>]* <command> <nclients>");
     System.err.println();
-    System.err.println("Options:");
+    System.err.println("General Options:");
     System.err.println(" nomapred        Run multiple clients using threads " +
       "(rather than use mapreduce)");
     System.err.println(" rows            Rows each client runs. Default: One million");
+    System.err.println();
+    System.err.println("Table Creation / Write Tests:");
     System.err.println(" table           Alternate table name. Default: 'TestTable'");
     System.err.println(" compress        Compression type to use (GZ, LZO, ...). Default: 'NONE'");
     System.err.println(" flushCommits    Used to determine if the test should flush the table. " +
@@ -1377,13 +1379,15 @@ public class PerformanceEvaluation extends Configured implements Tool {
     System.err.println(" writeToWAL      Set writeToWAL on puts. Default: True");
     System.err.println(" presplit        Create presplit table. Recommended for accurate perf " +
       "analysis (see guide).  Default: disabled");
-    System.err.println(" inmemory        Tries to keep the HFiles of the CF inmemory as far as " +
-      "possible.  Not guaranteed that reads are always served from inmemory.  Default: false");
     System.err.println(" usetags         Writes tags along with KVs.  Use with HFile V3. " +
       "Default : false");
     System.err.println(" numoftags        Specify the no of tags that would be needed. " +
       "This works only if usetags is true.");
     System.err.println();
+    System.err.println("Read Tests:");
+    System.err.println(" inmemory        Tries to keep the HFiles of the CF inmemory as far as " +
+      "possible.  Not guaranteed that reads are always served from inmemory.  Default: false");
+    System.err.println();
     System.err.println(" Note: -D properties will be applied to the conf used. ");
     System.err.println("  For example: ");
     System.err.println("   -Dmapreduce.output.fileoutputformat.compress=true");

http://git-wip-us.apache.org/repos/asf/hbase/blob/7cab2472/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
index 651bc86..a31cc06 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
@@ -1827,56 +1827,58 @@ public class PerformanceEvaluation extends Configured implements Tool {
     System.err.println("Usage: java " + className + " \\");
     System.err.println("  <OPTIONS> [-D<property=value>]* <command> <nclients>");
     System.err.println();
-    System.err.println("Options:");
+    System.err.println("General Options:");
     System.err.println(" nomapred        Run multiple clients using threads " +
       "(rather than use mapreduce)");
-    System.err.println(" rows            Rows each client runs. Default: One million");
-    System.err.println(" size            Total size in GiB. Mutually exclusive with --rows. " +
-      "Default: 1.0.");
+    System.err.println(" oneCon          all the threads share the same connection. Default: False");
     System.err.println(" sampleRate      Execute test on a sample of total " +
       "rows. Only supported by randomRead. Default: 1.0");
+    System.err.println(" period          Report every 'period' rows: " +
+      "Default: opts.perClientRunRows / 10");
+    System.err.println(" cycles          How many times to cycle the test. Defaults: 1.");
     System.err.println(" traceRate       Enable HTrace spans. Initiate tracing every N rows. " +
       "Default: 0");
+    System.err.println(" latency         Set to report operation latencies. Default: False");
+    System.err.println(" measureAfter    Start to measure the latency once 'measureAfter'" +
+        " rows have been treated. Default: 0");
+    System.err.println(" valueSize       Pass value size to use: Default: 1024");
+    System.err.println(" valueRandom     Set if we should vary value size between 0 and " +
+        "'valueSize'; set on read for stats on size: Default: Not set.");
+    System.err.println();
+    System.err.println("Table Creation / Write Tests:");
     System.err.println(" table           Alternate table name. Default: 'TestTable'");
-    System.err.println(" multiGet        If >0, when doing RandomRead, perform multiple gets " +
-      "instead of single gets. Default: 0");
+    System.err.println(" rows            Rows each client runs. Default: One million");
+    System.err.println(" size            Total size in GiB. Mutually exclusive with --rows. " +
+      "Default: 1.0.");
     System.err.println(" compress        Compression type to use (GZ, LZO, ...). Default: 'NONE'");
     System.err.println(" flushCommits    Used to determine if the test should flush the table. " +
       "Default: false");
+    System.err.println(" valueZipf       Set if we should vary value size between 0 and " +
+        "'valueSize' in zipf form: Default: Not set.");
     System.err.println(" writeToWAL      Set writeToWAL on puts. Default: True");
     System.err.println(" autoFlush       Set autoFlush on htable. Default: False");
-    System.err.println(" oneCon          all the threads share the same connection. Default: False");
     System.err.println(" presplit        Create presplit table. Recommended for accurate perf " +
       "analysis (see guide).  Default: disabled");
-    System.err.println(" inmemory        Tries to keep the HFiles of the CF " +
-      "inmemory as far as possible. Not guaranteed that reads are always served " +
-      "from memory.  Default: false");
     System.err.println(" usetags         Writes tags along with KVs. Use with HFile V3. " +
       "Default: false");
     System.err.println(" numoftags       Specify the no of tags that would be needed. " +
        "This works only if usetags is true.");
+    System.err.println(" splitPolicy     Specify a custom RegionSplitPolicy for the table.");
+    System.err.println(" columns         Columns to write per row. Default: 1");
+    System.err.println();
+    System.err.println("Read Tests:");
     System.err.println(" filterAll       Helps to filter out all the rows on the server side"
         + " there by not returning any thing back to the client.  Helps to check the server side"
         + " performance.  Uses FilterAllFilter internally. ");
-    System.err.println(" latency         Set to report operation latencies. Default: False");
-    System.err.println(" measureAfter    Start to measure the latency once 'measureAfter'" +
-        " rows have been treated. Default: 0");
-    System.err.println(" bloomFilter      Bloom filter type, one of " + Arrays.toString(BloomType.values()));
-    System.err.println(" valueSize       Pass value size to use: Default: 1024");
-    System.err.println(" valueRandom     Set if we should vary value size between 0 and " +
-        "'valueSize'; set on read for stats on size: Default: Not set.");
-    System.err.println(" valueZipf       Set if we should vary value size between 0 and " +
-        "'valueSize' in zipf form: Default: Not set.");
-    System.err.println(" period          Report every 'period' rows: " +
-      "Default: opts.perClientRunRows / 10");
     System.err.println(" multiGet        Batch gets together into groups of N. Only supported " +
       "by randomRead. Default: disabled");
+    System.err.println(" inmemory        Tries to keep the HFiles of the CF " +
+      "inmemory as far as possible. Not guaranteed that reads are always served " +
+      "from memory.  Default: false");
+    System.err.println(" bloomFilter      Bloom filter type, one of " + Arrays.toString(BloomType.values()));
     System.err.println(" addColumns      Adds columns to scans/gets explicitly. Default: true");
     System.err.println(" replicas        Enable region replica testing. Defaults: 1.");
-    System.err.println(" cycles          How many times to cycle the test. Defaults: 1.");
-    System.err.println(" splitPolicy     Specify a custom RegionSplitPolicy for the table.");
     System.err.println(" randomSleep     Do a random sleep before each get between 0 and entered value. Defaults: 0");
-    System.err.println(" columns         Columns to write per row. Default: 1");
     System.err.println(" caching         Scan caching to use. Default: 30");
     System.err.println();
     System.err.println(" Note: -D properties will be applied to the conf used. ");


[06/22] hbase git commit: HBASE-15219 Canary tool does not return non-zero exit code when one of regions is in stuck state

Posted by sy...@apache.org.
HBASE-15219 Canary tool does not return non-zero exit code when one of regions is in stuck state


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

Branch: refs/heads/hbase-12439
Commit: df829ea7d1b4d2ef745e29d2b25b12966000eeb2
Parents: 703e975
Author: tedyu <yu...@gmail.com>
Authored: Wed Feb 10 02:36:46 2016 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Wed Feb 10 02:36:46 2016 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/tool/Canary.java    | 55 ++++++++++++++++----
 src/main/asciidoc/_chapters/ops_mgt.adoc        | 11 ++++
 2 files changed, 57 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/df829ea7/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
index 837688e..b2cca32 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
@@ -94,9 +94,11 @@ import org.apache.hadoop.util.ToolRunner;
 public final class Canary implements Tool {
   // Sink interface used by the canary to outputs information
   public interface Sink {
+    public long getReadFailureCount();
     public void publishReadFailure(HRegionInfo region, Exception e);
     public void publishReadFailure(HRegionInfo region, HColumnDescriptor column, Exception e);
     public void publishReadTiming(HRegionInfo region, HColumnDescriptor column, long msTime);
+    public long getWriteFailureCount();
     public void publishWriteFailure(HRegionInfo region, Exception e);
     public void publishWriteFailure(HRegionInfo region, HColumnDescriptor column, Exception e);
     public void publishWriteTiming(HRegionInfo region, HColumnDescriptor column, long msTime);
@@ -111,13 +113,23 @@ public final class Canary implements Tool {
   // Simple implementation of canary sink that allows to plot on
   // file or standard output timings or failures.
   public static class StdOutSink implements Sink {
+    protected AtomicLong readFailureCount = new AtomicLong(0),
+        writeFailureCount = new AtomicLong(0);
+
+    @Override
+    public long getReadFailureCount() {
+      return readFailureCount.get();
+    }
+
     @Override
     public void publishReadFailure(HRegionInfo region, Exception e) {
+      readFailureCount.incrementAndGet();
       LOG.error(String.format("read from region %s failed", region.getRegionNameAsString()), e);
     }
 
     @Override
     public void publishReadFailure(HRegionInfo region, HColumnDescriptor column, Exception e) {
+      readFailureCount.incrementAndGet();
       LOG.error(String.format("read from region %s column family %s failed",
                 region.getRegionNameAsString(), column.getNameAsString()), e);
     }
@@ -129,12 +141,19 @@ public final class Canary implements Tool {
     }
 
     @Override
+    public long getWriteFailureCount() {
+      return writeFailureCount.get();
+    }
+
+    @Override
     public void publishWriteFailure(HRegionInfo region, Exception e) {
+      writeFailureCount.incrementAndGet();
       LOG.error(String.format("write to region %s failed", region.getRegionNameAsString()), e);
     }
 
     @Override
     public void publishWriteFailure(HRegionInfo region, HColumnDescriptor column, Exception e) {
+      writeFailureCount.incrementAndGet();
       LOG.error(String.format("write to region %s column family %s failed",
         region.getRegionNameAsString(), column.getNameAsString()), e);
     }
@@ -150,6 +169,7 @@ public final class Canary implements Tool {
 
     @Override
     public void publishReadFailure(String table, String server) {
+      readFailureCount.incrementAndGet();
       LOG.error(String.format("Read from table:%s on region server:%s", table, server));
     }
 
@@ -435,6 +455,7 @@ public final class Canary implements Tool {
   private boolean regionServerMode = false;
   private boolean regionServerAllRegions = false;
   private boolean writeSniffing = false;
+  private boolean treatFailureAsError = false;
   private TableName writeTableName = DEFAULT_WRITE_TABLE_NAME;
 
   private ExecutorService executor; // threads to retrieve data from regionservers
@@ -498,6 +519,8 @@ public final class Canary implements Tool {
           this.regionServerAllRegions = true;
         } else if(cmd.equals("-writeSniffing")) {
           this.writeSniffing = true;
+        } else if(cmd.equals("-treatFailureAsError")) {
+          this.treatFailureAsError = true;
         } else if (cmd.equals("-e")) {
           this.useRegExp = true;
         } else if (cmd.equals("-t")) {
@@ -603,7 +626,7 @@ public final class Canary implements Tool {
             }
           }
 
-          if (this.failOnError && monitor.hasError()) {
+          if (this.failOnError && monitor.finalCheckForErrors()) {
             monitorThread.interrupt();
             System.exit(monitor.errorCode);
           }
@@ -639,6 +662,7 @@ public final class Canary implements Tool {
         " default is true");
     System.err.println("   -t <N>         timeout for a check, default is 600000 (milisecs)");
     System.err.println("   -writeSniffing enable the write sniffing in canary");
+    System.err.println("   -treatFailureAsError treats read / write failure as error");
     System.err.println("   -writeTable    The table used for write sniffing."
         + " Default is hbase:canary");
     System.err
@@ -666,11 +690,12 @@ public final class Canary implements Tool {
     if (this.regionServerMode) {
       monitor =
           new RegionServerMonitor(connection, monitorTargets, this.useRegExp,
-              (ExtendedSink) this.sink, this.executor, this.regionServerAllRegions);
+              (ExtendedSink) this.sink, this.executor, this.regionServerAllRegions,
+              this.treatFailureAsError);
     } else {
       monitor =
           new RegionMonitor(connection, monitorTargets, this.useRegExp, this.sink, this.executor,
-              this.writeSniffing, this.writeTableName);
+              this.writeSniffing, this.writeTableName, this.treatFailureAsError);
     }
     return monitor;
   }
@@ -682,6 +707,7 @@ public final class Canary implements Tool {
     protected Admin admin;
     protected String[] targets;
     protected boolean useRegExp;
+    protected boolean treatFailureAsError;
     protected boolean initialized = false;
 
     protected boolean done = false;
@@ -697,18 +723,27 @@ public final class Canary implements Tool {
       return errorCode != 0;
     }
 
+    public boolean finalCheckForErrors() {
+      if (errorCode != 0) {
+        return true;
+      }
+      return treatFailureAsError &&
+          (sink.getReadFailureCount() > 0 || sink.getWriteFailureCount() > 0);
+    }
+
     @Override
     public void close() throws IOException {
       if (this.admin != null) this.admin.close();
     }
 
     protected Monitor(Connection connection, String[] monitorTargets, boolean useRegExp, Sink sink,
-        ExecutorService executor) {
+        ExecutorService executor, boolean treatFailureAsError) {
       if (null == connection) throw new IllegalArgumentException("connection shall not be null");
 
       this.connection = connection;
       this.targets = monitorTargets;
       this.useRegExp = useRegExp;
+      this.treatFailureAsError = treatFailureAsError;
       this.sink = sink;
       this.executor = executor;
     }
@@ -748,8 +783,9 @@ public final class Canary implements Tool {
     private int checkPeriod;
 
     public RegionMonitor(Connection connection, String[] monitorTargets, boolean useRegExp,
-        Sink sink, ExecutorService executor, boolean writeSniffing, TableName writeTableName) {
-      super(connection, monitorTargets, useRegExp, sink, executor);
+        Sink sink, ExecutorService executor, boolean writeSniffing, TableName writeTableName,
+        boolean treatFailureAsError) {
+      super(connection, monitorTargets, useRegExp, sink, executor, treatFailureAsError);
       Configuration conf = connection.getConfiguration();
       this.writeSniffing = writeSniffing;
       this.writeTableName = writeTableName;
@@ -993,8 +1029,9 @@ public final class Canary implements Tool {
     private boolean allRegions;
 
     public RegionServerMonitor(Connection connection, String[] monitorTargets, boolean useRegExp,
-        ExtendedSink sink, ExecutorService executor, boolean allRegions) {
-      super(connection, monitorTargets, useRegExp, sink, executor);
+        ExtendedSink sink, ExecutorService executor, boolean allRegions,
+        boolean treatFailureAsError) {
+      super(connection, monitorTargets, useRegExp, sink, executor, treatFailureAsError);
       this.allRegions = allRegions;
     }
 
@@ -1089,7 +1126,7 @@ public final class Canary implements Tool {
         }
       } catch (InterruptedException e) {
         this.errorCode = ERROR_EXIT_CODE;
-        LOG.error("Sniff regionserver failed!", e);
+        LOG.error("Sniff regionserver interrupted!", e);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/df829ea7/src/main/asciidoc/_chapters/ops_mgt.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/ops_mgt.adoc b/src/main/asciidoc/_chapters/ops_mgt.adoc
index 0f93785..578d66f 100644
--- a/src/main/asciidoc/_chapters/ops_mgt.adoc
+++ b/src/main/asciidoc/_chapters/ops_mgt.adoc
@@ -93,6 +93,7 @@ Usage: bin/hbase org.apache.hadoop.hbase.tool.Canary [opts] [table1 [table2]...]
    -f <B>         stop whole program if first error occurs, default is true
    -t <N>         timeout for a check, default is 600000 (milliseconds)
    -writeSniffing enable the write sniffing in canary
+   -treatFailureAsError treats read / write failure as error
    -writeTable    The table used for write sniffing. Default is hbase:canary
    -D<configProperty>=<value> assigning or override the configuration params
 ----
@@ -215,6 +216,16 @@ $ ${HBASE_HOME}/bin/hbase canary -writeSniffing -writeTable ns:canary
 The default value size of each put is 10 bytes and you can set it by the config key:
 `hbase.canary.write.value.size`.
 
+==== Treat read / write failure as error
+
+By default, the canary tool only logs read failure, due to e.g. RetriesExhaustedException,
+while returning normal exit code. To treat read / write failure as error, you can run canary
+with the `-treatFailureAsError` option. When enabled, read / write failure would result in error
+exit code.
+----
+$ ${HBASE_HOME}/bin/hbase canary --treatFailureAsError
+----
+
 ==== Running Canary in a Kerberos-enabled Cluster
 
 To run Canary in a Kerberos-enabled cluster, configure the following two properties in _hbase-site.xml_:


[04/22] hbase git commit: HBASE-15216 Canary does not accept config params from command line (Vishal Khandelwal)

Posted by sy...@apache.org.
HBASE-15216 Canary does not accept config params from command line (Vishal Khandelwal)


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

Branch: refs/heads/hbase-12439
Commit: bab812df289a3bb33407446a8910e3bb5c6195e0
Parents: b6328eb
Author: Andrew Purtell <ap...@apache.org>
Authored: Tue Feb 9 22:45:37 2016 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Tue Feb 9 22:45:37 2016 -0800

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hbase/tool/Canary.java  | 9 +++++++++
 src/main/asciidoc/_chapters/ops_mgt.adoc                    | 1 +
 2 files changed, 10 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/bab812df/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
index 151be42..837688e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
@@ -76,6 +76,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.ReflectionUtils;
 import org.apache.hadoop.hbase.util.RegionSplitter;
+import org.apache.hadoop.util.GenericOptionsParser;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 
@@ -640,6 +641,8 @@ public final class Canary implements Tool {
     System.err.println("   -writeSniffing enable the write sniffing in canary");
     System.err.println("   -writeTable    The table used for write sniffing."
         + " Default is hbase:canary");
+    System.err
+        .println("   -D<configProperty>=<value> assigning or override the configuration params");
     System.exit(USAGE_EXIT_CODE);
   }
 
@@ -1184,7 +1187,13 @@ public final class Canary implements Tool {
 
   public static void main(String[] args) throws Exception {
     final Configuration conf = HBaseConfiguration.create();
+
+    // loading the generic options to conf
+    new GenericOptionsParser(conf, args);
+
     int numThreads = conf.getInt("hbase.canary.threads.num", MAX_THREADS_NUM);
+    LOG.info("Number of exection threads " + numThreads);
+
     ExecutorService executor = new ScheduledThreadPoolExecutor(numThreads);
 
     Class<? extends Sink> sinkClass =

http://git-wip-us.apache.org/repos/asf/hbase/blob/bab812df/src/main/asciidoc/_chapters/ops_mgt.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/ops_mgt.adoc b/src/main/asciidoc/_chapters/ops_mgt.adoc
index 13835c0..0f93785 100644
--- a/src/main/asciidoc/_chapters/ops_mgt.adoc
+++ b/src/main/asciidoc/_chapters/ops_mgt.adoc
@@ -94,6 +94,7 @@ Usage: bin/hbase org.apache.hadoop.hbase.tool.Canary [opts] [table1 [table2]...]
    -t <N>         timeout for a check, default is 600000 (milliseconds)
    -writeSniffing enable the write sniffing in canary
    -writeTable    The table used for write sniffing. Default is hbase:canary
+   -D<configProperty>=<value> assigning or override the configuration params
 ----
 
 This tool will return non zero error codes to user for collaborating with other monitoring tools, such as Nagios.


[17/22] hbase git commit: HBASE-15198 RPC client not using Codec and CellBlock for puts by default-addendum.

Posted by sy...@apache.org.
HBASE-15198 RPC client not using Codec and CellBlock for puts by default-addendum.


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

Branch: refs/heads/hbase-12439
Commit: 12982d1957d46276882e447c8fe854e0d16a17b0
Parents: ab50c7c
Author: anoopsjohn <an...@gmail.com>
Authored: Thu Feb 11 20:00:02 2016 +0530
Committer: anoopsjohn <an...@gmail.com>
Committed: Fri Feb 12 07:37:45 2016 +0530

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/client/TestAsyncProcess.java   | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/12982d19/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
index fb8b20b..645cc42 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
@@ -391,6 +391,11 @@ public class TestAsyncProcess {
         byte[] row, boolean useCache, boolean retry, int replicaId) throws IOException {
       return new RegionLocations(loc1);
     }
+
+    @Override
+    public boolean hasCellBlockSupport() {
+      return false;
+    }
   }
 
   /**


[22/22] hbase git commit: Revert "HBASE-9393 Hbase does not closing a closed socket resulting in many CLOSE_WAIT"

Posted by sy...@apache.org.
Revert "HBASE-9393 Hbase does not closing a closed socket resulting in many CLOSE_WAIT"

This reverts commit c8d133186b85a4e7298bab3376ad96899860b1c7.


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

Branch: refs/heads/hbase-12439
Commit: 3897c4e102b223847e1ceedf363df59fe24eb294
Parents: 1419628
Author: Sean Busbey <bu...@cloudera.com>
Authored: Sat Feb 13 11:18:53 2016 -0600
Committer: Sean Busbey <bu...@cloudera.com>
Committed: Sat Feb 13 11:18:53 2016 -0600

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/io/hfile/HFile.java | 34 ++++----------------
 .../hadoop/hbase/io/hfile/HFileBlock.java       | 20 +-----------
 .../hadoop/hbase/io/hfile/HFileReaderImpl.java  |  9 ------
 3 files changed, 7 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/3897c4e1/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
index 03d681e..1e1835f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
@@ -41,7 +41,6 @@ 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.CanUnbuffer;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
@@ -475,11 +474,6 @@ public class HFile {
 
     @VisibleForTesting
     boolean prefetchComplete();
-
-    /**
-     * To close only the stream's socket. HBASE-9393
-     */
-    void unbufferStream();
   }
 
   /**
@@ -496,8 +490,8 @@ public class HFile {
    */
   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="SF_SWITCH_FALLTHROUGH",
       justification="Intentional")
-  private static Reader openReader(Path path, FSDataInputStreamWrapper fsdis, long size,
-      CacheConfig cacheConf, HFileSystem hfs, Configuration conf) throws IOException {
+  private static Reader pickReaderVersion(Path path, FSDataInputStreamWrapper fsdis,
+      long size, CacheConfig cacheConf, HFileSystem hfs, Configuration conf) throws IOException {
     FixedFileTrailer trailer = null;
     try {
       boolean isHBaseChecksum = fsdis.shouldUseHBaseChecksum();
@@ -519,22 +513,6 @@ public class HFile {
         LOG.warn("Error closing fsdis FSDataInputStreamWrapper", t2);
       }
       throw new CorruptHFileException("Problem reading HFile Trailer from file " + path, t);
-    } finally {
-      unbufferStream(fsdis);
-    }
-  }
-
-  static void unbufferStream(FSDataInputStreamWrapper fsdis) {
-    boolean useHBaseChecksum = fsdis.shouldUseHBaseChecksum();
-    final FSDataInputStream stream = fsdis.getStream(useHBaseChecksum);
-    if (stream != null && stream.getWrappedStream() instanceof CanUnbuffer) {
-      // Enclosing unbuffer() in try-catch just to be on defensive side.
-      try {
-        stream.unbuffer();
-      } catch (Throwable e) {
-        LOG.error("Failed to unbuffer the stream so possibly there may be a TCP socket connection "
-            + "left open in CLOSE_WAIT state.", e);
-      }
     }
   }
 
@@ -563,7 +541,7 @@ public class HFile {
     } else {
       hfs = (HFileSystem)fs;
     }
-    return openReader(path, fsdis, size, cacheConf, hfs, conf);
+    return pickReaderVersion(path, fsdis, size, cacheConf, hfs, conf);
   }
 
   /**
@@ -578,8 +556,8 @@ public class HFile {
       FileSystem fs, Path path, CacheConfig cacheConf, Configuration conf) throws IOException {
     Preconditions.checkNotNull(cacheConf, "Cannot create Reader with null CacheConf");
     FSDataInputStreamWrapper stream = new FSDataInputStreamWrapper(fs, path);
-    return openReader(path, stream, fs.getFileStatus(path).getLen(), cacheConf, stream.getHfs(),
-      conf);
+    return pickReaderVersion(path, stream, fs.getFileStatus(path).getLen(),
+      cacheConf, stream.getHfs(), conf);
   }
 
   /**
@@ -589,7 +567,7 @@ public class HFile {
       FSDataInputStream fsdis, long size, CacheConfig cacheConf, Configuration conf)
       throws IOException {
     FSDataInputStreamWrapper wrapper = new FSDataInputStreamWrapper(fsdis);
-    return openReader(path, wrapper, size, cacheConf, null, conf);
+    return pickReaderVersion(path, wrapper, size, cacheConf, null, conf);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/3897c4e1/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 79b3e1c..e7a1e5e 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
@@ -33,10 +33,10 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.fs.HFileSystem;
+import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
 import org.apache.hadoop.hbase.io.ByteArrayOutputStream;
 import org.apache.hadoop.hbase.io.ByteBuffInputStream;
 import org.apache.hadoop.hbase.io.ByteBufferSupportDataOutputStream;
-import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.io.encoding.HFileBlockDecodingContext;
 import org.apache.hadoop.hbase.io.encoding.HFileBlockDefaultDecodingContext;
@@ -1311,11 +1311,6 @@ public class HFileBlock implements Cacheable {
 
     void setIncludesMemstoreTS(boolean includesMemstoreTS);
     void setDataBlockEncoder(HFileDataBlockEncoder encoder);
-
-    /**
-     * To close only the stream's socket. HBASE-9393
-     */
-    void unbufferStream();
   }
 
   /**
@@ -1763,19 +1758,6 @@ public class HFileBlock implements Cacheable {
     public String toString() {
       return "hfs=" + hfs + ", path=" + pathName + ", fileContext=" + fileContext;
     }
-
-    @Override
-    public void unbufferStream() {
-      // To handle concurrent reads, ensure that no other client is accessing the streams while we
-      // unbuffer it.
-      if (streamLock.tryLock()) {
-        try {
-          HFile.unbufferStream(this.streamWrapper);
-        } finally {
-          streamLock.unlock();
-        }
-      }
-    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/3897c4e1/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 f676e60..b2f5ded 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
@@ -575,10 +575,6 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
 
     @Override
     public void close() {
-      if (!pread) {
-        // For seek + pread stream socket should be closed when the scanner is closed. HBASE-9393
-        reader.unbufferStream();
-      }
       this.returnBlocks(true);
     }
 
@@ -1902,9 +1898,4 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
   public int getMajorVersion() {
     return 3;
   }
-
-  @Override
-  public void unbufferStream() {
-    fsBlockReader.unbufferStream();
-  }
 }


[20/22] hbase git commit: HBASE-9393 Hbase does not closing a closed socket resulting in many CLOSE_WAIT

Posted by sy...@apache.org.
HBASE-9393 Hbase does not closing a closed socket resulting in many CLOSE_WAIT

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

Branch: refs/heads/hbase-12439
Commit: c8d133186b85a4e7298bab3376ad96899860b1c7
Parents: 454e45f
Author: Ashish Singhi <as...@huawei.com>
Authored: Thu Jan 28 14:23:26 2016 +0530
Committer: stack <st...@apache.org>
Committed: Fri Feb 12 13:41:09 2016 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/io/hfile/HFile.java | 34 ++++++++++++++++----
 .../hadoop/hbase/io/hfile/HFileBlock.java       | 20 +++++++++++-
 .../hadoop/hbase/io/hfile/HFileReaderImpl.java  |  9 ++++++
 3 files changed, 56 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c8d13318/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
index 1e1835f..03d681e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
@@ -41,6 +41,7 @@ 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.CanUnbuffer;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
@@ -474,6 +475,11 @@ public class HFile {
 
     @VisibleForTesting
     boolean prefetchComplete();
+
+    /**
+     * To close only the stream's socket. HBASE-9393
+     */
+    void unbufferStream();
   }
 
   /**
@@ -490,8 +496,8 @@ public class HFile {
    */
   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="SF_SWITCH_FALLTHROUGH",
       justification="Intentional")
-  private static Reader pickReaderVersion(Path path, FSDataInputStreamWrapper fsdis,
-      long size, CacheConfig cacheConf, HFileSystem hfs, Configuration conf) throws IOException {
+  private static Reader openReader(Path path, FSDataInputStreamWrapper fsdis, long size,
+      CacheConfig cacheConf, HFileSystem hfs, Configuration conf) throws IOException {
     FixedFileTrailer trailer = null;
     try {
       boolean isHBaseChecksum = fsdis.shouldUseHBaseChecksum();
@@ -513,6 +519,22 @@ public class HFile {
         LOG.warn("Error closing fsdis FSDataInputStreamWrapper", t2);
       }
       throw new CorruptHFileException("Problem reading HFile Trailer from file " + path, t);
+    } finally {
+      unbufferStream(fsdis);
+    }
+  }
+
+  static void unbufferStream(FSDataInputStreamWrapper fsdis) {
+    boolean useHBaseChecksum = fsdis.shouldUseHBaseChecksum();
+    final FSDataInputStream stream = fsdis.getStream(useHBaseChecksum);
+    if (stream != null && stream.getWrappedStream() instanceof CanUnbuffer) {
+      // Enclosing unbuffer() in try-catch just to be on defensive side.
+      try {
+        stream.unbuffer();
+      } catch (Throwable e) {
+        LOG.error("Failed to unbuffer the stream so possibly there may be a TCP socket connection "
+            + "left open in CLOSE_WAIT state.", e);
+      }
     }
   }
 
@@ -541,7 +563,7 @@ public class HFile {
     } else {
       hfs = (HFileSystem)fs;
     }
-    return pickReaderVersion(path, fsdis, size, cacheConf, hfs, conf);
+    return openReader(path, fsdis, size, cacheConf, hfs, conf);
   }
 
   /**
@@ -556,8 +578,8 @@ public class HFile {
       FileSystem fs, Path path, CacheConfig cacheConf, Configuration conf) throws IOException {
     Preconditions.checkNotNull(cacheConf, "Cannot create Reader with null CacheConf");
     FSDataInputStreamWrapper stream = new FSDataInputStreamWrapper(fs, path);
-    return pickReaderVersion(path, stream, fs.getFileStatus(path).getLen(),
-      cacheConf, stream.getHfs(), conf);
+    return openReader(path, stream, fs.getFileStatus(path).getLen(), cacheConf, stream.getHfs(),
+      conf);
   }
 
   /**
@@ -567,7 +589,7 @@ public class HFile {
       FSDataInputStream fsdis, long size, CacheConfig cacheConf, Configuration conf)
       throws IOException {
     FSDataInputStreamWrapper wrapper = new FSDataInputStreamWrapper(fsdis);
-    return pickReaderVersion(path, wrapper, size, cacheConf, null, conf);
+    return openReader(path, wrapper, size, cacheConf, null, conf);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/c8d13318/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 e7a1e5e..79b3e1c 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
@@ -33,10 +33,10 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.fs.HFileSystem;
-import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
 import org.apache.hadoop.hbase.io.ByteArrayOutputStream;
 import org.apache.hadoop.hbase.io.ByteBuffInputStream;
 import org.apache.hadoop.hbase.io.ByteBufferSupportDataOutputStream;
+import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.io.encoding.HFileBlockDecodingContext;
 import org.apache.hadoop.hbase.io.encoding.HFileBlockDefaultDecodingContext;
@@ -1311,6 +1311,11 @@ public class HFileBlock implements Cacheable {
 
     void setIncludesMemstoreTS(boolean includesMemstoreTS);
     void setDataBlockEncoder(HFileDataBlockEncoder encoder);
+
+    /**
+     * To close only the stream's socket. HBASE-9393
+     */
+    void unbufferStream();
   }
 
   /**
@@ -1758,6 +1763,19 @@ public class HFileBlock implements Cacheable {
     public String toString() {
       return "hfs=" + hfs + ", path=" + pathName + ", fileContext=" + fileContext;
     }
+
+    @Override
+    public void unbufferStream() {
+      // To handle concurrent reads, ensure that no other client is accessing the streams while we
+      // unbuffer it.
+      if (streamLock.tryLock()) {
+        try {
+          HFile.unbufferStream(this.streamWrapper);
+        } finally {
+          streamLock.unlock();
+        }
+      }
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/c8d13318/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 b2f5ded..f676e60 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
@@ -575,6 +575,10 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
 
     @Override
     public void close() {
+      if (!pread) {
+        // For seek + pread stream socket should be closed when the scanner is closed. HBASE-9393
+        reader.unbufferStream();
+      }
       this.returnBlocks(true);
     }
 
@@ -1898,4 +1902,9 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
   public int getMajorVersion() {
     return 3;
   }
+
+  @Override
+  public void unbufferStream() {
+    fsBlockReader.unbufferStream();
+  }
 }


[03/22] hbase git commit: HBASE-15238 HFileReaderV2 prefetch overreaches; runs off the end of the data

Posted by sy...@apache.org.
HBASE-15238 HFileReaderV2 prefetch overreaches; runs off the end of the data

    M hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ChecksumUtil.java
      Cleanup trace message and include offset; makes debug the easier.

    M hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java
      Fix incorrect data member javadoc.

    M hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
      Pass along the offset we are checksumming at.

    M hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpljava
      Add trace logging for debugging and set the end of the prefetch to be
      last datablock, not size minus trailersize (there is the root indices
      and file info to be skipped)


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

Branch: refs/heads/hbase-12439
Commit: b6328eb80336e4a30ddec1e03751572f9bec77cf
Parents: 7cab247
Author: stack <st...@apache.org>
Authored: Tue Feb 9 11:35:33 2016 -0800
Committer: stack <st...@apache.org>
Committed: Tue Feb 9 20:31:44 2016 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/io/hfile/ChecksumUtil.java     | 29 ++++++++++----------
 .../hadoop/hbase/io/hfile/FixedFileTrailer.java |  9 +++---
 .../hadoop/hbase/io/hfile/HFileBlock.java       |  9 +++---
 .../hadoop/hbase/io/hfile/HFileReaderImpl.java  | 18 ++++++++----
 .../hadoop/hbase/io/hfile/TestChecksum.java     |  2 +-
 5 files changed, 37 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b6328eb8/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 402caa8..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
@@ -38,11 +38,11 @@ public class ChecksumUtil {
   /** This is used to reserve space in a byte buffer */
   private static byte[] DUMMY_VALUE = new byte[128 * HFileBlock.CHECKSUM_SIZE];
 
-  /** 
-   * This is used by unit tests to make checksum failures throw an 
-   * exception instead of returning null. Returning a null value from 
-   * checksum validation will cause the higher layer to retry that 
-   * read with hdfs-level checksums. Instead, we would like checksum 
+  /**
+   * This is used by unit tests to make checksum failures throw an
+   * exception instead of returning null. Returning a null value from
+   * checksum validation will cause the higher layer to retry that
+   * read with hdfs-level checksums. Instead, we would like checksum
    * failures to cause the entire unit test to fail.
    */
   private static boolean generateExceptions = false;
@@ -86,7 +86,7 @@ public class ChecksumUtil {
    * The header is extracted from the specified HFileBlock while the
    * data-to-be-verified is extracted from 'data'.
    */
-  static boolean validateBlockChecksum(String pathName, HFileBlock block,
+  static boolean validateBlockChecksum(String pathName, long offset, HFileBlock block,
     byte[] data, int hdrSize) throws IOException {
 
     // If this is an older version of the block that does not have
@@ -100,7 +100,7 @@ public class ChecksumUtil {
     }
 
     // Get a checksum object based on the type of checksum that is
-    // set in the HFileBlock header. A ChecksumType.NULL indicates that 
+    // set in the HFileBlock header. A ChecksumType.NULL indicates that
     // the caller is not interested in validating checksums, so we
     // always return true.
     ChecksumType cktype = ChecksumType.codeToType(block.getChecksumType());
@@ -116,12 +116,13 @@ public class ChecksumUtil {
     assert dataChecksum != null;
     int sizeWithHeader =  block.getOnDiskDataSizeWithHeader();
     if (LOG.isTraceEnabled()) {
-      LOG.info("length of data = " + data.length
-          + " OnDiskDataSizeWithHeader = " + sizeWithHeader
-          + " checksum type = " + cktype.getName()
-          + " file =" + pathName
-          + " header size = " + hdrSize
-          + " bytesPerChecksum = " + bytesPerChecksum);
+      LOG.info("dataLength=" + data.length
+          + ", sizeWithHeader=" + sizeWithHeader
+          + ", checksumType=" + cktype.getName()
+          + ", file=" + pathName
+          + ", offset=" + offset
+          + ", headerSize=" + hdrSize
+          + ", bytesPerChecksum=" + bytesPerChecksum);
     }
     try {
       dataChecksum.verifyChunkedSums(ByteBuffer.wrap(data, 0, sizeWithHeader),
@@ -140,7 +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) * 
+    return numChunks(datasize, bytesPerChecksum) *
                      HFileBlock.CHECKSUM_SIZE;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b6328eb8/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java
index 72f550a..ef6370e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java
@@ -43,8 +43,7 @@ import org.apache.hadoop.hbase.util.Bytes;
  * trailer size is fixed within a given {@link HFile} format version only, but
  * we always store the version number as the last four-byte integer of the file.
  * The version number itself is split into two portions, a major 
- * version and a minor version. 
- * The last three bytes of a file is the major
+ * version and a minor version. The last three bytes of a file are the major
  * version and a single preceding byte is the minor number. The major version
  * determines which readers/writers to use to read/write a hfile while a minor
  * version determines smaller changes in hfile format that do not need a new
@@ -52,7 +51,6 @@ import org.apache.hadoop.hbase.util.Bytes;
  */
 @InterfaceAudience.Private
 public class FixedFileTrailer {
-
   /**
    * We store the comparator class name as a fixed-length field in the trailer.
    */
@@ -67,8 +65,9 @@ public class FixedFileTrailer {
   /**
    * In version 1, the offset to the data block index. Starting from version 2,
    * the meaning of this field is the offset to the section of the file that
-   * should be loaded at the time the file is being opened, and as of the time
-   * of writing, this happens to be the offset of the file info section.
+   * should be loaded at the time the file is being opened: i.e. on open we load
+   * the root index, file info, etc. See http://hbase.apache.org/book.html#_hfile_format_2
+   * in the reference guide.
    */
   private long loadOnOpenDataOffset;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b6328eb8/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 e0719aa..e7a1e5e 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
@@ -1694,7 +1694,7 @@ public class HFileBlock implements Cacheable {
         b.assumeUncompressed();
       }
 
-      if (verifyChecksum && !validateBlockChecksum(b, onDiskBlock, hdrSize)) {
+      if (verifyChecksum && !validateBlockChecksum(b, offset, onDiskBlock, hdrSize)) {
         return null;             // checksum mismatch
       }
 
@@ -1743,9 +1743,10 @@ public class HFileBlock implements Cacheable {
      * If there is a checksum mismatch, then return false. Otherwise
      * return true.
      */
-    protected boolean validateBlockChecksum(HFileBlock block,  byte[] data, int hdrSize)
-        throws IOException {
-      return ChecksumUtil.validateBlockChecksum(pathName, block, data, hdrSize);
+    protected boolean validateBlockChecksum(HFileBlock block, long offset, byte[] data,
+        int hdrSize)
+    throws IOException {
+      return ChecksumUtil.validateBlockChecksum(pathName, offset, block, data, hdrSize);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/b6328eb8/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 a873280..b2f5ded 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
@@ -248,10 +248,14 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
     if (cacheConf.shouldPrefetchOnOpen()) {
       PrefetchExecutor.request(path, new Runnable() {
         public void run() {
+          long offset = 0;
+          long end = 0;
           try {
-            long offset = 0;
-            long end = fileSize - getTrailer().getTrailerSize();
+            end = getTrailer().getLoadOnOpenDataOffset();
             HFileBlock prevBlock = null;
+            if (LOG.isTraceEnabled()) {
+              LOG.trace("File=" + path.toString() + ", offset=" + offset + ", end=" + end);
+            }
             while (offset < end) {
               if (Thread.interrupted()) {
                 break;
@@ -273,11 +277,11 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
           } catch (IOException e) {
             // IOExceptions are probably due to region closes (relocation, etc.)
             if (LOG.isTraceEnabled()) {
-              LOG.trace("Exception encountered while prefetching " + path + ":", e);
+              LOG.trace("File=" + path.toString() + ", offset=" + offset + ", end=" + end, e);
             }
           } catch (Exception e) {
             // Other exceptions are interesting
-            LOG.warn("Exception encountered while prefetching " + path + ":", e);
+            LOG.warn("File=" + path.toString() + ", offset=" + offset + ", end=" + end, e);
           } finally {
             PrefetchExecutor.complete(path);
           }
@@ -1457,9 +1461,11 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
     if (dataBlockIndexReader == null) {
       throw new IOException("Block index not loaded");
     }
-    if (dataBlockOffset < 0 || dataBlockOffset >= trailer.getLoadOnOpenDataOffset()) {
+    long trailerOffset = trailer.getLoadOnOpenDataOffset();
+    if (dataBlockOffset < 0 || dataBlockOffset >= trailerOffset) {
       throw new IOException("Requested block is out of range: " + dataBlockOffset +
-        ", lastDataBlockOffset: " + trailer.getLastDataBlockOffset());
+        ", lastDataBlockOffset: " + trailer.getLastDataBlockOffset() +
+        ", trailer.getLoadOnOpenDataOffset: " + trailerOffset);
     }
     // For any given block from any given file, synchronize reads for said
     // block.

http://git-wip-us.apache.org/repos/asf/hbase/blob/b6328eb8/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 1767fb2..e8a2882 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
@@ -349,7 +349,7 @@ public class TestChecksum {
     }
 
     @Override
-    protected boolean validateBlockChecksum(HFileBlock block, 
+    protected boolean validateBlockChecksum(HFileBlock block, long offset,
       byte[] data, int hdrSize) throws IOException {
       return false;  // checksum validation failure
     }


[11/22] hbase git commit: HBASE-15204 Try to estimate the cell count for adding into WALEdit (Ram)

Posted by sy...@apache.org.
HBASE-15204 Try to estimate the cell count for adding into WALEdit (Ram)


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

Branch: refs/heads/hbase-12439
Commit: fec97338931f2617ddb99bf7faad67d0a0ee2ddf
Parents: 1942a99
Author: ramkrishna <ra...@gmail.com>
Authored: Thu Feb 11 09:09:25 2016 +0530
Committer: ramkrishna <ra...@gmail.com>
Committed: Thu Feb 11 09:09:25 2016 +0530

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/HRegion.java      | 20 ++++++++++++++++----
 .../hadoop/hbase/regionserver/wal/WALEdit.java  | 11 ++++++++++-
 2 files changed, 26 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/fec97338/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 ac846b6..3e6c092 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
@@ -2951,7 +2951,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     Set<byte[]> deletesCfSet = null;
     long currentNonceGroup = HConstants.NO_NONCE;
     long currentNonce = HConstants.NO_NONCE;
-    WALEdit walEdit = new WALEdit(replay);
+    WALEdit walEdit = null;
     boolean locked = false;
     // reference family maps directly so coprocessors can mutate them if desired
     Map<byte[], List<Cell>>[] familyMaps = new Map[batchOp.operations.length];
@@ -2962,6 +2962,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     int noOfPuts = 0;
     int noOfDeletes = 0;
     WriteEntry writeEntry = null;
+    int cellCount = 0;
     /** Keep track of the locks we hold so we can release them in finally clause */
     List<RowLock> acquiredRowLocks = Lists.newArrayListWithCapacity(batchOp.operations.length);
     try {
@@ -2990,7 +2991,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
         lastIndexExclusive++;
         numReadyToWrite++;
-
+        if (replay) {
+          for (List<Cell> cells : mutation.getFamilyCellMap().values()) {
+            cellCount += cells.size();
+          }
+        }
         if (mutation instanceof Put) {
           // If Column Families stay consistent through out all of the
           // individual puts then metrics can be reported as a multiput across
@@ -3041,8 +3046,15 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
           noOfDeletes++;
         }
         rewriteCellTags(familyMaps[i], mutation);
+        WALEdit fromCP = batchOp.walEditsFromCoprocessors[i];
+        if (fromCP != null) {
+          cellCount += fromCP.size();
+        }
+        for (List<Cell> cells : familyMaps[i].values()) {
+          cellCount += cells.size();
+        }
       }
-
+      walEdit = new WALEdit(cellCount, replay);
       lock(this.updatesLock.readLock(), numReadyToWrite);
       locked = true;
 
@@ -3082,7 +3094,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         if (nonceGroup != currentNonceGroup || nonce != currentNonce) {
           // Write what we have so far for nonces out to WAL
           appendCurrentNonces(m, replay, walEdit, now, currentNonceGroup, currentNonce);
-          walEdit = new WALEdit(replay);
+          walEdit = new WALEdit(cellCount, replay);
           currentNonceGroup = nonceGroup;
           currentNonce = nonce;
         }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fec97338/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java
index cea2ee7..346a8ed 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java
@@ -99,7 +99,7 @@ public class WALEdit implements Writable, HeapSize {
   private final int VERSION_2 = -1;
   private final boolean isReplay;
 
-  private ArrayList<Cell> cells = new ArrayList<Cell>(1);
+  private ArrayList<Cell> cells = null;
 
   public static final WALEdit EMPTY_WALEDIT = new WALEdit();
 
@@ -117,7 +117,16 @@ public class WALEdit implements Writable, HeapSize {
   }
 
   public WALEdit(boolean isReplay) {
+    this(1, isReplay);
+  }
+
+  public WALEdit(int cellCount) {
+    this(cellCount, false);
+  }
+
+  public WALEdit(int cellCount, boolean isReplay) {
     this.isReplay = isReplay;
+    cells = new ArrayList<Cell>(cellCount);
   }
 
   /**


[09/22] hbase git commit: HBASE-14192 Fix REST Cluster Constructor with String List

Posted by sy...@apache.org.
HBASE-14192 Fix REST Cluster Constructor with String List


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

Branch: refs/heads/hbase-12439
Commit: abb6cdce718c3c475d76b1941980710cf0c136f7
Parents: 2963d59
Author: Andrew Purtell <ap...@apache.org>
Authored: Wed Feb 10 12:33:56 2016 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Feb 10 12:33:56 2016 -0800

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/abb6cdce/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Cluster.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Cluster.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Cluster.java
index 2ad0541..549a1b2 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Cluster.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Cluster.java
@@ -47,7 +47,7 @@ public class Cluster {
    * @param nodes a list of service locations, in 'host:port' format
    */
   public Cluster(List<String> nodes) {
-    nodes.addAll(nodes);
+    this.nodes.addAll(nodes);
   }
 
   /**


[13/22] hbase git commit: HBASE-15253 Small bug in CellUtil.matchingRow(Cell, byte[]) (Ram)

Posted by sy...@apache.org.
HBASE-15253 Small bug in CellUtil.matchingRow(Cell, byte[]) (Ram)


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

Branch: refs/heads/hbase-12439
Commit: a975408b7c90e2d545a7a490687cddb717d43807
Parents: 29a192e
Author: ramkrishna <ra...@gmail.com>
Authored: Thu Feb 11 21:49:41 2016 +0530
Committer: ramkrishna <ra...@gmail.com>
Committed: Thu Feb 11 21:51:19 2016 +0530

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/a975408b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
index 7242791..85b3913 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
@@ -460,7 +460,7 @@ public final class CellUtil {
 
   public static boolean matchingRow(final Cell left, final byte[] buf) {
     if (buf == null) {
-      return left.getQualifierLength() == 0;
+      return left.getRowLength() == 0;
     }
     return matchingRow(left, buf, 0, buf.length);
   }


[10/22] hbase git commit: HBASE-15223 Make convertScanToString public for Spark

Posted by sy...@apache.org.
HBASE-15223 Make convertScanToString public for Spark


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

Branch: refs/heads/hbase-12439
Commit: 1942a99b831bb4c41c0e09d6b93df5e1d060f58e
Parents: abb6cdc
Author: Jerry He <je...@apache.org>
Authored: Wed Feb 10 15:02:58 2016 -0800
Committer: Jerry He <je...@apache.org>
Committed: Wed Feb 10 15:02:58 2016 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/mapreduce/TableInputFormatBase.java   | 10 +++++-----
 .../hadoop/hbase/mapreduce/TableMapReduceUtil.java     |  4 ++--
 .../org/apache/hadoop/hbase/mapreduce/TableSplit.java  | 13 ++++++++++++-
 3 files changed, 19 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/1942a99b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java
index 918232f..b2f115c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java
@@ -266,7 +266,7 @@ extends InputFormat<ImmutableBytesWritable, Result> {
         }
         List<InputSplit> splits = new ArrayList<InputSplit>(1);
         long regionSize = sizeCalculator.getRegionSize(regLoc.getRegionInfo().getRegionName());
-        TableSplit split = new TableSplit(tableName,
+        TableSplit split = new TableSplit(tableName, scan,
             HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY, regLoc
                 .getHostnamePort().split(Addressing.HOSTNAME_PORT_SEPARATOR)[0], regionSize);
         splits.add(split);
@@ -309,7 +309,7 @@ extends InputFormat<ImmutableBytesWritable, Result> {
   
           byte[] regionName = location.getRegionInfo().getRegionName();
           long regionSize = sizeCalculator.getRegionSize(regionName);
-          TableSplit split = new TableSplit(tableName,
+          TableSplit split = new TableSplit(tableName, scan,
             splitStart, splitStop, regionLocation, regionSize);
           splits.add(split);
           if (LOG.isDebugEnabled()) {
@@ -397,9 +397,9 @@ extends InputFormat<ImmutableBytesWritable, Result> {
         byte[] splitKey = getSplitKey(ts.getStartRow(), ts.getEndRow(), isTextKey);
          //Set the size of child TableSplit as 1/2 of the region size. The exact size of the
          // MapReduce input splits is not far off.
-        TableSplit t1 = new TableSplit(tableName, ts.getStartRow(), splitKey, regionLocation,
+        TableSplit t1 = new TableSplit(tableName, scan, ts.getStartRow(), splitKey, regionLocation,
                 regionSize / 2);
-        TableSplit t2 = new TableSplit(tableName, splitKey, ts.getEndRow(), regionLocation,
+        TableSplit t2 = new TableSplit(tableName, scan, splitKey, ts.getEndRow(), regionLocation,
                 regionSize - regionSize / 2);
         resultList.add(t1);
         resultList.add(t2);
@@ -426,7 +426,7 @@ extends InputFormat<ImmutableBytesWritable, Result> {
             break;
           }
         }
-        TableSplit t = new TableSplit(tableName, splitStartKey, splitEndKey,
+        TableSplit t = new TableSplit(tableName, scan, splitStartKey, splitEndKey,
                 regionLocation, totalSize);
         resultList.add(t);
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1942a99b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
index d43c4d9..37e4e44 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
@@ -561,7 +561,7 @@ public class TableMapReduceUtil {
    * @return The scan saved in a Base64 encoded string.
    * @throws IOException When writing the scan fails.
    */
-  static String convertScanToString(Scan scan) throws IOException {
+  public static String convertScanToString(Scan scan) throws IOException {
     ClientProtos.Scan proto = ProtobufUtil.toScan(scan);
     return Base64.encodeBytes(proto.toByteArray());
   }
@@ -573,7 +573,7 @@ public class TableMapReduceUtil {
    * @return The newly created Scan instance.
    * @throws IOException When reading the scan instance fails.
    */
-  static Scan convertStringToScan(String base64) throws IOException {
+  public static Scan convertStringToScan(String base64) throws IOException {
     byte [] decoded = Base64.decode(base64);
     ClientProtos.Scan scan;
     try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/1942a99b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSplit.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSplit.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSplit.java
index e8e59a2..850db81 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSplit.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSplit.java
@@ -301,7 +301,18 @@ implements Writable, Comparable<TableSplit> {
     StringBuilder sb = new StringBuilder();
     sb.append("HBase table split(");
     sb.append("table name: ").append(tableName);
-    sb.append(", scan: ").append(scan);
+    // null scan input is represented by ""
+    String printScan = "";
+    if (!scan.equals("")) {
+      try {
+        // get the real scan here in toString, not the Base64 string
+        printScan = TableMapReduceUtil.convertStringToScan(scan).toString();
+      }
+      catch (IOException e) {
+        printScan = "";
+      }
+    }
+    sb.append(", scan: ").append(printScan);
     sb.append(", start row: ").append(Bytes.toStringBinary(startRow));
     sb.append(", end row: ").append(Bytes.toStringBinary(endRow));
     sb.append(", region location: ").append(regionLocation);


[15/22] hbase git commit: HBASE-14919 Refactoring for in-memory flush and compaction

Posted by sy...@apache.org.
HBASE-14919 Refactoring for in-memory flush and compaction

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

Branch: refs/heads/hbase-12439
Commit: 25dfc112dd76134a9a3ce1f2e88c4075aef76557
Parents: a975408
Author: eshcar <es...@yahoo-inc.com>
Authored: Mon Feb 8 23:35:02 2016 +0200
Committer: stack <st...@apache.org>
Committed: Thu Feb 11 10:39:01 2016 -0800

----------------------------------------------------------------------
 .../hbase/regionserver/AbstractMemStore.java    | 497 +++++++++++
 .../hadoop/hbase/regionserver/CellSet.java      | 183 ++++
 .../hbase/regionserver/CellSkipListSet.java     | 185 ----
 .../hbase/regionserver/DefaultMemStore.java     | 859 +------------------
 .../hadoop/hbase/regionserver/HStore.java       |  22 +-
 .../hbase/regionserver/ImmutableSegment.java    |  72 ++
 .../regionserver/ImmutableSegmentAdapter.java   | 107 +++
 .../hadoop/hbase/regionserver/MemStore.java     |  16 +-
 .../hbase/regionserver/MemStoreScanner.java     | 348 ++++++++
 .../hbase/regionserver/MemStoreSnapshot.java    |  13 +-
 .../regionserver/MutableCellSetSegment.java     | 153 ++++
 .../MutableCellSetSegmentScanner.java           | 258 ++++++
 .../hbase/regionserver/MutableSegment.java      |  57 ++
 .../hadoop/hbase/regionserver/Segment.java      | 218 +++++
 .../hbase/regionserver/SegmentFactory.java      |  89 ++
 .../hbase/regionserver/SegmentScanner.java      | 152 ++++
 .../hbase/regionserver/StoreFlushContext.java   |   2 +-
 .../apache/hadoop/hbase/io/TestHeapSize.java    |  49 +-
 .../hbase/regionserver/TestCellSkipListSet.java |  13 +-
 .../hbase/regionserver/TestDefaultMemStore.java | 133 +--
 .../hbase/regionserver/TestHMobStore.java       |  29 +-
 .../hadoop/hbase/regionserver/TestHRegion.java  | 150 ++--
 .../regionserver/TestMemStoreChunkPool.java     |  29 +-
 .../hadoop/hbase/regionserver/TestStore.java    |  11 +-
 24 files changed, 2380 insertions(+), 1265 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/25dfc112/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMemStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMemStore.java
new file mode 100644
index 0000000..18d2f8a
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMemStore.java
@@ -0,0 +1,497 @@
+/**
+ *
+ * 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 com.google.common.annotations.VisibleForTesting;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NavigableSet;
+import java.util.SortedSet;
+
+import org.apache.commons.logging.Log;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.ClassSize;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+
+/**
+ * An abstract class, which implements the behaviour shared by all concrete memstore instances.
+ */
+@InterfaceAudience.Private
+public abstract class AbstractMemStore implements MemStore {
+
+  private static final long NO_SNAPSHOT_ID = -1;
+
+  private final Configuration conf;
+  private final CellComparator comparator;
+
+  // active segment absorbs write operations
+  private volatile MutableSegment active;
+  // Snapshot of memstore.  Made for flusher.
+  private volatile ImmutableSegment snapshot;
+  protected volatile long snapshotId;
+  // Used to track when to flush
+  private volatile long timeOfOldestEdit;
+
+  public final static long FIXED_OVERHEAD = ClassSize.align(
+      ClassSize.OBJECT +
+          (4 * ClassSize.REFERENCE) +
+          (2 * Bytes.SIZEOF_LONG));
+
+  public final static long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD +
+      2 * (ClassSize.ATOMIC_LONG + ClassSize.TIMERANGE_TRACKER +
+      ClassSize.CELL_SKIPLIST_SET + ClassSize.CONCURRENT_SKIPLISTMAP));
+
+
+  protected AbstractMemStore(final Configuration conf, final CellComparator c) {
+    this.conf = conf;
+    this.comparator = c;
+    resetCellSet();
+    this.snapshot = SegmentFactory.instance().createImmutableSegment(conf, c, 0);
+    this.snapshotId = NO_SNAPSHOT_ID;
+  }
+
+  protected void resetCellSet() {
+    // Reset heap to not include any keys
+    this.active = SegmentFactory.instance().createMutableSegment(
+        conf, comparator, DEEP_OVERHEAD);
+    this.timeOfOldestEdit = Long.MAX_VALUE;
+  }
+
+  /*
+  * Calculate how the MemStore size has changed.  Includes overhead of the
+  * backing Map.
+  * @param cell
+  * @param notPresent True if the cell was NOT present in the set.
+  * @return change in size
+  */
+  static long heapSizeChange(final Cell cell, final boolean notPresent) {
+    return notPresent ? ClassSize.align(ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY
+        + CellUtil.estimatedHeapSizeOf(cell)) : 0;
+  }
+
+  /**
+   * Updates the wal with the lowest sequence id (oldest entry) that is still in memory
+   * @param onlyIfMoreRecent a flag that marks whether to update the sequence id no matter what or
+   *                      only if it is greater than the previous sequence id
+   */
+  public abstract void updateLowestUnflushedSequenceIdInWal(boolean onlyIfMoreRecent);
+
+  /**
+   * Write an update
+   * @param cell the cell to be added
+   * @return approximate size of the passed cell & newly added cell which maybe different than the
+   *         passed-in cell
+   */
+  @Override
+  public long add(Cell cell) {
+    Cell toAdd = maybeCloneWithAllocator(cell);
+    return internalAdd(toAdd);
+  }
+
+  /**
+   * Update or insert the specified Cells.
+   * <p>
+   * For each Cell, insert into MemStore.  This will atomically upsert the
+   * value for that row/family/qualifier.  If a Cell did already exist,
+   * it will then be removed.
+   * <p>
+   * Currently the memstoreTS is kept at 0 so as each insert happens, it will
+   * be immediately visible.  May want to change this so it is atomic across
+   * all Cells.
+   * <p>
+   * This is called under row lock, so Get operations will still see updates
+   * atomically.  Scans will only see each Cell update as atomic.
+   *
+   * @param cells the cells to be updated
+   * @param readpoint readpoint below which we can safely remove duplicate KVs
+   * @return change in memstore size
+   */
+  @Override
+  public long upsert(Iterable<Cell> cells, long readpoint) {
+    long size = 0;
+    for (Cell cell : cells) {
+      size += upsert(cell, readpoint);
+    }
+    return size;
+  }
+
+  /**
+   * @return Oldest timestamp of all the Cells in the MemStore
+   */
+  @Override
+  public long timeOfOldestEdit() {
+    return timeOfOldestEdit;
+  }
+
+
+  /**
+   * Write a delete
+   * @param deleteCell the cell to be deleted
+   * @return approximate size of the passed key and value.
+   */
+  @Override
+  public long delete(Cell deleteCell) {
+    Cell toAdd = maybeCloneWithAllocator(deleteCell);
+    long s = internalAdd(toAdd);
+    return s;
+  }
+
+  /**
+   * An override on snapshot so the no arg version of the method implies zero seq num,
+   * like for cases without wal
+   */
+  public MemStoreSnapshot snapshot() {
+    return snapshot(0);
+  }
+
+  /**
+   * The passed snapshot was successfully persisted; it can be let go.
+   * @param id Id of the snapshot to clean out.
+   * @see MemStore#snapshot(long)
+   */
+  @Override
+  public void clearSnapshot(long id) throws UnexpectedStateException {
+    if (this.snapshotId != id) {
+      throw new UnexpectedStateException("Current snapshot id is " + this.snapshotId + ",passed "
+          + id);
+    }
+    // OK. Passed in snapshot is same as current snapshot. If not-empty,
+    // create a new snapshot and let the old one go.
+    Segment oldSnapshot = this.snapshot;
+    if (!this.snapshot.isEmpty()) {
+      this.snapshot = SegmentFactory.instance().createImmutableSegment(
+          getComparator(), 0);
+    }
+    this.snapshotId = NO_SNAPSHOT_ID;
+    oldSnapshot.close();
+  }
+
+  /**
+   * Get the entire heap usage for this MemStore not including keys in the
+   * snapshot.
+   */
+  @Override
+  public long heapSize() {
+    return getActive().getSize();
+  }
+
+  /**
+   * On flush, how much memory we will clear from the active cell set.
+   *
+   * @return size of data that is going to be flushed from active set
+   */
+  @Override
+  public long getFlushableSize() {
+    long snapshotSize = getSnapshot().getSize();
+    return snapshotSize > 0 ? snapshotSize : keySize();
+  }
+
+
+  /**
+   * @return a list containing a single memstore scanner.
+   */
+  @Override
+  public List<KeyValueScanner> getScanners(long readPt) throws IOException {
+    return Collections.<KeyValueScanner> singletonList(new MemStoreScanner(this, readPt));
+  }
+
+  @Override
+  public long getSnapshotSize() {
+    return getSnapshot().getSize();
+  }
+
+  @Override
+  public String toString() {
+    StringBuffer buf = new StringBuffer();
+    int i = 1;
+    try {
+      for (Segment segment : getListOfSegments()) {
+        buf.append("Segment (" + i + ") " + segment.toString() + "; ");
+        i++;
+      }
+    } catch (IOException e){
+      return e.toString();
+    }
+    return buf.toString();
+  }
+
+  protected void rollbackInSnapshot(Cell cell) {
+    // If the key is in the snapshot, delete it. We should not update
+    // this.size, because that tracks the size of only the memstore and
+    // not the snapshot. The flush of this snapshot to disk has not
+    // yet started because Store.flush() waits for all rwcc transactions to
+    // commit before starting the flush to disk.
+    snapshot.rollback(cell);
+  }
+
+  protected void rollbackInActive(Cell cell) {
+    // If the key is in the memstore, delete it. Update this.size.
+    long sz = active.rollback(cell);
+    if (sz != 0) {
+      setOldestEditTimeToNow();
+    }
+  }
+
+  protected Configuration getConfiguration() {
+    return conf;
+  }
+
+  protected void dump(Log log) {
+    active.dump(log);
+    snapshot.dump(log);
+  }
+
+
+  /**
+   * Inserts the specified Cell into MemStore and deletes any existing
+   * versions of the same row/family/qualifier as the specified Cell.
+   * <p>
+   * First, the specified Cell is inserted into the Memstore.
+   * <p>
+   * If there are any existing Cell in this MemStore with the same row,
+   * family, and qualifier, they are removed.
+   * <p>
+   * Callers must hold the read lock.
+   *
+   * @param cell the cell to be updated
+   * @param readpoint readpoint below which we can safely remove duplicate KVs
+   * @return change in size of MemStore
+   */
+  private long upsert(Cell cell, long readpoint) {
+    // Add the Cell to the MemStore
+    // Use the internalAdd method here since we (a) already have a lock
+    // and (b) cannot safely use the MSLAB here without potentially
+    // hitting OOME - see TestMemStore.testUpsertMSLAB for a
+    // test that triggers the pathological case if we don't avoid MSLAB
+    // here.
+    long addedSize = internalAdd(cell);
+
+    // Get the Cells for the row/family/qualifier regardless of timestamp.
+    // For this case we want to clean up any other puts
+    Cell firstCell = KeyValueUtil.createFirstOnRow(
+        cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
+        cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
+        cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength());
+    SortedSet<Cell> ss = active.tailSet(firstCell);
+    Iterator<Cell> it = ss.iterator();
+    // versions visible to oldest scanner
+    int versionsVisible = 0;
+    while (it.hasNext()) {
+      Cell cur = it.next();
+
+      if (cell == cur) {
+        // ignore the one just put in
+        continue;
+      }
+      // check that this is the row and column we are interested in, otherwise bail
+      if (CellUtil.matchingRow(cell, cur) && CellUtil.matchingQualifier(cell, cur)) {
+        // only remove Puts that concurrent scanners cannot possibly see
+        if (cur.getTypeByte() == KeyValue.Type.Put.getCode() &&
+            cur.getSequenceId() <= readpoint) {
+          if (versionsVisible >= 1) {
+            // if we get here we have seen at least one version visible to the oldest scanner,
+            // which means we can prove that no scanner will see this version
+
+            // false means there was a change, so give us the size.
+            long delta = heapSizeChange(cur, true);
+            addedSize -= delta;
+            active.incSize(-delta);
+            it.remove();
+            setOldestEditTimeToNow();
+          } else {
+            versionsVisible++;
+          }
+        }
+      } else {
+        // past the row or column, done
+        break;
+      }
+    }
+    return addedSize;
+  }
+
+  /*
+   * @param a
+   * @param b
+   * @return Return lowest of a or b or null if both a and b are null
+   */
+  protected Cell getLowest(final Cell a, final Cell b) {
+    if (a == null) {
+      return b;
+    }
+    if (b == null) {
+      return a;
+    }
+    return comparator.compareRows(a, b) <= 0? a: b;
+  }
+
+  /*
+   * @param key Find row that follows this one.  If null, return first.
+   * @param set Set to look in for a row beyond <code>row</code>.
+   * @return Next row or null if none found.  If one found, will be a new
+   * KeyValue -- can be destroyed by subsequent calls to this method.
+   */
+  protected Cell getNextRow(final Cell key,
+      final NavigableSet<Cell> set) {
+    Cell result = null;
+    SortedSet<Cell> tail = key == null? set: set.tailSet(key);
+    // Iterate until we fall into the next row; i.e. move off current row
+    for (Cell cell: tail) {
+      if (comparator.compareRows(cell, key) <= 0) {
+        continue;
+      }
+      // Note: Not suppressing deletes or expired cells.  Needs to be handled
+      // by higher up functions.
+      result = cell;
+      break;
+    }
+    return result;
+  }
+
+  /**
+   * Given the specs of a column, update it, first by inserting a new record,
+   * then removing the old one.  Since there is only 1 KeyValue involved, the memstoreTS
+   * will be set to 0, thus ensuring that they instantly appear to anyone. The underlying
+   * store will ensure that the insert/delete each are atomic. A scanner/reader will either
+   * get the new value, or the old value and all readers will eventually only see the new
+   * value after the old was removed.
+   */
+  @VisibleForTesting
+  @Override
+  public long updateColumnValue(byte[] row, byte[] family, byte[] qualifier,
+      long newValue, long now) {
+    Cell firstCell = KeyValueUtil.createFirstOnRow(row, family, qualifier);
+    // Is there a Cell in 'snapshot' with the same TS? If so, upgrade the timestamp a bit.
+    Cell snc = snapshot.getFirstAfter(firstCell);
+    if(snc != null) {
+      // is there a matching Cell in the snapshot?
+      if (CellUtil.matchingRow(snc, firstCell) && CellUtil.matchingQualifier(snc, firstCell)) {
+        if (snc.getTimestamp() == now) {
+          now += 1;
+        }
+      }
+    }
+    // logic here: the new ts MUST be at least 'now'. But it could be larger if necessary.
+    // But the timestamp should also be max(now, mostRecentTsInMemstore)
+
+    // so we cant add the new Cell w/o knowing what's there already, but we also
+    // want to take this chance to delete some cells. So two loops (sad)
+
+    SortedSet<Cell> ss = getActive().tailSet(firstCell);
+    for (Cell cell : ss) {
+      // if this isnt the row we are interested in, then bail:
+      if (!CellUtil.matchingColumn(cell, family, qualifier)
+          || !CellUtil.matchingRow(cell, firstCell)) {
+        break; // rows dont match, bail.
+      }
+
+      // if the qualifier matches and it's a put, just RM it out of the active.
+      if (cell.getTypeByte() == KeyValue.Type.Put.getCode() &&
+          cell.getTimestamp() > now && CellUtil.matchingQualifier(firstCell, cell)) {
+        now = cell.getTimestamp();
+      }
+    }
+
+    // create or update (upsert) a new Cell with
+    // 'now' and a 0 memstoreTS == immediately visible
+    List<Cell> cells = new ArrayList<Cell>(1);
+    cells.add(new KeyValue(row, family, qualifier, now, Bytes.toBytes(newValue)));
+    return upsert(cells, 1L);
+  }
+
+  private Cell maybeCloneWithAllocator(Cell cell) {
+    return active.maybeCloneWithAllocator(cell);
+  }
+
+  /**
+   * Internal version of add() that doesn't clone Cells with the
+   * allocator, and doesn't take the lock.
+   *
+   * Callers should ensure they already have the read lock taken
+   */
+  private long internalAdd(final Cell toAdd) {
+    long s = active.add(toAdd);
+    setOldestEditTimeToNow();
+    checkActiveSize();
+    return s;
+  }
+
+  private void setOldestEditTimeToNow() {
+    if (timeOfOldestEdit == Long.MAX_VALUE) {
+      timeOfOldestEdit = EnvironmentEdgeManager.currentTime();
+    }
+  }
+
+  protected long keySize() {
+    return heapSize() - DEEP_OVERHEAD;
+  }
+
+  protected CellComparator getComparator() {
+    return comparator;
+  }
+
+  protected MutableSegment getActive() {
+    return active;
+  }
+
+  protected ImmutableSegment getSnapshot() {
+    return snapshot;
+  }
+
+  protected AbstractMemStore setSnapshot(ImmutableSegment snapshot) {
+    this.snapshot = snapshot;
+    return this;
+  }
+
+  protected void setSnapshotSize(long snapshotSize) {
+    getSnapshot().setSize(snapshotSize);
+  }
+
+  /**
+   * Check whether anything need to be done based on the current active set size
+   */
+  protected abstract void checkActiveSize();
+
+  /**
+   * Returns a list of Store segment scanners, one per each store segment
+   * @param readPt the version number required to initialize the scanners
+   * @return a list of Store segment scanners, one per each store segment
+   */
+  protected abstract List<SegmentScanner> getListOfScanners(long readPt) throws IOException;
+
+  /**
+   * Returns an ordered list of segments from most recent to oldest in memstore
+   * @return an ordered list of segments from most recent to oldest in memstore
+   */
+  protected abstract List<Segment> getListOfSegments() throws IOException;
+
+  public long getActiveSize() {
+    return getActive().getSize();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/25dfc112/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellSet.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellSet.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellSet.java
new file mode 100644
index 0000000..4433302
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellSet.java
@@ -0,0 +1,183 @@
+/**
+ *
+ * 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.util.Collection;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.NavigableSet;
+import java.util.SortedSet;
+import java.util.concurrent.ConcurrentNavigableMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * A {@link java.util.Set} of {@link Cell}s, where an add will overwrite the entry if already
+ * exists in the set.  The call to add returns true if no value in the backing map or false if
+ * there was an entry with same key (though value may be different).
+ * implementation is tolerant of concurrent get and set and won't throw
+ * ConcurrentModificationException when iterating.
+ */
+@InterfaceAudience.Private
+public class CellSet implements NavigableSet<Cell>  {
+  // Implemented on top of a {@link java.util.concurrent.ConcurrentSkipListMap}
+  // Differ from CSLS in one respect, where CSLS does "Adds the specified element to this set if it
+  // is not already present.", this implementation "Adds the specified element to this set EVEN
+  // if it is already present overwriting what was there previous".
+  // Otherwise, has same attributes as ConcurrentSkipListSet
+  private final ConcurrentNavigableMap<Cell, Cell> delegatee;
+
+  CellSet(final CellComparator c) {
+    this.delegatee = new ConcurrentSkipListMap<Cell, Cell>(c);
+  }
+
+  CellSet(final ConcurrentNavigableMap<Cell, Cell> m) {
+    this.delegatee = m;
+  }
+
+  public Cell ceiling(Cell e) {
+    throw new UnsupportedOperationException("Not implemented");
+  }
+
+  public Iterator<Cell> descendingIterator() {
+    return this.delegatee.descendingMap().values().iterator();
+  }
+
+  public NavigableSet<Cell> descendingSet() {
+    throw new UnsupportedOperationException("Not implemented");
+  }
+
+  public Cell floor(Cell e) {
+    throw new UnsupportedOperationException("Not implemented");
+  }
+
+  public SortedSet<Cell> headSet(final Cell toElement) {
+    return headSet(toElement, false);
+  }
+
+  public NavigableSet<Cell> headSet(final Cell toElement,
+      boolean inclusive) {
+    return new CellSet(this.delegatee.headMap(toElement, inclusive));
+  }
+
+  public Cell higher(Cell e) {
+    throw new UnsupportedOperationException("Not implemented");
+  }
+
+  public Iterator<Cell> iterator() {
+    return this.delegatee.values().iterator();
+  }
+
+  public Cell lower(Cell e) {
+    throw new UnsupportedOperationException("Not implemented");
+  }
+
+  public Cell pollFirst() {
+    throw new UnsupportedOperationException("Not implemented");
+  }
+
+  public Cell pollLast() {
+    throw new UnsupportedOperationException("Not implemented");
+  }
+
+  public SortedSet<Cell> subSet(Cell fromElement, Cell toElement) {
+    throw new UnsupportedOperationException("Not implemented");
+  }
+
+  public NavigableSet<Cell> subSet(Cell fromElement,
+      boolean fromInclusive, Cell toElement, boolean toInclusive) {
+    throw new UnsupportedOperationException("Not implemented");
+  }
+
+  public SortedSet<Cell> tailSet(Cell fromElement) {
+    return tailSet(fromElement, true);
+  }
+
+  public NavigableSet<Cell> tailSet(Cell fromElement, boolean inclusive) {
+    return new CellSet(this.delegatee.tailMap(fromElement, inclusive));
+  }
+
+  public Comparator<? super Cell> comparator() {
+    throw new UnsupportedOperationException("Not implemented");
+  }
+
+  public Cell first() {
+    return this.delegatee.get(this.delegatee.firstKey());
+  }
+
+  public Cell last() {
+    return this.delegatee.get(this.delegatee.lastKey());
+  }
+
+  public boolean add(Cell e) {
+    return this.delegatee.put(e, e) == null;
+  }
+
+  public boolean addAll(Collection<? extends Cell> c) {
+    throw new UnsupportedOperationException("Not implemented");
+  }
+
+  public void clear() {
+    this.delegatee.clear();
+  }
+
+  public boolean contains(Object o) {
+    //noinspection SuspiciousMethodCalls
+    return this.delegatee.containsKey(o);
+  }
+
+  public boolean containsAll(Collection<?> c) {
+    throw new UnsupportedOperationException("Not implemented");
+  }
+
+  public boolean isEmpty() {
+    return this.delegatee.isEmpty();
+  }
+
+  public boolean remove(Object o) {
+    return this.delegatee.remove(o) != null;
+  }
+
+  public boolean removeAll(Collection<?> c) {
+    throw new UnsupportedOperationException("Not implemented");
+  }
+
+  public boolean retainAll(Collection<?> c) {
+    throw new UnsupportedOperationException("Not implemented");
+  }
+
+  public Cell get(Cell kv) {
+    return this.delegatee.get(kv);
+  }
+
+  public int size() {
+    return this.delegatee.size();
+  }
+
+  public Object[] toArray() {
+    throw new UnsupportedOperationException("Not implemented");
+  }
+
+  public <T> T[] toArray(T[] a) {
+    throw new UnsupportedOperationException("Not implemented");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/25dfc112/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellSkipListSet.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellSkipListSet.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellSkipListSet.java
deleted file mode 100644
index e9941b3..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellSkipListSet.java
+++ /dev/null
@@ -1,185 +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 java.util.Collection;
-import java.util.Comparator;
-import java.util.Iterator;
-import java.util.NavigableSet;
-import java.util.SortedSet;
-import java.util.concurrent.ConcurrentNavigableMap;
-import java.util.concurrent.ConcurrentSkipListMap;
-
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellComparator;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-
-/**
- * A {@link java.util.Set} of {@link Cell}s implemented on top of a
- * {@link java.util.concurrent.ConcurrentSkipListMap}.  Works like a
- * {@link java.util.concurrent.ConcurrentSkipListSet} in all but one regard:
- * An add will overwrite if already an entry for the added key.  In other words,
- * where CSLS does "Adds the specified element to this set if it is not already
- * present.", this implementation "Adds the specified element to this set EVEN
- * if it is already present overwriting what was there previous".  The call to
- * add returns true if no value in the backing map or false if there was an
- * entry with same key (though value may be different).
- * <p>Otherwise,
- * has same attributes as ConcurrentSkipListSet: e.g. tolerant of concurrent
- * get and set and won't throw ConcurrentModificationException when iterating.
- */
-@InterfaceAudience.Private
-public class CellSkipListSet implements NavigableSet<Cell> {
-  private final ConcurrentNavigableMap<Cell, Cell> delegatee;
-
-  CellSkipListSet(final CellComparator c) {
-    this.delegatee = new ConcurrentSkipListMap<Cell, Cell>(c);
-  }
-
-  CellSkipListSet(final ConcurrentNavigableMap<Cell, Cell> m) {
-    this.delegatee = m;
-  }
-
-  public Cell ceiling(Cell e) {
-    throw new UnsupportedOperationException("Not implemented");
-  }
-
-  public Iterator<Cell> descendingIterator() {
-    return this.delegatee.descendingMap().values().iterator();
-  }
-
-  public NavigableSet<Cell> descendingSet() {
-    throw new UnsupportedOperationException("Not implemented");
-  }
-
-  public Cell floor(Cell e) {
-    throw new UnsupportedOperationException("Not implemented");
-  }
-
-  public SortedSet<Cell> headSet(final Cell toElement) {
-    return headSet(toElement, false);
-  }
-
-  public NavigableSet<Cell> headSet(final Cell toElement,
-      boolean inclusive) {
-    return new CellSkipListSet(this.delegatee.headMap(toElement, inclusive));
-  }
-
-  public Cell higher(Cell e) {
-    throw new UnsupportedOperationException("Not implemented");
-  }
-
-  public Iterator<Cell> iterator() {
-    return this.delegatee.values().iterator();
-  }
-
-  public Cell lower(Cell e) {
-    throw new UnsupportedOperationException("Not implemented");
-  }
-
-  public Cell pollFirst() {
-    throw new UnsupportedOperationException("Not implemented");
-  }
-
-  public Cell pollLast() {
-    throw new UnsupportedOperationException("Not implemented");
-  }
-
-  public SortedSet<Cell> subSet(Cell fromElement, Cell toElement) {
-    throw new UnsupportedOperationException("Not implemented");
-  }
-
-  public NavigableSet<Cell> subSet(Cell fromElement,
-      boolean fromInclusive, Cell toElement, boolean toInclusive) {
-    throw new UnsupportedOperationException("Not implemented");
-  }
-
-  public SortedSet<Cell> tailSet(Cell fromElement) {
-    return tailSet(fromElement, true);
-  }
-
-  public NavigableSet<Cell> tailSet(Cell fromElement, boolean inclusive) {
-    return new CellSkipListSet(this.delegatee.tailMap(fromElement, inclusive));
-  }
-
-  public Comparator<? super Cell> comparator() {
-    throw new UnsupportedOperationException("Not implemented");
-  }
-
-  public Cell first() {
-    return this.delegatee.get(this.delegatee.firstKey());
-  }
-
-  public Cell last() {
-    return this.delegatee.get(this.delegatee.lastKey());
-  }
-
-  public boolean add(Cell e) {
-    return this.delegatee.put(e, e) == null;
-  }
-
-  public boolean addAll(Collection<? extends Cell> c) {
-    throw new UnsupportedOperationException("Not implemented");
-  }
-
-  public void clear() {
-    this.delegatee.clear();
-  }
-
-  public boolean contains(Object o) {
-    //noinspection SuspiciousMethodCalls
-    return this.delegatee.containsKey(o);
-  }
-
-  public boolean containsAll(Collection<?> c) {
-    throw new UnsupportedOperationException("Not implemented");
-  }
-
-  public boolean isEmpty() {
-    return this.delegatee.isEmpty();
-  }
-
-  public boolean remove(Object o) {
-    return this.delegatee.remove(o) != null;
-  }
-
-  public boolean removeAll(Collection<?> c) {
-    throw new UnsupportedOperationException("Not implemented");
-  }
-
-  public boolean retainAll(Collection<?> c) {
-    throw new UnsupportedOperationException("Not implemented");
-  }
-
-  public Cell get(Cell kv) {
-    return this.delegatee.get(kv);
-  }
-
-  public int size() {
-    return this.delegatee.size();
-  }
-
-  public Object[] toArray() {
-    throw new UnsupportedOperationException("Not implemented");
-  }
-
-  public <T> T[] toArray(T[] a) {
-    throw new UnsupportedOperationException("Not implemented");
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/25dfc112/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java
index f61d871..82d40b6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java
@@ -19,35 +19,22 @@
 
 package org.apache.hadoop.hbase.regionserver;
 
+import java.io.IOException;
 import java.lang.management.ManagementFactory;
 import java.lang.management.RuntimeMXBean;
 import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Iterator;
 import java.util.List;
-import java.util.NavigableSet;
-import java.util.SortedSet;
-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.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
-import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.io.TimeRange;
-import org.apache.hadoop.hbase.util.ByteRange;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.ClassSize;
-import org.apache.hadoop.hbase.util.CollectionBackedScanner;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.ReflectionUtils;
-import org.apache.htrace.Trace;
 
 /**
  * The MemStore holds in-memory modifications to the Store.  Modifications
@@ -66,40 +53,8 @@ import org.apache.htrace.Trace;
  * in KV size.
  */
 @InterfaceAudience.Private
-public class DefaultMemStore implements MemStore {
+public class DefaultMemStore extends AbstractMemStore {
   private static final Log LOG = LogFactory.getLog(DefaultMemStore.class);
-  static final String USEMSLAB_KEY = "hbase.hregion.memstore.mslab.enabled";
-  private static final boolean USEMSLAB_DEFAULT = true;
-  static final String MSLAB_CLASS_NAME = "hbase.regionserver.mslab.class";
-
-  private Configuration conf;
-
-  // MemStore.  Use a CellSkipListSet rather than SkipListSet because of the
-  // better semantics.  The Map will overwrite if passed a key it already had
-  // whereas the Set will not add new Cell if key is same though value might be
-  // different.  Value is not important -- just make sure always same
-  // reference passed.
-  volatile CellSkipListSet cellSet;
-
-  // Snapshot of memstore.  Made for flusher.
-  volatile CellSkipListSet snapshot;
-
-  final CellComparator comparator;
-
-  // Used to track own heapSize
-  final AtomicLong size;
-  private volatile long snapshotSize;
-
-  // Used to track when to flush
-  volatile long timeOfOldestEdit = Long.MAX_VALUE;
-
-  TimeRangeTracker timeRangeTracker;
-  TimeRangeTracker snapshotTimeRangeTracker;
-
-  volatile MemStoreLAB allocator;
-  volatile MemStoreLAB snapshotAllocator;
-  volatile long snapshotId;
-  volatile boolean tagsPresent;
 
   /**
    * Default constructor. Used for tests.
@@ -112,183 +67,54 @@ public class DefaultMemStore implements MemStore {
    * Constructor.
    * @param c Comparator
    */
-  public DefaultMemStore(final Configuration conf,
-                  final CellComparator c) {
-    this.conf = conf;
-    this.comparator = c;
-    this.cellSet = new CellSkipListSet(c);
-    this.snapshot = new CellSkipListSet(c);
-    timeRangeTracker = new TimeRangeTracker();
-    snapshotTimeRangeTracker = new TimeRangeTracker();
-    this.size = new AtomicLong(DEEP_OVERHEAD);
-    this.snapshotSize = 0;
-    if (conf.getBoolean(USEMSLAB_KEY, USEMSLAB_DEFAULT)) {
-      String className = conf.get(MSLAB_CLASS_NAME, HeapMemStoreLAB.class.getName());
-      this.allocator = ReflectionUtils.instantiateWithCustomCtor(className,
-          new Class[] { Configuration.class }, new Object[] { conf });
-    } else {
-      this.allocator = null;
-    }
+  public DefaultMemStore(final Configuration conf, final CellComparator c) {
+    super(conf, c);
   }
 
   void dump() {
-    for (Cell cell: this.cellSet) {
-      LOG.info(cell);
-    }
-    for (Cell cell: this.snapshot) {
-      LOG.info(cell);
-    }
+    super.dump(LOG);
   }
 
   /**
    * Creates a snapshot of the current memstore.
    * Snapshot must be cleared by call to {@link #clearSnapshot(long)}
+   * @param flushOpSeqId the sequence id that is attached to the flush operation in the wal
    */
   @Override
-  public MemStoreSnapshot snapshot() {
+  public MemStoreSnapshot snapshot(long flushOpSeqId) {
     // If snapshot currently has entries, then flusher failed or didn't call
     // cleanup.  Log a warning.
-    if (!this.snapshot.isEmpty()) {
+    if (!getSnapshot().isEmpty()) {
       LOG.warn("Snapshot called again without clearing previous. " +
           "Doing nothing. Another ongoing flush or did we fail last attempt?");
     } else {
       this.snapshotId = EnvironmentEdgeManager.currentTime();
-      this.snapshotSize = keySize();
-      if (!this.cellSet.isEmpty()) {
-        this.snapshot = this.cellSet;
-        this.cellSet = new CellSkipListSet(this.comparator);
-        this.snapshotTimeRangeTracker = this.timeRangeTracker;
-        this.timeRangeTracker = new TimeRangeTracker();
-        // Reset heap to not include any keys
-        this.size.set(DEEP_OVERHEAD);
-        this.snapshotAllocator = this.allocator;
-        // Reset allocator so we get a fresh buffer for the new memstore
-        if (allocator != null) {
-          String className = conf.get(MSLAB_CLASS_NAME, HeapMemStoreLAB.class.getName());
-          this.allocator = ReflectionUtils.instantiateWithCustomCtor(className,
-              new Class[] { Configuration.class }, new Object[] { conf });
-        }
-        timeOfOldestEdit = Long.MAX_VALUE;
+      if (!getActive().isEmpty()) {
+        ImmutableSegment immutableSegment = SegmentFactory.instance().
+            createImmutableSegment(getConfiguration(), getActive());
+        setSnapshot(immutableSegment);
+        setSnapshotSize(keySize());
+        resetCellSet();
       }
     }
-    MemStoreSnapshot memStoreSnapshot = new MemStoreSnapshot(this.snapshotId, snapshot.size(), this.snapshotSize,
-        this.snapshotTimeRangeTracker, new CollectionBackedScanner(snapshot, this.comparator),
-        this.tagsPresent);
-    this.tagsPresent = false;
-    return memStoreSnapshot;
-  }
-
-  /**
-   * The passed snapshot was successfully persisted; it can be let go.
-   * @param id Id of the snapshot to clean out.
-   * @throws UnexpectedStateException
-   * @see #snapshot()
-   */
-  @Override
-  public void clearSnapshot(long id) throws UnexpectedStateException {
-    MemStoreLAB tmpAllocator = null;
-    if (this.snapshotId != id) {
-      throw new UnexpectedStateException("Current snapshot id is " + this.snapshotId + ",passed "
-          + id);
-    }
-    // OK. Passed in snapshot is same as current snapshot. If not-empty,
-    // create a new snapshot and let the old one go.
-    if (!this.snapshot.isEmpty()) {
-      this.snapshot = new CellSkipListSet(this.comparator);
-      this.snapshotTimeRangeTracker = new TimeRangeTracker();
-    }
-    this.snapshotSize = 0;
-    this.snapshotId = -1;
-    if (this.snapshotAllocator != null) {
-      tmpAllocator = this.snapshotAllocator;
-      this.snapshotAllocator = null;
-    }
-    if (tmpAllocator != null) {
-      tmpAllocator.close();
-    }
-  }
-
-  @Override
-  public long getFlushableSize() {
-    return this.snapshotSize > 0 ? this.snapshotSize : keySize();
-  }
+    return new MemStoreSnapshot(this.snapshotId, getSnapshot());
 
-  @Override
-  public long getSnapshotSize() {
-    return this.snapshotSize;
   }
 
-  /**
-   * Write an update
-   * @param cell
-   * @return approximate size of the passed Cell.
-   */
   @Override
-  public long add(Cell cell) {
-    Cell toAdd = maybeCloneWithAllocator(cell);
-    return internalAdd(toAdd);
+  protected List<SegmentScanner> getListOfScanners(long readPt) throws IOException {
+    List<SegmentScanner> list = new ArrayList<SegmentScanner>(2);
+    list.add(0, getActive().getSegmentScanner(readPt));
+    list.add(1, getSnapshot().getSegmentScanner(readPt));
+    return list;
   }
 
   @Override
-  public long timeOfOldestEdit() {
-    return timeOfOldestEdit;
-  }
-
-  private boolean addToCellSet(Cell e) {
-    boolean b = this.cellSet.add(e);
-    // In no tags case this NoTagsKeyValue.getTagsLength() is a cheap call.
-    // When we use ACL CP or Visibility CP which deals with Tags during
-    // mutation, the TagRewriteCell.getTagsLength() is a cheaper call. We do not
-    // parse the byte[] to identify the tags length.
-    if(e.getTagsLength() > 0) {
-      tagsPresent = true;
-    }
-    setOldestEditTimeToNow();
-    return b;
-  }
-
-  private boolean removeFromCellSet(Cell e) {
-    boolean b = this.cellSet.remove(e);
-    setOldestEditTimeToNow();
-    return b;
-  }
-
-  void setOldestEditTimeToNow() {
-    if (timeOfOldestEdit == Long.MAX_VALUE) {
-      timeOfOldestEdit = EnvironmentEdgeManager.currentTime();
-    }
-  }
-
-  /**
-   * Internal version of add() that doesn't clone Cells with the
-   * allocator, and doesn't take the lock.
-   *
-   * Callers should ensure they already have the read lock taken
-   */
-  private long internalAdd(final Cell toAdd) {
-    long s = heapSizeChange(toAdd, addToCellSet(toAdd));
-    timeRangeTracker.includeTimestamp(toAdd);
-    this.size.addAndGet(s);
-    return s;
-  }
-
-  private Cell maybeCloneWithAllocator(Cell cell) {
-    if (allocator == null) {
-      return cell;
-    }
-
-    int len = KeyValueUtil.length(cell);
-    ByteRange alloc = allocator.allocateBytes(len);
-    if (alloc == null) {
-      // The allocation was too large, allocator decided
-      // not to do anything with it.
-      return cell;
-    }
-    assert alloc.getBytes() != null;
-    KeyValueUtil.appendToByteArray(cell, alloc.getBytes(), alloc.getOffset());
-    KeyValue newKv = new KeyValue(alloc.getBytes(), alloc.getOffset(), len);
-    newKv.setSequenceId(cell.getSequenceId());
-    return newKv;
+  protected List<Segment> getListOfSegments() throws IOException {
+    List<Segment> list = new ArrayList<Segment>(2);
+    list.add(0, getActive());
+    list.add(1, getSnapshot());
+    return list;
   }
 
   /**
@@ -301,39 +127,8 @@ public class DefaultMemStore implements MemStore {
    */
   @Override
   public void rollback(Cell cell) {
-    // If the key is in the snapshot, delete it. We should not update
-    // this.size, because that tracks the size of only the memstore and
-    // not the snapshot. The flush of this snapshot to disk has not
-    // yet started because Store.flush() waits for all rwcc transactions to
-    // commit before starting the flush to disk.
-    Cell found = this.snapshot.get(cell);
-    if (found != null && found.getSequenceId() == cell.getSequenceId()) {
-      this.snapshot.remove(cell);
-      long sz = heapSizeChange(cell, true);
-      this.snapshotSize -= sz;
-    }
-    // If the key is in the memstore, delete it. Update this.size.
-    found = this.cellSet.get(cell);
-    if (found != null && found.getSequenceId() == cell.getSequenceId()) {
-      removeFromCellSet(cell);
-      long s = heapSizeChange(cell, true);
-      this.size.addAndGet(-s);
-    }
-  }
-
-  /**
-   * Write a delete
-   * @param deleteCell
-   * @return approximate size of the passed key and value.
-   */
-  @Override
-  public long delete(Cell deleteCell) {
-    long s = 0;
-    Cell toAdd = maybeCloneWithAllocator(deleteCell);
-    s += heapSizeChange(toAdd, addToCellSet(toAdd));
-    timeRangeTracker.includeTimestamp(toAdd);
-    this.size.addAndGet(s);
-    return s;
+    rollbackInSnapshot(cell);
+    rollbackInActive(cell);
   }
 
   /**
@@ -342,604 +137,29 @@ public class DefaultMemStore implements MemStore {
    * @return Next row or null if none found.
    */
   Cell getNextRow(final Cell cell) {
-    return getLowest(getNextRow(cell, this.cellSet), getNextRow(cell, this.snapshot));
-  }
-
-  /*
-   * @param a
-   * @param b
-   * @return Return lowest of a or b or null if both a and b are null
-   */
-  private Cell getLowest(final Cell a, final Cell b) {
-    if (a == null) {
-      return b;
-    }
-    if (b == null) {
-      return a;
-    }
-    return comparator.compareRows(a, b) <= 0? a: b;
+    return getLowest(
+        getNextRow(cell, getActive().getCellSet()),
+        getNextRow(cell, getSnapshot().getCellSet()));
   }
 
-  /*
-   * @param key Find row that follows this one.  If null, return first.
-   * @param map Set to look in for a row beyond <code>row</code>.
-   * @return Next row or null if none found.  If one found, will be a new
-   * KeyValue -- can be destroyed by subsequent calls to this method.
-   */
-  private Cell getNextRow(final Cell key,
-      final NavigableSet<Cell> set) {
-    Cell result = null;
-    SortedSet<Cell> tail = key == null? set: set.tailSet(key);
-    // Iterate until we fall into the next row; i.e. move off current row
-    for (Cell cell: tail) {
-      if (comparator.compareRows(cell, key) <= 0)
-        continue;
-      // Note: Not suppressing deletes or expired cells.  Needs to be handled
-      // by higher up functions.
-      result = cell;
-      break;
-    }
-    return result;
+  @Override public void updateLowestUnflushedSequenceIdInWal(boolean onlyIfMoreRecent) {
   }
 
   /**
-   * Only used by tests. TODO: Remove
-   *
-   * Given the specs of a column, update it, first by inserting a new record,
-   * then removing the old one.  Since there is only 1 KeyValue involved, the memstoreTS
-   * will be set to 0, thus ensuring that they instantly appear to anyone. The underlying
-   * store will ensure that the insert/delete each are atomic. A scanner/reader will either
-   * get the new value, or the old value and all readers will eventually only see the new
-   * value after the old was removed.
-   *
-   * @param row
-   * @param family
-   * @param qualifier
-   * @param newValue
-   * @param now
-   * @return  Timestamp
+   * @return Total memory occupied by this MemStore.
    */
   @Override
-  public long updateColumnValue(byte[] row,
-                                byte[] family,
-                                byte[] qualifier,
-                                long newValue,
-                                long now) {
-    Cell firstCell = KeyValueUtil.createFirstOnRow(row, family, qualifier);
-    // Is there a Cell in 'snapshot' with the same TS? If so, upgrade the timestamp a bit.
-    SortedSet<Cell> snSs = snapshot.tailSet(firstCell);
-    if (!snSs.isEmpty()) {
-      Cell snc = snSs.first();
-      // is there a matching Cell in the snapshot?
-      if (CellUtil.matchingRow(snc, firstCell) && CellUtil.matchingQualifier(snc, firstCell)) {
-        if (snc.getTimestamp() == now) {
-          // poop,
-          now += 1;
-        }
-      }
-    }
-
-    // logic here: the new ts MUST be at least 'now'. But it could be larger if necessary.
-    // But the timestamp should also be max(now, mostRecentTsInMemstore)
-
-    // so we cant add the new Cell w/o knowing what's there already, but we also
-    // want to take this chance to delete some cells. So two loops (sad)
-
-    SortedSet<Cell> ss = cellSet.tailSet(firstCell);
-    for (Cell cell : ss) {
-      // if this isnt the row we are interested in, then bail:
-      if (!CellUtil.matchingColumn(cell, family, qualifier)
-          || !CellUtil.matchingRow(cell, firstCell)) {
-        break; // rows dont match, bail.
-      }
-
-      // if the qualifier matches and it's a put, just RM it out of the cellSet.
-      if (cell.getTypeByte() == KeyValue.Type.Put.getCode() &&
-          cell.getTimestamp() > now && CellUtil.matchingQualifier(firstCell, cell)) {
-        now = cell.getTimestamp();
-      }
-    }
-
-    // create or update (upsert) a new Cell with
-    // 'now' and a 0 memstoreTS == immediately visible
-    List<Cell> cells = new ArrayList<Cell>(1);
-    cells.add(new KeyValue(row, family, qualifier, now, Bytes.toBytes(newValue)));
-    return upsert(cells, 1L);
-  }
-
-  /**
-   * Update or insert the specified KeyValues.
-   * <p>
-   * For each KeyValue, insert into MemStore.  This will atomically upsert the
-   * value for that row/family/qualifier.  If a KeyValue did already exist,
-   * it will then be removed.
-   * <p>
-   * This is called under row lock, so Get operations will still see updates
-   * atomically.  Scans will only see each KeyValue update as atomic.
-   *
-   * @param readpoint readpoint below which we can safely remove duplicate KVs
-   * @return change in memstore size
-   */
-  @Override
-  public long upsert(Iterable<Cell> cells, long readpoint) {
-    long size = 0;
-    for (Cell cell : cells) {
-      size += upsert(cell, readpoint);
-    }
-    return size;
-  }
-
-  /**
-   * Inserts the specified KeyValue into MemStore and deletes any existing
-   * versions of the same row/family/qualifier as the specified KeyValue.
-   * <p>
-   * First, the specified KeyValue is inserted into the Memstore.
-   * <p>
-   * If there are any existing KeyValues in this MemStore with the same row,
-   * family, and qualifier, they are removed.
-   * <p>
-   * Callers must hold the read lock.
-   * @param readpoint Smallest outstanding readpoint; below which we can remove duplicate Cells.
-   * @return change in size of MemStore
-   */
-  private long upsert(Cell cell, long readpoint) {
-    // Add the Cell to the MemStore
-    // Use the internalAdd method here since we (a) already have a lock
-    // and (b) cannot safely use the MSLAB here without potentially
-    // hitting OOME - see TestMemStore.testUpsertMSLAB for a
-    // test that triggers the pathological case if we don't avoid MSLAB
-    // here.
-    long addedSize = internalAdd(cell);
-
-    // Get the Cells for the row/family/qualifier regardless of timestamp.
-    // For this case we want to clean up any other puts
-    Cell firstCell = KeyValueUtil.createFirstOnRow(
-        cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
-        cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
-        cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength());
-    SortedSet<Cell> ss = cellSet.tailSet(firstCell);
-    Iterator<Cell> it = ss.iterator();
-    // Versions visible to oldest scanner.
-    int versionsVisible = 0;
-    while ( it.hasNext() ) {
-      Cell cur = it.next();
-
-      if (cell == cur) {
-        // ignore the one just put in
-        continue;
-      }
-      // check that this is the row and column we are interested in, otherwise bail
-      if (CellUtil.matchingRow(cell, cur) && CellUtil.matchingQualifier(cell, cur)) {
-        // only remove Puts that concurrent scanners cannot possibly see
-        if (cur.getTypeByte() == KeyValue.Type.Put.getCode() &&
-            cur.getSequenceId() <= readpoint) {
-          if (versionsVisible >= 1) {
-            // if we get here we have seen at least one version visible to the oldest scanner,
-            // which means we can prove that no scanner will see this version
-
-            // false means there was a change, so give us the size.
-            long delta = heapSizeChange(cur, true);
-            addedSize -= delta;
-            this.size.addAndGet(-delta);
-            it.remove();
-            setOldestEditTimeToNow();
-          } else {
-            versionsVisible++;
-          }
-        }
-      } else {
-        // past the row or column, done
-        break;
-      }
-    }
-    return addedSize;
-  }
-
-  /**
-   * @return scanner on memstore and snapshot in this order.
-   */
-  @Override
-  public List<KeyValueScanner> getScanners(long readPt) {
-    return Collections.<KeyValueScanner> singletonList(new MemStoreScanner(readPt));
-  }
-
-  /**
-   * Check if this memstore may contain the required keys
-   * @param scan scan
-   * @param store holds reference to cf
-   * @param oldestUnexpiredTS
-   * @return False if the key definitely does not exist in this Memstore
-   */
-  public boolean shouldSeek(Scan scan, Store store, long oldestUnexpiredTS) {
-    byte[] cf = store.getFamily().getName();
-    TimeRange timeRange = scan.getColumnFamilyTimeRange().get(cf);
-    if (timeRange == null) {
-      timeRange = scan.getTimeRange();
-    }
-    return (timeRangeTracker.includesTimeRange(timeRange) ||
-        snapshotTimeRangeTracker.includesTimeRange(timeRange))
-        && (Math.max(timeRangeTracker.getMaximumTimestamp(),
-                     snapshotTimeRangeTracker.getMaximumTimestamp()) >=
-            oldestUnexpiredTS);
-  }
-
-  /*
-   * MemStoreScanner implements the KeyValueScanner.
-   * It lets the caller scan the contents of a memstore -- both current
-   * map and snapshot.
-   * This behaves as if it were a real scanner but does not maintain position.
-   */
-  protected class MemStoreScanner extends NonLazyKeyValueScanner {
-    // Next row information for either cellSet or snapshot
-    private Cell cellSetNextRow = null;
-    private Cell snapshotNextRow = null;
-
-    // last iterated Cells for cellSet and snapshot (to restore iterator state after reseek)
-    private Cell cellSetItRow = null;
-    private Cell snapshotItRow = null;
-
-    // iterator based scanning.
-    private Iterator<Cell> cellSetIt;
-    private Iterator<Cell> snapshotIt;
-
-    // The cellSet and snapshot at the time of creating this scanner
-    private CellSkipListSet cellSetAtCreation;
-    private CellSkipListSet snapshotAtCreation;
-
-    // the pre-calculated Cell to be returned by peek() or next()
-    private Cell theNext;
-
-    // The allocator and snapshot allocator at the time of creating this scanner
-    volatile MemStoreLAB allocatorAtCreation;
-    volatile MemStoreLAB snapshotAllocatorAtCreation;
-
-    // A flag represents whether could stop skipping Cells for MVCC
-    // if have encountered the next row. Only used for reversed scan
-    private boolean stopSkippingCellsIfNextRow = false;
-
-    private long readPoint;
-
-    /*
-    Some notes...
-
-     So memstorescanner is fixed at creation time. this includes pointers/iterators into
-    existing kvset/snapshot.  during a snapshot creation, the kvset is null, and the
-    snapshot is moved.  since kvset is null there is no point on reseeking on both,
-      we can save us the trouble. During the snapshot->hfile transition, the memstore
-      scanner is re-created by StoreScanner#updateReaders().  StoreScanner should
-      potentially do something smarter by adjusting the existing memstore scanner.
-
-      But there is a greater problem here, that being once a scanner has progressed
-      during a snapshot scenario, we currently iterate past the kvset then 'finish' up.
-      if a scan lasts a little while, there is a chance for new entries in kvset to
-      become available but we will never see them.  This needs to be handled at the
-      StoreScanner level with coordination with MemStoreScanner.
-
-      Currently, this problem is only partly managed: during the small amount of time
-      when the StoreScanner has not yet created a new MemStoreScanner, we will miss
-      the adds to kvset in the MemStoreScanner.
-    */
-
-    MemStoreScanner(long readPoint) {
-      super();
-
-      this.readPoint = readPoint;
-      cellSetAtCreation = cellSet;
-      snapshotAtCreation = snapshot;
-      if (allocator != null) {
-        this.allocatorAtCreation = allocator;
-        this.allocatorAtCreation.incScannerCount();
-      }
-      if (snapshotAllocator != null) {
-        this.snapshotAllocatorAtCreation = snapshotAllocator;
-        this.snapshotAllocatorAtCreation.incScannerCount();
-      }
-      if (Trace.isTracing() && Trace.currentSpan() != null) {
-        Trace.currentSpan().addTimelineAnnotation("Creating MemStoreScanner");
-      }
-    }
-
-    /**
-     * Lock on 'this' must be held by caller.
-     * @param it
-     * @return Next Cell
-     */
-    private Cell getNext(Iterator<Cell> it) {
-      Cell startCell = theNext;
-      Cell v = null;
-      try {
-        while (it.hasNext()) {
-          v = it.next();
-          if (v.getSequenceId() <= this.readPoint) {
-            return v;
-          }
-          if (stopSkippingCellsIfNextRow && startCell != null
-              && comparator.compareRows(v, startCell) > 0) {
-            return null;
-          }
-        }
-
-        return null;
-      } finally {
-        if (v != null) {
-          // in all cases, remember the last Cell iterated to
-          if (it == snapshotIt) {
-            snapshotItRow = v;
-          } else {
-            cellSetItRow = v;
-          }
-        }
-      }
-    }
-
-    /**
-     *  Set the scanner at the seek key.
-     *  Must be called only once: there is no thread safety between the scanner
-     *   and the memStore.
-     * @param key seek value
-     * @return false if the key is null or if there is no data
-     */
-    @Override
-    public synchronized boolean seek(Cell key) {
-      if (key == null) {
-        close();
-        return false;
-      }
-      // kvset and snapshot will never be null.
-      // if tailSet can't find anything, SortedSet is empty (not null).
-      cellSetIt = cellSetAtCreation.tailSet(key).iterator();
-      snapshotIt = snapshotAtCreation.tailSet(key).iterator();
-      cellSetItRow = null;
-      snapshotItRow = null;
-
-      return seekInSubLists(key);
-    }
-
-
-    /**
-     * (Re)initialize the iterators after a seek or a reseek.
-     */
-    private synchronized boolean seekInSubLists(Cell key){
-      cellSetNextRow = getNext(cellSetIt);
-      snapshotNextRow = getNext(snapshotIt);
-
-      // Calculate the next value
-      theNext = getLowest(cellSetNextRow, snapshotNextRow);
-
-      // has data
-      return (theNext != null);
-    }
-
-
-    /**
-     * Move forward on the sub-lists set previously by seek.
-     * @param key seek value (should be non-null)
-     * @return true if there is at least one KV to read, false otherwise
-     */
-    @Override
-    public synchronized boolean reseek(Cell key) {
-      /*
-      See HBASE-4195 & HBASE-3855 & HBASE-6591 for the background on this implementation.
-      This code is executed concurrently with flush and puts, without locks.
-      Two points must be known when working on this code:
-      1) It's not possible to use the 'kvTail' and 'snapshot'
-       variables, as they are modified during a flush.
-      2) The ideal implementation for performance would use the sub skip list
-       implicitly pointed by the iterators 'kvsetIt' and
-       'snapshotIt'. Unfortunately the Java API does not offer a method to
-       get it. So we remember the last keys we iterated to and restore
-       the reseeked set to at least that point.
-       */
-      cellSetIt = cellSetAtCreation.tailSet(getHighest(key, cellSetItRow)).iterator();
-      snapshotIt = snapshotAtCreation.tailSet(getHighest(key, snapshotItRow)).iterator();
-
-      return seekInSubLists(key);
-    }
-
-
-    @Override
-    public synchronized Cell peek() {
-      //DebugPrint.println(" MS@" + hashCode() + " peek = " + getLowest());
-      return theNext;
-    }
-
-    @Override
-    public synchronized Cell next() {
-      if (theNext == null) {
-          return null;
-      }
-
-      final Cell ret = theNext;
-
-      // Advance one of the iterators
-      if (theNext == cellSetNextRow) {
-        cellSetNextRow = getNext(cellSetIt);
-      } else {
-        snapshotNextRow = getNext(snapshotIt);
-      }
-
-      // Calculate the next value
-      theNext = getLowest(cellSetNextRow, snapshotNextRow);
-
-      //long readpoint = ReadWriteConsistencyControl.getThreadReadPoint();
-      //DebugPrint.println(" MS@" + hashCode() + " next: " + theNext + " next_next: " +
-      //    getLowest() + " threadpoint=" + readpoint);
-      return ret;
-    }
-
-    /*
-     * Returns the lower of the two key values, or null if they are both null.
-     * This uses comparator.compare() to compare the KeyValue using the memstore
-     * comparator.
-     */
-    private Cell getLowest(Cell first, Cell second) {
-      if (first == null && second == null) {
-        return null;
-      }
-      if (first != null && second != null) {
-        int compare = comparator.compare(first, second);
-        return (compare <= 0 ? first : second);
-      }
-      return (first != null ? first : second);
-    }
-
-    /*
-     * Returns the higher of the two cells, or null if they are both null.
-     * This uses comparator.compare() to compare the Cell using the memstore
-     * comparator.
-     */
-    private Cell getHighest(Cell first, Cell second) {
-      if (first == null && second == null) {
-        return null;
-      }
-      if (first != null && second != null) {
-        int compare = comparator.compare(first, second);
-        return (compare > 0 ? first : second);
-      }
-      return (first != null ? first : second);
-    }
-
-    public synchronized void close() {
-      this.cellSetNextRow = null;
-      this.snapshotNextRow = null;
-
-      this.cellSetIt = null;
-      this.snapshotIt = null;
-
-      if (allocatorAtCreation != null) {
-        this.allocatorAtCreation.decScannerCount();
-        this.allocatorAtCreation = null;
-      }
-      if (snapshotAllocatorAtCreation != null) {
-        this.snapshotAllocatorAtCreation.decScannerCount();
-        this.snapshotAllocatorAtCreation = null;
-      }
-
-      this.cellSetItRow = null;
-      this.snapshotItRow = null;
-    }
-
-    /**
-     * MemStoreScanner returns max value as sequence id because it will
-     * always have the latest data among all files.
-     */
-    @Override
-    public long getSequenceID() {
-      return Long.MAX_VALUE;
-    }
-
-    @Override
-    public boolean shouldUseScanner(Scan scan, Store store, long oldestUnexpiredTS) {
-      return shouldSeek(scan, store, oldestUnexpiredTS);
-    }
-
-    /**
-     * Seek scanner to the given key first. If it returns false(means
-     * peek()==null) or scanner's peek row is bigger than row of given key, seek
-     * the scanner to the previous row of given key
-     */
-    @Override
-    public synchronized boolean backwardSeek(Cell key) {
-      seek(key);
-      if (peek() == null || comparator.compareRows(peek(), key) > 0) {
-        return seekToPreviousRow(key);
-      }
-      return true;
-    }
-
-    /**
-     * Separately get the KeyValue before the specified key from kvset and
-     * snapshotset, and use the row of higher one as the previous row of
-     * specified key, then seek to the first KeyValue of previous row
-     */
-    @Override
-    public synchronized boolean seekToPreviousRow(Cell originalKey) {
-      boolean keepSeeking = false;
-      Cell key = originalKey;
-      do {
-        Cell firstKeyOnRow = CellUtil.createFirstOnRow(key);
-        SortedSet<Cell> cellHead = cellSetAtCreation.headSet(firstKeyOnRow);
-        Cell cellSetBeforeRow = cellHead.isEmpty() ? null : cellHead.last();
-        SortedSet<Cell> snapshotHead = snapshotAtCreation
-            .headSet(firstKeyOnRow);
-        Cell snapshotBeforeRow = snapshotHead.isEmpty() ? null : snapshotHead
-            .last();
-        Cell lastCellBeforeRow = getHighest(cellSetBeforeRow, snapshotBeforeRow);
-        if (lastCellBeforeRow == null) {
-          theNext = null;
-          return false;
-        }
-        Cell firstKeyOnPreviousRow = CellUtil.createFirstOnRow(lastCellBeforeRow);
-        this.stopSkippingCellsIfNextRow = true;
-        seek(firstKeyOnPreviousRow);
-        this.stopSkippingCellsIfNextRow = false;
-        if (peek() == null
-            || comparator.compareRows(peek(), firstKeyOnPreviousRow) > 0) {
-          keepSeeking = true;
-          key = firstKeyOnPreviousRow;
-          continue;
-        } else {
-          keepSeeking = false;
-        }
-      } while (keepSeeking);
-      return true;
-    }
-
-    @Override
-    public synchronized boolean seekToLastRow() {
-      Cell first = cellSetAtCreation.isEmpty() ? null : cellSetAtCreation
-          .last();
-      Cell second = snapshotAtCreation.isEmpty() ? null
-          : snapshotAtCreation.last();
-      Cell higherCell = getHighest(first, second);
-      if (higherCell == null) {
-        return false;
-      }
-      Cell firstCellOnLastRow = CellUtil.createFirstOnRow(higherCell);
-      if (seek(firstCellOnLastRow)) {
-        return true;
-      } else {
-        return seekToPreviousRow(higherCell);
-      }
-
-    }
-  }
-
-  public final static long FIXED_OVERHEAD = ClassSize.align(ClassSize.OBJECT
-      + (9 * ClassSize.REFERENCE) + (3 * Bytes.SIZEOF_LONG) + Bytes.SIZEOF_BOOLEAN);
-
-  public final static long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD +
-      ClassSize.ATOMIC_LONG + (2 * ClassSize.TIMERANGE_TRACKER) +
-      (2 * ClassSize.CELL_SKIPLIST_SET) + (2 * ClassSize.CONCURRENT_SKIPLISTMAP));
-
-  /*
-   * Calculate how the MemStore size has changed.  Includes overhead of the
-   * backing Map.
-   * @param cell
-   * @param notpresent True if the cell was NOT present in the set.
-   * @return Size
-   */
-  static long heapSizeChange(final Cell cell, final boolean notpresent) {
-    return notpresent ? ClassSize.align(ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY
-        + CellUtil.estimatedHeapSizeOf(cell)) : 0;
-  }
-
-  private long keySize() {
-    return heapSize() - DEEP_OVERHEAD;
+  public long size() {
+    return heapSize();
   }
 
   /**
-   * Get the entire heap usage for this MemStore not including keys in the
-   * snapshot.
+   * Check whether anything need to be done based on the current active set size
+   * Nothing need to be done for the DefaultMemStore
    */
   @Override
-  public long heapSize() {
-    return size.get();
-  }
-
-  @Override
-  public long size() {
-    return heapSize();
+  protected void checkActiveSize() {
+    return;
   }
 
   /**
@@ -978,9 +198,6 @@ public class DefaultMemStore implements MemStore {
     LOG.info("memstore2 estimated size=" + size);
     final int seconds = 30;
     LOG.info("Waiting " + seconds + " seconds while heap dump is taken");
-    for (int i = 0; i < seconds; i++) {
-      // Thread.sleep(1000);
-    }
     LOG.info("Exiting.");
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/25dfc112/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 c65326a..5c29fb4 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
@@ -18,6 +18,13 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableCollection;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.net.InetSocketAddress;
@@ -91,13 +98,6 @@ import org.apache.hadoop.hbase.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.ImmutableCollection;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
-
 /**
  * A Store holds a column family in a Region.  Its a memstore and a set of zero
  * or more StoreFiles, which stretch backwards over time.
@@ -1636,7 +1636,7 @@ public class HStore implements Store {
       this.lock.readLock().unlock();
     }
 
-    LOG.debug(getRegionInfo().getEncodedName() + " - "  + getColumnFamilyName()
+    LOG.debug(getRegionInfo().getEncodedName() + " - " + getColumnFamilyName()
         + ": Initiating " + (request.isMajor() ? "major" : "minor") + " compaction"
         + (request.isAllFiles() ? " (all files)" : ""));
     this.region.reportCompactionRequestStart(request.isMajor());
@@ -1990,8 +1990,6 @@ public class HStore implements Store {
   }
 
   /**
-   * Used in tests. TODO: Remove
-   *
    * Updates the value for the given row/family/qualifier. This function will always be seen as
    * atomic by other readers because it only puts a single KV to memstore. Thus no read/write
    * control necessary.
@@ -2002,6 +2000,7 @@ public class HStore implements Store {
    * @return memstore size delta
    * @throws IOException
    */
+  @VisibleForTesting
   public long updateColumnValue(byte [] row, byte [] f,
                                 byte [] qualifier, long newValue)
       throws IOException {
@@ -2055,7 +2054,8 @@ public class HStore implements Store {
      */
     @Override
     public void prepare() {
-      this.snapshot = memstore.snapshot();
+      // passing the current sequence number of the wal - to allow bookkeeping in the memstore
+      this.snapshot = memstore.snapshot(cacheFlushSeqNum);
       this.cacheFlushCount = snapshot.getCellsCount();
       this.cacheFlushSize = snapshot.getSize();
       committedFiles = new ArrayList<Path>(1);

http://git-wip-us.apache.org/repos/asf/hbase/blob/25dfc112/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ImmutableSegment.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ImmutableSegment.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ImmutableSegment.java
new file mode 100644
index 0000000..cfcd81e
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ImmutableSegment.java
@@ -0,0 +1,72 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver;
+
+import org.apache.commons.lang.NotImplementedException;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * ImmutableSegment is an abstract class that extends the API supported by a {@link Segment},
+ * and is not needed for a {@link MutableSegment}. Specifically, the method
+ * {@link ImmutableSegment#getKeyValueScanner()} builds a special scanner for the
+ * {@link MemStoreSnapshot} object.
+ * In addition, this class overrides methods that are not likely to be supported by an immutable
+ * segment, e.g. {@link Segment#rollback(Cell)} and {@link Segment#getCellSet()}, which
+ * can be very inefficient.
+ */
+@InterfaceAudience.Private
+public abstract class ImmutableSegment extends Segment {
+
+  public ImmutableSegment(Segment segment) {
+    super(segment);
+  }
+
+  /**
+   * Removes the given cell from this segment.
+   * By default immutable store segment can not rollback
+   * It may be invoked by tests in specific cases where it is known to be supported {@link
+   * ImmutableSegmentAdapter}
+   */
+  @Override
+  public long rollback(Cell cell) {
+    return 0;
+  }
+
+  /**
+   * Returns a set of all the cells in the segment.
+   * The implementation of this method might be very inefficient for some immutable segments
+   * that do not maintain a cell set. Therefore by default this method is not supported.
+   * It may be invoked by tests in specific cases where it is known to be supported {@link
+   * ImmutableSegmentAdapter}
+   */
+  @Override
+  public CellSet getCellSet() {
+    throw new NotImplementedException("Immutable Segment does not support this operation by " +
+        "default");
+  }
+
+  /**
+   * Builds a special scanner for the MemStoreSnapshot object that may be different than the
+   * general segment scanner.
+   * @return a special scanner for the MemStoreSnapshot object
+   */
+  public abstract KeyValueScanner getKeyValueScanner();
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/25dfc112/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ImmutableSegmentAdapter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ImmutableSegmentAdapter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ImmutableSegmentAdapter.java
new file mode 100644
index 0000000..058865a
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ImmutableSegmentAdapter.java
@@ -0,0 +1,107 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver;
+
+import org.apache.commons.logging.Log;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.util.CollectionBackedScanner;
+
+/**
+ * This segment is adapting a mutable segment making it into an immutable segment.
+ * This is used when a mutable segment is moved to being a snapshot or pushed into a compaction
+ * pipeline, that consists only of immutable segments.
+ * The compaction may generate different type of immutable segment
+ */
+@InterfaceAudience.Private
+public class ImmutableSegmentAdapter extends ImmutableSegment {
+
+  final private MutableSegment adaptee;
+
+  public ImmutableSegmentAdapter(MutableSegment segment) {
+    super(segment);
+    this.adaptee = segment;
+  }
+
+  @Override
+  public KeyValueScanner getKeyValueScanner() {
+    return new CollectionBackedScanner(adaptee.getCellSet(), adaptee.getComparator());
+  }
+
+  @Override
+  public SegmentScanner getSegmentScanner(long readPoint) {
+    return adaptee.getSegmentScanner(readPoint);
+  }
+
+  @Override
+  public boolean isEmpty() {
+    return adaptee.isEmpty();
+  }
+
+  @Override
+  public int getCellsCount() {
+    return adaptee.getCellsCount();
+  }
+
+  @Override
+  public long add(Cell cell) {
+    return adaptee.add(cell);
+  }
+
+  @Override
+  public Cell getFirstAfter(Cell cell) {
+    return adaptee.getFirstAfter(cell);
+  }
+
+  @Override
+  public void close() {
+    adaptee.close();
+  }
+
+  @Override
+  public Cell maybeCloneWithAllocator(Cell cell) {
+    return adaptee.maybeCloneWithAllocator(cell);
+  }
+
+  @Override
+  public Segment setSize(long size) {
+    adaptee.setSize(size);
+    return this;
+  }
+
+  @Override
+  public long getSize() {
+    return adaptee.getSize();
+  }
+
+  @Override
+  public long rollback(Cell cell) {
+    return adaptee.rollback(cell);
+  }
+
+  @Override
+  public CellSet getCellSet() {
+    return adaptee.getCellSet();
+  }
+
+  @Override
+  public void dump(Log log) {
+    adaptee.dump(log);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/25dfc112/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStore.java
index e9f8103..a10ccd9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStore.java
@@ -17,10 +17,11 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
+import java.io.IOException;
 import java.util.List;
 
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.io.HeapSize;
 
 /**
@@ -41,10 +42,19 @@ public interface MemStore extends HeapSize {
   MemStoreSnapshot snapshot();
 
   /**
+   * Creates a snapshot of the current memstore. Snapshot must be cleared by call to
+   * {@link #clearSnapshot(long)}.
+   * @param flushOpSeqId the current sequence number of the wal; to be attached to the flushed
+   *                     segment
+   * @return {@link MemStoreSnapshot}
+   */
+  MemStoreSnapshot snapshot(long flushOpSeqId);
+
+  /**
    * Clears the current snapshot of the Memstore.
    * @param id
    * @throws UnexpectedStateException
-   * @see #snapshot()
+   * @see #snapshot(long)
    */
   void clearSnapshot(long id) throws UnexpectedStateException;
 
@@ -128,7 +138,7 @@ public interface MemStore extends HeapSize {
    * @return scanner over the memstore. This might include scanner over the snapshot when one is
    * present.
    */
-  List<KeyValueScanner> getScanners(long readPt);
+  List<KeyValueScanner> getScanners(long readPt) throws IOException;
 
   /**
    * @return Total memory occupied by this MemStore.


[19/22] hbase git commit: HBASE-15219 Revert pending verification of test result

Posted by sy...@apache.org.
HBASE-15219 Revert pending verification of test result


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

Branch: refs/heads/hbase-12439
Commit: 454e45f50df0038e2c67368c47f9395ebee7cf69
Parents: 85e1d9a
Author: tedyu <yu...@gmail.com>
Authored: Fri Feb 12 12:06:48 2016 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Fri Feb 12 12:06:48 2016 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/tool/Canary.java    | 55 ++++----------------
 src/main/asciidoc/_chapters/ops_mgt.adoc        | 11 ----
 2 files changed, 9 insertions(+), 57 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/454e45f5/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
index 9248c71..3c7ae64 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
@@ -94,11 +94,9 @@ import org.apache.hadoop.util.ToolRunner;
 public final class Canary implements Tool {
   // Sink interface used by the canary to outputs information
   public interface Sink {
-    public long getReadFailureCount();
     public void publishReadFailure(HRegionInfo region, Exception e);
     public void publishReadFailure(HRegionInfo region, HColumnDescriptor column, Exception e);
     public void publishReadTiming(HRegionInfo region, HColumnDescriptor column, long msTime);
-    public long getWriteFailureCount();
     public void publishWriteFailure(HRegionInfo region, Exception e);
     public void publishWriteFailure(HRegionInfo region, HColumnDescriptor column, Exception e);
     public void publishWriteTiming(HRegionInfo region, HColumnDescriptor column, long msTime);
@@ -113,23 +111,13 @@ public final class Canary implements Tool {
   // Simple implementation of canary sink that allows to plot on
   // file or standard output timings or failures.
   public static class StdOutSink implements Sink {
-    protected AtomicLong readFailureCount = new AtomicLong(0),
-        writeFailureCount = new AtomicLong(0);
-
-    @Override
-    public long getReadFailureCount() {
-      return readFailureCount.get();
-    }
-
     @Override
     public void publishReadFailure(HRegionInfo region, Exception e) {
-      readFailureCount.incrementAndGet();
       LOG.error(String.format("read from region %s failed", region.getRegionNameAsString()), e);
     }
 
     @Override
     public void publishReadFailure(HRegionInfo region, HColumnDescriptor column, Exception e) {
-      readFailureCount.incrementAndGet();
       LOG.error(String.format("read from region %s column family %s failed",
                 region.getRegionNameAsString(), column.getNameAsString()), e);
     }
@@ -141,19 +129,12 @@ public final class Canary implements Tool {
     }
 
     @Override
-    public long getWriteFailureCount() {
-      return writeFailureCount.get();
-    }
-
-    @Override
     public void publishWriteFailure(HRegionInfo region, Exception e) {
-      writeFailureCount.incrementAndGet();
       LOG.error(String.format("write to region %s failed", region.getRegionNameAsString()), e);
     }
 
     @Override
     public void publishWriteFailure(HRegionInfo region, HColumnDescriptor column, Exception e) {
-      writeFailureCount.incrementAndGet();
       LOG.error(String.format("write to region %s column family %s failed",
         region.getRegionNameAsString(), column.getNameAsString()), e);
     }
@@ -169,7 +150,6 @@ public final class Canary implements Tool {
 
     @Override
     public void publishReadFailure(String table, String server) {
-      readFailureCount.incrementAndGet();
       LOG.error(String.format("Read from table:%s on region server:%s", table, server));
     }
 
@@ -455,7 +435,6 @@ public final class Canary implements Tool {
   private boolean regionServerMode = false;
   private boolean regionServerAllRegions = false;
   private boolean writeSniffing = false;
-  private boolean treatFailureAsError = false;
   private TableName writeTableName = DEFAULT_WRITE_TABLE_NAME;
 
   private ExecutorService executor; // threads to retrieve data from regionservers
@@ -519,8 +498,6 @@ public final class Canary implements Tool {
           this.regionServerAllRegions = true;
         } else if(cmd.equals("-writeSniffing")) {
           this.writeSniffing = true;
-        } else if(cmd.equals("-treatFailureAsError")) {
-          this.treatFailureAsError = true;
         } else if (cmd.equals("-e")) {
           this.useRegExp = true;
         } else if (cmd.equals("-t")) {
@@ -625,7 +602,7 @@ public final class Canary implements Tool {
             }
           }
 
-          if (this.failOnError && monitor.finalCheckForErrors()) {
+          if (this.failOnError && monitor.hasError()) {
             monitorThread.interrupt();
             return monitor.errorCode;
           }
@@ -661,7 +638,6 @@ public final class Canary implements Tool {
         " default is true");
     System.err.println("   -t <N>         timeout for a check, default is 600000 (milisecs)");
     System.err.println("   -writeSniffing enable the write sniffing in canary");
-    System.err.println("   -treatFailureAsError treats read / write failure as error");
     System.err.println("   -writeTable    The table used for write sniffing."
         + " Default is hbase:canary");
     System.err
@@ -689,12 +665,11 @@ public final class Canary implements Tool {
     if (this.regionServerMode) {
       monitor =
           new RegionServerMonitor(connection, monitorTargets, this.useRegExp,
-              (ExtendedSink) this.sink, this.executor, this.regionServerAllRegions,
-              this.treatFailureAsError);
+              (ExtendedSink) this.sink, this.executor, this.regionServerAllRegions);
     } else {
       monitor =
           new RegionMonitor(connection, monitorTargets, this.useRegExp, this.sink, this.executor,
-              this.writeSniffing, this.writeTableName, this.treatFailureAsError);
+              this.writeSniffing, this.writeTableName);
     }
     return monitor;
   }
@@ -706,7 +681,6 @@ public final class Canary implements Tool {
     protected Admin admin;
     protected String[] targets;
     protected boolean useRegExp;
-    protected boolean treatFailureAsError;
     protected boolean initialized = false;
 
     protected boolean done = false;
@@ -722,27 +696,18 @@ public final class Canary implements Tool {
       return errorCode != 0;
     }
 
-    public boolean finalCheckForErrors() {
-      if (errorCode != 0) {
-        return true;
-      }
-      return treatFailureAsError &&
-          (sink.getReadFailureCount() > 0 || sink.getWriteFailureCount() > 0);
-    }
-
     @Override
     public void close() throws IOException {
       if (this.admin != null) this.admin.close();
     }
 
     protected Monitor(Connection connection, String[] monitorTargets, boolean useRegExp, Sink sink,
-        ExecutorService executor, boolean treatFailureAsError) {
+        ExecutorService executor) {
       if (null == connection) throw new IllegalArgumentException("connection shall not be null");
 
       this.connection = connection;
       this.targets = monitorTargets;
       this.useRegExp = useRegExp;
-      this.treatFailureAsError = treatFailureAsError;
       this.sink = sink;
       this.executor = executor;
     }
@@ -782,9 +747,8 @@ public final class Canary implements Tool {
     private int checkPeriod;
 
     public RegionMonitor(Connection connection, String[] monitorTargets, boolean useRegExp,
-        Sink sink, ExecutorService executor, boolean writeSniffing, TableName writeTableName,
-        boolean treatFailureAsError) {
-      super(connection, monitorTargets, useRegExp, sink, executor, treatFailureAsError);
+        Sink sink, ExecutorService executor, boolean writeSniffing, TableName writeTableName) {
+      super(connection, monitorTargets, useRegExp, sink, executor);
       Configuration conf = connection.getConfiguration();
       this.writeSniffing = writeSniffing;
       this.writeTableName = writeTableName;
@@ -1028,9 +992,8 @@ public final class Canary implements Tool {
     private boolean allRegions;
 
     public RegionServerMonitor(Connection connection, String[] monitorTargets, boolean useRegExp,
-        ExtendedSink sink, ExecutorService executor, boolean allRegions,
-        boolean treatFailureAsError) {
-      super(connection, monitorTargets, useRegExp, sink, executor, treatFailureAsError);
+        ExtendedSink sink, ExecutorService executor, boolean allRegions) {
+      super(connection, monitorTargets, useRegExp, sink, executor);
       this.allRegions = allRegions;
     }
 
@@ -1125,7 +1088,7 @@ public final class Canary implements Tool {
         }
       } catch (InterruptedException e) {
         this.errorCode = ERROR_EXIT_CODE;
-        LOG.error("Sniff regionserver interrupted!", e);
+        LOG.error("Sniff regionserver failed!", e);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/454e45f5/src/main/asciidoc/_chapters/ops_mgt.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/ops_mgt.adoc b/src/main/asciidoc/_chapters/ops_mgt.adoc
index 578d66f..0f93785 100644
--- a/src/main/asciidoc/_chapters/ops_mgt.adoc
+++ b/src/main/asciidoc/_chapters/ops_mgt.adoc
@@ -93,7 +93,6 @@ Usage: bin/hbase org.apache.hadoop.hbase.tool.Canary [opts] [table1 [table2]...]
    -f <B>         stop whole program if first error occurs, default is true
    -t <N>         timeout for a check, default is 600000 (milliseconds)
    -writeSniffing enable the write sniffing in canary
-   -treatFailureAsError treats read / write failure as error
    -writeTable    The table used for write sniffing. Default is hbase:canary
    -D<configProperty>=<value> assigning or override the configuration params
 ----
@@ -216,16 +215,6 @@ $ ${HBASE_HOME}/bin/hbase canary -writeSniffing -writeTable ns:canary
 The default value size of each put is 10 bytes and you can set it by the config key:
 `hbase.canary.write.value.size`.
 
-==== Treat read / write failure as error
-
-By default, the canary tool only logs read failure, due to e.g. RetriesExhaustedException,
-while returning normal exit code. To treat read / write failure as error, you can run canary
-with the `-treatFailureAsError` option. When enabled, read / write failure would result in error
-exit code.
-----
-$ ${HBASE_HOME}/bin/hbase canary --treatFailureAsError
-----
-
 ==== Running Canary in a Kerberos-enabled Cluster
 
 To run Canary in a Kerberos-enabled cluster, configure the following two properties in _hbase-site.xml_:


[16/22] hbase git commit: HBASE-15255 Add pointer to linkedin blog on putting jvm logs on fast disk

Posted by sy...@apache.org.
HBASE-15255 Add pointer to linkedin blog on putting jvm logs on fast disk


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

Branch: refs/heads/hbase-12439
Commit: ab50c7c8c64aa00da14167b01a9e599098afe4bc
Parents: 25dfc11
Author: stack <st...@apache.org>
Authored: Thu Feb 11 13:18:28 2016 -0800
Committer: stack <st...@apache.org>
Committed: Thu Feb 11 13:18:28 2016 -0800

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/ab50c7c8/src/main/asciidoc/_chapters/performance.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/performance.adoc b/src/main/asciidoc/_chapters/performance.adoc
index 5155f0a..ee7933c 100644
--- a/src/main/asciidoc/_chapters/performance.adoc
+++ b/src/main/asciidoc/_chapters/performance.adoc
@@ -48,6 +48,11 @@ Use a 64-bit platform (and 64-bit JVM).
 Watch out for swapping.
 Set `swappiness` to 0.
 
+[[perf.os.cpu]]
+=== CPU
+Make sure you have set up your Hadoop to use native, hardware checksumming.
+See link:[hadoop.native.lib].
+
 [[perf.network]]
 == Network
 
@@ -137,6 +142,9 @@ It describes configurations to lower the amount of young GC during write-heavy l
 If you do not have HBASE-8163 installed, and you are trying to improve your young GC times, one trick to consider -- courtesy of our Liang Xie -- is to set the GC config `-XX:PretenureSizeThreshold` in _hbase-env.sh_ to be just smaller than the size of `hbase.hregion.memstore.mslab.chunksize` so MSLAB allocations happen in the tenured space directly rather than first in the young gen.
 You'd do this because these MSLAB allocations are going to likely make it to the old gen anyways and rather than pay the price of a copies between s0 and s1 in eden space followed by the copy up from young to old gen after the MSLABs have achieved sufficient tenure, save a bit of YGC churn and allocate in the old gen directly.
 
+Other sources of long GCs can be the JVM itself logging.
+See link:https://engineering.linkedin.com/blog/2016/02/eliminating-large-jvm-gc-pauses-caused-by-background-io-traffic[Eliminating Large JVM GC Pauses Caused by Background IO Traffic]
+
 For more information about GC logs, see <<trouble.log.gc>>.
 
 Consider also enabling the off-heap Block Cache.