HBASE-16280 Use hash based map in SequenceIdAccounting
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java
index 7b9eb0b..41292a5 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java
@@ -2631,16 +2631,16 @@
       }
     }
     return result;
-    }
+  }
 
-    public static int findCommonPrefix(byte[] left, byte[] right, int leftLength, int rightLength,
-        int leftOffset, int rightOffset) {
-      int length = Math.min(leftLength, rightLength);
-      int result = 0;
+  public static int findCommonPrefix(byte[] left, byte[] right, int leftLength, int rightLength,
+      int leftOffset, int rightOffset) {
+    int length = Math.min(leftLength, rightLength);
+    int result = 0;
 
-      while (result < length && left[leftOffset + result] == right[rightOffset + result]) {
-        result++;
-      }
-      return result;
+    while (result < length && left[leftOffset + result] == right[rightOffset + result]) {
+      result++;
     }
+    return result;
+  }
 }
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ImmutableByteArray.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ImmutableByteArray.java
new file mode 100644
index 0000000..afd1ebf
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ImmutableByteArray.java
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.util;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * Mainly used as keys for HashMap.
+ */
+@InterfaceAudience.Private
+public final class ImmutableByteArray {
+
+  private final byte[] b;
+
+  private ImmutableByteArray(byte[] b) {
+    this.b = b;
+  }
+
+  @Override
+  public int hashCode() {
+    return Bytes.hashCode(b);
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj == null || obj.getClass() != ImmutableByteArray.class) {
+      return false;
+    }
+    return Bytes.equals(b, ((ImmutableByteArray) obj).b);
+  }
+
+  public static ImmutableByteArray wrap(byte[] b) {
+    return new ImmutableByteArray(b);
+  }
+
+  public String toStringUtf8() {
+    return Bytes.toString(b);
+  }
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceIdAccounting.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceIdAccounting.java
index 53c501f..62dea53 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceIdAccounting.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceIdAccounting.java
@@ -17,28 +17,39 @@
  */
 package org.apache.hadoop.hbase.regionserver.wal;
 
-import com.google.common.collect.Maps;
+import com.google.common.annotations.VisibleForTesting;
+
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.TreeMap;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ConcurrentSkipListMap;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.ImmutableByteArray;
 
 /**
  * Accounting of sequence ids per region and then by column family. So we can our accounting
- * current, call startCacheFlush and then finishedCacheFlush or abortCacheFlush so this instance
- * can keep abreast of the state of sequence id persistence. Also call update per append.
+ * current, call startCacheFlush and then finishedCacheFlush or abortCacheFlush so this instance can
+ * keep abreast of the state of sequence id persistence. Also call update per append.
+ * <p>
+ * For the implementation, we assume that all the {@code encodedRegionName} passed in is gotten by
+ * {@link HRegionInfo#getEncodedNameAsBytes()}. So it is safe to use it as a hash key. And for
+ * family name, we use {@link ImmutableByteArray} as key. This is because hash based map is much
+ * faster than RBTree or CSLM and here we are on the critical write path. See HBASE-16278 for more
+ * details.
  */
