You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by jm...@apache.org on 2013/08/29 22:45:06 UTC

svn commit: r1518818 [1/2] - in /hbase/branches/0.95: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ hbase-common/src/main/java/org/apache/hadoop/hbase/ hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/ hbase-common/src/main/j...

Author: jmhsieh
Date: Thu Aug 29 20:45:05 2013
New Revision: 1518818

URL: http://svn.apache.org/r1518818
Log:
HBASE-9247 Cleanup Key/KV/Meta/MetaKey Comparators

Modified:
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java
    hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
    hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java
    hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java
    hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/CopyKeyDataBlockEncoder.java
    hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DataBlockEncoder.java
    hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DiffKeyDeltaEncoder.java
    hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/FastDiffDeltaEncoder.java
    hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/PrefixKeyDeltaEncoder.java
    hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java
    hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java
    hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeCodec.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileReader.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileWriter.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV2.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/GetClosestRowBeforeTracker.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterBase.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ByteBloomFilter.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilter.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterBase.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterWriter.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTreeEncoding.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestFixedFileTrailer.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFilePerformance.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV2.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestReseekTo.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStore.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java?rev=1518818&r1=1518817&r2=1518818&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java Thu Aug 29 20:45:05 2013
@@ -19,16 +19,6 @@
 
 package org.apache.hadoop.hbase.client;
 
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellScannable;
-import org.apache.hadoop.hbase.CellScanner;
-import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValue.SplitKeyValue;
-import org.apache.hadoop.hbase.util.Bytes;
-
 import java.nio.BufferOverflowException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
@@ -39,6 +29,15 @@ import java.util.Map;
 import java.util.NavigableMap;
 import java.util.TreeMap;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellScannable;
+import org.apache.hadoop.hbase.CellScanner;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.util.Bytes;
+
 /**
  * Single row result of a {@link Get} or {@link Scan} query.<p>
  *
@@ -537,8 +536,7 @@ public class Result implements CellScann
     }
     this.familyMap = new TreeMap<byte[], NavigableMap<byte[], NavigableMap<Long, byte[]>>>(Bytes.BYTES_COMPARATOR);
     for(KeyValue kv : this.kvs) {
-      SplitKeyValue splitKV = kv.split();
-      byte [] family = splitKV.getFamily();
+      byte [] family = kv.getFamily();
       NavigableMap<byte[], NavigableMap<Long, byte[]>> columnMap =
         familyMap.get(family);
       if(columnMap == null) {
@@ -546,7 +544,7 @@ public class Result implements CellScann
           (Bytes.BYTES_COMPARATOR);
         familyMap.put(family, columnMap);
       }
-      byte [] qualifier = splitKV.getQualifier();
+      byte [] qualifier = kv.getQualifier();
       NavigableMap<Long, byte[]> versionMap = columnMap.get(qualifier);
       if(versionMap == null) {
         versionMap = new TreeMap<Long, byte[]>(new Comparator<Long>() {
@@ -556,8 +554,9 @@ public class Result implements CellScann
         });
         columnMap.put(qualifier, versionMap);
       }
-      Long timestamp = Bytes.toLong(splitKV.getTimestamp());
-      byte [] value = splitKV.getValue();
+      Long timestamp = kv.getTimestamp();
+      byte [] value = kv.getValue();
+
       versionMap.put(timestamp, value);
     }
     return this.familyMap;

Modified: hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java?rev=1518818&r1=1518817&r2=1518818&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java (original)
+++ hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java Thu Aug 29 20:45:05 2013
@@ -54,8 +54,8 @@ import com.google.common.primitives.Long
  * <p>
  * Instances of this class are immutable. They do not implement Comparable but Comparators are
  * provided. Comparators change with context, whether user table or a catalog table comparison. Its
- * critical you use the appropriate comparator. There are Comparators for KeyValue instances and
- * then for just the Key portion of a KeyValue used mostly by HFile.
+ * critical you use the appropriate comparator. There are Comparators for normal HFiles, Meta's
+ * Hfiles, and bloom filter keys.
  * <p>
  * KeyValue wraps a byte array and takes offsets and lengths into passed array at where to start
  * interpreting the content as KeyValue. The KeyValue format inside a byte array is:
@@ -71,9 +71,6 @@ import com.google.common.primitives.Long
 public class KeyValue implements Cell, HeapSize, Cloneable {
   static final Log LOG = LogFactory.getLog(KeyValue.class);
 
-  // TODO: Group Key-only comparators and operations into a Key class, just
-  // for neatness sake, if can figure what to call it.
-
   /**
    * Colon character in UTF-8
    */
@@ -83,16 +80,10 @@ public class KeyValue implements Cell, H
     new byte[]{COLUMN_FAMILY_DELIMITER};
 
   /**
-   * Comparator for plain key/values; i.e. non-catalog table key/values.
-   */
-  public static final KVComparator COMPARATOR = new KVComparator();
-
-  /**
-   * Comparator for plain key; i.e. non-catalog table key.  Works on Key portion
+   * Comparator for plain key/values; i.e. non-catalog table key/values. Works on Key portion
    * of KeyValue only.
    */
-  public static final KeyComparator KEY_COMPARATOR = new KeyComparator();
-
+  public static final KVComparator COMPARATOR = new KVComparator();
   /**
    * A {@link KVComparator} for <code>.META.</code> catalog table
    * {@link KeyValue}s.
@@ -100,21 +91,9 @@ public class KeyValue implements Cell, H
   public static final KVComparator META_COMPARATOR = new MetaComparator();
 
   /**
-   * Get the appropriate row comparator for the specified table.
-   *
-   * Hopefully we can get rid of this, I added this here because it's replacing
-   * something in HSK.  We should move completely off of that.
-   *
-   * @param tableName  The table name.
-   * @return The comparator.
+   * Needed for Bloom Filters.
    */
-  @Deprecated // use TableName#getRowComparator
-  public static KeyComparator getRowComparator(TableName tableName) {
-     if(TableName.META_TABLE_NAME.equals(tableName)) {
-      return META_COMPARATOR.getRawComparator();
-    }
-    return COMPARATOR.getRawComparator();
-  }
+  public static final KVComparator RAW_COMPARATOR = new RawBytesComparator();
 
   /** Size of the key length field in bytes*/
   public static final int KEY_LENGTH_SIZE = Bytes.SIZEOF_INT;
@@ -229,9 +208,11 @@ public class KeyValue implements Cell, H
   public static final KeyValue LOWESTKEY =
     new KeyValue(HConstants.EMPTY_BYTE_ARRAY, HConstants.LATEST_TIMESTAMP);
 
-  private byte [] bytes = null;
-  private int offset = 0;
-  private int length = 0;
+  ////
+  // KeyValue core instance fields.
+  private byte [] bytes = null;  // an immutable byte array that contains the KV
+  private int offset = 0;  // offset into bytes buffer KV starts at
+  private int length = 0;  // length of the KV starting from offset.
 
   /**
    * @return True if a delete type, a {@link KeyValue.Type#Delete} or
@@ -247,11 +228,11 @@ public class KeyValue implements Cell, H
   // used to achieve atomic operations in the memstore.
   @Override
   public long getMvccVersion() {
-    return memstoreTS;
+    return mvcc;
   }
 
   public void setMvccVersion(long mvccVersion){
-    this.memstoreTS = mvccVersion;
+    this.mvcc = mvccVersion;
   }
 
   @Deprecated
@@ -264,8 +245,8 @@ public class KeyValue implements Cell, H
     setMvccVersion(memstoreTS);
   }
 
-  // default value is 0, aka DNC
-  private long memstoreTS = 0;
+  // multi-version concurrency control version.  default value is 0, aka do not care.
+  private long mvcc = 0;  // this value is not part of a serialized KeyValue (not in HFiles)
 
   /** Dragon time over, return to normal business */
 
@@ -502,7 +483,7 @@ public class KeyValue implements Cell, H
    * @param vlength
    * @return The newly created byte array.
    */
