View Javadoc

1   /**
2    * Copyright The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase;
21  
22  import static org.apache.hadoop.hbase.util.Bytes.len;
23  
24  import java.io.DataInput;
25  import java.io.DataOutput;
26  import java.io.IOException;
27  import java.io.InputStream;
28  import java.io.OutputStream;
29  import java.nio.ByteBuffer;
30  import java.util.Arrays;
31  import java.util.Comparator;
32  import java.util.HashMap;
33  import java.util.Map;
34  
35  import org.apache.commons.logging.Log;
36  import org.apache.commons.logging.LogFactory;
37  import org.apache.hadoop.classification.InterfaceAudience;
38  import org.apache.hadoop.hbase.io.HeapSize;
39  import org.apache.hadoop.hbase.util.Bytes;
40  import org.apache.hadoop.hbase.util.ClassSize;
41  import org.apache.hadoop.io.IOUtils;
42  import org.apache.hadoop.io.RawComparator;
43  
44  import com.google.common.primitives.Longs;
45  
46  /**
47   * An HBase Key/Value. This is the fundamental HBase Type.  
48   * <p>
49   * HBase applications and users should use the Cell interface and avoid directly using KeyValue
50   * and member functions not defined in Cell.
51   * <p>
52   * If being used client-side, the primary methods to access individual fields are {@link #getRow()},
53   * {@link #getFamily()}, {@link #getQualifier()}, {@link #getTimestamp()}, and {@link #getValue()}.
54   * These methods allocate new byte arrays and return copies. Avoid their use server-side.
55   * <p>
56   * Instances of this class are immutable. They do not implement Comparable but Comparators are
57   * provided. Comparators change with context, whether user table or a catalog table comparison. Its
58   * critical you use the appropriate comparator. There are Comparators for normal HFiles, Meta's
59   * Hfiles, and bloom filter keys.
60   * <p>
61   * KeyValue wraps a byte array and takes offsets and lengths into passed array at where to start
62   * interpreting the content as KeyValue. The KeyValue format inside a byte array is:
63   * <code>&lt;keylength> &lt;valuelength> &lt;key> &lt;value></code> Key is further decomposed as:
64   * <code>&lt;rowlength> &lt;row> &lt;columnfamilylength> &lt;columnfamily> &lt;columnqualifier>
65   * &lt;timestamp> &lt;keytype></code>
66   * The <code>rowlength</code> maximum is <code>Short.MAX_SIZE</code>, column family length maximum
67   * is <code>Byte.MAX_SIZE</code>, and column qualifier + key length must be <
68   * <code>Integer.MAX_SIZE</code>. The column does not contain the family/qualifier delimiter,
69   * {@link #COLUMN_FAMILY_DELIMITER}
70   */
71  @InterfaceAudience.Private
72  public class KeyValue implements Cell, HeapSize, Cloneable {
73    static final Log LOG = LogFactory.getLog(KeyValue.class);
74  
75    /**
76     * Colon character in UTF-8
77     */
78    public static final char COLUMN_FAMILY_DELIMITER = ':';
79  
80    public static final byte[] COLUMN_FAMILY_DELIM_ARRAY =
81      new byte[]{COLUMN_FAMILY_DELIMITER};
82  
83    /**
84     * Comparator for plain key/values; i.e. non-catalog table key/values. Works on Key portion
85     * of KeyValue only.
86     */
87    public static final KVComparator COMPARATOR = new KVComparator();
88    /**
89     * A {@link KVComparator} for <code>hbase:meta</code> catalog table
90     * {@link KeyValue}s.
91     */
92    public static final KVComparator META_COMPARATOR = new MetaComparator();
93  
94    /**
95     * Needed for Bloom Filters.
96     */
97    public static final KVComparator RAW_COMPARATOR = new RawBytesComparator();
98  
99    /** Size of the key length field in bytes*/
100   public static final int KEY_LENGTH_SIZE = Bytes.SIZEOF_INT;
101 
102   /** Size of the key type field in bytes */
103   public static final int TYPE_SIZE = Bytes.SIZEOF_BYTE;
104 
105   /** Size of the row length field in bytes */
106   public static final int ROW_LENGTH_SIZE = Bytes.SIZEOF_SHORT;
107 
108   /** Size of the family length field in bytes */
109   public static final int FAMILY_LENGTH_SIZE = Bytes.SIZEOF_BYTE;
110 
111   /** Size of the timestamp field in bytes */
112   public static final int TIMESTAMP_SIZE = Bytes.SIZEOF_LONG;
113 
114   // Size of the timestamp and type byte on end of a key -- a long + a byte.
115   public static final int TIMESTAMP_TYPE_SIZE = TIMESTAMP_SIZE + TYPE_SIZE;
116 
117   // Size of the length shorts and bytes in key.
118   public static final int KEY_INFRASTRUCTURE_SIZE = ROW_LENGTH_SIZE
119       + FAMILY_LENGTH_SIZE + TIMESTAMP_TYPE_SIZE;
120 
121   // How far into the key the row starts at. First thing to read is the short
122   // that says how long the row is.
123   public static final int ROW_OFFSET =
124     Bytes.SIZEOF_INT /*keylength*/ +
125     Bytes.SIZEOF_INT /*valuelength*/;
126 
127   // Size of the length ints in a KeyValue datastructure.
128   public static final int KEYVALUE_INFRASTRUCTURE_SIZE = ROW_OFFSET;
129 
130   /**
131    * Computes the number of bytes that a <code>KeyValue</code> instance with the provided
132    * characteristics would take up for its underlying data structure.
133    *
134    * @param rlength row length
135    * @param flength family length
136    * @param qlength qualifier length
137    * @param vlength value length
138    *
139    * @return the <code>KeyValue</code> data structure length
140    */
141   public static long getKeyValueDataStructureSize(int rlength,
142       int flength, int qlength, int vlength) {
143     return KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE +
144             getKeyDataStructureSize(rlength, flength, qlength) + vlength;
145   }
146 
147 
148   /**
149    * Computes the number of bytes that a <code>KeyValue</code> instance with the provided
150    * characteristics would take up in its underlying data structure for the key.
151    *
152    * @param rlength row length
153    * @param flength family length
154    * @param qlength qualifier length
155    *
156    * @return the key data structure length
157    */
158   public static long getKeyDataStructureSize(int rlength, int flength, int qlength) {
159     return KeyValue.KEY_INFRASTRUCTURE_SIZE + rlength + flength + qlength;
160   }
161 
162   /**
163    * Key type.
164    * Has space for other key types to be added later.  Cannot rely on
165    * enum ordinals . They change if item is removed or moved.  Do our own codes.
166    */
167   public static enum Type {
168     Minimum((byte)0),
169     Put((byte)4),
170 
171     Delete((byte)8),
172     DeleteFamilyVersion((byte)10),
173     DeleteColumn((byte)12),
174     DeleteFamily((byte)14),
175 
176     // Maximum is used when searching; you look from maximum on down.
177     Maximum((byte)255);
178 
179     private final byte code;
180 
181     Type(final byte c) {
182       this.code = c;
183     }
184 
185     public byte getCode() {
186       return this.code;
187     }
188 
189     /**
190      * Cannot rely on enum ordinals . They change if item is removed or moved.
191      * Do our own codes.
192      * @param b
193      * @return Type associated with passed code.
194      */
195     public static Type codeToType(final byte b) {
196       for (Type t : Type.values()) {
197         if (t.getCode() == b) {
198           return t;
199         }
200       }
201       throw new RuntimeException("Unknown code " + b);
202     }
203   }
204 
205   /**
206    * Lowest possible key.
207    * Makes a Key with highest possible Timestamp, empty row and column.  No
208    * key can be equal or lower than this one in memstore or in store file.
209    */
210   public static final KeyValue LOWESTKEY =
211     new KeyValue(HConstants.EMPTY_BYTE_ARRAY, HConstants.LATEST_TIMESTAMP);
212 
213   ////
214   // KeyValue core instance fields.
215   private byte [] bytes = null;  // an immutable byte array that contains the KV
216   private int offset = 0;  // offset into bytes buffer KV starts at
217   private int length = 0;  // length of the KV starting from offset.
218 
219   /**
220    * @return True if a delete type, a {@link KeyValue.Type#Delete} or
221    * a {KeyValue.Type#DeleteFamily} or a {@link KeyValue.Type#DeleteColumn}
222    * KeyValue type.
223    */
224   public static boolean isDelete(byte t) {
225     return Type.Delete.getCode() <= t && t <= Type.DeleteFamily.getCode();
226   }
227 
228   /** Here be dragons **/
229 
230   // used to achieve atomic operations in the memstore.
231   @Override
232   public long getMvccVersion() {
233     return mvcc;
234   }
235 
236   public void setMvccVersion(long mvccVersion){
237     this.mvcc = mvccVersion;
238   }
239 
240   @Deprecated
241   public long getMemstoreTS() {
242     return getMvccVersion();
243   }
244 
245   @Deprecated
246   public void setMemstoreTS(long memstoreTS) {
247     setMvccVersion(memstoreTS);
248   }
249 
250   // multi-version concurrency control version.  default value is 0, aka do not care.
251   private long mvcc = 0;  // this value is not part of a serialized KeyValue (not in HFiles)
252 
253   /** Dragon time over, return to normal business */
254 
255 
256   /** Writable Constructor -- DO NOT USE */
257   public KeyValue() {}
258 
259   /**
260    * Creates a KeyValue from the start of the specified byte array.
261    * Presumes <code>bytes</code> content is formatted as a KeyValue blob.
262    * @param bytes byte array
263    */
264   public KeyValue(final byte [] bytes) {
265     this(bytes, 0);
266   }
267 
268   /**
269    * Creates a KeyValue from the specified byte array and offset.
270    * Presumes <code>bytes</code> content starting at <code>offset</code> is
271    * formatted as a KeyValue blob.
272    * @param bytes byte array
273    * @param offset offset to start of KeyValue
274    */
275   public KeyValue(final byte [] bytes, final int offset) {
276     this(bytes, offset, getLength(bytes, offset));
277   }
278 
279   /**
280    * Creates a KeyValue from the specified byte array, starting at offset, and
281    * for length <code>length</code>.
282    * @param bytes byte array
283    * @param offset offset to start of the KeyValue
284    * @param length length of the KeyValue
285    */
286   public KeyValue(final byte [] bytes, final int offset, final int length) {
287     this.bytes = bytes;
288     this.offset = offset;
289     this.length = length;
290   }
291 
292   /**
293    * Creates a KeyValue from the specified byte array, starting at offset, and
294    * for length <code>length</code>.
295    *
296    * @param bytes  byte array
297    * @param offset offset to start of the KeyValue
298    * @param length length of the KeyValue
299    * @param ts
300    */
301   public KeyValue(final byte[] bytes, final int offset, final int length, long ts) {
302     this(bytes, offset, length, null, 0, 0, null, 0, 0, ts, Type.Maximum, null, 0, 0);
303   }
304 
305   /** Constructors that build a new backing byte array from fields */
306 
307   /**
308    * Constructs KeyValue structure filled with null value.
309    * Sets type to {@link KeyValue.Type#Maximum}
310    * @param row - row key (arbitrary byte array)
311    * @param timestamp
312    */
313   public KeyValue(final byte [] row, final long timestamp) {
314     this(row, null, null, timestamp, Type.Maximum, null);
315   }
316 
317   /**
318    * Constructs KeyValue structure filled with null value.
319    * @param row - row key (arbitrary byte array)
320    * @param timestamp
321    */
322   public KeyValue(final byte [] row, final long timestamp, Type type) {
323     this(row, null, null, timestamp, type, null);
324   }
325 
326   /**
327    * Constructs KeyValue structure filled with null value.
328    * Sets type to {@link KeyValue.Type#Maximum}
329    * @param row - row key (arbitrary byte array)
330    * @param family family name
331    * @param qualifier column qualifier
332    */
333   public KeyValue(final byte [] row, final byte [] family,
334       final byte [] qualifier) {
335     this(row, family, qualifier, HConstants.LATEST_TIMESTAMP, Type.Maximum);
336   }
337 
338   /**
339    * Constructs KeyValue structure filled with null value.
340    * @param row - row key (arbitrary byte array)
341    * @param family family name
342    * @param qualifier column qualifier
343    */
344   public KeyValue(final byte [] row, final byte [] family,
345       final byte [] qualifier, final byte [] value) {
346     this(row, family, qualifier, HConstants.LATEST_TIMESTAMP, Type.Put, value);
347   }
348 
349   /**
350    * Constructs KeyValue structure filled with specified values.
351    * @param row row key
352    * @param family family name
353    * @param qualifier column qualifier
354    * @param timestamp version timestamp
355    * @param type key type
356    * @throws IllegalArgumentException
357    */
358   public KeyValue(final byte[] row, final byte[] family,
359       final byte[] qualifier, final long timestamp, Type type) {
360     this(row, family, qualifier, timestamp, type, null);
361   }
362 
363   /**
364    * Constructs KeyValue structure filled with specified values.
365    * @param row row key
366    * @param family family name
367    * @param qualifier column qualifier
368    * @param timestamp version timestamp
369    * @param value column value
370    * @throws IllegalArgumentException
371    */
372   public KeyValue(final byte[] row, final byte[] family,
373       final byte[] qualifier, final long timestamp, final byte[] value) {
374     this(row, family, qualifier, timestamp, Type.Put, value);
375   }
376 
377   /**
378    * Constructs KeyValue structure filled with specified values.
379    * @param row row key
380    * @param family family name
381    * @param qualifier column qualifier
382    * @param timestamp version timestamp
383    * @param type key type
384    * @param value column value
385    * @throws IllegalArgumentException
386    */
387   public KeyValue(final byte[] row, final byte[] family,
388       final byte[] qualifier, final long timestamp, Type type,
389       final byte[] value) {
390     this(row, 0, len(row),   family, 0, len(family),   qualifier, 0, len(qualifier),
391         timestamp, type,   value, 0, len(value));
392   }
393 
394   /**
395    * Constructs KeyValue structure filled with specified values.
396    * @param row row key
397    * @param family family name
398    * @param qualifier column qualifier
399    * @param qoffset qualifier offset
400    * @param qlength qualifier length
401    * @param timestamp version timestamp
402    * @param type key type
403    * @param value column value
404    * @param voffset value offset
405    * @param vlength value length
406    * @throws IllegalArgumentException
407    */
408     @Deprecated // removing redundant
409   public KeyValue(byte [] row, byte [] family,
410       byte [] qualifier, int qoffset, int qlength, long timestamp, Type type,
411       byte [] value, int voffset, int vlength) {
412       this(row, 0, len(row),
413 	   family, 0, len(family),
414 	   qualifier,qoffset, qlength, timestamp, type,
415 	   value, voffset, vlength);
416   }
417 
418   /**
419    * Constructs KeyValue structure filled with specified values.
420    * <p>
421    * Column is split into two fields, family and qualifier.
422    * @param row row key
423    * @param roffset row offset
424    * @param rlength row length
425    * @param family family name
426    * @param foffset family offset
427    * @param flength family length
428    * @param qualifier column qualifier
429    * @param qoffset qualifier offset
430    * @param qlength qualifier length
431    * @param timestamp version timestamp
432    * @param type key type
433    * @param value column value
434    * @param voffset value offset
435    * @param vlength value length
436    * @throws IllegalArgumentException
437    */
438   public KeyValue(final byte [] row, final int roffset, final int rlength,
439       final byte [] family, final int foffset, final int flength,
440       final byte [] qualifier, final int qoffset, final int qlength,
441       final long timestamp, final Type type,
442       final byte [] value, final int voffset, final int vlength) {
443     this.bytes = createByteArray(row, roffset, rlength,
444         family, foffset, flength, qualifier, qoffset, qlength,
445         timestamp, type, value, voffset, vlength);
446     this.length = bytes.length;
447     this.offset = 0;
448   }
449 
450   /**
451    * Constructs an empty KeyValue structure, with specified sizes.
452    * This can be used to partially fill up KeyValues.
453    * <p>
454    * Column is split into two fields, family and qualifier.
455    * @param rlength row length
456    * @param flength family length
457    * @param qlength qualifier length
458    * @param timestamp version timestamp
459    * @param type key type
460    * @param vlength value length
461    * @throws IllegalArgumentException
462    */
463   public KeyValue(final int rlength,
464       final int flength,
465       final int qlength,
466       final long timestamp, final Type type,
467       final int vlength) {
468     this.bytes = createEmptyByteArray(rlength,
469         flength, qlength,
470         timestamp, type, vlength);
471     this.length = bytes.length;
472     this.offset = 0;
473   }
474 
475 
476   public KeyValue(byte[] row, int roffset, int rlength,
477                   byte[] family, int foffset, int flength,
478                   ByteBuffer qualifier, long ts, Type type, ByteBuffer value) {
479     this.bytes = createByteArray(row, roffset, rlength, family, foffset, flength,
480         qualifier, 0, qualifier == null ? 0 : qualifier.remaining(), ts, type,
481         value, 0, value == null ? 0 : value.remaining());
482     this.length = bytes.length;
483     this.offset = 0;
484   }
485 
486   public KeyValue(Cell c) {
487     this(c.getRowArray(), c.getRowOffset(), (int)c.getRowLength(),
488         c.getFamilyArray(), c.getFamilyOffset(), (int)c.getFamilyLength(), 
489         c.getQualifierArray(), c.getQualifierOffset(), (int) c.getQualifierLength(),
490         c.getTimestamp(), Type.codeToType(c.getTypeByte()), 
491         c.getValueArray(), c.getValueOffset(), c.getValueLength());
492   }
493   
494   /**
495    * Create an empty byte[] representing a KeyValue
496    * All lengths are preset and can be filled in later.
497    * @param rlength
498    * @param flength
499    * @param qlength
500    * @param timestamp
501    * @param type
502    * @param vlength
503    * @return The newly created byte array.
504    */
505   private static byte[] createEmptyByteArray(final int rlength, int flength,
506       int qlength, final long timestamp, final Type type, int vlength) {
507     if (rlength > Short.MAX_VALUE) {
508       throw new IllegalArgumentException("Row > " + Short.MAX_VALUE);
509     }
510     if (flength > Byte.MAX_VALUE) {
511       throw new IllegalArgumentException("Family > " + Byte.MAX_VALUE);
512     }
513     // Qualifier length
514     if (qlength > Integer.MAX_VALUE - rlength - flength) {
515       throw new IllegalArgumentException("Qualifier > " + Integer.MAX_VALUE);
516     }
517     // Key length
518     long longkeylength = getKeyDataStructureSize(rlength, flength, qlength);
519     if (longkeylength > Integer.MAX_VALUE) {
520       throw new IllegalArgumentException("keylength " + longkeylength + " > " +
521         Integer.MAX_VALUE);
522     }
523     int keylength = (int)longkeylength;
524     // Value length
525     if (vlength > HConstants.MAXIMUM_VALUE_LENGTH) { // FindBugs INT_VACUOUS_COMPARISON
526       throw new IllegalArgumentException("Valuer > " +
527           HConstants.MAXIMUM_VALUE_LENGTH);
528     }
529 
530     // Allocate right-sized byte array.
531     byte [] bytes =
532         new byte[(int) getKeyValueDataStructureSize(rlength, flength, qlength, vlength)];
533     // Write the correct size markers
534     int pos = 0;
535     pos = Bytes.putInt(bytes, pos, keylength);
536     pos = Bytes.putInt(bytes, pos, vlength);
537     pos = Bytes.putShort(bytes, pos, (short)(rlength & 0x0000ffff));
538     pos += rlength;
539     pos = Bytes.putByte(bytes, pos, (byte)(flength & 0x0000ff));
540     pos += flength + qlength;
541     pos = Bytes.putLong(bytes, pos, timestamp);
542     pos = Bytes.putByte(bytes, pos, type.getCode());
543     return bytes;
544   }
545 
546   /**
547    * Constructs KeyValue structure filled with specified values. Uses the provided buffer as its
548    * backing data buffer.
549    * <p>
550    * Column is split into two fields, family and qualifier.
551    *
552    * @param buffer the bytes buffer to use
553    * @param row row key
554    * @param roffset row offset
555    * @param rlength row length
556    * @param family family name
557    * @param foffset family offset
558    * @param flength family length
559    * @param qualifier column qualifier
560    * @param qoffset qualifier offset
561    * @param qlength qualifier length
562    * @param timestamp version timestamp
563    * @param type key type
564    * @param value column value
565    * @param voffset value offset
566    * @param vlength value length
567    * @throws IllegalArgumentException an illegal value was passed or there is insufficient space
568    * remaining in the buffer
569    */
570   @Deprecated  // removing redundant
571   public KeyValue(byte [] buffer,
572       final byte [] row, final int roffset, final int rlength,
573       final byte [] family, final int foffset, final int flength,
574       final byte [] qualifier, final int qoffset, final int qlength,
575       final long timestamp, final Type type,
576       final byte [] value, final int voffset, final int vlength) {
577 
578     this(buffer, 0,
579         row, roffset, rlength,
580         family, foffset, flength,
581         qualifier, qoffset, qlength,
582         timestamp, type,
583         value, voffset, vlength);
584   }
585 
586   /**
587    * Constructs KeyValue structure filled with specified values. Uses the provided buffer as the
588    * data buffer.
589    * <p>
590    * Column is split into two fields, family and qualifier.
591    *
592    * @param buffer the bytes buffer to use
593    * @param boffset buffer offset
594    * @param row row key
595    * @param roffset row offset
596    * @param rlength row length
597    * @param family family name
598    * @param foffset family offset
599    * @param flength family length
600    * @param qualifier column qualifier
601    * @param qoffset qualifier offset
602    * @param qlength qualifier length
603    * @param timestamp version timestamp
604    * @param type key type
605    * @param value column value
606    * @param voffset value offset
607    * @param vlength value length
608    * @throws IllegalArgumentException an illegal value was passed or there is insufficient space
609    * remaining in the buffer
610    */
611   @Deprecated // removing redundant
612   public KeyValue(byte [] buffer, final int boffset,
613       final byte [] row, final int roffset, final int rlength,
614       final byte [] family, final int foffset, final int flength,
615       final byte [] qualifier, final int qoffset, final int qlength,
616       final long timestamp, final Type type,
617       final byte [] value, final int voffset, final int vlength) {
618 
619     this.bytes  = buffer;
620     this.length = writeByteArray(buffer, boffset,
621         row, roffset, rlength,
622         family, foffset, flength, qualifier, qoffset, qlength,
623         timestamp, type, value, voffset, vlength);
624     this.offset = boffset;
625   }
626 
627   /**
628    * Checks the parameters passed to a constructor.
629    *
630    * @param row row key
631    * @param rlength row length
632    * @param family family name
633    * @param flength family length
634    * @param qlength qualifier length
635    * @param vlength value length
636    *
637    * @throws IllegalArgumentException an illegal value was passed
638    */
639   private static void checkParameters(final byte [] row, final int rlength,
640       final byte [] family, int flength, int qlength, int vlength)
641           throws IllegalArgumentException {
642 
643     if (rlength > Short.MAX_VALUE) {
644       throw new IllegalArgumentException("Row > " + Short.MAX_VALUE);
645     }
646     if (row == null) {
647       throw new IllegalArgumentException("Row is null");
648     }
649     // Family length
650     flength = family == null ? 0 : flength;
651     if (flength > Byte.MAX_VALUE) {
652       throw new IllegalArgumentException("Family > " + Byte.MAX_VALUE);
653     }
654     // Qualifier length
655     if (qlength > Integer.MAX_VALUE - rlength - flength) {
656       throw new IllegalArgumentException("Qualifier > " + Integer.MAX_VALUE);
657     }
658     // Key length
659     long longKeyLength = getKeyDataStructureSize(rlength, flength, qlength);
660     if (longKeyLength > Integer.MAX_VALUE) {
661       throw new IllegalArgumentException("keylength " + longKeyLength + " > " +
662           Integer.MAX_VALUE);
663     }
664     // Value length
665     if (vlength > HConstants.MAXIMUM_VALUE_LENGTH) { // FindBugs INT_VACUOUS_COMPARISON
666       throw new IllegalArgumentException("Value length " + vlength + " > " +
667           HConstants.MAXIMUM_VALUE_LENGTH);
668     }
669   }
670 
671   /**
672    * Write KeyValue format into the provided byte array.
673    *
674    * @param buffer the bytes buffer to use
675    * @param boffset buffer offset
676    * @param row row key
677    * @param roffset row offset
678    * @param rlength row length
679    * @param family family name
680    * @param foffset family offset
681    * @param flength family length
682    * @param qualifier column qualifier
683    * @param qoffset qualifier offset
684    * @param qlength qualifier length
685    * @param timestamp version timestamp
686    * @param type key type
687    * @param value column value
688    * @param voffset value offset
689    * @param vlength value length
690    *
691    * @return The number of useful bytes in the buffer.
692    *
693    * @throws IllegalArgumentException an illegal value was passed or there is insufficient space
694    * remaining in the buffer
695    */
696   private static int writeByteArray(byte [] buffer, final int boffset,
697       final byte [] row, final int roffset, final int rlength,
698       final byte [] family, final int foffset, int flength,
699       final byte [] qualifier, final int qoffset, int qlength,
700       final long timestamp, final Type type,
701       final byte [] value, final int voffset, int vlength) {
702 
703     checkParameters(row, rlength, family, flength, qlength, vlength);
704 
705     int keyLength = (int) getKeyDataStructureSize(rlength, flength, qlength);
706     int keyValueLength = (int) getKeyValueDataStructureSize(rlength, flength, qlength, vlength);
707     if (keyValueLength > buffer.length - boffset) {
708       throw new IllegalArgumentException("Buffer size " + (buffer.length - boffset) + " < " +
709           keyValueLength);
710     }
711 
712     // Write key, value and key row length.
713     int pos = boffset;
714     pos = Bytes.putInt(buffer, pos, keyLength);
715     pos = Bytes.putInt(buffer, pos, vlength);
716     pos = Bytes.putShort(buffer, pos, (short)(rlength & 0x0000ffff));
717     pos = Bytes.putBytes(buffer, pos, row, roffset, rlength);
718     pos = Bytes.putByte(buffer, pos, (byte) (flength & 0x0000ff));
719     if (flength != 0) {
720       pos = Bytes.putBytes(buffer, pos, family, foffset, flength);
721     }
722     if (qlength != 0) {
723       pos = Bytes.putBytes(buffer, pos, qualifier, qoffset, qlength);
724     }
725     pos = Bytes.putLong(buffer, pos, timestamp);
726     pos = Bytes.putByte(buffer, pos, type.getCode());
727     if (value != null && value.length > 0) {
728       pos = Bytes.putBytes(buffer, pos, value, voffset, vlength);
729     }
730 
731     return keyValueLength;
732   }
733 
734   /**
735    * Write KeyValue format into a byte array.
736    *
737    * @param row row key
738    * @param roffset row offset
739    * @param rlength row length
740    * @param family family name
741    * @param foffset family offset
742    * @param flength family length
743    * @param qualifier column qualifier, a byte[] or a byte Buffer
744    * @param qoffset qualifier offset
745    * @param qlength qualifier length
746    * @param timestamp version timestamp
747    * @param type key type
748    * @param value column value, a byte[] or a byte Buffer
749    * @param voffset value offset
750    * @param vlength value length
751    * @return The newly created byte array.
752    */
753   private static byte [] createByteArray(final byte [] row, final int roffset,
754       final int rlength, final byte [] family, final int foffset, int flength,
755       final Object qualifier, final int qoffset, int qlength,
756       final long timestamp, final Type type,
757       final Object value, final int voffset, int vlength) {
758 
759     checkParameters(row, rlength, family, flength, qlength, vlength);
760 
761     // Allocate right-sized byte array.
762     int keyLength = (int) getKeyDataStructureSize(rlength, flength, qlength);
763     byte [] bytes =
764         new byte[(int) getKeyValueDataStructureSize(rlength, flength, qlength, vlength)];
765     // Write key, value and key row length.
766     int pos = 0;
767     pos = Bytes.putInt(bytes, pos, keyLength);
768     pos = Bytes.putInt(bytes, pos, vlength);
769     pos = Bytes.putShort(bytes, pos, (short)(rlength & 0x0000ffff));
770     pos = Bytes.putBytes(bytes, pos, row, roffset, rlength);
771     pos = Bytes.putByte(bytes, pos, (byte)(flength & 0x0000ff));
772     if(flength != 0) {
773       pos = Bytes.putBytes(bytes, pos, family, foffset, flength);
774     }
775     if (qlength > 0) {
776       if (qualifier instanceof ByteBuffer) {
777         pos = Bytes.putByteBuffer(bytes, pos, (ByteBuffer) qualifier);
778       } else {
779         pos = Bytes.putBytes(bytes, pos, (byte[]) qualifier, qoffset, qlength);
780       }
781     }
782     pos = Bytes.putLong(bytes, pos, timestamp);
783     pos = Bytes.putByte(bytes, pos, type.getCode());
784     if (vlength > 0) {
785       if (value instanceof ByteBuffer) {
786         pos = Bytes.putByteBuffer(bytes, pos, (ByteBuffer) value);
787       } else {
788         pos = Bytes.putBytes(bytes, pos, (byte[]) value, voffset, vlength);
789       }
790     }
791     return bytes;
792   }
793 
794   /**
795    * Needed doing 'contains' on List.  Only compares the key portion, not the value.
796    */
797   @Override
798   public boolean equals(Object other) {
799     if (!(other instanceof Cell)) {
800       return false;
801     }
802     return CellComparator.equals(this, (Cell)other);
803   }
804 
805   @Override
806   public int hashCode() {
807     byte[] b = getBuffer();
808     int start = getOffset(), end = getOffset() + getLength();
809     int h = b[start++];
810     for (int i = start; i < end; i++) {
811       h = (h * 13) ^ b[i];
812     }
813     return h;
814   }
815 
816   //---------------------------------------------------------------------------
817   //
818   //  KeyValue cloning
819   //
820   //---------------------------------------------------------------------------
821 
822   /**
823    * Clones a KeyValue.  This creates a copy, re-allocating the buffer.
824    * @return Fully copied clone of this KeyValue
825    * @throws CloneNotSupportedException
826    */
827   @Override
828   public KeyValue clone() throws CloneNotSupportedException {
829     super.clone();
830     byte [] b = new byte[this.length];
831     System.arraycopy(this.bytes, this.offset, b, 0, this.length);
832     KeyValue ret = new KeyValue(b, 0, b.length);
833     // Important to clone the memstoreTS as well - otherwise memstore's
834     // update-in-place methods (eg increment) will end up creating
835     // new entries
836     ret.setMvccVersion(mvcc);
837     return ret;
838   }
839 
840   /**
841    * Creates a shallow copy of this KeyValue, reusing the data byte buffer.
842    * http://en.wikipedia.org/wiki/Object_copy
843    * @return Shallow copy of this KeyValue
844    */
845   public KeyValue shallowCopy() {
846     KeyValue shallowCopy = new KeyValue(this.bytes, this.offset, this.length);
847     shallowCopy.setMvccVersion(this.mvcc);
848     return shallowCopy;
849   }
850 
851   //---------------------------------------------------------------------------
852   //
853   //  String representation
854   //
855   //---------------------------------------------------------------------------
856 
857   public String toString() {
858     if (this.bytes == null || this.bytes.length == 0) {
859       return "empty";
860     }
861     return keyToString(this.bytes, this.offset + ROW_OFFSET, getKeyLength()) +
862       "/vlen=" + getValueLength() + "/mvcc=" + mvcc;
863   }
864 
865   /**
866    * @param k Key portion of a KeyValue.
867    * @return Key as a String, empty string if k is null. 
868    */
869   public static String keyToString(final byte [] k) {
870     if (k == null) { 
871       return "";
872     }
873     return keyToString(k, 0, k.length);
874   }
875 
876   /**
877    * Use for logging.
878    * @param b Key portion of a KeyValue.
879    * @param o Offset to start of key
880    * @param l Length of key.
881    * @return Key as a String.
882    */
883   /**
884    * Produces a string map for this key/value pair. Useful for programmatic use
885    * and manipulation of the data stored in an HLogKey, for example, printing
886    * as JSON. Values are left out due to their tendency to be large. If needed,
887    * they can be added manually.
888    *
889    * @return the Map<String,?> containing data from this key
890    */
891   public Map<String, Object> toStringMap() {
892     Map<String, Object> stringMap = new HashMap<String, Object>();
893     stringMap.put("row", Bytes.toStringBinary(getRow()));
894     stringMap.put("family", Bytes.toStringBinary(getFamily()));
895     stringMap.put("qualifier", Bytes.toStringBinary(getQualifier()));
896     stringMap.put("timestamp", getTimestamp());
897     stringMap.put("vlen", getValueLength());
898     return stringMap;
899   }
900 
901   public static String keyToString(final byte [] b, final int o, final int l) {
902     if (b == null) return "";
903     int rowlength = Bytes.toShort(b, o);
904     String row = Bytes.toStringBinary(b, o + Bytes.SIZEOF_SHORT, rowlength);
905     int columnoffset = o + Bytes.SIZEOF_SHORT + 1 + rowlength;
906     int familylength = b[columnoffset - 1];
907     int columnlength = l - ((columnoffset - o) + TIMESTAMP_TYPE_SIZE);
908     String family = familylength == 0? "":
909       Bytes.toStringBinary(b, columnoffset, familylength);
910     String qualifier = columnlength == 0? "":
911       Bytes.toStringBinary(b, columnoffset + familylength,
912       columnlength - familylength);
913     long timestamp = Bytes.toLong(b, o + (l - TIMESTAMP_TYPE_SIZE));
914     String timestampStr = humanReadableTimestamp(timestamp);
915     byte type = b[o + l - 1];
916     return row + "/" + family +
917       (family != null && family.length() > 0? ":" :"") +
918       qualifier + "/" + timestampStr + "/" + Type.codeToType(type);
919   }
920 
921   public static String humanReadableTimestamp(final long timestamp) {
922     if (timestamp == HConstants.LATEST_TIMESTAMP) {
923       return "LATEST_TIMESTAMP";
924     }
925     if (timestamp == HConstants.OLDEST_TIMESTAMP) {
926       return "OLDEST_TIMESTAMP";
927     }
928     return String.valueOf(timestamp);
929   }
930 
931   //---------------------------------------------------------------------------
932   //
933   //  Public Member Accessors
934   //
935   //---------------------------------------------------------------------------
936 
937   /**
938    * @return The byte array backing this KeyValue.
939    */
940   public byte [] getBuffer() {
941     return this.bytes;
942   }
943 
944   /**
945    * @return Offset into {@link #getBuffer()} at which this KeyValue starts.
946    */
947   public int getOffset() {
948     return this.offset;
949   }
950 
951   /**
952    * @return Length of bytes this KeyValue occupies in {@link #getBuffer()}.
953    */
954   public int getLength() {
955     return length;
956   }
957 
958   //---------------------------------------------------------------------------
959   //
960   //  Length and Offset Calculators
961   //
962   //---------------------------------------------------------------------------
963 
964   /**
965    * Determines the total length of the KeyValue stored in the specified
966    * byte array and offset.  Includes all headers.
967    * @param bytes byte array
968    * @param offset offset to start of the KeyValue
969    * @return length of entire KeyValue, in bytes
970    */
971   private static int getLength(byte [] bytes, int offset) {
972     return ROW_OFFSET +
973         Bytes.toInt(bytes, offset) +
974         Bytes.toInt(bytes, offset + Bytes.SIZEOF_INT);
975   }
976 
977   /**
978    * @return Key offset in backing buffer..
979    */
980   public int getKeyOffset() {
981     return this.offset + ROW_OFFSET;
982   }
983 
984   public String getKeyString() {
985     return Bytes.toStringBinary(getBuffer(), getKeyOffset(), getKeyLength());
986   }
987 
988   /**
989    * @return Length of key portion.
990    */
991   public int getKeyLength() {
992     return Bytes.toInt(this.bytes, this.offset);
993   }
994 
995   /**
996    * @return the backing array of the entire KeyValue (all KeyValue fields are in a single array)
997    */
998   @Override
999   public byte[] getValueArray() {
1000     return bytes;
1001   }
1002 
1003   /**
1004    * @return Value offset
1005    */
1006   @Override
1007   public int getValueOffset() {
1008     return getKeyOffset() + getKeyLength();
1009   }
1010 
1011   /**
1012    * @return Value length
1013    */
1014   @Override
1015   public int getValueLength() {
1016     return Bytes.toInt(this.bytes, this.offset + Bytes.SIZEOF_INT);
1017   }
1018 
1019   /**
1020    * @return the backing array of the entire KeyValue (all KeyValue fields are in a single array)
1021    */
1022   @Override
1023   public byte[] getRowArray() {
1024     return bytes;
1025   }
1026 
1027   /**
1028    * @return Row offset
1029    */
1030   @Override
1031   public int getRowOffset() {
1032     return getKeyOffset() + Bytes.SIZEOF_SHORT;
1033   }
1034 
1035   /**
1036    * @return Row length
1037    */
1038   @Override
1039   public short getRowLength() {
1040     return Bytes.toShort(this.bytes, getKeyOffset());
1041   }
1042 
1043   /**
1044    * @return the backing array of the entire KeyValue (all KeyValue fields are in a single array)
1045    */
1046   @Override
1047   public byte[] getFamilyArray() {
1048     return bytes;
1049   }
1050 
1051   /**
1052    * @return Family offset
1053    */
1054   @Override
1055   public int getFamilyOffset() {
1056     return getFamilyOffset(getRowLength());
1057   }
1058 
1059   /**
1060    * @return Family offset
1061    */
1062   @Deprecated // making private
1063   public  int getFamilyOffset(int rlength) {
1064     return this.offset + ROW_OFFSET + Bytes.SIZEOF_SHORT + rlength + Bytes.SIZEOF_BYTE;
1065   }
1066 
1067   /**
1068    * @return Family length
1069    */
1070   @Override
1071   public byte getFamilyLength() {
1072     return getFamilyLength(getFamilyOffset());
1073   }
1074 
1075   /**
1076    * @return Family length
1077    */
1078   public byte getFamilyLength(int foffset) {
1079     return this.bytes[foffset-1];
1080   }
1081 
1082   /**
1083    * @return the backing array of the entire KeyValue (all KeyValue fields are in a single array)
1084    */
1085   @Override
1086   public byte[] getQualifierArray() {
1087     return bytes;
1088   }
1089 
1090   /**
1091    * @return Qualifier offset
1092    */
1093   @Override
1094   public int getQualifierOffset() {
1095     return getQualifierOffset(getFamilyOffset());
1096   }
1097 
1098   /**
1099    * @return Qualifier offset
1100    */
1101   @Deprecated // making private
1102   public int getQualifierOffset(int foffset) {
1103     return foffset + getFamilyLength(foffset);
1104   }
1105 
1106   /**
1107    * @return Qualifier length
1108    */
1109   @Override
1110   public int getQualifierLength() {
1111     return getQualifierLength(getRowLength(),getFamilyLength());
1112   }
1113 
1114   /**
1115    * @return Qualifier length
1116    */
1117   @Deprecated // making private
1118   public int getQualifierLength(int rlength, int flength) {
1119     return getKeyLength() - (int) getKeyDataStructureSize(rlength, flength, 0);
1120   }
1121 
1122   /**
1123    * @return Column (family + qualifier) length
1124    */
1125   private int getTotalColumnLength(int rlength, int foffset) {
1126     int flength = getFamilyLength(foffset);
1127     int qlength = getQualifierLength(rlength,flength);
1128     return flength + qlength;
1129   }
1130 
1131   /**
1132    * @return Timestamp offset
1133    */
1134   public int getTimestampOffset() {
1135     return getTimestampOffset(getKeyLength());
1136   }
1137 
1138   /**
1139    * @param keylength Pass if you have it to save on a int creation.
1140    * @return Timestamp offset
1141    */
1142   @Deprecated // making private
1143   public int getTimestampOffset(final int keylength) {
1144     return getKeyOffset() + keylength - TIMESTAMP_TYPE_SIZE;
1145   }
1146 
1147   /**
1148    * @return True if this KeyValue has a LATEST_TIMESTAMP timestamp.
1149    */
1150   public boolean isLatestTimestamp() {
1151     return Bytes.equals(getBuffer(), getTimestampOffset(), Bytes.SIZEOF_LONG,
1152       HConstants.LATEST_TIMESTAMP_BYTES, 0, Bytes.SIZEOF_LONG);
1153   }
1154 
1155   /**
1156    * @param now Time to set into <code>this</code> IFF timestamp ==
1157    * {@link HConstants#LATEST_TIMESTAMP} (else, its a noop).
1158    * @return True is we modified this.
1159    */
1160   public boolean updateLatestStamp(final byte [] now) {
1161     if (this.isLatestTimestamp()) {
1162       int tsOffset = getTimestampOffset();
1163       System.arraycopy(now, 0, this.bytes, tsOffset, Bytes.SIZEOF_LONG);
1164       // clear cache or else getTimestamp() possibly returns an old value
1165       return true;
1166     }
1167     return false;
1168   }
1169 
1170   //---------------------------------------------------------------------------
1171   //
1172   //  Methods that return copies of fields
1173   //
1174   //---------------------------------------------------------------------------
1175 
1176   /**
1177    * Do not use unless you have to.  Used internally for compacting and testing.
1178    *
1179    * Use {@link #getRow()}, {@link #getFamily()}, {@link #getQualifier()}, and
1180    * {@link #getValue()} if accessing a KeyValue client-side.
1181    * @return Copy of the key portion only.
1182    */
1183   public byte [] getKey() {
1184     int keylength = getKeyLength();
1185     byte [] key = new byte[keylength];
1186     System.arraycopy(getBuffer(), getKeyOffset(), key, 0, keylength);
1187     return key;
1188   }
1189 
1190   /**
1191    * Returns value in a new byte array.
1192    * Primarily for use client-side. If server-side, use
1193    * {@link #getBuffer()} with appropriate offsets and lengths instead to
1194    * save on allocations.
1195    * @return Value in a new byte array.
1196    */
1197   @Deprecated // use CellUtil.getValueArray()
1198   public byte [] getValue() {
1199     return CellUtil.cloneValue(this);
1200   }
1201 
1202   /**
1203    * Primarily for use client-side.  Returns the row of this KeyValue in a new
1204    * byte array.<p>
1205    *
1206    * If server-side, use {@link #getBuffer()} with appropriate offsets and
1207    * lengths instead.
1208    * @return Row in a new byte array.
1209    */
1210   @Deprecated // use CellUtil.getRowArray()
1211   public byte [] getRow() {
1212     return CellUtil.cloneRow(this);
1213   }
1214 
1215   /**
1216    *
1217    * @return Timestamp
1218    */
1219   @Override
1220   public long getTimestamp() {
1221     return getTimestamp(getKeyLength());
1222   }
1223 
1224   /**
1225    * @param keylength Pass if you have it to save on a int creation.
1226    * @return Timestamp
1227    */
1228   long getTimestamp(final int keylength) {
1229     int tsOffset = getTimestampOffset(keylength);
1230     return Bytes.toLong(this.bytes, tsOffset);
1231   }
1232 
1233   /**
1234    * @return Type of this KeyValue.
1235    */
1236   @Deprecated // use getTypeByte()
1237   public byte getType() {
1238     return getTypeByte();
1239   }
1240 
1241   /**
1242    * @return KeyValue.TYPE byte representation
1243    */
1244   @Override
1245   public byte getTypeByte() {
1246     return this.bytes[this.offset + getKeyLength() - 1 + ROW_OFFSET];
1247   }
1248 
1249   /**
1250    * @return True if a delete type, a {@link KeyValue.Type#Delete} or
1251    * a {KeyValue.Type#DeleteFamily} or a {@link KeyValue.Type#DeleteColumn}
1252    * KeyValue type.
1253    */
1254   @Deprecated // use CellUtil#isDelete
1255   public boolean isDelete() {
1256     return KeyValue.isDelete(getType());
1257   }
1258 
1259   /**
1260    * @return True if this KV is a {@link KeyValue.Type#Delete} type.
1261    */
1262   public boolean isDeleteType() {
1263     // TODO: Fix this method name vis-a-vis isDelete!
1264     return getTypeByte() == Type.Delete.getCode();
1265   }
1266 
1267   /**
1268    * @return True if this KV is a delete family type.
1269    */
1270   public boolean isDeleteFamily() {
1271     return getTypeByte() == Type.DeleteFamily.getCode();
1272   }
1273 
1274   /**
1275    * @return True if this KV is a delete family-version type.
1276    */
1277   public boolean isDeleteFamilyVersion() {
1278     return getTypeByte() == Type.DeleteFamilyVersion.getCode();
1279   }
1280 
1281   /**
1282    *
1283    * @return True if this KV is a delete family or column type.
1284    */
1285   public boolean isDeleteColumnOrFamily() {
1286     int t = getTypeByte();
1287     return t == Type.DeleteColumn.getCode() || t == Type.DeleteFamily.getCode();
1288   }
1289 
1290   /**
1291    * Primarily for use client-side.  Returns the family of this KeyValue in a
1292    * new byte array.<p>
1293    *
1294    * If server-side, use {@link #getBuffer()} with appropriate offsets and
1295    * lengths instead.
1296    * @return Returns family. Makes a copy.
1297    */
1298   @Deprecated // use CellUtil.getFamilyArray
1299   public byte [] getFamily() {
1300     return CellUtil.cloneFamily(this);
1301   }
1302 
1303   /**
1304    * Primarily for use client-side.  Returns the column qualifier of this
1305    * KeyValue in a new byte array.<p>
1306    *
1307    * If server-side, use {@link #getBuffer()} with appropriate offsets and
1308    * lengths instead.
1309    * Use {@link #getBuffer()} with appropriate offsets and lengths instead.
1310    * @return Returns qualifier. Makes a copy.
1311    */
1312   @Deprecated // use CellUtil.getQualifierArray
1313   public byte [] getQualifier() {
1314     return CellUtil.cloneQualifier(this);
1315   }
1316 
1317   //---------------------------------------------------------------------------
1318   //
1319   //  Compare specified fields against those contained in this KeyValue
1320   //
1321   //---------------------------------------------------------------------------
1322 
1323   /**
1324    * @param family
1325    * @return True if matching families.
1326    */
1327   public boolean matchingFamily(final byte [] family) {
1328     if (this.length == 0 || this.bytes.length == 0) {
1329       return false;
1330     }
1331     return Bytes.equals(family, 0, family.length,
1332         this.bytes, getFamilyOffset(), getFamilyLength());
1333   }
1334 
1335   /**
1336    * @param qualifier
1337    * @return True if matching qualifiers.
1338    */
1339   public boolean matchingQualifier(final byte [] qualifier) {
1340     return matchingQualifier(qualifier, 0, qualifier.length);
1341   }
1342 
1343   public boolean matchingQualifier(final byte [] qualifier, int offset, int length) {
1344     return Bytes.equals(qualifier, offset, length,
1345         this.bytes, getQualifierOffset(), getQualifierLength());
1346   }
1347 
1348   public boolean matchingQualifier(final KeyValue other) {
1349     return matchingQualifier(other.getBuffer(), other.getQualifierOffset(),
1350         other.getQualifierLength());
1351   }
1352 
1353   public boolean matchingRow(final byte [] row) {
1354     return matchingRow(row, 0, row.length);
1355   }
1356 
1357   public boolean matchingRow(final byte[] row, int offset, int length) {
1358     return Bytes.equals(row, offset, length,
1359         this.bytes, getRowOffset(), getRowLength());
1360   }
1361 
1362   public boolean matchingRow(KeyValue other) {
1363     return matchingRow(other.getBuffer(), other.getRowOffset(),
1364         other.getRowLength());
1365   }
1366 
1367   /**
1368    *
1369    * @param family column family
1370    * @param qualifier column qualifier
1371    * @return True if column matches
1372    */
1373   public boolean matchingColumn(final byte[] family, final byte[] qualifier) {
1374     return matchingColumn(family, 0, len(family), qualifier, 0, len(qualifier));
1375   }
1376 
1377   /**
1378    * Checks if column matches.
1379    *
1380    * @param family family name
1381    * @param foffset family offset
1382    * @param flength family length
1383    * @param qualifier column qualifier
1384    * @param qoffset qualifier offset
1385    * @param qlength qualifier length
1386    *
1387    * @return True if column matches
1388    */
1389   public boolean matchingColumn(final byte [] family, final int foffset, final int flength,
1390       final byte [] qualifier, final int qoffset, final int qlength) {
1391     int rl = getRowLength();
1392     int o = getFamilyOffset(rl);
1393     int fl = getFamilyLength(o);
1394     if (!Bytes.equals(family, foffset, flength, this.bytes, o, fl)) {
1395       return false;
1396     }
1397 
1398     int ql = getQualifierLength(rl, fl);
1399     if (qualifier == null || qlength == 0) {
1400       return (ql == 0);
1401     }
1402     return Bytes.equals(qualifier, qoffset, qlength, this.bytes, o + fl, ql);
1403   }
1404 
1405   /**
1406    * Creates a new KeyValue that only contains the key portion (the value is
1407    * set to be null).
1408    *
1409    * TODO only used by KeyOnlyFilter -- move there.
1410    * @param lenAsVal replace value with the actual value length (false=empty)
1411    */
1412   public KeyValue createKeyOnly(boolean lenAsVal) {
1413     // KV format:  <keylen:4><valuelen:4><key:keylen><value:valuelen>
1414     // Rebuild as: <keylen:4><0:4><key:keylen>
1415     int dataLen = lenAsVal? Bytes.SIZEOF_INT : 0;
1416     byte [] newBuffer = new byte[getKeyLength() + ROW_OFFSET + dataLen];
1417     System.arraycopy(this.bytes, this.offset, newBuffer, 0,
1418         Math.min(newBuffer.length,this.length));
1419     Bytes.putInt(newBuffer, Bytes.SIZEOF_INT, dataLen);
1420     if (lenAsVal) {
1421       Bytes.putInt(newBuffer, newBuffer.length - dataLen, this.getValueLength());
1422     }
1423     return new KeyValue(newBuffer);
1424   }
1425 
1426   /**
1427    * Splits a column in {@code family:qualifier} form into separate byte arrays. An empty qualifier
1428    * (ie, {@code fam:}) is parsed as <code>{ fam, EMPTY_BYTE_ARRAY }</code> while no delimiter (ie,
1429    * {@code fam}) is parsed as an array of one element, <code>{ fam }</code>.
1430    * <p>
1431    * Don't forget, HBase DOES support empty qualifiers. (see HBASE-9549)
1432    * </p>
1433    * <p>
1434    * Not recommend to be used as this is old-style API.
1435    * </p>
1436    * @param c The column.
1437    * @return The parsed column.
1438    */
1439   public static byte [][] parseColumn(byte [] c) {
1440     final int index = getDelimiter(c, 0, c.length, COLUMN_FAMILY_DELIMITER);
1441     if (index == -1) {
1442       // If no delimiter, return array of size 1
1443       return new byte [][] { c };
1444     } else if(index == c.length - 1) {
1445       // family with empty qualifier, return array size 2
1446       byte [] family = new byte[c.length-1];
1447       System.arraycopy(c, 0, family, 0, family.length);
1448       return new byte [][] { family, HConstants.EMPTY_BYTE_ARRAY};
1449     }
1450     // Family and column, return array size 2
1451     final byte [][] result = new byte [2][];
1452     result[0] = new byte [index];
1453     System.arraycopy(c, 0, result[0], 0, index);
1454     final int len = c.length - (index + 1);
1455     result[1] = new byte[len];
1456     System.arraycopy(c, index + 1 /* Skip delimiter */, result[1], 0, len);
1457     return result;
1458   }
1459 
1460   /**
1461    * Makes a column in family:qualifier form from separate byte arrays.
1462    * <p>
1463    * Not recommended for usage as this is old-style API.
1464    * @param family
1465    * @param qualifier
1466    * @return family:qualifier
1467    */
1468   public static byte [] makeColumn(byte [] family, byte [] qualifier) {
1469     return Bytes.add(family, COLUMN_FAMILY_DELIM_ARRAY, qualifier);
1470   }
1471 
1472   /**
1473    * This function is only used in Meta key comparisons so its error message
1474    * is specific for meta key errors.
1475    */
1476   static int getRequiredDelimiterInReverse(final byte [] b,
1477       final int offset, final int length, final int delimiter) {
1478     int index = getDelimiterInReverse(b, offset, length, delimiter);
1479     if (index < 0) {
1480       throw new IllegalArgumentException("hbase:meta key must have two '" + (char)delimiter + "' "
1481         + "delimiters and have the following format: '<table>,<key>,<etc>'");
1482     }
1483     return index;
1484   }
1485 
1486   /**
1487    * @param b
1488    * @param delimiter
1489    * @return Index of delimiter having started from start of <code>b</code>
1490    * moving rightward.
1491    */
1492   public static int getDelimiter(final byte [] b, int offset, final int length,
1493       final int delimiter) {
1494     if (b == null) {
1495       throw new IllegalArgumentException("Passed buffer is null");
1496     }
1497     int result = -1;
1498     for (int i = offset; i < length + offset; i++) {
1499       if (b[i] == delimiter) {
1500         result = i;
1501         break;
1502       }
1503     }
1504     return result;
1505   }
1506 
1507   /**
1508    * Find index of passed delimiter walking from end of buffer backwards.
1509    * @param b
1510    * @param delimiter
1511    * @return Index of delimiter
1512    */
1513   public static int getDelimiterInReverse(final byte [] b, final int offset,
1514       final int length, final int delimiter) {
1515     if (b == null) {
1516       throw new IllegalArgumentException("Passed buffer is null");
1517     }
1518     int result = -1;
1519     for (int i = (offset + length) - 1; i >= offset; i--) {
1520       if (b[i] == delimiter) {
1521         result = i;
1522         break;
1523       }
1524     }
1525     return result;
1526   }
1527 
1528   /**
1529    * A {@link KVComparator} for <code>hbase:meta</code> catalog table
1530    * {@link KeyValue}s.
1531    */
1532   public static class MetaComparator extends KVComparator {
1533     /**
1534      * Compare key portion of a {@link KeyValue} for keys in <code>hbase:meta</code>
1535      * table.
1536      */
1537     @Override
1538     public int compareRows(byte [] left, int loffset, int llength,
1539         byte [] right, int roffset, int rlength) {
1540       int leftDelimiter = getDelimiter(left, loffset, llength,
1541           HConstants.DELIMITER);
1542       int rightDelimiter = getDelimiter(right, roffset, rlength,
1543           HConstants.DELIMITER);
1544       if (leftDelimiter < 0 && rightDelimiter >= 0) {
1545         // Nothing between hbase:meta and regionid.  Its first key.
1546         return -1;
1547       } else if (rightDelimiter < 0 && leftDelimiter >= 0) {
1548         return 1;
1549       } else if (leftDelimiter < 0 && rightDelimiter < 0) {
1550         return 0;
1551       }
1552       // Compare up to the delimiter
1553       int result = Bytes.compareTo(left, loffset, leftDelimiter - loffset,
1554           right, roffset, rightDelimiter - roffset);
1555       if (result != 0) {
1556         return result;
1557       }
1558       // Compare middle bit of the row.
1559       // Move past delimiter
1560       leftDelimiter++;
1561       rightDelimiter++;
1562       int leftFarDelimiter = getRequiredDelimiterInReverse(left, leftDelimiter,
1563           llength - (leftDelimiter - loffset), HConstants.DELIMITER);
1564       int rightFarDelimiter = getRequiredDelimiterInReverse(right,
1565           rightDelimiter, rlength - (rightDelimiter - roffset),
1566           HConstants.DELIMITER);
1567       // Now compare middlesection of row.
1568       result = super.compareRows(left, leftDelimiter,
1569           leftFarDelimiter - leftDelimiter, right, rightDelimiter,
1570           rightFarDelimiter - rightDelimiter);
1571       if (result != 0) {
1572         return result;
1573       }
1574       // Compare last part of row, the rowid.
1575       leftFarDelimiter++;
1576       rightFarDelimiter++;
1577       result = Bytes.compareTo(left, leftFarDelimiter, llength - (leftFarDelimiter - loffset),
1578           right, rightFarDelimiter, rlength - (rightFarDelimiter - roffset));
1579       return result;
1580     }
1581 
1582     /**
1583      * Don't do any fancy Block Index splitting tricks.
1584      */
1585     @Override
1586     public byte[] getShortMidpointKey(final byte[] leftKey, final byte[] rightKey) {
1587       return Arrays.copyOf(rightKey, rightKey.length);
1588     }
1589 
1590     /**
1591      * The HFileV2 file format's trailer contains this class name.  We reinterpret this and
1592      * instantiate the appropriate comparator.
1593      * TODO: With V3 consider removing this.
1594      * @return legacy class name for FileFileTrailer#comparatorClassName
1595      */
1596     @Override
1597     public String getLegacyKeyComparatorName() {
1598       return "org.apache.hadoop.hbase.KeyValue$MetaKeyComparator";
1599     }
1600 
1601     @Override
1602     protected Object clone() throws CloneNotSupportedException {
1603       return new MetaComparator();
1604     }
1605 
1606     /**
1607      * Override the row key comparison to parse and compare the meta row key parts.
1608      */
1609     @Override
1610     protected int compareRowKey(final Cell l, final Cell r) {
1611       byte[] left = l.getRowArray();
1612       int loffset = l.getRowOffset();
1613       int llength = l.getRowLength();
1614       byte[] right = r.getRowArray();
1615       int roffset = r.getRowOffset();
1616       int rlength = r.getRowLength();
1617       return compareRows(left, loffset, llength, right, roffset, rlength);
1618     }
1619   }
1620 
1621   /**
1622    * Compare KeyValues.  When we compare KeyValues, we only compare the Key
1623    * portion.  This means two KeyValues with same Key but different Values are
1624    * considered the same as far as this Comparator is concerned.
1625    */
1626   public static class KVComparator implements RawComparator<Cell>, SamePrefixComparator<byte[]> {
1627 
1628     /**
1629      * The HFileV2 file format's trailer contains this class name.  We reinterpret this and
1630      * instantiate the appropriate comparator.
1631      * TODO: With V3 consider removing this.
1632      * @return legacy class name for FileFileTrailer#comparatorClassName
1633      */
1634     public String getLegacyKeyComparatorName() {
1635       return "org.apache.hadoop.hbase.KeyValue$KeyComparator";
1636     }
1637 
1638     @Override // RawComparator
1639     public int compare(byte[] l, int loff, int llen, byte[] r, int roff, int rlen) {
1640       return compareFlatKey(l,loff,llen, r,roff,rlen);
1641     }
1642 
1643     
1644     /**
1645      * Compares the only the user specified portion of a Key.  This is overridden by MetaComparator.
1646      * @param left
1647      * @param right
1648      * @return 0 if equal, <0 if left smaller, >0 if right smaller
1649      */
1650     protected int compareRowKey(final Cell left, final Cell right) {
1651       return Bytes.compareTo(
1652           left.getRowArray(),  left.getRowOffset(),  left.getRowLength(),
1653           right.getRowArray(), right.getRowOffset(), right.getRowLength());
1654     }
1655 
1656     /**
1657      * Compares left to right assuming that left,loffset,llength and right,roffset,rlength are
1658      * full KVs laid out in a flat byte[]s.
1659      * @param left
1660      * @param loffset
1661      * @param llength
1662      * @param right
1663      * @param roffset
1664      * @param rlength
1665      * @return  0 if equal, <0 if left smaller, >0 if right smaller
1666      */
1667     public int compareFlatKey(byte[] left, int loffset, int llength,
1668         byte[] right, int roffset, int rlength) {
1669       // Compare row
1670       short lrowlength = Bytes.toShort(left, loffset);
1671       short rrowlength = Bytes.toShort(right, roffset);
1672       int compare = compareRows(left, loffset + Bytes.SIZEOF_SHORT,
1673           lrowlength, right, roffset + Bytes.SIZEOF_SHORT, rrowlength);
1674       if (compare != 0) {
1675         return compare;
1676       }
1677 
1678       // Compare the rest of the two KVs without making any assumptions about
1679       // the common prefix. This function will not compare rows anyway, so we
1680       // don't need to tell it that the common prefix includes the row.
1681       return compareWithoutRow(0, left, loffset, llength, right, roffset,
1682           rlength, rrowlength);
1683     }
1684 
1685     public int compareFlatKey(byte[] left, byte[] right) {
1686       return compareFlatKey(left, 0, left.length, right, 0, right.length);
1687     }
1688 
1689     /**
1690      * Compares the Key of a cell -- with fields being more significant in this order:
1691      * rowkey, colfam/qual, timestamp, type, mvcc
1692      */
1693     public int compare(final Cell left, final Cell right) {
1694       // compare row
1695       int compare = compareRowKey(left, right);
1696       if (compare != 0) {
1697         return compare;
1698       }
1699 
1700       // compare vs minimum
1701       byte ltype = left.getTypeByte();
1702       byte rtype = right.getTypeByte();
1703       // If the column is not specified, the "minimum" key type appears the
1704       // latest in the sorted order, regardless of the timestamp. This is used
1705       // for specifying the last key/value in a given row, because there is no
1706       // "lexicographically last column" (it would be infinitely long). The
1707       // "maximum" key type does not need this behavior.
1708       int lcfqLen = left.getFamilyLength() + left.getQualifierLength() ;
1709       int rcfqLen = right.getFamilyLength() + right.getQualifierLength() ;
1710       if (lcfqLen == 0 && ltype == Type.Minimum.getCode()) {
1711         // left is "bigger", i.e. it appears later in the sorted order
1712         return 1;
1713       }
1714       if (rcfqLen == 0 && rtype == Type.Minimum.getCode()) {
1715         return -1;
1716       }
1717 
1718 
1719       // compare col family / col fam + qual
1720       // If left family size is not equal to right family size, we need not
1721       // compare the qualifiers.
1722       compare = Bytes.compareTo(
1723         left.getFamilyArray(),  left.getFamilyOffset(),  left.getFamilyLength(),
1724         right.getFamilyArray(), right.getFamilyOffset(), right.getFamilyLength());
1725       if (compare != 0) {
1726         return compare;
1727       }
1728 
1729       // Compare qualifier
1730       compare = Bytes.compareTo(
1731           left.getQualifierArray(), left.getQualifierOffset(), left.getQualifierLength(),
1732           right.getQualifierArray(), right.getQualifierOffset(), right.getQualifierLength());
1733       if (compare!= 0) {
1734         return compare;
1735       }
1736 
1737       // compare timestamp
1738       long ltimestamp = left.getTimestamp();
1739       long rtimestamp = right.getTimestamp();
1740       compare = compareTimestamps(ltimestamp, rtimestamp);
1741       if (compare != 0) {
1742         return compare;
1743       }
1744 
1745       // Compare types. Let the delete types sort ahead of puts; i.e. types
1746       // of higher numbers sort before those of lesser numbers. Maximum (255)
1747       // appears ahead of everything, and minimum (0) appears after
1748       // everything.
1749       compare = (0xff & rtype) - (0xff & ltype);
1750       if (compare != 0) {
1751         return compare;
1752       }
1753 
1754       // compare Mvcc Version
1755       // Negate this comparison so later edits show up first
1756       return -Longs.compare(left.getMvccVersion(), right.getMvccVersion());
1757     }
1758 
1759     public int compareTimestamps(final KeyValue left, final KeyValue right) {
1760       // Compare timestamps
1761       long ltimestamp = left.getTimestamp(left.getKeyLength());
1762       long rtimestamp = right.getTimestamp(right.getKeyLength());
1763       return compareTimestamps(ltimestamp, rtimestamp);
1764     }
1765 
1766     /**
1767      * @param left
1768      * @param right
1769      * @return Result comparing rows.
1770      */
1771     public int compareRows(final KeyValue left, final KeyValue right) {
1772       return compareRows(left.getBuffer(),left.getRowOffset(), left.getRowLength(),
1773       right.getBuffer(), right.getRowOffset(), right.getRowLength());
1774     }
1775 
1776     /**
1777      * Get the b[],o,l for left and right rowkey portions and compare.
1778      * @param left
1779      * @param loffset
1780      * @param llength
1781      * @param right
1782      * @param roffset
1783      * @param rlength
1784      * @return 0 if equal, <0 if left smaller, >0 if right smaller
1785      */
1786     public int compareRows(byte [] left, int loffset, int llength,
1787         byte [] right, int roffset, int rlength) {
1788       return Bytes.compareTo(left, loffset, llength, right, roffset, rlength);
1789     }
1790 
1791     int compareColumns(final KeyValue left, final short lrowlength,
1792         final KeyValue right, final short rrowlength) {
1793       int lfoffset = left.getFamilyOffset(lrowlength);
1794       int rfoffset = right.getFamilyOffset(rrowlength);
1795       int lclength = left.getTotalColumnLength(lrowlength,lfoffset);
1796       int rclength = right.getTotalColumnLength(rrowlength, rfoffset);
1797       int lfamilylength = left.getFamilyLength(lfoffset);
1798       int rfamilylength = right.getFamilyLength(rfoffset);
1799       return compareColumns(left.getBuffer(), lfoffset,
1800           lclength, lfamilylength,
1801         right.getBuffer(), rfoffset, rclength, rfamilylength);
1802     }
1803 
1804     protected int compareColumns(
1805         byte [] left, int loffset, int llength, final int lfamilylength,
1806         byte [] right, int roffset, int rlength, final int rfamilylength) {
1807       // Compare family portion first.
1808       int diff = Bytes.compareTo(left, loffset, lfamilylength,
1809         right, roffset, rfamilylength);
1810       if (diff != 0) {
1811         return diff;
1812       }
1813       // Compare qualifier portion
1814       return Bytes.compareTo(left, loffset + lfamilylength,
1815         llength - lfamilylength,
1816         right, roffset + rfamilylength, rlength - rfamilylength);
1817       }
1818 
1819     static int compareTimestamps(final long ltimestamp, final long rtimestamp) {
1820       // The below older timestamps sorting ahead of newer timestamps looks
1821       // wrong but it is intentional. This way, newer timestamps are first
1822       // found when we iterate over a memstore and newer versions are the
1823       // first we trip over when reading from a store file.
1824       if (ltimestamp < rtimestamp) {
1825         return 1;
1826       } else if (ltimestamp > rtimestamp) {
1827         return -1;
1828       }
1829       return 0;
1830     }
1831 
1832     /**
1833      * Overridden
1834      * @param commonPrefix
1835      * @param left
1836      * @param loffset
1837      * @param llength
1838      * @param right
1839      * @param roffset
1840      * @param rlength
1841      * @return 0 if equal, <0 if left smaller, >0 if right smaller
1842      */
1843     @Override // SamePrefixComparator
1844     public int compareIgnoringPrefix(int commonPrefix, byte[] left,
1845         int loffset, int llength, byte[] right, int roffset, int rlength) {
1846       // Compare row
1847       short lrowlength = Bytes.toShort(left, loffset);
1848       short rrowlength;
1849 
1850       int comparisonResult = 0;
1851       if (commonPrefix < ROW_LENGTH_SIZE) {
1852         // almost nothing in common
1853         rrowlength = Bytes.toShort(right, roffset);
1854         comparisonResult = compareRows(left, loffset + ROW_LENGTH_SIZE,
1855             lrowlength, right, roffset + ROW_LENGTH_SIZE, rrowlength);
1856       } else { // the row length is the same
1857         rrowlength = lrowlength;
1858         if (commonPrefix < ROW_LENGTH_SIZE + rrowlength) {
1859           // The rows are not the same. Exclude the common prefix and compare
1860           // the rest of the two rows.
1861           int common = commonPrefix - ROW_LENGTH_SIZE;
1862           comparisonResult = compareRows(
1863               left, loffset + common + ROW_LENGTH_SIZE, lrowlength - common,
1864               right, roffset + common + ROW_LENGTH_SIZE, rrowlength - common);
1865         }
1866       }
1867       if (comparisonResult != 0) {
1868         return comparisonResult;
1869       }
1870 
1871       assert lrowlength == rrowlength;
1872       return compareWithoutRow(commonPrefix, left, loffset, llength, right,
1873           roffset, rlength, lrowlength);
1874     }
1875 
1876     /**
1877      * Compare columnFamily, qualifier, timestamp, and key type (everything
1878      * except the row). This method is used both in the normal comparator and
1879      * the "same-prefix" comparator. Note that we are assuming that row portions
1880      * of both KVs have already been parsed and found identical, and we don't
1881      * validate that assumption here.
1882      * @param commonPrefix
1883      *          the length of the common prefix of the two key-values being
1884      *          compared, including row length and row
1885      */
1886     private int compareWithoutRow(int commonPrefix, byte[] left, int loffset,
1887         int llength, byte[] right, int roffset, int rlength, short rowlength) {
1888       /***
1889        * KeyValue Format and commonLength:
1890        * |_keyLen_|_valLen_|_rowLen_|_rowKey_|_famiLen_|_fami_|_Quali_|....
1891        * ------------------|-------commonLength--------|--------------
1892        */
1893       int commonLength = ROW_LENGTH_SIZE + FAMILY_LENGTH_SIZE + rowlength;
1894 
1895       // commonLength + TIMESTAMP_TYPE_SIZE
1896       int commonLengthWithTSAndType = TIMESTAMP_TYPE_SIZE + commonLength;
1897       // ColumnFamily + Qualifier length.
1898       int lcolumnlength = llength - commonLengthWithTSAndType;
1899       int rcolumnlength = rlength - commonLengthWithTSAndType;
1900 
1901       byte ltype = left[loffset + (llength - 1)];
1902       byte rtype = right[roffset + (rlength - 1)];
1903 
1904       // If the column is not specified, the "minimum" key type appears the
1905       // latest in the sorted order, regardless of the timestamp. This is used
1906       // for specifying the last key/value in a given row, because there is no
1907       // "lexicographically last column" (it would be infinitely long). The
1908       // "maximum" key type does not need this behavior.
1909       if (lcolumnlength == 0 && ltype == Type.Minimum.getCode()) {
1910         // left is "bigger", i.e. it appears later in the sorted order
1911         return 1;
1912       }
1913       if (rcolumnlength == 0 && rtype == Type.Minimum.getCode()) {
1914         return -1;
1915       }
1916 
1917       int lfamilyoffset = commonLength + loffset;
1918       int rfamilyoffset = commonLength + roffset;
1919 
1920       // Column family length.
1921       int lfamilylength = left[lfamilyoffset - 1];
1922       int rfamilylength = right[rfamilyoffset - 1];
1923       // If left family size is not equal to right family size, we need not
1924       // compare the qualifiers.
1925       boolean sameFamilySize = (lfamilylength == rfamilylength);
1926       int common = 0;
1927       if (commonPrefix > 0) {
1928         common = Math.max(0, commonPrefix - commonLength);
1929         if (!sameFamilySize) {
1930           // Common should not be larger than Math.min(lfamilylength,
1931           // rfamilylength).
1932           common = Math.min(common, Math.min(lfamilylength, rfamilylength));
1933         } else {
1934           common = Math.min(common, Math.min(lcolumnlength, rcolumnlength));
1935         }
1936       }
1937       if (!sameFamilySize) {
1938         // comparing column family is enough.
1939         return Bytes.compareTo(left, lfamilyoffset + common, lfamilylength
1940             - common, right, rfamilyoffset + common, rfamilylength - common);
1941       }
1942       // Compare family & qualifier together.
1943       final int comparison = Bytes.compareTo(left, lfamilyoffset + common,
1944           lcolumnlength - common, right, rfamilyoffset + common,
1945           rcolumnlength - common);
1946       if (comparison != 0) {
1947         return comparison;
1948       }
1949 
1950       ////
1951       // Next compare timestamps.
1952       long ltimestamp = Bytes.toLong(left,
1953           loffset + (llength - TIMESTAMP_TYPE_SIZE));
1954       long rtimestamp = Bytes.toLong(right,
1955           roffset + (rlength - TIMESTAMP_TYPE_SIZE));
1956       int compare = compareTimestamps(ltimestamp, rtimestamp);
1957       if (compare != 0) {
1958         return compare;
1959       }
1960 
1961       // Compare types. Let the delete types sort ahead of puts; i.e. types
1962       // of higher numbers sort before those of lesser numbers. Maximum (255)
1963       // appears ahead of everything, and minimum (0) appears after
1964       // everything.
1965       return (0xff & rtype) - (0xff & ltype);
1966     }
1967 
1968     /**
1969      * Compares the row and column of two keyvalues for equality
1970      * @param left
1971      * @param right
1972      * @return True if same row and column.
1973      */
1974     public boolean matchingRowColumn(final KeyValue left,
1975         final KeyValue right) {
1976       short lrowlength = left.getRowLength();
1977       short rrowlength = right.getRowLength();
1978 
1979       // TsOffset = end of column data. just comparing Row+CF length of each
1980       if ((left.getTimestampOffset() - left.getOffset()) !=
1981           (right.getTimestampOffset() - right.getOffset())) {
1982         return false;
1983       }
1984 
1985       if (!matchingRows(left, lrowlength, right, rrowlength)) {
1986         return false;
1987       }
1988 
1989       int lfoffset = left.getFamilyOffset(lrowlength);
1990       int rfoffset = right.getFamilyOffset(rrowlength);
1991       int lclength = left.getTotalColumnLength(lrowlength,lfoffset);
1992       int rclength = right.getTotalColumnLength(rrowlength, rfoffset);
1993       int lfamilylength = left.getFamilyLength(lfoffset);
1994       int rfamilylength = right.getFamilyLength(rfoffset);
1995       int ccRes = compareColumns(left.getBuffer(), lfoffset, lclength, lfamilylength,
1996           right.getBuffer(), rfoffset, rclength, rfamilylength);
1997       return ccRes == 0;
1998     }
1999 
2000     /**
2001      * Compares the row of two keyvalues for equality
2002      * @param left
2003      * @param right
2004      * @return True if rows match.
2005      */
2006     public boolean matchingRows(final KeyValue left, final KeyValue right) {
2007       short lrowlength = left.getRowLength();
2008       short rrowlength = right.getRowLength();
2009       return matchingRows(left, lrowlength, right, rrowlength);
2010     }
2011 
2012     /**
2013      * @param left
2014      * @param lrowlength
2015      * @param right
2016      * @param rrowlength
2017      * @return True if rows match.
2018      */
2019     private boolean matchingRows(final KeyValue left, final short lrowlength,
2020         final KeyValue right, final short rrowlength) {
2021       return lrowlength == rrowlength &&
2022           Bytes.equals(left.getBuffer(), left.getRowOffset(), lrowlength,
2023               right.getBuffer(), right.getRowOffset(), rrowlength);
2024     }
2025 
2026     public byte[] calcIndexKey(byte[] lastKeyOfPreviousBlock, byte[] firstKeyInBlock) {
2027       byte[] fakeKey = getShortMidpointKey(lastKeyOfPreviousBlock, firstKeyInBlock);
2028       if (compareFlatKey(fakeKey, firstKeyInBlock) > 0) {
2029         LOG.error("Unexpected getShortMidpointKey result, fakeKey:"
2030             + Bytes.toStringBinary(fakeKey) + ", firstKeyInBlock:"
2031             + Bytes.toStringBinary(firstKeyInBlock));
2032         return firstKeyInBlock;
2033       }
2034       if (lastKeyOfPreviousBlock != null && compareFlatKey(lastKeyOfPreviousBlock, fakeKey) >= 0) {
2035         LOG.error("Unexpected getShortMidpointKey result, lastKeyOfPreviousBlock:" +
2036             Bytes.toStringBinary(lastKeyOfPreviousBlock) + ", fakeKey:" +
2037             Bytes.toStringBinary(fakeKey));
2038         return firstKeyInBlock;
2039       }
2040       return fakeKey;
2041     }
2042 
2043     /**
2044      * This is a HFile block index key optimization.
2045      * @param leftKey
2046      * @param rightKey
2047      * @return 0 if equal, <0 if left smaller, >0 if right smaller
2048      */
2049     public byte[] getShortMidpointKey(final byte[] leftKey, final byte[] rightKey) {
2050       if (rightKey == null) {
2051         throw new IllegalArgumentException("rightKey can not be null");
2052       }
2053       if (leftKey == null) {
2054         return Arrays.copyOf(rightKey, rightKey.length);
2055       }
2056       if (compareFlatKey(leftKey, rightKey) >= 0) {
2057         throw new IllegalArgumentException("Unexpected input, leftKey:" + Bytes.toString(leftKey)
2058           + ", rightKey:" + Bytes.toString(rightKey));
2059       }
2060 
2061       short leftRowLength = Bytes.toShort(leftKey, 0);
2062       short rightRowLength = Bytes.toShort(rightKey, 0);
2063       int leftCommonLength = ROW_LENGTH_SIZE + FAMILY_LENGTH_SIZE + leftRowLength;
2064       int rightCommonLength = ROW_LENGTH_SIZE + FAMILY_LENGTH_SIZE + rightRowLength;
2065       int leftCommonLengthWithTSAndType = TIMESTAMP_TYPE_SIZE + leftCommonLength;
2066       int rightCommonLengthWithTSAndType = TIMESTAMP_TYPE_SIZE + rightCommonLength;
2067       int leftColumnLength = leftKey.length - leftCommonLengthWithTSAndType;
2068       int rightColumnLength = rightKey.length - rightCommonLengthWithTSAndType;
2069       // rows are equal
2070       if (leftRowLength == rightRowLength && compareRows(leftKey, ROW_LENGTH_SIZE, leftRowLength,
2071         rightKey, ROW_LENGTH_SIZE, rightRowLength) == 0) {
2072         // Compare family & qualifier together.
2073         int comparison = Bytes.compareTo(leftKey, leftCommonLength, leftColumnLength, rightKey,
2074           rightCommonLength, rightColumnLength);
2075         // same with "row + family + qualifier", return rightKey directly
2076         if (comparison == 0) {
2077           return Arrays.copyOf(rightKey, rightKey.length);
2078         }
2079         // "family + qualifier" are different, generate a faked key per rightKey
2080         byte[] newKey = Arrays.copyOf(rightKey, rightKey.length);
2081         Bytes.putLong(newKey, rightKey.length - TIMESTAMP_TYPE_SIZE, HConstants.LATEST_TIMESTAMP);
2082         Bytes.putByte(newKey, rightKey.length - TYPE_SIZE, Type.Maximum.getCode());
2083         return newKey;
2084       }
2085       // rows are different
2086       short minLength = leftRowLength < rightRowLength ? leftRowLength : rightRowLength;
2087       short diffIdx = 0;
2088       while (diffIdx < minLength
2089           && leftKey[ROW_LENGTH_SIZE + diffIdx] == rightKey[ROW_LENGTH_SIZE + diffIdx]) {
2090         diffIdx++;
2091       }
2092       if (diffIdx >= minLength) {
2093         // leftKey's row is prefix of rightKey's. we can optimize it in future
2094         return Arrays.copyOf(rightKey, rightKey.length);
2095       }
2096       int diffByte = leftKey[ROW_LENGTH_SIZE + diffIdx];
2097       if ((0xff & diffByte) < 0xff && (diffByte + 1) <
2098           (rightKey[ROW_LENGTH_SIZE + diffIdx] & 0xff)) {
2099         byte[] newRowKey = new byte[diffIdx + 1];
2100         System.arraycopy(leftKey, ROW_LENGTH_SIZE, newRowKey, 0, diffIdx);
2101         newRowKey[diffIdx] = (byte) (diffByte + 1);
2102         int rightFamilyLength = rightKey[rightCommonLength - 1];
2103         byte[] family = null;
2104         if (rightFamilyLength > 0) {
2105           family = new byte[rightFamilyLength];
2106           System.arraycopy(rightKey, rightCommonLength, family, 0, rightFamilyLength);
2107         }
2108         int rightQualifierLength = rightColumnLength - rightFamilyLength;
2109         byte[] qualifier = null;
2110         if (rightQualifierLength > 0) {
2111           qualifier = new byte[rightQualifierLength];
2112           System.arraycopy(rightKey, rightCommonLength + rightFamilyLength, qualifier, 0,
2113             rightQualifierLength);
2114         }
2115         return new KeyValue(newRowKey, null, null, HConstants.LATEST_TIMESTAMP,
2116           Type.Maximum).getKey();
2117       }
2118       // the following is optimizable in future
2119       return Arrays.copyOf(rightKey, rightKey.length);
2120     }
2121 
2122     @Override
2123     protected Object clone() throws CloneNotSupportedException {
2124       return new KVComparator();
2125     }
2126 
2127   }
2128 
2129   /**
2130    * Creates a KeyValue that is last on the specified row id. That is,
2131    * every other possible KeyValue for the given row would compareTo()
2132    * less than the result of this call.
2133    * @param row row key
2134    * @return Last possible KeyValue on passed <code>row</code>
2135    */
2136   public static KeyValue createLastOnRow(final byte[] row) {
2137     return new KeyValue(row, null, null, HConstants.LATEST_TIMESTAMP, Type.Minimum);
2138   }
2139 
2140   /**
2141    * Create a KeyValue that is smaller than all other possible KeyValues
2142    * for the given row. That is any (valid) KeyValue on 'row' would sort
2143    * _after_ the result.
2144    *
2145    * @param row - row key (arbitrary byte array)
2146    * @return First possible KeyValue on passed <code>row</code>
2147    */
2148   public static KeyValue createFirstOnRow(final byte [] row) {
2149     return createFirstOnRow(row, HConstants.LATEST_TIMESTAMP);
2150   }
2151 
2152   /**
2153    * Create a KeyValue that is smaller than all other possible KeyValues
2154    * for the given row. That is any (valid) KeyValue on 'row' would sort
2155    * _after_ the result.
2156    *
2157    * @param row - row key (arbitrary byte array)
2158    * @return First possible KeyValue on passed <code>row</code>
2159    */
2160   public static KeyValue createFirstOnRow(final byte [] row, int roffset, short rlength) {
2161     return new KeyValue(row, roffset, rlength,
2162         null, 0, 0, null, 0, 0, HConstants.LATEST_TIMESTAMP, Type.Maximum, null, 0, 0);
2163   }
2164 
2165   /**
2166    * Creates a KeyValue that is smaller than all other KeyValues that
2167    * are older than the passed timestamp.
2168    * @param row - row key (arbitrary byte array)
2169    * @param ts - timestamp
2170    * @return First possible key on passed <code>row</code> and timestamp.
2171    */
2172   public static KeyValue createFirstOnRow(final byte [] row,
2173       final long ts) {
2174     return new KeyValue(row, null, null, ts, Type.Maximum);
2175   }
2176 
2177   /**
2178    * Create a KeyValue for the specified row, family and qualifier that would be
2179    * smaller than all other possible KeyValues that have the same row,family,qualifier.
2180    * Used for seeking.
2181    * @param row - row key (arbitrary byte array)
2182    * @param family - family name
2183    * @param qualifier - column qualifier
2184    * @return First possible key on passed <code>row</code>, and column.
2185    */
2186   public static KeyValue createFirstOnRow(final byte [] row, final byte [] family,
2187       final byte [] qualifier) {
2188     return new KeyValue(row, family, qualifier, HConstants.LATEST_TIMESTAMP, Type.Maximum);
2189   }
2190 
2191   /**
2192    * Create a Delete Family KeyValue for the specified row and family that would
2193    * be smaller than all other possible Delete Family KeyValues that have the
2194    * same row and family.
2195    * Used for seeking.
2196    * @param row - row key (arbitrary byte array)
2197    * @param family - family name
2198    * @return First Delete Family possible key on passed <code>row</code>.
2199    */
2200   public static KeyValue createFirstDeleteFamilyOnRow(final byte [] row,
2201       final byte [] family) {
2202     return new KeyValue(row, family, null, HConstants.LATEST_TIMESTAMP,
2203         Type.DeleteFamily);
2204   }
2205 
2206   /**
2207    * @param row - row key (arbitrary byte array)
2208    * @param f - family name
2209    * @param q - column qualifier
2210    * @param ts - timestamp
2211    * @return First possible key on passed <code>row</code>, column and timestamp
2212    */
2213   public static KeyValue createFirstOnRow(final byte [] row, final byte [] f,
2214       final byte [] q, final long ts) {
2215     return new KeyValue(row, f, q, ts, Type.Maximum);
2216   }
2217 
2218   /**
2219    * Create a KeyValue for the specified row, family and qualifier that would be
2220    * smaller than all other possible KeyValues that have the same row,
2221    * family, qualifier.
2222    * Used for seeking.
2223    * @param row row key
2224    * @param roffset row offset
2225    * @param rlength row length
2226    * @param family family name
2227    * @param foffset family offset
2228    * @param flength family length
2229    * @param qualifier column qualifier
2230    * @param qoffset qualifier offset
2231    * @param qlength qualifier length
2232    * @return First possible key on passed Row, Family, Qualifier.
2233    */
2234   public static KeyValue createFirstOnRow(final byte [] row,
2235       final int roffset, final int rlength, final byte [] family,
2236       final int foffset, final int flength, final byte [] qualifier,
2237       final int qoffset, final int qlength) {
2238     return new KeyValue(row, roffset, rlength, family,
2239         foffset, flength, qualifier, qoffset, qlength,
2240         HConstants.LATEST_TIMESTAMP, Type.Maximum, null, 0, 0);
2241   }
2242 
2243   /**
2244    * Create a KeyValue for the specified row, family and qualifier that would be
2245    * smaller than all other possible KeyValues that have the same row,
2246    * family, qualifier.
2247    * Used for seeking.
2248    *
2249    * @param buffer the buffer to use for the new <code>KeyValue</code> object
2250    * @param row the value key
2251    * @param family family name
2252    * @param qualifier column qualifier
2253    *
2254    * @return First possible key on passed Row, Family, Qualifier.
2255    *
2256    * @throws IllegalArgumentException The resulting <code>KeyValue</code> object would be larger
2257    * than the provided buffer or than <code>Integer.MAX_VALUE</code>
2258    */
2259   public static KeyValue createFirstOnRow(byte [] buffer, final byte [] row,
2260       final byte [] family, final byte [] qualifier)
2261           throws IllegalArgumentException {
2262 
2263     return createFirstOnRow(buffer, 0, row, 0, row.length,
2264         family, 0, family.length,
2265         qualifier, 0, qualifier.length);
2266   }
2267 
2268   /**
2269    * Create a KeyValue for the specified row, family and qualifier that would be
2270    * smaller than all other possible KeyValues that have the same row,
2271    * family, qualifier.
2272    * Used for seeking.
2273    *
2274    * @param buffer the buffer to use for the new <code>KeyValue</code> object
2275    * @param boffset buffer offset
2276    * @param row the value key
2277    * @param roffset row offset
2278    * @param rlength row length
2279    * @param family family name
2280    * @param foffset family offset
2281    * @param flength family length
2282    * @param qualifier column qualifier
2283    * @param qoffset qualifier offset
2284    * @param qlength qualifier length
2285    *
2286    * @return First possible key on passed Row, Family, Qualifier.
2287    *
2288    * @throws IllegalArgumentException The resulting <code>KeyValue</code> object would be larger
2289    * than the provided buffer or than <code>Integer.MAX_VALUE</code>
2290    */
2291   public static KeyValue createFirstOnRow(byte [] buffer, final int boffset,
2292       final byte [] row, final int roffset, final int rlength,
2293       final byte [] family, final int foffset, final int flength,
2294       final byte [] qualifier, final int qoffset, final int qlength)
2295           throws IllegalArgumentException {
2296 
2297     long lLength = getKeyValueDataStructureSize(rlength, flength, qlength, 0);
2298 
2299     if (lLength > Integer.MAX_VALUE) {
2300       throw new IllegalArgumentException("KeyValue length " + lLength + " > " + Integer.MAX_VALUE);
2301     }
2302     int iLength = (int) lLength;
2303     if (buffer.length - boffset < iLength) {
2304       throw new IllegalArgumentException("Buffer size " + (buffer.length - boffset) + " < " +
2305           iLength);
2306     }
2307 
2308     int len = writeByteArray(buffer, boffset, row, roffset, rlength, family, foffset, flength,
2309         qualifier, qoffset, qlength, HConstants.LATEST_TIMESTAMP, KeyValue.Type.Maximum,
2310         null, 0, 0);
2311     return new KeyValue(buffer, boffset, len);
2312   }
2313 
2314   /**
2315    * Create a KeyValue for the specified row, family and qualifier that would be
2316    * larger than or equal to all other possible KeyValues that have the same
2317    * row, family, qualifier.
2318    * Used for reseeking.
2319    * @param row row key
2320    * @param roffset row offset
2321    * @param rlength row length
2322    * @param family family name
2323    * @param foffset family offset
2324    * @param flength family length
2325    * @param qualifier column qualifier
2326    * @param qoffset qualifier offset
2327    * @param qlength qualifier length
2328    * @return Last possible key on passed row, family, qualifier.
2329    */
2330   public static KeyValue createLastOnRow(final byte [] row,
2331       final int roffset, final int rlength, final byte [] family,
2332       final int foffset, final int flength, final byte [] qualifier,
2333       final int qoffset, final int qlength) {
2334     return new KeyValue(row, roffset, rlength, family,
2335         foffset, flength, qualifier, qoffset, qlength,
2336         HConstants.OLDEST_TIMESTAMP, Type.Minimum, null, 0, 0);
2337   }
2338 
2339   /**
2340    * Similar to {@link #createLastOnRow(byte[], int, int, byte[], int, int,
2341    * byte[], int, int)} but creates the last key on the row/column of this KV
2342    * (the value part of the returned KV is always empty). Used in creating
2343    * "fake keys" for the multi-column Bloom filter optimization to skip the
2344    * row/column we already know is not in the file.
2345    * @return the last key on the row/column of the given key-value pair
2346    */
2347   public KeyValue createLastOnRowCol() {
2348     return new KeyValue(
2349         bytes, getRowOffset(), getRowLength(),
2350         bytes, getFamilyOffset(), getFamilyLength(),
2351         bytes, getQualifierOffset(), getQualifierLength(),
2352         HConstants.OLDEST_TIMESTAMP, Type.Minimum, null, 0, 0);
2353   }
2354 
2355   /**
2356    * Creates the first KV with the row/family/qualifier of this KV and the
2357    * given timestamp. Uses the "maximum" KV type that guarantees that the new
2358    * KV is the lowest possible for this combination of row, family, qualifier,
2359    * and timestamp. This KV's own timestamp is ignored. While this function
2360    * copies the value from this KV, it is normally used on key-only KVs.
2361    */
2362   public KeyValue createFirstOnRowColTS(long ts) {
2363     return new KeyValue(
2364         bytes, getRowOffset(), getRowLength(),
2365         bytes, getFamilyOffset(), getFamilyLength(),
2366         bytes, getQualifierOffset(), getQualifierLength(),
2367         ts, Type.Maximum, bytes, getValueOffset(), getValueLength());
2368   }
2369 
2370   /**
2371    * @param b
2372    * @return A KeyValue made of a byte array that holds the key-only part.
2373    * Needed to convert hfile index members to KeyValues.
2374    */
2375   public static KeyValue createKeyValueFromKey(final byte [] b) {
2376     return createKeyValueFromKey(b, 0, b.length);
2377   }
2378 
2379   /**
2380    * @param bb
2381    * @return A KeyValue made of a byte buffer that holds the key-only part.
2382    * Needed to convert hfile index members to KeyValues.
2383    */
2384   public static KeyValue createKeyValueFromKey(final ByteBuffer bb) {
2385     return createKeyValueFromKey(bb.array(), bb.arrayOffset(), bb.limit());
2386   }
2387 
2388   /**
2389    * @param b
2390    * @param o
2391    * @param l
2392    * @return A KeyValue made of a byte array that holds the key-only part.
2393    * Needed to convert hfile index members to KeyValues.
2394    */
2395   public static KeyValue createKeyValueFromKey(final byte [] b, final int o,
2396       final int l) {
2397     byte [] newb = new byte[l + ROW_OFFSET];
2398     System.arraycopy(b, o, newb, ROW_OFFSET, l);
2399     Bytes.putInt(newb, 0, l);
2400     Bytes.putInt(newb, Bytes.SIZEOF_INT, 0);
2401     return new KeyValue(newb);
2402   }
2403 
2404   /**
2405    * @param in Where to read bytes from.  Creates a byte array to hold the KeyValue
2406    * backing bytes copied from the steam.
2407    * @return KeyValue created by deserializing from <code>in</code> OR if we find a length
2408    * of zero, we will return null which can be useful marking a stream as done.
2409    * @throws IOException
2410    */
2411   public static KeyValue create(final DataInput in) throws IOException {
2412     return create(in.readInt(), in);
2413   }
2414 
2415   /**
2416    * Create a KeyValue reading <code>length</code> from <code>in</code>
2417    * @param length
2418    * @param in
2419    * @return Created KeyValue OR if we find a length of zero, we will return null which
2420    * can be useful marking a stream as done.
2421    * @throws IOException
2422    */
2423   public static KeyValue create(int length, final DataInput in) throws IOException {
2424 
2425     if (length <= 0) {
2426       if (length == 0) return null;
2427       throw new IOException("Failed read " + length + " bytes, stream corrupt?");
2428     }
2429 
2430     // This is how the old Writables.readFrom used to deserialize.  Didn't even vint.
2431     byte [] bytes = new byte[length];
2432     in.readFully(bytes);
2433     return new KeyValue(bytes, 0, length);
2434   }
2435 
2436   /**
2437    * Create a KeyValue reading from the raw InputStream.
2438    * Named <code>iscreate</code> so doesn't clash with {@link #create(DataInput)}
2439    * @param in
2440    * @return Created KeyValue OR if we find a length of zero, we will return null which
2441    * can be useful marking a stream as done.
2442    * @throws IOException
2443    */
2444   public static KeyValue iscreate(final InputStream in) throws IOException {
2445     byte [] intBytes = new byte[Bytes.SIZEOF_INT];
2446     int bytesRead = 0;
2447     while (bytesRead < intBytes.length) {
2448       int n = in.read(intBytes, bytesRead, intBytes.length - bytesRead);
2449       if (n < 0) {
2450         if (bytesRead == 0) return null; // EOF at start is ok
2451         throw new IOException("Failed read of int, read " + bytesRead + " bytes");
2452       }
2453       bytesRead += n;
2454     }
2455     // TODO: perhaps some sanity check is needed here.
2456     byte [] bytes = new byte[Bytes.toInt(intBytes)];
2457     IOUtils.readFully(in, bytes, 0, bytes.length);
2458     return new KeyValue(bytes, 0, bytes.length);
2459   }
2460 
2461   /**
2462    * Write out a KeyValue in the manner in which we used to when KeyValue was a Writable.
2463    * @param kv
2464    * @param out
2465    * @return Length written on stream
2466    * @throws IOException
2467    * @see #create(DataInput) for the inverse function
2468    */
2469   public static long write(final KeyValue kv, final DataOutput out) throws IOException {
2470     // This is how the old Writables write used to serialize KVs.  Need to figure way to make it
2471     // work for all implementations.
2472     int length = kv.getLength();
2473     out.writeInt(length);
2474     out.write(kv.getBuffer(), kv.getOffset(), length);
2475     return length + Bytes.SIZEOF_INT;
2476   }
2477 
2478   /**
2479    * Write out a KeyValue in the manner in which we used to when KeyValue was a Writable but do
2480    * not require a {@link DataOutput}, just take plain {@link OutputStream}
2481    * Named <code>oswrite</code> so does not clash with {@link #write(KeyValue, DataOutput)}
2482    * @param kv
2483    * @param out
2484    * @return Length written on stream
2485    * @throws IOException
2486    * @see #create(DataInput) for the inverse function
2487    * @see #write(KeyValue, DataOutput)
2488    */
2489   public static long oswrite(final KeyValue kv, final OutputStream out) throws IOException {
2490     int length = kv.getLength();
2491     // This does same as DataOuput#writeInt (big-endian, etc.)
2492     out.write(Bytes.toBytes(length));
2493     out.write(kv.getBuffer(), kv.getOffset(), length);
2494     return length + Bytes.SIZEOF_INT;
2495   }
2496 
2497   /**
2498    * Comparator that compares row component only of a KeyValue.
2499    */
2500   public static class RowOnlyComparator implements Comparator<KeyValue> {
2501     final KVComparator comparator;
2502 
2503     public RowOnlyComparator(final KVComparator c) {
2504       this.comparator = c;
2505     }
2506 
2507     public int compare(KeyValue left, KeyValue right) {
2508       return comparator.compareRows(left, right);
2509     }
2510   }
2511 
2512   /**
2513    * Avoids redundant comparisons for better performance.
2514    * 
2515    * TODO get rid of this wart
2516    */
2517   public interface SamePrefixComparator<T> {
2518     /**
2519      * Compare two keys assuming that the first n bytes are the same.
2520      * @param commonPrefix How many bytes are the same.
2521      */
2522     int compareIgnoringPrefix(
2523       int commonPrefix, byte[] left, int loffset, int llength, byte[] right, int roffset, int rlength
2524     );
2525   }
2526 
2527   /**
2528    * This is a TEST only Comparator used in TestSeekTo and TestReseekTo.
2529    */
2530   public static class RawBytesComparator extends KVComparator {
2531     /**
2532      * The HFileV2 file format's trailer contains this class name.  We reinterpret this and
2533      * instantiate the appropriate comparator.
2534      * TODO: With V3 consider removing this.
2535      * @return legacy class name for FileFileTrailer#comparatorClassName
2536      */
2537     public String getLegacyKeyComparatorName() {
2538       return "org.apache.hadoop.hbase.util.Bytes$ByteArrayComparator";
2539     }
2540 
2541     public int compareFlatKey(byte[] left, int loffset, int llength, byte[] right,
2542         int roffset, int rlength) {
2543       return Bytes.BYTES_RAWCOMPARATOR.compare(left,  loffset, llength, right, roffset, rlength);
2544     }
2545 
2546     public byte[] calcIndexKey(byte[] lastKeyOfPreviousBlock, byte[] firstKeyInBlock) {
2547       return firstKeyInBlock;
2548     }
2549 
2550   }
2551 
2552   /**
2553    * HeapSize implementation
2554    *
2555    * We do not count the bytes in the rowCache because it should be empty for a KeyValue in the
2556    * MemStore.
2557    */
2558   @Override
2559   public long heapSize() {
2560     int sum = 0;
2561     sum += ClassSize.OBJECT;// the KeyValue object itself
2562     sum += ClassSize.REFERENCE;// pointer to "bytes"
2563     sum += ClassSize.align(ClassSize.ARRAY);// "bytes"
2564     sum += ClassSize.align(length);// number of bytes of data in the "bytes" array
2565     sum += 2 * Bytes.SIZEOF_INT;// offset, length
2566     sum += Bytes.SIZEOF_LONG;// memstoreTS
2567     return ClassSize.align(sum);
2568   }
2569 
2570   // -----
2571   // KV tags stubs
2572   @Override
2573   public int getTagsOffset() {
2574     throw new UnsupportedOperationException("Not implemented");
2575   }
2576 
2577   @Override
2578   public short getTagsLength() {
2579     throw new UnsupportedOperationException("Not implemented");
2580   }
2581 
2582   @Override
2583   public byte[] getTagsArray() {
2584     throw new UnsupportedOperationException("Not implemented");
2585   }
2586 
2587 }