+@InterfaceAudience.Private
 class SequenceIdAccounting {
+
   private static final Log LOG = LogFactory.getLog(SequenceIdAccounting.class);
   /**
    * This lock ties all operations on {@link SequenceIdAccounting#flushingSequenceIds} and
@@ -69,9 +80,8 @@
    * <p>If flush fails, currently server is aborted so no need to restore previous sequence ids.
    * <p>Needs to be concurrent Maps because we use putIfAbsent updating oldest.
    */
-  private final ConcurrentMap<byte[], ConcurrentMap<byte[], Long>> lowestUnflushedSequenceIds
-    = new ConcurrentSkipListMap<byte[], ConcurrentMap<byte[], Long>>(
-      Bytes.BYTES_COMPARATOR);
+  private final ConcurrentMap<byte[], ConcurrentMap<ImmutableByteArray, Long>>
+    lowestUnflushedSequenceIds = new ConcurrentHashMap<>();
 
   /**
    * Map of encoded region names and family names to their lowest or OLDEST sequence/edit id
@@ -79,8 +89,7 @@
    * {@link #lowestUnflushedSequenceIds} while the lock {@link #tieLock} is held
    * (so movement between the Maps is atomic).
    */
-  private final Map<byte[], Map<byte[], Long>> flushingSequenceIds =
-    new TreeMap<byte[], Map<byte[], Long>>(Bytes.BYTES_COMPARATOR);
+  private final Map<byte[], Map<ImmutableByteArray, Long>> flushingSequenceIds = new HashMap<>();
 
  /**
   * Map of region encoded names to the latest/highest region sequence id.  Updated on each
@@ -90,7 +99,7 @@
   * use {@link HRegionInfo#getEncodedNameAsBytes()} as keys. For a given region, it always returns
   * the same array.
   */
-  private Map<byte[], Long> highestSequenceIds = new HashMap<byte[], Long>();
+  private Map<byte[], Long> highestSequenceIds = new HashMap<>();
 
   /**
    * Returns the lowest unflushed sequence id for the region.
@@ -98,33 +107,39 @@
    * @return Lowest outstanding unflushed sequenceid for <code>encodedRegionName</code>. Will
    * return {@link HConstants#NO_SEQNUM} when none.
    */
-  long getLowestSequenceId(final byte [] encodedRegionName) {
-    synchronized (this.tieLock)  {
-      Map<byte[], Long> m = this.flushingSequenceIds.get(encodedRegionName);
-      long flushingLowest = m != null? getLowestSequenceId(m): Long.MAX_VALUE;
+  long getLowestSequenceId(final byte[] encodedRegionName) {
+    synchronized (this.tieLock) {
+      Map<?, Long> m = this.flushingSequenceIds.get(encodedRegionName);
+      long flushingLowest = m != null ? getLowestSequenceId(m) : Long.MAX_VALUE;
       m = this.lowestUnflushedSequenceIds.get(encodedRegionName);
-      long unflushedLowest = m != null? getLowestSequenceId(m): HConstants.NO_SEQNUM;
+      long unflushedLowest = m != null ? getLowestSequenceId(m) : HConstants.NO_SEQNUM;
       return Math.min(flushingLowest, unflushedLowest);
     }
   }
 
   /**
    * @param encodedRegionName
-   * @param familyName 
+   * @param familyName
    * @return Lowest outstanding unflushed sequenceid for <code>encodedRegionname</code> and
-   * <code>familyName</code>. Returned sequenceid may be for an edit currently being flushed.
+   *         <code>familyName</code>. Returned sequenceid may be for an edit currently being
+   *         flushed.
    */
-  long getLowestSequenceId(final byte [] encodedRegionName, final byte [] familyName) {
+  long getLowestSequenceId(final byte[] encodedRegionName, final byte[] familyName) {
+    ImmutableByteArray familyNameWrapper = ImmutableByteArray.wrap(familyName);
     synchronized (this.tieLock) {
-      Map<byte[], Long> m = this.flushingSequenceIds.get(encodedRegionName);
+      Map<ImmutableByteArray, Long> m = this.flushingSequenceIds.get(encodedRegionName);
       if (m != null) {
-        Long lowest = m.get(familyName);
-        if (lowest != null) return lowest;
+        Long lowest = m.get(familyNameWrapper);
+        if (lowest != null) {
+          return lowest;
+        }
       }
       m = this.lowestUnflushedSequenceIds.get(encodedRegionName);
       if (m != null) {
-        Long lowest = m.get(familyName);
-        if (lowest != null) return lowest;
+        Long lowest = m.get(familyNameWrapper);
+        if (lowest != null) {
+          return lowest;
+        }
       }
     }
     return HConstants.NO_SEQNUM;
@@ -155,9 +170,9 @@
     Long l = Long.valueOf(sequenceid);
     this.highestSequenceIds.put(encodedRegionName, l);
     if (lowest) {
-      ConcurrentMap<byte[], Long> m = getOrCreateLowestSequenceIds(encodedRegionName);
+      ConcurrentMap<ImmutableByteArray, Long> m = getOrCreateLowestSequenceIds(encodedRegionName);
       for (byte[] familyName : families) {
-        m.putIfAbsent(familyName, l);
+        m.putIfAbsent(ImmutableByteArray.wrap(familyName), l);
       }
     }
   }
@@ -167,49 +182,56 @@
    */
   void updateStore(byte[] encodedRegionName, byte[] familyName, Long sequenceId,
       boolean onlyIfGreater) {
-    if(sequenceId == null) return;
-    Long highest = this.highestSequenceIds.get(encodedRegionName);
-    if(highest == null || sequenceId > highest) {
-      this.highestSequenceIds.put(encodedRegionName,sequenceId);
+    if (sequenceId == null) {
+      return;
     }
+    Long highest = this.highestSequenceIds.get(encodedRegionName);
+    if (highest == null || sequenceId > highest) {
+      this.highestSequenceIds.put(encodedRegionName, sequenceId);
+    }
+    ImmutableByteArray familyNameWrapper = ImmutableByteArray.wrap(familyName);
     synchronized (this.tieLock) {
-      ConcurrentMap<byte[], Long> m = getOrCreateLowestSequenceIds(encodedRegionName);
+      ConcurrentMap<ImmutableByteArray, Long> m = getOrCreateLowestSequenceIds(encodedRegionName);
       boolean replaced = false;
       while (!replaced) {
-        Long oldSeqId = m.get(familyName);
+        Long oldSeqId = m.get(familyNameWrapper);
         if (oldSeqId == null) {
-          m.put(familyName, sequenceId);
+          m.put(familyNameWrapper, sequenceId);
           replaced = true;
         } else if (onlyIfGreater) {
           if (sequenceId > oldSeqId) {
-            replaced = m.replace(familyName, oldSeqId, sequenceId);
+            replaced = m.replace(familyNameWrapper, oldSeqId, sequenceId);
           } else {
             return;
           }
         } else { // replace even if sequence id is not greater than oldSeqId
-          m.put(familyName, sequenceId);
+          m.put(familyNameWrapper, sequenceId);
           return;
         }
       }
     }
   }
 
-  ConcurrentMap<byte[], Long> getOrCreateLowestSequenceIds(byte[] encodedRegionName) {
+  @VisibleForTesting
+  ConcurrentMap<ImmutableByteArray, Long> getOrCreateLowestSequenceIds(byte[] encodedRegionName) {
     // Intentionally, this access is done outside of this.regionSequenceIdLock. Done per append.
-    ConcurrentMap<byte[], Long> m = this.lowestUnflushedSequenceIds.get(encodedRegionName);
-    if (m != null) return m;
-    m = new ConcurrentSkipListMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
+    ConcurrentMap<ImmutableByteArray, Long> m = this.lowestUnflushedSequenceIds
+        .get(encodedRegionName);
+    if (m != null) {
+      return m;
+    }
+    m = new ConcurrentHashMap<>();
     // Another thread may have added it ahead of us.
-    ConcurrentMap<byte[], Long> alreadyPut =
-        this.lowestUnflushedSequenceIds.putIfAbsent(encodedRegionName, m);
-    return alreadyPut == null? m : alreadyPut;
+    ConcurrentMap<ImmutableByteArray, Long> alreadyPut = this.lowestUnflushedSequenceIds
+        .putIfAbsent(encodedRegionName, m);
+    return alreadyPut == null ? m : alreadyPut;
   }
 
   /**
    * @param sequenceids Map to search for lowest value.
    * @return Lowest value found in <code>sequenceids</code>.
    */
-  static long getLowestSequenceId(Map<byte[], Long> sequenceids) {
+  private static long getLowestSequenceId(Map<?, Long> sequenceids) {
     long lowest = HConstants.NO_SEQNUM;
     for (Long sid: sequenceids.values()) {
       if (lowest == HConstants.NO_SEQNUM || sid.longValue() < lowest) {
@@ -222,13 +244,14 @@
   /**
    * @param src
    * @return New Map that has same keys as <code>src</code> but instead of a Map for a value, it
-   * instead has found the smallest sequence id and it returns that as the value instead.
+   *         instead has found the smallest sequence id and it returns that as the value instead.
    */
-  private <T extends Map<byte[], Long>> Map<byte[], Long> flattenToLowestSequenceId(
-      Map<byte[], T> src) {
-    if (src == null || src.isEmpty()) return null;
-    Map<byte[], Long> tgt = Maps.newHashMap();
-    for (Map.Entry<byte[], T> entry: src.entrySet()) {
+  private <T extends Map<?, Long>> Map<byte[], Long> flattenToLowestSequenceId(Map<byte[], T> src) {
+    if (src == null || src.isEmpty()) {
+      return null;
+    }
+    Map<byte[], Long> tgt = new HashMap<>();
+    for (Map.Entry<byte[], T> entry : src.entrySet()) {
       long lowestSeqId = getLowestSequenceId(entry.getValue());
       if (lowestSeqId != HConstants.NO_SEQNUM) {
         tgt.put(entry.getKey(), lowestSeqId);
@@ -247,20 +270,23 @@
    * oldest/lowest outstanding edit.
    */
   Long startCacheFlush(final byte[] encodedRegionName, final Set<byte[]> families) {
-    Map<byte[], Long> oldSequenceIds = null;
+    Map<ImmutableByteArray, Long> oldSequenceIds = null;
     Long lowestUnflushedInRegion = HConstants.NO_SEQNUM;
     synchronized (tieLock) {
-      Map<byte[], Long> m = this.lowestUnflushedSequenceIds.get(encodedRegionName);
+      Map<ImmutableByteArray, Long> m = this.lowestUnflushedSequenceIds.get(encodedRegionName);
       if (m != null) {
         // NOTE: Removal from this.lowestUnflushedSequenceIds must be done in controlled
         // circumstance because another concurrent thread now may add sequenceids for this family
         // (see above in getOrCreateLowestSequenceId). Make sure you are ok with this. Usually it
         // is fine because updates are blocked when this method is called. Make sure!!!
-        for (byte[] familyName: families) {
-          Long seqId = m.remove(familyName);
+        for (byte[] familyName : families) {
+          ImmutableByteArray familyNameWrapper = ImmutableByteArray.wrap(familyName);
+          Long seqId = m.remove(familyNameWrapper);
           if (seqId != null) {
-            if (oldSequenceIds == null) oldSequenceIds = Maps.newTreeMap(Bytes.BYTES_COMPARATOR);
-            oldSequenceIds.put(familyName, seqId);
+            if (oldSequenceIds == null) {
+              oldSequenceIds = new HashMap<>();
+            }
+            oldSequenceIds.put(familyNameWrapper, seqId);
           }
         }
         if (oldSequenceIds != null && !oldSequenceIds.isEmpty()) {
@@ -293,7 +319,7 @@
     return lowestUnflushedInRegion;
   }
 
-  void completeCacheFlush(final byte [] encodedRegionName) {
+  void completeCacheFlush(final byte[] encodedRegionName) {
     synchronized (tieLock) {
       this.flushingSequenceIds.remove(encodedRegionName);
     }
@@ -302,16 +328,16 @@
   void abortCacheFlush(final byte[] encodedRegionName) {
     // Method is called when we are crashing down because failed write flush AND it is called
     // if we fail prepare. The below is for the fail prepare case; we restore the old sequence ids.
-    Map<byte[], Long> flushing = null;
-    Map<byte[], Long> tmpMap = new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
+    Map<ImmutableByteArray, Long> flushing = null;
+    Map<ImmutableByteArray, Long> tmpMap = new HashMap<>();
     // Here we are moving sequenceids from flushing back to unflushed; doing opposite of what
     // happened in startCacheFlush. During prepare phase, we have update lock on the region so
     // no edits should be coming in via append.
     synchronized (tieLock) {
       flushing = this.flushingSequenceIds.remove(encodedRegionName);
       if (flushing != null) {
-        Map<byte[], Long> unflushed = getOrCreateLowestSequenceIds(encodedRegionName);
-        for (Map.Entry<byte[], Long> e: flushing.entrySet()) {
+        Map<ImmutableByteArray, Long> unflushed = getOrCreateLowestSequenceIds(encodedRegionName);
+        for (Map.Entry<ImmutableByteArray, Long> e: flushing.entrySet()) {
           // Set into unflushed the 'old' oldest sequenceid and if any value in flushed with this
           // value, it will now be in tmpMap.
           tmpMap.put(e.getKey(), unflushed.put(e.getKey(), e.getValue()));
@@ -322,12 +348,12 @@
     // Here we are doing some 'test' to see if edits are going in out of order. What is it for?
     // Carried over from old code.
     if (flushing != null) {
-      for (Map.Entry<byte[], Long> e : flushing.entrySet()) {
+      for (Map.Entry<ImmutableByteArray, Long> e : flushing.entrySet()) {
         Long currentId = tmpMap.get(e.getKey());
         if (currentId != null && currentId.longValue() <= e.getValue().longValue()) {
-          String errorStr = Bytes.toString(encodedRegionName) + " family " +
-            Bytes.toString(e.getKey()) + " acquired edits out of order current memstore seq=" +
-              currentId + ", previous oldest unflushed id=" + e.getValue();
+          String errorStr = Bytes.toString(encodedRegionName) + " family "
+              + e.getKey().toStringUtf8() + " acquired edits out of order current memstore seq="
+              + currentId + ", previous oldest unflushed id=" + e.getValue();
           LOG.error(errorStr);
           Runtime.getRuntime().halt(1);
         }
@@ -338,57 +364,63 @@
   /**
    * See if passed <code>sequenceids</code> are lower -- i.e. earlier -- than any outstanding
    * sequenceids, sequenceids we are holding on to in this accounting instance.
-   * @param sequenceids Keyed by encoded region name. Cannot be null (doesn't make
-   * sense for it to be null).
+   * @param sequenceids Keyed by encoded region name. Cannot be null (doesn't make sense for it to
+   *          be null).
    * @return true if all sequenceids are lower, older than, the old sequenceids in this instance.
    */
-   boolean areAllLower(Map<byte[], Long> sequenceids) {
-     Map<byte[], Long> flushing = null;
-     Map<byte[], Long> unflushed = null;
-     synchronized (this.tieLock) {
-       // Get a flattened -- only the oldest sequenceid -- copy of current flushing and unflushed
-       // data structures to use in tests below.
-       flushing = flattenToLowestSequenceId(this.flushingSequenceIds);
-       unflushed = flattenToLowestSequenceId(this.lowestUnflushedSequenceIds);
-     }
+  boolean areAllLower(Map<byte[], Long> sequenceids) {
+    Map<byte[], Long> flushing = null;
+    Map<byte[], Long> unflushed = null;
+    synchronized (this.tieLock) {
+      // Get a flattened -- only the oldest sequenceid -- copy of current flushing and unflushed
+      // data structures to use in tests below.
+      flushing = flattenToLowestSequenceId(this.flushingSequenceIds);
+      unflushed = flattenToLowestSequenceId(this.lowestUnflushedSequenceIds);
+    }
     for (Map.Entry<byte[], Long> e : sequenceids.entrySet()) {
       long oldestFlushing = Long.MAX_VALUE;
       long oldestUnflushed = Long.MAX_VALUE;
-      if (flushing != null) {
-        if (flushing.containsKey(e.getKey())) oldestFlushing = flushing.get(e.getKey());
+      if (flushing != null && flushing.containsKey(e.getKey())) {
+        oldestFlushing = flushing.get(e.getKey());
       }
-      if (unflushed != null) {
-        if (unflushed.containsKey(e.getKey())) oldestUnflushed = unflushed.get(e.getKey());
+      if (unflushed != null && unflushed.containsKey(e.getKey())) {
+        oldestUnflushed = unflushed.get(e.getKey());
       }
       long min = Math.min(oldestFlushing, oldestUnflushed);
-      if (min <= e.getValue()) return false;
+      if (min <= e.getValue()) {
+        return false;
+      }
     }
     return true;
   }
 
-   /**
-    * Iterates over the given Map and compares sequence ids with corresponding
-    * entries in {@link #oldestUnflushedRegionSequenceIds}. If a region in
-    * {@link #oldestUnflushedRegionSequenceIds} has a sequence id less than that passed
-    * in <code>sequenceids</code> then return it.
-    * @param sequenceids Sequenceids keyed by encoded region name.
-    * @return regions found in this instance with sequence ids less than those passed in.
-    */
-   byte[][] findLower(Map<byte[], Long> sequenceids) {
-     List<byte[]> toFlush = null;
-     // Keeping the old behavior of iterating unflushedSeqNums under oldestSeqNumsLock.
-     synchronized (tieLock) {
-       for (Map.Entry<byte[], Long> e: sequenceids.entrySet()) {
-         Map<byte[], Long> m = this.lowestUnflushedSequenceIds.get(e.getKey());
-         if (m == null) continue;
-         // The lowest sequence id outstanding for this region.
-         long lowest = getLowestSequenceId(m);
-         if (lowest != HConstants.NO_SEQNUM && lowest <= e.getValue()) {
-           if (toFlush == null) toFlush = new ArrayList<byte[]>();
-           toFlush.add(e.getKey());
-         }
-       }
-     }
-     return toFlush == null? null: toFlush.toArray(new byte[][] { HConstants.EMPTY_BYTE_ARRAY });
-   }
+  /**
+   * Iterates over the given Map and compares sequence ids with corresponding entries in
+   * {@link #oldestUnflushedRegionSequenceIds}. If a region in
+   * {@link #oldestUnflushedRegionSequenceIds} has a sequence id less than that passed in
+   * <code>sequenceids</code> then return it.
+   * @param sequenceids Sequenceids keyed by encoded region name.
+   * @return regions found in this instance with sequence ids less than those passed in.
+   */
+  byte[][] findLower(Map<byte[], Long> sequenceids) {
+    List<byte[]> toFlush = null;
+    // Keeping the old behavior of iterating unflushedSeqNums under oldestSeqNumsLock.
+    synchronized (tieLock) {
+      for (Map.Entry<byte[], Long> e : sequenceids.entrySet()) {
+        Map<ImmutableByteArray, Long> m = this.lowestUnflushedSequenceIds.get(e.getKey());
+        if (m == null) {
+          continue;
+        }
+        // The lowest sequence id outstanding for this region.
+        long lowest = getLowestSequenceId(m);
+        if (lowest != HConstants.NO_SEQNUM && lowest <= e.getValue()) {
+          if (toFlush == null) {
+            toFlush = new ArrayList<byte[]>();
+          }
+          toFlush.add(e.getKey());
+        }
+      }
+    }
+    return toFlush == null ? null : toFlush.toArray(new byte[0][]);
+  }
 }
\ No newline at end of file