-  static byte[] createEmptyByteArray(final int rlength, int flength,
+  private static byte[] createEmptyByteArray(final int rlength, int flength,
       int qlength, final long timestamp, final Type type, int vlength) {
     if (rlength > Short.MAX_VALUE) {
       throw new IllegalArgumentException("Row > " + Short.MAX_VALUE);
@@ -756,7 +737,7 @@ public class KeyValue implements Cell, H
    * @param vlength value length
    * @return The newly created byte array.
    */
-  static byte [] createByteArray(final byte [] row, final int roffset,
+  private static byte [] createByteArray(final byte [] row, final int roffset,
       final int rlength, final byte [] family, final int foffset, int flength,
       final byte [] qualifier, final int qoffset, int qlength,
       final long timestamp, final Type type,
@@ -790,46 +771,6 @@ public class KeyValue implements Cell, H
   }
 
   /**
-   * Write KeyValue format into a byte array.
-   * <p>
-   * Takes column in the form <code>family:qualifier</code>
-   * @param row - row key (arbitrary byte array)
-   * @param roffset
-   * @param rlength
-   * @param column
-   * @param coffset
-   * @param clength
-   * @param timestamp
-   * @param type
-   * @param value
-   * @param voffset
-   * @param vlength
-   * @return The newly created byte array.
-   */
-  static byte [] createByteArray(final byte [] row, final int roffset,
-        final int rlength,
-      final byte [] column, final int coffset, int clength,
-      final long timestamp, final Type type,
-      final byte [] value, final int voffset, int vlength) {
-    // If column is non-null, figure where the delimiter is at.
-    int delimiteroffset = 0;
-    if (column != null && column.length > 0) {
-      delimiteroffset = getFamilyDelimiterIndex(column, coffset, clength);
-      if (delimiteroffset > Byte.MAX_VALUE) {
-        throw new IllegalArgumentException("Family > " + Byte.MAX_VALUE);
-      }
-    } else {
-      return createByteArray(row,roffset,rlength,null,0,0,null,0,0,timestamp,
-          type,value,voffset,vlength);
-    }
-    int flength = delimiteroffset-coffset;
-    int qlength = clength - flength - 1;
-    return createByteArray(row, roffset, rlength, column, coffset,
-        flength, column, delimiteroffset+1, qlength, timestamp, type,
-        value, voffset, vlength);
-  }
-
-  /**
    * Needed doing 'contains' on List.  Only compares the key portion, not the value.
    */
   @Override
@@ -871,7 +812,7 @@ public class KeyValue implements Cell, H
     // Important to clone the memstoreTS as well - otherwise memstore's
     // update-in-place methods (eg increment) will end up creating
     // new entries
-    ret.setMvccVersion(memstoreTS);
+    ret.setMvccVersion(mvcc);
     return ret;
   }
 
@@ -882,7 +823,7 @@ public class KeyValue implements Cell, H
    */
   public KeyValue shallowCopy() {
     KeyValue shallowCopy = new KeyValue(this.bytes, this.offset, this.length);
-    shallowCopy.setMvccVersion(this.memstoreTS);
+    shallowCopy.setMvccVersion(this.mvcc);
     return shallowCopy;
   }
 
@@ -897,7 +838,7 @@ public class KeyValue implements Cell, H
       return "empty";
     }
     return keyToString(this.bytes, this.offset + ROW_OFFSET, getKeyLength()) +
-      "/vlen=" + getValueLength() + "/mvcc=" + memstoreTS;
+      "/vlen=" + getValueLength() + "/mvcc=" + mvcc;
   }
 
   /**
@@ -1162,15 +1103,6 @@ public class KeyValue implements Cell, H
   /**
    * @return Column (family + qualifier) length
    */
-  public int getTotalColumnLength() {
-    int rlength = getRowLength();
-    int foffset = getFamilyOffset(rlength);
-    return getTotalColumnLength(rlength,foffset);
-  }
-
-  /**
-   * @return Column (family + qualifier) length
-   */
   private int getTotalColumnLength(int rlength, int foffset) {
     int flength = getFamilyLength(foffset);
     int qlength = getQualifierLength(rlength,flength);
@@ -1202,14 +1134,6 @@ public class KeyValue implements Cell, H
   }
 
   /**
-   * @return True if this is a "fake" KV created for internal seeking purposes,
-   * which should not be seen by user code
-   */
-  public boolean isInternal() {
-    byte type = getType();
-    return type == Type.Minimum.code || type == Type.Maximum.code;
-  }
-  /**
    * @param now Time to set into <code>this</code> IFF timestamp ==
    * {@link HConstants#LATEST_TIMESTAMP} (else, its a noop).
    * @return True is we modified this.
@@ -1406,76 +1330,6 @@ public class KeyValue implements Cell, H
 
   //---------------------------------------------------------------------------
   //
-  //  KeyValue splitter
-  //
-  //---------------------------------------------------------------------------
-
-  /**
-   * Utility class that splits a KeyValue buffer into separate byte arrays.
-   * <p>
-   * Should get rid of this if we can, but is very useful for debugging.
-   */
-  public static class SplitKeyValue {
-    private byte [][] split;
-    SplitKeyValue() {
-      this.split = new byte[6][];
-    }
-    public void setRow(byte [] value) { this.split[0] = value; }
-    public void setFamily(byte [] value) { this.split[1] = value; }
-    public void setQualifier(byte [] value) { this.split[2] = value; }
-    public void setTimestamp(byte [] value) { this.split[3] = value; }
-    public void setType(byte [] value) { this.split[4] = value; }
-    public void setValue(byte [] value) { this.split[5] = value; }
-    public byte [] getRow() { return this.split[0]; }
-    public byte [] getFamily() { return this.split[1]; }
-    public byte [] getQualifier() { return this.split[2]; }
-    public byte [] getTimestamp() { return this.split[3]; }
-    public byte [] getType() { return this.split[4]; }
-    public byte [] getValue() { return this.split[5]; }
-  }
-
-  public SplitKeyValue split() {
-    SplitKeyValue split = new SplitKeyValue();
-    int splitOffset = this.offset;
-    int keyLen = Bytes.toInt(bytes, splitOffset);
-    splitOffset += Bytes.SIZEOF_INT;
-    int valLen = Bytes.toInt(bytes, splitOffset);
-    splitOffset += Bytes.SIZEOF_INT;
-    short rowLen = Bytes.toShort(bytes, splitOffset);
-    splitOffset += Bytes.SIZEOF_SHORT;
-    byte [] row = new byte[rowLen];
-    System.arraycopy(bytes, splitOffset, row, 0, rowLen);
-    splitOffset += rowLen;
-    split.setRow(row);
-    byte famLen = bytes[splitOffset];
-    splitOffset += Bytes.SIZEOF_BYTE;
-    byte [] family = new byte[famLen];
-    System.arraycopy(bytes, splitOffset, family, 0, famLen);
-    splitOffset += famLen;
-    split.setFamily(family);
-    int colLen = keyLen -
-      (rowLen + famLen + Bytes.SIZEOF_SHORT + Bytes.SIZEOF_BYTE +
-      Bytes.SIZEOF_LONG + Bytes.SIZEOF_BYTE);
-    byte [] qualifier = new byte[colLen];
-    System.arraycopy(bytes, splitOffset, qualifier, 0, colLen);
-    splitOffset += colLen;
-    split.setQualifier(qualifier);
-    byte [] timestamp = new byte[Bytes.SIZEOF_LONG];
-    System.arraycopy(bytes, splitOffset, timestamp, 0, Bytes.SIZEOF_LONG);
-    splitOffset += Bytes.SIZEOF_LONG;
-    split.setTimestamp(timestamp);
-    byte [] type = new byte[1];
-    type[0] = bytes[splitOffset];
-    splitOffset += Bytes.SIZEOF_BYTE;
-    split.setType(type);
-    byte [] value = new byte[valLen];
-    System.arraycopy(bytes, splitOffset, value, 0, valLen);
-    split.setValue(value);
-    return split;
-  }
-
-  //---------------------------------------------------------------------------
-  //
   //  Compare specified fields against those contained in this KeyValue
   //
   //---------------------------------------------------------------------------
@@ -1485,22 +1339,13 @@ public class KeyValue implements Cell, H
    * @return True if matching families.
    */
   public boolean matchingFamily(final byte [] family) {
-    return matchingFamily(family, 0, family.length);
-  }
-
-  public boolean matchingFamily(final byte[] family, int offset, int length) {
     if (this.length == 0 || this.bytes.length == 0) {
       return false;
     }
-    return Bytes.equals(family, offset, length,
+    return Bytes.equals(family, 0, family.length,
         this.bytes, getFamilyOffset(), getFamilyLength());
   }
 
-  public boolean matchingFamily(final KeyValue other) {
-    return matchingFamily(other.getBuffer(), other.getFamilyOffset(),
-        other.getFamilyLength());
-  }
-
   /**
    * @param qualifier
    * @return True if matching qualifiers.
@@ -1534,18 +1379,6 @@ public class KeyValue implements Cell, H
   }
 
   /**
-   * @param column Column minus its delimiter
-   * @return True if column matches.
-   */
-  public boolean matchingColumnNoDelimiter(final byte [] column) {
-    int rl = getRowLength();
-    int o = getFamilyOffset(rl);
-    int fl = getFamilyLength(o);
-    int l = fl + getQualifierLength(rl,fl);
-    return Bytes.equals(column, 0, column.length, this.bytes, o, l);
-  }
-
-  /**
    *
    * @param family column family
    * @param qualifier column qualifier
@@ -1584,47 +1417,6 @@ public class KeyValue implements Cell, H
   }
 
   /**
-   * @param left
-   * @param loffset
-   * @param llength
-   * @param lfamilylength Offset of family delimiter in left column.
-   * @param right
-   * @param roffset
-   * @param rlength
-   * @param rfamilylength Offset of family delimiter in right column.
-   * @return The result of the comparison.
-   */
-  static int compareColumns(final byte [] left, final int loffset,
-      final int llength, final int lfamilylength,
-      final byte [] right, final int roffset, final int rlength,
-      final int rfamilylength) {
-    // Compare family portion first.
-    int diff = Bytes.compareTo(left, loffset, lfamilylength,
-      right, roffset, rfamilylength);
-    if (diff != 0) {
-      return diff;
-    }
-    // Compare qualifier portion
-    return Bytes.compareTo(left, loffset + lfamilylength,
-      llength - lfamilylength,
-      right, roffset + rfamilylength, rlength - rfamilylength);
-  }
-
-  /**
-   * @return True if non-null row and column.
-   */
-  public boolean nonNullRowAndColumn() {
-    return getRowLength() > 0 && !isEmptyColumn();
-  }
-
-  /**
-   * @return True if column is empty.
-   */
-  public boolean isEmptyColumn() {
-    return getQualifierLength() == 0;
-  }
-
-  /**
    * Creates a new KeyValue that only contains the key portion (the value is
    * set to be null).
    * @param lenAsVal replace value with the actual value length (false=empty)
@@ -1685,26 +1477,6 @@ public class KeyValue implements Cell, H
   }
 
   /**
-   * @param b
-   * @return Index of the family-qualifier colon delimiter character in passed
-   * buffer.
-   */
-  public static int getFamilyDelimiterIndex(final byte [] b, final int offset,
-      final int length) {
-    return getRequiredDelimiter(b, offset, length, COLUMN_FAMILY_DELIMITER);
-  }
-
-  private static int getRequiredDelimiter(final byte [] b,
-      final int offset, final int length, final int delimiter) {
-    int index = getDelimiter(b, offset, length, delimiter);
-    if (index < 0) {
-      throw new IllegalArgumentException("No " + (char)delimiter + " in <" +
-        Bytes.toString(b) + ">" + ", length=" + length + ", offset=" + offset);
-    }
-    return index;
-  }
-
-  /**
    * This function is only used in Meta key comparisons so its error message
    * is specific for meta key errors.
    */
@@ -1765,29 +1537,13 @@ public class KeyValue implements Cell, H
    * {@link KeyValue}s.
    */
   public static class MetaComparator extends KVComparator {
-    private final KeyComparator rawcomparator = new MetaKeyComparator();
-
-    public KeyComparator getRawComparator() {
-      return this.rawcomparator;
-    }
-
-    @Override
-    protected Object clone() throws CloneNotSupportedException {
-      return new MetaComparator();
-    }
-
     /**
-     * Override the row key comparision to parse and compare the meta row key parts.
+     * Compare key portion of a {@link KeyValue} for keys in <code>.META.</code>
+     * table.
      */
     @Override
-    protected int compareRowKey(final Cell l, final Cell r) {
-      byte[] left = l.getRowArray();
-      int loffset = l.getRowOffset();
-      int llength = l.getRowLength();
-      byte[] right = r.getRowArray();
-      int roffset = r.getRowOffset();
-      int rlength = r.getRowLength();
-
+    public int compareRows(byte [] left, int loffset, int llength,
+        byte [] right, int roffset, int rlength) {
       int leftDelimiter = getDelimiter(left, loffset, llength,
           HConstants.DELIMITER);
       int rightDelimiter = getDelimiter(right, roffset, rlength,
@@ -1816,45 +1572,127 @@ public class KeyValue implements Cell, H
           rightDelimiter, rlength - (rightDelimiter - roffset),
           HConstants.DELIMITER);
       // Now compare middlesection of row.
-      result = Bytes.compareTo(
-          left,  leftDelimiter,  leftFarDelimiter - leftDelimiter,
-          right, rightDelimiter, rightFarDelimiter - rightDelimiter);
+      result = super.compareRows(left, leftDelimiter,
+          leftFarDelimiter - leftDelimiter, right, rightDelimiter,
+          rightFarDelimiter - rightDelimiter);
       if (result != 0) {
         return result;
       }
       // Compare last part of row, the rowid.
       leftFarDelimiter++;
       rightFarDelimiter++;
-      result = Bytes.compareTo(
-          left,  leftFarDelimiter,  llength - (leftFarDelimiter - loffset),
+      result = Bytes.compareTo(left, leftFarDelimiter, llength - (leftFarDelimiter - loffset),
           right, rightFarDelimiter, rlength - (rightFarDelimiter - roffset));
       return result;
     }
+
+    /**
+     * Don't do any fancy Block Index splitting tricks.
+     */
+    @Override
+    public byte[] getShortMidpointKey(final byte[] leftKey, final byte[] rightKey) {
+      return Arrays.copyOf(rightKey, rightKey.length);
+    }
+
+    /**
+     * The HFileV2 file format's trailer contains this class name.  We reinterpret this and
+     * instantiate the appropriate comparator.
+     * TODO: With V3 consider removing this.
+     * @return legacy class name for FileFileTrailer#comparatorClassName
+     */
+    @Override
+    public String getLegacyKeyComparatorName() {
+      return "org.apache.hadoop.hbase.KeyValue$MetaKeyComparator";
+    }
+
+    @Override
+    protected Object clone() throws CloneNotSupportedException {
+      return new MetaComparator();
+    }
+
+    /**
+     * Override the row key comparison to parse and compare the meta row key parts.
+     */
+    @Override
+    protected int compareRowKey(final Cell l, final Cell r) {
+      byte[] left = l.getRowArray();
+      int loffset = l.getRowOffset();
+      int llength = l.getRowLength();
+      byte[] right = r.getRowArray();
+      int roffset = r.getRowOffset();
+      int rlength = r.getRowLength();
+      return compareRows(left, loffset, llength, right, roffset, rlength);
+    }
   }
 
   /**
    * Compare KeyValues.  When we compare KeyValues, we only compare the Key
    * portion.  This means two KeyValues with same Key but different Values are
    * considered the same as far as this Comparator is concerned.
-   * Hosts a {@link KeyComparator}.
    */
-  public static class KVComparator implements java.util.Comparator<Cell> {
-    private final KeyComparator rawcomparator = new KeyComparator();
+  public static class KVComparator implements RawComparator<Cell>, SamePrefixComparator<byte[]> {
 
     /**
-     * @return RawComparator that can compare the Key portion of a KeyValue.
-     * Used in hfile where indices are the Key portion of a KeyValue.
+     * The HFileV2 file format's trailer contains this class name.  We reinterpret this and
+     * instantiate the appropriate comparator.
+     * TODO: With V3 consider removing this.
+     * @return legacy class name for FileFileTrailer#comparatorClassName
      */
-    public KeyComparator getRawComparator() {
-      return this.rawcomparator;
+    public String getLegacyKeyComparatorName() {
+      return "org.apache.hadoop.hbase.KeyValue$KeyComparator";
+    }
+
+    @Override // RawComparator
+    public int compare(byte[] l, int loff, int llen, byte[] r, int roff, int rlen) {
+      return compareFlatKey(l,loff,llen, r,roff,rlen);
     }
 
+    
+    /**
+     * Compares the only the user specified portion of a Key.  This is overridden by MetaComparator.
+     * @param left
+     * @param right
+     * @return 0 if equal, <0 if left smaller, >0 if right smaller
+     */
     protected int compareRowKey(final Cell left, final Cell right) {
       return Bytes.compareTo(
           left.getRowArray(),  left.getRowOffset(),  left.getRowLength(),
           right.getRowArray(), right.getRowOffset(), right.getRowLength());
     }
-    
+
+    /**
+     * Compares left to right assuming that left,loffset,llength and right,roffset,rlength are
+     * full KVs laid out in a flat byte[]s.
+     * @param left
+     * @param loffset
+     * @param llength
+     * @param right
+     * @param roffset
+     * @param rlength
+     * @return  0 if equal, <0 if left smaller, >0 if right smaller
+     */
+    public int compareFlatKey(byte[] left, int loffset, int llength,
+        byte[] right, int roffset, int rlength) {
+      // Compare row
+      short lrowlength = Bytes.toShort(left, loffset);
+      short rrowlength = Bytes.toShort(right, roffset);
+      int compare = compareRows(left, loffset + Bytes.SIZEOF_SHORT,
+          lrowlength, right, roffset + Bytes.SIZEOF_SHORT, rrowlength);
+      if (compare != 0) {
+        return compare;
+      }
+
+      // Compare the rest of the two KVs without making any assumptions about
+      // the common prefix. This function will not compare rows anyway, so we
+      // don't need to tell it that the common prefix includes the row.
+      return compareWithoutRow(0, left, loffset, llength, right, roffset,
+          rlength, rrowlength);
+    }
+
+    public int compareFlatKey(byte[] left, byte[] right) {
+      return compareFlatKey(left, 0, left.length, right, 0, right.length);
+    }
+
     /**
      * Compares the Key of a cell -- with fields being more significant in this order:
      * rowkey, colfam/qual, timestamp, type, mvcc
@@ -1866,7 +1704,6 @@ public class KeyValue implements Cell, H
         return compare;
       }
 
-
       // compare vs minimum
       byte ltype = left.getTypeByte();
       byte rtype = right.getTypeByte();
@@ -1895,7 +1732,7 @@ public class KeyValue implements Cell, H
       if (compare != 0) {
         return compare;
       }
-      
+
       // Compare qualifier
       compare = Bytes.compareTo(
           left.getQualifierArray(), left.getQualifierOffset(), left.getQualifierLength(),
@@ -1904,11 +1741,10 @@ public class KeyValue implements Cell, H
         return compare;
       }
 
-      
       // compare timestamp
       long ltimestamp = left.getTimestamp();
       long rtimestamp = right.getTimestamp();
-      compare = KeyComparator.compareTimestamps(ltimestamp, rtimestamp);
+      compare = compareTimestamps(ltimestamp, rtimestamp);
       if (compare != 0) {
         return compare;
       }
@@ -1921,24 +1757,17 @@ public class KeyValue implements Cell, H
       if (compare != 0) {
         return compare;
       }
-      
-      // compare Mvcc Version
 
+      // compare Mvcc Version
       // Negate this comparison so later edits show up first
       return -Longs.compare(left.getMvccVersion(), right.getMvccVersion());
     }
 
     public int compareTimestamps(final KeyValue left, final KeyValue right) {
-      return compareTimestamps(left, left.getKeyLength(), right,
-        right.getKeyLength());
-    }
-
-    int compareTimestamps(final KeyValue left, final int lkeylength,
-        final KeyValue right, final int rkeylength) {
       // Compare timestamps
-      long ltimestamp = left.getTimestamp(lkeylength);
-      long rtimestamp = right.getTimestamp(rkeylength);
-      return KeyComparator.compareTimestamps(ltimestamp, rtimestamp);
+      long ltimestamp = left.getTimestamp(left.getKeyLength());
+      long rtimestamp = right.getTimestamp(right.getKeyLength());
+      return compareTimestamps(ltimestamp, rtimestamp);
     }
 
     /**
@@ -1947,47 +1776,23 @@ public class KeyValue implements Cell, H
      * @return Result comparing rows.
      */
     public int compareRows(final KeyValue left, final KeyValue right) {
-      return compareRows(left, left.getRowLength(), right,
-          right.getRowLength());
+      return compareRows(left.getBuffer(),left.getRowOffset(), left.getRowLength(),
+      right.getBuffer(), right.getRowOffset(), right.getRowLength());
     }
 
     /**
+     * Get the b[],o,l for left and right rowkey portions and compare.
      * @param left
-     * @param lrowlength Length of left row.
+     * @param loffset
+     * @param llength
      * @param right
-     * @param rrowlength Length of right row.
-     * @return Result comparing rows.
-     */
-    public int compareRows(final KeyValue left, final short lrowlength,
-        final KeyValue right, final short rrowlength) {
-      return getRawComparator().compareRows(left.getBuffer(),
-          left.getRowOffset(), lrowlength,
-        right.getBuffer(), right.getRowOffset(), rrowlength);
-    }
-
-    /**
-     * @param left
-     * @param row - row key (arbitrary byte array)
-     * @return RawComparator
+     * @param roffset
+     * @param rlength
+     * @return 0 if equal, <0 if left smaller, >0 if right smaller
      */
-    public int compareRows(final KeyValue left, final byte [] row) {
-      return getRawComparator().compareRows(left.getBuffer(),
-          left.getRowOffset(), left.getRowLength(), row, 0, row.length);
-    }
-
     public int compareRows(byte [] left, int loffset, int llength,
         byte [] right, int roffset, int rlength) {
-      return getRawComparator().compareRows(left, loffset, llength,
-        right, roffset, rlength);
-    }
-
-    public int compareColumns(final KeyValue left, final byte [] right,
-        final int roffset, final int rlength, final int rfamilyoffset) {
-      int offset = left.getFamilyOffset();
-      int length = left.getFamilyLength() + left.getQualifierLength();
-      return getRawComparator().compareColumns(left.getBuffer(), offset, length,
-        left.getFamilyLength(offset),
-        right, roffset, rlength, rfamilyoffset);
+      return Bytes.compareTo(left, loffset, llength, right, roffset, rlength);
     }
 
     int compareColumns(final KeyValue left, final short lrowlength,
@@ -1998,92 +1803,334 @@ public class KeyValue implements Cell, H
       int rclength = right.getTotalColumnLength(rrowlength, rfoffset);
       int lfamilylength = left.getFamilyLength(lfoffset);
       int rfamilylength = right.getFamilyLength(rfoffset);
-      return getRawComparator().compareColumns(left.getBuffer(), lfoffset,
+      return compareColumns(left.getBuffer(), lfoffset,
           lclength, lfamilylength,
         right.getBuffer(), rfoffset, rclength, rfamilylength);
     }
 
-    /**
-     * Compares the row and column of two keyvalues for equality
-     * @param left
-     * @param right
-     * @return True if same row and column.
-     */
-    public boolean matchingRowColumn(final KeyValue left,
-        final KeyValue right) {
-      short lrowlength = left.getRowLength();
-      short rrowlength = right.getRowLength();
-      // TsOffset = end of column data. just comparing Row+CF length of each
-      return ((left.getTimestampOffset() - left.getOffset()) ==
-              (right.getTimestampOffset() - right.getOffset())) &&
-        matchingRows(left, lrowlength, right, rrowlength) &&
-        compareColumns(left, lrowlength, right, rrowlength) == 0;
-    }
+    protected int compareColumns(
+        byte [] left, int loffset, int llength, final int lfamilylength,
+        byte [] right, int roffset, int rlength, final int rfamilylength) {
+      // Compare family portion first.
+      int diff = Bytes.compareTo(left, loffset, lfamilylength,
+        right, roffset, rfamilylength);
+      if (diff != 0) {
+        return diff;
+      }
+      // Compare qualifier portion
+      return Bytes.compareTo(left, loffset + lfamilylength,
+        llength - lfamilylength,
+        right, roffset + rfamilylength, rlength - rfamilylength);
+      }
 
-    /**
-     * @param left
-     * @param right
-     * @return True if rows match.
-     */
-    public boolean matchingRows(final KeyValue left, final byte [] right) {
-      return Bytes.equals(left.getBuffer(), left.getRowOffset(), left.getRowLength(),
-          right, 0, right.length);
+    static int compareTimestamps(final long ltimestamp, final long rtimestamp) {
+      // The below older timestamps sorting ahead of newer timestamps looks
+      // wrong but it is intentional. This way, newer timestamps are first
+      // found when we iterate over a memstore and newer versions are the
+      // first we trip over when reading from a store file.
+      if (ltimestamp < rtimestamp) {
+        return 1;
+      } else if (ltimestamp > rtimestamp) {
+        return -1;
+      }
+      return 0;
     }
 
     /**
-     * Compares the row of two keyvalues for equality
+     * Overridden
+     * @param commonPrefix
      * @param left
+     * @param loffset
+     * @param llength
      * @param right
-     * @return True if rows match.
+     * @param roffset
+     * @param rlength
+     * @return 0 if equal, <0 if left smaller, >0 if right smaller
      */
-    public boolean matchingRows(final KeyValue left, final KeyValue right) {
-      short lrowlength = left.getRowLength();
-      short rrowlength = right.getRowLength();
-      return matchingRows(left, lrowlength, right, rrowlength);
+    @Override // SamePrefixComparator
+    public int compareIgnoringPrefix(int commonPrefix, byte[] left,
+        int loffset, int llength, byte[] right, int roffset, int rlength) {
+      // Compare row
+      short lrowlength = Bytes.toShort(left, loffset);
+      short rrowlength;
+
+      int comparisonResult = 0;
+      if (commonPrefix < ROW_LENGTH_SIZE) {
+        // almost nothing in common
+        rrowlength = Bytes.toShort(right, roffset);
+        comparisonResult = compareRows(left, loffset + ROW_LENGTH_SIZE,
+            lrowlength, right, roffset + ROW_LENGTH_SIZE, rrowlength);
+      } else { // the row length is the same
+        rrowlength = lrowlength;
+        if (commonPrefix < ROW_LENGTH_SIZE + rrowlength) {
+          // The rows are not the same. Exclude the common prefix and compare
+          // the rest of the two rows.
+          int common = commonPrefix - ROW_LENGTH_SIZE;
+          comparisonResult = compareRows(
+              left, loffset + common + ROW_LENGTH_SIZE, lrowlength - common,
+              right, roffset + common + ROW_LENGTH_SIZE, rrowlength - common);
+        }
+      }
+      if (comparisonResult != 0) {
+        return comparisonResult;
+      }
+
+      assert lrowlength == rrowlength;
+      return compareWithoutRow(commonPrefix, left, loffset, llength, right,
+          roffset, rlength, lrowlength);
     }
 
     /**
-     * @param left
-     * @param lrowlength
-     * @param right
-     * @param rrowlength
-     * @return True if rows match.
-     */
-    public boolean matchingRows(final KeyValue left, final short lrowlength,
-        final KeyValue right, final short rrowlength) {
-      return lrowlength == rrowlength &&
-          Bytes.equals(left.getBuffer(), left.getRowOffset(), lrowlength,
-              right.getBuffer(), right.getRowOffset(), rrowlength);
-    }
+     * Compare columnFamily, qualifier, timestamp, and key type (everything
+     * except the row). This method is used both in the normal comparator and
+     * the "same-prefix" comparator. Note that we are assuming that row portions
+     * of both KVs have already been parsed and found identical, and we don't
+     * validate that assumption here.
+     * @param commonPrefix
+     *          the length of the common prefix of the two key-values being
+     *          compared, including row length and row
+     */
+    private int compareWithoutRow(int commonPrefix, byte[] left, int loffset,
+        int llength, byte[] right, int roffset, int rlength, short rowlength) {
+      /***
+       * KeyValue Format and commonLength:
+       * |_keyLen_|_valLen_|_rowLen_|_rowKey_|_famiLen_|_fami_|_Quali_|....
+       * ------------------|-------commonLength--------|--------------
+       */
+      int commonLength = ROW_LENGTH_SIZE + FAMILY_LENGTH_SIZE + rowlength;
+
+      // commonLength + TIMESTAMP_TYPE_SIZE
+      int commonLengthWithTSAndType = TIMESTAMP_TYPE_SIZE + commonLength;
+      // ColumnFamily + Qualifier length.
+      int lcolumnlength = llength - commonLengthWithTSAndType;
+      int rcolumnlength = rlength - commonLengthWithTSAndType;
+
+      byte ltype = left[loffset + (llength - 1)];
+      byte rtype = right[roffset + (rlength - 1)];
+
+      // If the column is not specified, the "minimum" key type appears the
+      // latest in the sorted order, regardless of the timestamp. This is used
+      // for specifying the last key/value in a given row, because there is no
+      // "lexicographically last column" (it would be infinitely long). The
+      // "maximum" key type does not need this behavior.
+      if (lcolumnlength == 0 && ltype == Type.Minimum.getCode()) {
+        // left is "bigger", i.e. it appears later in the sorted order
+        return 1;
+      }
+      if (rcolumnlength == 0 && rtype == Type.Minimum.getCode()) {
+        return -1;
+      }
+
+      int lfamilyoffset = commonLength + loffset;
+      int rfamilyoffset = commonLength + roffset;
+
+      // Column family length.
+      int lfamilylength = left[lfamilyoffset - 1];
+      int rfamilylength = right[rfamilyoffset - 1];
+      // If left family size is not equal to right family size, we need not
+      // compare the qualifiers.
+      boolean sameFamilySize = (lfamilylength == rfamilylength);
+      int common = 0;
+      if (commonPrefix > 0) {
+        common = Math.max(0, commonPrefix - commonLength);
+        if (!sameFamilySize) {
+          // Common should not be larger than Math.min(lfamilylength,
+          // rfamilylength).
+          common = Math.min(common, Math.min(lfamilylength, rfamilylength));
+        } else {
+          common = Math.min(common, Math.min(lcolumnlength, rcolumnlength));
+        }
+      }
+      if (!sameFamilySize) {
+        // comparing column family is enough.
+        return Bytes.compareTo(left, lfamilyoffset + common, lfamilylength
+            - common, right, rfamilyoffset + common, rfamilylength - common);
+      }
+      // Compare family & qualifier together.
+      final int comparison = Bytes.compareTo(left, lfamilyoffset + common,
+          lcolumnlength - common, right, rfamilyoffset + common,
+          rcolumnlength - common);
+      if (comparison != 0) {
+        return comparison;
+      }
+
+      ////
+      // Next compare timestamps.
+      long ltimestamp = Bytes.toLong(left,
+          loffset + (llength - TIMESTAMP_TYPE_SIZE));
+      long rtimestamp = Bytes.toLong(right,
+          roffset + (rlength - TIMESTAMP_TYPE_SIZE));
+      int compare = compareTimestamps(ltimestamp, rtimestamp);
+      if (compare != 0) {
+        return compare;
+      }
 
-    public boolean matchingRows(final byte [] left, final int loffset,
-        final int llength,
-        final byte [] right, final int roffset, final int rlength) {
-      return Bytes.equals(left, loffset, llength,
-          right, roffset, rlength);
+      // Compare types. Let the delete types sort ahead of puts; i.e. types
+      // of higher numbers sort before those of lesser numbers. Maximum (255)
+      // appears ahead of everything, and minimum (0) appears after
+      // everything.
+      return (0xff & rtype) - (0xff & ltype);
     }
 
     /**
-     * Compares the row and timestamp of two keys
-     * Was called matchesWithoutColumn in HStoreKey.
-     * @param right Key to compare against.
-     * @return True if same row and timestamp is greater than the timestamp in
-     * <code>right</code>
+     * Compares the row and column of two keyvalues for equality
+     * @param left
+     * @param right
+     * @return True if same row and column.
      */
-    public boolean matchingRowsGreaterTimestamp(final KeyValue left,
+    public boolean matchingRowColumn(final KeyValue left,
         final KeyValue right) {
       short lrowlength = left.getRowLength();
       short rrowlength = right.getRowLength();
+
+      // TsOffset = end of column data. just comparing Row+CF length of each
+      if ((left.getTimestampOffset() - left.getOffset()) !=
+          (right.getTimestampOffset() - right.getOffset())) {
+        return false;
+      }
+
       if (!matchingRows(left, lrowlength, right, rrowlength)) {
         return false;
       }
-      return left.getTimestamp() >= right.getTimestamp();
+
+      int lfoffset = left.getFamilyOffset(lrowlength);
+      int rfoffset = right.getFamilyOffset(rrowlength);
+      int lclength = left.getTotalColumnLength(lrowlength,lfoffset);
+      int rclength = right.getTotalColumnLength(rrowlength, rfoffset);
+      int lfamilylength = left.getFamilyLength(lfoffset);
+      int rfamilylength = right.getFamilyLength(rfoffset);
+      int ccRes = compareColumns(left.getBuffer(), lfoffset, lclength, lfamilylength,
+          right.getBuffer(), rfoffset, rclength, rfamilylength);
+      return ccRes == 0;
+    }
+
+    /**
+     * Compares the row of two keyvalues for equality
+     * @param left
+     * @param right
+     * @return True if rows match.
+     */
+    public boolean matchingRows(final KeyValue left, final KeyValue right) {
+      short lrowlength = left.getRowLength();
+      short rrowlength = right.getRowLength();
+      return matchingRows(left, lrowlength, right, rrowlength);
+    }
+
+    /**
+     * @param left
+     * @param lrowlength
+     * @param right
+     * @param rrowlength
+     * @return True if rows match.
+     */
+    private boolean matchingRows(final KeyValue left, final short lrowlength,
+        final KeyValue right, final short rrowlength) {
+      return lrowlength == rrowlength &&
+          Bytes.equals(left.getBuffer(), left.getRowOffset(), lrowlength,
+              right.getBuffer(), right.getRowOffset(), rrowlength);
+    }
+
+    public byte[] calcIndexKey(byte[] lastKeyOfPreviousBlock, byte[] firstKeyInBlock) {
+      byte[] fakeKey = getShortMidpointKey(lastKeyOfPreviousBlock, firstKeyInBlock);
+      if (compareFlatKey(fakeKey, firstKeyInBlock) > 0) {
+        LOG.error("Unexpected getShortMidpointKey result, fakeKey:"
+            + Bytes.toStringBinary(fakeKey) + ", firstKeyInBlock:"
+            + Bytes.toStringBinary(firstKeyInBlock));
+        return firstKeyInBlock;
+      }
+      if (lastKeyOfPreviousBlock != null && compareFlatKey(lastKeyOfPreviousBlock, fakeKey) >= 0) {
+        LOG.error("Unexpected getShortMidpointKey result, lastKeyOfPreviousBlock:" +
+            Bytes.toStringBinary(lastKeyOfPreviousBlock) + ", fakeKey:" +
+            Bytes.toStringBinary(fakeKey));
+        return firstKeyInBlock;
+      }
+      return fakeKey;
+    }
+
+    /**
+     * This is a HFile block index key optimization.
+     * @param leftKey
+     * @param rightKey
+     * @return 0 if equal, <0 if left smaller, >0 if right smaller
+     */
+    public byte[] getShortMidpointKey(final byte[] leftKey, final byte[] rightKey) {
+      if (rightKey == null) {
+        throw new IllegalArgumentException("rightKey can not be null");
+      }
+      if (leftKey == null) {
+        return Arrays.copyOf(rightKey, rightKey.length);
+      }
+      if (compareFlatKey(leftKey, rightKey) >= 0) {
+        throw new IllegalArgumentException("Unexpected input, leftKey:" + Bytes.toString(leftKey)
+          + ", rightKey:" + Bytes.toString(rightKey));
+      }
+
+      short leftRowLength = Bytes.toShort(leftKey, 0);
+      short rightRowLength = Bytes.toShort(rightKey, 0);
+      int leftCommonLength = ROW_LENGTH_SIZE + FAMILY_LENGTH_SIZE + leftRowLength;
+      int rightCommonLength = ROW_LENGTH_SIZE + FAMILY_LENGTH_SIZE + rightRowLength;
+      int leftCommonLengthWithTSAndType = TIMESTAMP_TYPE_SIZE + leftCommonLength;
+      int rightCommonLengthWithTSAndType = TIMESTAMP_TYPE_SIZE + rightCommonLength;
+      int leftColumnLength = leftKey.length - leftCommonLengthWithTSAndType;
+      int rightColumnLength = rightKey.length - rightCommonLengthWithTSAndType;
+      // rows are equal
+      if (leftRowLength == rightRowLength && compareRows(leftKey, ROW_LENGTH_SIZE, leftRowLength,
+        rightKey, ROW_LENGTH_SIZE, rightRowLength) == 0) {
+        // Compare family & qualifier together.
+        int comparison = Bytes.compareTo(leftKey, leftCommonLength, leftColumnLength, rightKey,
+          rightCommonLength, rightColumnLength);
+        // same with "row + family + qualifier", return rightKey directly
+        if (comparison == 0) {
+          return Arrays.copyOf(rightKey, rightKey.length);
+        }
+        // "family + qualifier" are different, generate a faked key per rightKey
+        byte[] newKey = Arrays.copyOf(rightKey, rightKey.length);
+        Bytes.putLong(newKey, rightKey.length - TIMESTAMP_TYPE_SIZE, HConstants.LATEST_TIMESTAMP);
+        Bytes.putByte(newKey, rightKey.length - TYPE_SIZE, Type.Maximum.getCode());
+        return newKey;
+      }
+      // rows are different
+      short minLength = leftRowLength < rightRowLength ? leftRowLength : rightRowLength;
+      short diffIdx = 0;
+      while (diffIdx < minLength
+          && leftKey[ROW_LENGTH_SIZE + diffIdx] == rightKey[ROW_LENGTH_SIZE + diffIdx]) {
+        diffIdx++;
+      }
+      if (diffIdx >= minLength) {
+        // leftKey's row is prefix of rightKey's. we can optimize it in future
+        return Arrays.copyOf(rightKey, rightKey.length);
+      }
+      int diffByte = leftKey[ROW_LENGTH_SIZE + diffIdx];
+      if ((0xff & diffByte) < 0xff && (diffByte + 1) <
+          (rightKey[ROW_LENGTH_SIZE + diffIdx] & 0xff)) {
+        byte[] newRowKey = new byte[diffIdx + 1];
+        System.arraycopy(leftKey, ROW_LENGTH_SIZE, newRowKey, 0, diffIdx);
+        newRowKey[diffIdx] = (byte) (diffByte + 1);
+        int rightFamilyLength = rightKey[rightCommonLength - 1];
+        byte[] family = null;
+        if (rightFamilyLength > 0) {
+          family = new byte[rightFamilyLength];
+          System.arraycopy(rightKey, rightCommonLength, family, 0, rightFamilyLength);
+        }
+        int rightQualifierLength = rightColumnLength - rightFamilyLength;
+        byte[] qualifier = null;
+        if (rightQualifierLength > 0) {
+          qualifier = new byte[rightQualifierLength];
+          System.arraycopy(rightKey, rightCommonLength + rightFamilyLength, qualifier, 0,
+            rightQualifierLength);
+        }
+        return new KeyValue(newRowKey, null, null, HConstants.LATEST_TIMESTAMP,
+          Type.Maximum).getKey();
+      }
+      // the following is optimizable in future
+      return Arrays.copyOf(rightKey, rightKey.length);
     }
 
     @Override
     protected Object clone() throws CloneNotSupportedException {
       return new KVComparator();
     }
+
   }
 
   /**
@@ -2452,10 +2499,10 @@ public class KeyValue implements Cell, H
   /**
    * Comparator that compares row component only of a KeyValue.
    */
-  public static class RowComparator implements Comparator<KeyValue> {
+  public static class RowOnlyComparator implements Comparator<KeyValue> {
     final KVComparator comparator;
 
-    public RowComparator(final KVComparator c) {
+    public RowOnlyComparator(final KVComparator c) {
       this.comparator = c;
     }
 
@@ -2464,69 +2511,11 @@ public class KeyValue implements Cell, H
     }
   }
 
+
   /**
-   * Compare key portion of a {@link KeyValue} for keys in <code>.META.</code>
-   * table.
-   */
-  public static class MetaKeyComparator extends KeyComparator {
-    public int compareRows(byte [] left, int loffset, int llength,
-        byte [] right, int roffset, int rlength) {
-      int leftDelimiter = getDelimiter(left, loffset, llength,
-          HConstants.DELIMITER);
-      int rightDelimiter = getDelimiter(right, roffset, rlength,
-          HConstants.DELIMITER);
-      if (leftDelimiter < 0 && rightDelimiter >= 0) {
-        // Nothing between .META. and regionid.  Its first key.
-        return -1;
-      } else if (rightDelimiter < 0 && leftDelimiter >= 0) {
-        return 1;
-      } else if (leftDelimiter < 0 && rightDelimiter < 0) {
-        return 0;
-      }
-      // Compare up to the delimiter
-      int result = Bytes.compareTo(left, loffset, leftDelimiter - loffset,
-          right, roffset, rightDelimiter - roffset);
-      if (result != 0) {
-        return result;
-      }
-      // Compare middle bit of the row.
-      // Move past delimiter
-      leftDelimiter++;
-      rightDelimiter++;
-      int leftFarDelimiter = getRequiredDelimiterInReverse(left, leftDelimiter,
-          llength - (leftDelimiter - loffset), HConstants.DELIMITER);
-      int rightFarDelimiter = getRequiredDelimiterInReverse(right,
-          rightDelimiter, rlength - (rightDelimiter - roffset),
-          HConstants.DELIMITER);
-      // Now compare middlesection of row.
-      result = super.compareRows(left, leftDelimiter,
-          leftFarDelimiter - leftDelimiter, right, rightDelimiter,
-          rightFarDelimiter - rightDelimiter);
-      if (result != 0) {
-        return result;
-      }
-      // Compare last part of row, the rowid.
-      leftFarDelimiter++;
-      rightFarDelimiter++;
-      result = compareRowid(left, leftFarDelimiter,
-          llength - (leftFarDelimiter - loffset),
-          right, rightFarDelimiter, rlength - (rightFarDelimiter - roffset));
-      return result;
-    }
-
-    @Override
-    public byte[] getShortMidpointKey(final byte[] leftKey, final byte[] rightKey) {
-      return Arrays.copyOf(rightKey, rightKey.length);
-    }
-
-    protected int compareRowid(byte[] left, int loffset, int llength,
-        byte[] right, int roffset, int rlength) {
-      return Bytes.compareTo(left, loffset, llength, right, roffset, rlength);
-    }
-  }
-
-  /**
-   * Avoids redundant comparisons for better performance.
+   * Avoids redundant comparisons for better performance.
+   * 
+   * TODO get rid of this wart
    */
   public interface SamePrefixComparator<T> {
     /**
@@ -2534,326 +2523,33 @@ public class KeyValue implements Cell, H
      * @param commonPrefix How many bytes are the same.
      */
     int compareIgnoringPrefix(
-      int commonPrefix, T left, int loffset, int llength, T right, int roffset, int rlength
+      int commonPrefix, byte[] left, int loffset, int llength, byte[] right, int roffset, int rlength
     );
   }
 
   /**
-   * Compare key portion of a {@link KeyValue}.
+   * This is a TEST only Comparator used in TestSeekTo and TestReseekTo.
    */
-  public static class KeyComparator
-      implements RawComparator<byte []>, SamePrefixComparator<byte[]> {
-
-    @Override
-    public int compare(byte[] left, int loffset, int llength, byte[] right,
-        int roffset, int rlength) {
-      // Compare row
-      short lrowlength = Bytes.toShort(left, loffset);
-      short rrowlength = Bytes.toShort(right, roffset);
-      int compare = compareRows(left, loffset + Bytes.SIZEOF_SHORT,
-          lrowlength, right, roffset + Bytes.SIZEOF_SHORT, rrowlength);
-      if (compare != 0) {
-        return compare;
-      }
-
-      // Compare the rest of the two KVs without making any assumptions about
-      // the common prefix. This function will not compare rows anyway, so we
-      // don't need to tell it that the common prefix includes the row.
-      return compareWithoutRow(0, left, loffset, llength, right, roffset,
-          rlength, rrowlength);
-    }
-
+  public static class RawBytesComparator extends KVComparator {
     /**
-     * Compare the two key-values, ignoring the prefix of the given length
-     * that is known to be the same between the two.
-     * @param commonPrefix the prefix length to ignore
+     * The HFileV2 file format's trailer contains this class name.  We reinterpret this and
+     * instantiate the appropriate comparator.
+     * TODO: With V3 consider removing this.
+     * @return legacy class name for FileFileTrailer#comparatorClassName
      */
-    @Override
-    public int compareIgnoringPrefix(int commonPrefix, byte[] left,
-        int loffset, int llength, byte[] right, int roffset, int rlength) {
-      // Compare row
-      short lrowlength = Bytes.toShort(left, loffset);
-      short rrowlength;
-
-      int comparisonResult = 0;
-      if (commonPrefix < ROW_LENGTH_SIZE) {
-        // almost nothing in common
-        rrowlength = Bytes.toShort(right, roffset);
-        comparisonResult = compareRows(left, loffset + ROW_LENGTH_SIZE,
-            lrowlength, right, roffset + ROW_LENGTH_SIZE, rrowlength);
-      } else { // the row length is the same
-        rrowlength = lrowlength;
-        if (commonPrefix < ROW_LENGTH_SIZE + rrowlength) {
-          // The rows are not the same. Exclude the common prefix and compare
-          // the rest of the two rows.
-          int common = commonPrefix - ROW_LENGTH_SIZE;
-          comparisonResult = compareRows(
-              left, loffset + common + ROW_LENGTH_SIZE, lrowlength - common,
-              right, roffset + common + ROW_LENGTH_SIZE, rrowlength - common);
-        }
-      }
-      if (comparisonResult != 0) {
-        return comparisonResult;
-      }
-
-      assert lrowlength == rrowlength;
-
-      return compareWithoutRow(commonPrefix, left, loffset, llength, right,
-          roffset, rlength, lrowlength);
+    public String getLegacyKeyComparatorName() {
+      return "org.apache.hadoop.hbase.util.Bytes$ByteArrayComparator";
     }
 
-    /**
-     * Compare columnFamily, qualifier, timestamp, and key type (everything
-     * except the row). This method is used both in the normal comparator and
-     * the "same-prefix" comparator. Note that we are assuming that row portions
-     * of both KVs have already been parsed and found identical, and we don't
-     * validate that assumption here.
-     * @param commonPrefix
-     *          the length of the common prefix of the two key-values being
-     *          compared, including row length and row
-     */
-    private int compareWithoutRow(int commonPrefix, byte[] left, int loffset,
-        int llength, byte[] right, int roffset, int rlength, short rowlength) {
-      /***
-       * KeyValue Format and commonLength:
-       * |_keyLen_|_valLen_|_rowLen_|_rowKey_|_famiLen_|_fami_|_Quali_|....
-       * ------------------|-------commonLength--------|--------------
-       */
-      int commonLength = ROW_LENGTH_SIZE + FAMILY_LENGTH_SIZE + rowlength;
-
-      // commonLength + TIMESTAMP_TYPE_SIZE
-      int commonLengthWithTSAndType = TIMESTAMP_TYPE_SIZE + commonLength;
-      // ColumnFamily + Qualifier length.
-      int lcolumnlength = llength - commonLengthWithTSAndType;
-      int rcolumnlength = rlength - commonLengthWithTSAndType;
-
-      byte ltype = left[loffset + (llength - 1)];
-      byte rtype = right[roffset + (rlength - 1)];
-
-      // If the column is not specified, the "minimum" key type appears the
-      // latest in the sorted order, regardless of the timestamp. This is used
-      // for specifying the last key/value in a given row, because there is no
-      // "lexicographically last column" (it would be infinitely long). The
-      // "maximum" key type does not need this behavior.
-      if (lcolumnlength == 0 && ltype == Type.Minimum.getCode()) {
-        // left is "bigger", i.e. it appears later in the sorted order
-        return 1;
-      }
-      if (rcolumnlength == 0 && rtype == Type.Minimum.getCode()) {
-        return -1;
-      }
-
-      int lfamilyoffset = commonLength + loffset;
-      int rfamilyoffset = commonLength + roffset;
-
-      // Column family length.
-      int lfamilylength = left[lfamilyoffset - 1];
-      int rfamilylength = right[rfamilyoffset - 1];
-      // If left family size is not equal to right family size, we need not
-      // compare the qualifiers.
-      boolean sameFamilySize = (lfamilylength == rfamilylength);
-      int common = 0;
-      if (commonPrefix > 0) {
-        common = Math.max(0, commonPrefix - commonLength);
-        if (!sameFamilySize) {
-          // Common should not be larger than Math.min(lfamilylength,
-          // rfamilylength).
-          common = Math.min(common, Math.min(lfamilylength, rfamilylength));
-        } else {
-          common = Math.min(common, Math.min(lcolumnlength, rcolumnlength));
-        }
-      }
-      if (!sameFamilySize) {
-        // comparing column family is enough.
-        return Bytes.compareTo(left, lfamilyoffset + common, lfamilylength
-            - common, right, rfamilyoffset + common, rfamilylength - common);
-      }
-      // Compare family & qualifier together.
-      final int comparison = Bytes.compareTo(left, lfamilyoffset + common,
-          lcolumnlength - common, right, rfamilyoffset + common,
-          rcolumnlength - common);
-      if (comparison != 0) {
-        return comparison;
-      }
-      return compareTimestampAndType(left, loffset, llength, right, roffset,
-          rlength, ltype, rtype);
-    }
-
-    private int compareTimestampAndType(byte[] left, int loffset, int llength,
-        byte[] right, int roffset, int rlength, byte ltype, byte rtype) {
-      int compare;
-      // Get timestamps.
-      long ltimestamp = Bytes.toLong(left,
-          loffset + (llength - TIMESTAMP_TYPE_SIZE));
-      long rtimestamp = Bytes.toLong(right,
-          roffset + (rlength - TIMESTAMP_TYPE_SIZE));
-      compare = compareTimestamps(ltimestamp, rtimestamp);
-      if (compare != 0) {
-        return compare;
-      }
-
-      // Compare types. Let the delete types sort ahead of puts; i.e. types
-      // of higher numbers sort before those of lesser numbers. Maximum (255)
-      // appears ahead of everything, and minimum (0) appears after
-      // everything.
-      return (0xff & rtype) - (0xff & ltype);
-    }
-
-    public int compare(byte[] left, byte[] right) {
-      return compare(left, 0, left.length, right, 0, right.length);
-    }
-
-    public int compareRows(byte [] left, int loffset, int llength,
-        byte [] right, int roffset, int rlength) {
-      return Bytes.compareTo(left, loffset, llength, right, roffset, rlength);
-    }
-
-    /**
-     * Generate a faked byte array if possible. It aims to:
-     * 1)reduce key length, which expects to reduce HFile index memory footprint
-     * 2)replace TS field with LATEST_TIMESTAMP(to avoid seeking previous block)
-     * see HBASE-7845 for more details
-     * we need to ensure: leftKey < newKey <= rightKey
-     * @param leftKey the previous block's real stop key usually
-     * @param rightKey the current block's real start key usually
-     * @return newKey: the newly generated faked key
-     */
-    protected byte[] getShortMidpointKey(final byte[] leftKey, final byte[] rightKey) {
-      if (rightKey == null) {
-        throw new IllegalArgumentException("rightKey can not be null");
-      }
-      if (leftKey == null) {
-        return Arrays.copyOf(rightKey, rightKey.length);
-      }
-      if (compare(leftKey, rightKey) >= 0) {
-        throw new IllegalArgumentException("Unexpected input, leftKey:" + Bytes.toString(leftKey)
-          + ", rightKey:" + Bytes.toString(rightKey));
-      }
-
-      short leftRowLength = Bytes.toShort(leftKey, 0);
-      short rightRowLength = Bytes.toShort(rightKey, 0);
-      int leftCommonLength = ROW_LENGTH_SIZE + FAMILY_LENGTH_SIZE + leftRowLength;
-      int rightCommonLength = ROW_LENGTH_SIZE + FAMILY_LENGTH_SIZE + rightRowLength;
-      int leftCommonLengthWithTSAndType = TIMESTAMP_TYPE_SIZE + leftCommonLength;
-      int rightCommonLengthWithTSAndType = TIMESTAMP_TYPE_SIZE + rightCommonLength;
-      int leftColumnLength = leftKey.length - leftCommonLengthWithTSAndType;
-      int rightColumnLength = rightKey.length - rightCommonLengthWithTSAndType;
-      // rows are equal
-      if (leftRowLength == rightRowLength && compareRows(leftKey, ROW_LENGTH_SIZE, leftRowLength,
-        rightKey, ROW_LENGTH_SIZE, rightRowLength) == 0) {
-        // Compare family & qualifier together.
-        int comparison = Bytes.compareTo(leftKey, leftCommonLength, leftColumnLength, rightKey,
-          rightCommonLength, rightColumnLength);
-        // same with "row + family + qualifier", return rightKey directly
-        if (comparison == 0) {
-          return Arrays.copyOf(rightKey, rightKey.length);
-        }
-        // "family + qualifier" are different, generate a faked key per rightKey
-        byte[] newKey = Arrays.copyOf(rightKey, rightKey.length);
-        Bytes.putLong(newKey, rightKey.length - TIMESTAMP_TYPE_SIZE, HConstants.LATEST_TIMESTAMP);
-        Bytes.putByte(newKey, rightKey.length - TYPE_SIZE, Type.Maximum.getCode());
-        return newKey;
-      }
-      // rows are different
-      short minLength = leftRowLength < rightRowLength ? leftRowLength : rightRowLength;
-      short diffIdx = 0;
-      while (diffIdx < minLength
-          && leftKey[ROW_LENGTH_SIZE + diffIdx] == rightKey[ROW_LENGTH_SIZE + diffIdx]) {
-        diffIdx++;
-      }
-      if (diffIdx >= minLength) {
-        // leftKey's row is prefix of rightKey's. we can optimize it in future
-        return Arrays.copyOf(rightKey, rightKey.length);
-      }
-      int diffByte = leftKey[ROW_LENGTH_SIZE + diffIdx];
-      if ((0xff & diffByte) < 0xff && (diffByte + 1) <
-          (rightKey[ROW_LENGTH_SIZE + diffIdx] & 0xff)) {
-        byte[] newRowKey = new byte[diffIdx + 1];
-        System.arraycopy(leftKey, ROW_LENGTH_SIZE, newRowKey, 0, diffIdx);
-        newRowKey[diffIdx] = (byte) (diffByte + 1);
-        int rightFamilyLength = rightKey[rightCommonLength - 1];
-        byte[] family = null;
-        if (rightFamilyLength > 0) {
-          family = new byte[rightFamilyLength];
-          System.arraycopy(rightKey, rightCommonLength, family, 0, rightFamilyLength);
-        }
-        int rightQualifierLength = rightColumnLength - rightFamilyLength;
-        byte[] qualifier = null;
-        if (rightQualifierLength > 0) {
-          qualifier = new byte[rightQualifierLength];
-          System.arraycopy(rightKey, rightCommonLength + rightFamilyLength, qualifier, 0,
-            rightQualifierLength);
-        }
-        return new KeyValue(newRowKey, null, null, HConstants.LATEST_TIMESTAMP,
-          Type.Maximum).getKey();
-      }
-      // the following is optimizable in future
-      return Arrays.copyOf(rightKey, rightKey.length);
-    }
-
-    protected int compareColumns(
-        byte [] left, int loffset, int llength, final int lfamilylength,
-        byte [] right, int roffset, int rlength, final int rfamilylength) {
-      return KeyValue.compareColumns(left, loffset, llength, lfamilylength,
-        right, roffset, rlength, rfamilylength);
-    }
-
-    static int compareTimestamps(final long ltimestamp, final long rtimestamp) {
-      // The below older timestamps sorting ahead of newer timestamps looks
-      // wrong but it is intentional. This way, newer timestamps are first
-      // found when we iterate over a memstore and newer versions are the
-      // first we trip over when reading from a store file.
-      if (ltimestamp < rtimestamp) {
-        return 1;
-      } else if (ltimestamp > rtimestamp) {
-        return -1;
-      }
-      return 0;
-    }
-
-    /**
-     * Generate a shorter faked key into index block. For example, consider a block boundary
-     * between the keys "the quick brown fox" and "the who test text".  We can use "the r" as the
-     * key for the index block entry since it is > all entries in the previous block and <= all
-     * entries in subsequent blocks.
-     *
-     * @param lastKeyOfPreviousBlock
-     * @param firstKeyInBlock
-     * @return a shortened null key, or if there are unexpected results, the firstKeyIn (new) Block
-     */
-    public byte[] calcIndexKey(byte[] lastKeyOfPreviousBlock, byte[] firstKeyInBlock) {
-      byte[] fakeKey = getShortMidpointKey(lastKeyOfPreviousBlock, firstKeyInBlock);
-      if (compare(fakeKey, firstKeyInBlock) > 0) {
-        LOG.error("Unexpected getShortMidpointKey result, fakeKey:"
-            + Bytes.toStringBinary(fakeKey) + ", firstKeyInBlock:"
-            + Bytes.toStringBinary(firstKeyInBlock));
-        return firstKeyInBlock;
-      }
-      if (lastKeyOfPreviousBlock != null && compare(lastKeyOfPreviousBlock, fakeKey) >= 0) {
-        LOG.error("Unexpected getShortMidpointKey result, lastKeyOfPreviousBlock:" +
-            Bytes.toStringBinary(lastKeyOfPreviousBlock) + ", fakeKey:" +
-            Bytes.toStringBinary(fakeKey));
-        return firstKeyInBlock;
-      }
-      return fakeKey;
-    }
-  }
-
-  /**
-   * This is a TEST only Comparator used in TestSeekTo and TestReseekTo.
-   */
-  @Deprecated
-  public static class RawKeyComparator extends KeyComparator {
-    RawComparator<byte []> getRawComparator() { return Bytes.BYTES_RAWCOMPARATOR; }
-    
-    public int compare(byte[] left, int loffset, int llength, byte[] right,
+    public int compareFlatKey(byte[] left, int loffset, int llength, byte[] right,
         int roffset, int rlength) {
-      return getRawComparator().compare(left,  loffset, llength, right, roffset, rlength);
+      return Bytes.BYTES_RAWCOMPARATOR.compare(left,  loffset, llength, right, roffset, rlength);
     }
-    
+
     public byte[] calcIndexKey(byte[] lastKeyOfPreviousBlock, byte[] firstKeyInBlock) {
       return firstKeyInBlock;
     }
+
   }
 
   /**
@@ -2874,6 +2570,8 @@ public class KeyValue implements Cell, H
     return ClassSize.align(sum);
   }
 
+  // -----
+  // KV tags stubs
   @Override
   public int getTagsOffset() {
     throw new UnsupportedOperationException("Not implememnted");

Modified: hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java?rev=1518818&r1=1518817&r2=1518818&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java (original)
+++ hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java Thu Aug 29 20:45:05 2013
@@ -20,7 +20,7 @@ package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hbase.KeyValue.KeyComparator;
+import org.apache.hadoop.hbase.KeyValue.KVComparator;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
@@ -335,10 +335,10 @@ public final class TableName implements 
    *
    * @return The comparator.
    */
-  public KeyComparator getRowComparator() {
+  public KVComparator getRowComparator() {
      if(TableName.META_TABLE_NAME.equals(this)) {
-      return KeyValue.META_COMPARATOR.getRawComparator();
+      return KeyValue.META_COMPARATOR;
     }
-    return KeyValue.COMPARATOR.getRawComparator();
+    return KeyValue.COMPARATOR;
   }
 }

Modified: hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java?rev=1518818&r1=1518817&r2=1518818&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java (original)
+++ hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java Thu Aug 29 20:45:05 2013
@@ -24,12 +24,12 @@ import java.nio.ByteBuffer;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValue.KVComparator;
 import org.apache.hadoop.hbase.KeyValue.SamePrefixComparator;
 import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
 import org.apache.hadoop.hbase.io.hfile.BlockType;
 import org.apache.hadoop.hbase.util.ByteBufferUtils;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.io.RawComparator;
 import org.apache.hadoop.io.WritableUtils;
 
 /**
@@ -113,14 +113,14 @@ abstract class BufferedDataBlockEncoder 
       BufferedEncodedSeeker<STATE extends SeekerState>
       implements EncodedSeeker {
 
-    protected final RawComparator<byte[]> comparator;
+    protected final KVComparator comparator;
     protected final SamePrefixComparator<byte[]> samePrefixComparator;
     protected ByteBuffer currentBuffer;
     protected STATE current = createSeekerState(); // always valid
     protected STATE previous = createSeekerState(); // may not be valid
 
     @SuppressWarnings("unchecked")
-    public BufferedEncodedSeeker(RawComparator<byte[]> comparator) {
+    public BufferedEncodedSeeker(KVComparator comparator) {
       this.comparator = comparator;
       if (comparator instanceof SamePrefixComparator) {
         this.samePrefixComparator = (SamePrefixComparator<byte[]>) comparator;
@@ -207,7 +207,7 @@ abstract class BufferedDataBlockEncoder 
           comp = samePrefixComparator.compareIgnoringPrefix(commonPrefix, key,
               offset, length, current.keyBuffer, 0, current.keyLength);
         } else {
-          comp = comparator.compare(key, offset, length,
+          comp = comparator.compareFlatKey(key, offset, length,
               current.keyBuffer, 0, current.keyLength);
         }
 

Modified: hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/CopyKeyDataBlockEncoder.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/CopyKeyDataBlockEncoder.java?rev=1518818&r1=1518817&r2=1518818&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/CopyKeyDataBlockEncoder.java (original)
+++ hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/CopyKeyDataBlockEncoder.java Thu Aug 29 20:45:05 2013
@@ -22,9 +22,9 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.KeyValue.KVComparator;
 import org.apache.hadoop.hbase.util.ByteBufferUtils;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.io.RawComparator;
 
 /**
  * Just copy data, do not do any kind of compression. Use for comparison and
@@ -67,7 +67,7 @@ public class CopyKeyDataBlockEncoder ext
   }
 
   @Override
-  public EncodedSeeker createSeeker(RawComparator<byte[]> comparator,
+  public EncodedSeeker createSeeker(KVComparator comparator,
       final boolean includesMemstoreTS) {
     return new BufferedEncodedSeeker<SeekerState>(comparator) {
       @Override

Modified: hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DataBlockEncoder.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DataBlockEncoder.java?rev=1518818&r1=1518817&r2=1518818&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DataBlockEncoder.java (original)
+++ hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DataBlockEncoder.java Thu Aug 29 20:45:05 2013
@@ -22,6 +22,7 @@ import java.nio.ByteBuffer;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValue.KVComparator;
 import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
 import org.apache.hadoop.io.RawComparator;
 
@@ -106,7 +107,7 @@ public interface DataBlockEncoder {
    * @return A newly created seeker.
    */
   EncodedSeeker createSeeker(
-    RawComparator<byte[]> comparator, boolean includesMemstoreTS
+    KVComparator comparator, boolean includesMemstoreTS
   );
 
   /**

Modified: hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DiffKeyDeltaEncoder.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DiffKeyDeltaEncoder.java?rev=1518818&r1=1518817&r2=1518818&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DiffKeyDeltaEncoder.java (original)
+++ hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DiffKeyDeltaEncoder.java Thu Aug 29 20:45:05 2013
@@ -23,6 +23,7 @@ import java.nio.ByteBuffer;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValue.KVComparator;
 import org.apache.hadoop.hbase.util.ByteBufferUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.RawComparator;
@@ -422,7 +423,7 @@ public class DiffKeyDeltaEncoder extends
   }
 
   @Override
-  public EncodedSeeker createSeeker(RawComparator<byte[]> comparator,
+  public EncodedSeeker createSeeker(KVComparator comparator,
       final boolean includesMemstoreTS) {
     return new BufferedEncodedSeeker<DiffSeekerState>(comparator) {
       private byte[] familyNameWithSize;

Modified: hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/FastDiffDeltaEncoder.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/FastDiffDeltaEncoder.java?rev=1518818&r1=1518817&r2=1518818&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/FastDiffDeltaEncoder.java (original)
+++ hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/FastDiffDeltaEncoder.java Thu Aug 29 20:45:05 2013
@@ -24,6 +24,7 @@ import java.nio.ByteBuffer;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValue.KVComparator;
 import org.apache.hadoop.hbase.util.ByteBufferUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.RawComparator;
@@ -417,7 +418,7 @@ public class FastDiffDeltaEncoder extend
   }
 
   @Override
-  public EncodedSeeker createSeeker(RawComparator<byte[]> comparator,
+  public EncodedSeeker createSeeker(KVComparator comparator,
       final boolean includesMemstoreTS) {
     return new BufferedEncodedSeeker<FastDiffSeekerState>(comparator) {
       private void decode(boolean isFirst) {

Modified: hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/PrefixKeyDeltaEncoder.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/PrefixKeyDeltaEncoder.java?rev=1518818&r1=1518817&r2=1518818&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/PrefixKeyDeltaEncoder.java (original)
+++ hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/PrefixKeyDeltaEncoder.java Thu Aug 29 20:45:05 2013
@@ -23,6 +23,7 @@ import java.nio.ByteBuffer;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValue.KVComparator;
 import org.apache.hadoop.hbase.util.ByteBufferUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.RawComparator;
@@ -164,7 +165,7 @@ public class PrefixKeyDeltaEncoder exten
   }
 
   @Override
-  public EncodedSeeker createSeeker(RawComparator<byte[]> comparator,
+  public EncodedSeeker createSeeker(KVComparator comparator,
       final boolean includesMemstoreTS) {
     return new BufferedEncodedSeeker<SeekerState>(comparator) {
       @Override

Modified: hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java?rev=1518818&r1=1518817&r2=1518818&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java (original)
+++ hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java Thu Aug 29 20:45:05 2013
@@ -1575,7 +1575,7 @@ public class Bytes {
    *         ranging from -(N + 1) to N - 1.
    */
   public static int binarySearch(byte [][]arr, byte []key, int offset,
-      int length, RawComparator<byte []> comparator) {
+      int length, RawComparator<?> comparator) {
     int low = 0;
     int high = arr.length - 1;