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