1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase;
21
22 import static org.apache.hadoop.hbase.util.Bytes.len;
23
24 import java.io.DataInput;
25 import java.io.DataOutput;
26 import java.io.EOFException;
27 import java.io.IOException;
28 import java.io.InputStream;
29 import java.io.OutputStream;
30 import java.nio.ByteBuffer;
31 import java.util.ArrayList;
32 import java.util.Arrays;
33 import java.util.Comparator;
34 import java.util.HashMap;
35 import java.util.List;
36 import java.util.Map;
37
38 import org.apache.commons.logging.Log;
39 import org.apache.commons.logging.LogFactory;
40 import org.apache.hadoop.hbase.classification.InterfaceAudience;
41 import org.apache.hadoop.hbase.io.HeapSize;
42 import org.apache.hadoop.hbase.util.Bytes;
43 import org.apache.hadoop.hbase.util.ClassSize;
44 import org.apache.hadoop.io.IOUtils;
45 import org.apache.hadoop.io.RawComparator;
46
47 import com.google.common.primitives.Longs;
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81 @InterfaceAudience.Private
82 public class KeyValue implements Cell, HeapSize, Cloneable {
83 private static final ArrayList<Tag> EMPTY_ARRAY_LIST = new ArrayList<Tag>();
84
85 static final Log LOG = LogFactory.getLog(KeyValue.class);
86
87
88
89
90 public static final char COLUMN_FAMILY_DELIMITER = ':';
91
92 public static final byte[] COLUMN_FAMILY_DELIM_ARRAY =
93 new byte[]{COLUMN_FAMILY_DELIMITER};
94
95
96
97
98
99 public static final KVComparator COMPARATOR = new KVComparator();
100
101
102
103
104 public static final KVComparator META_COMPARATOR = new MetaComparator();
105
106
107
108
109 public static final KVComparator RAW_COMPARATOR = new RawBytesComparator();
110
111
112 public static final int KEY_LENGTH_SIZE = Bytes.SIZEOF_INT;
113
114
115 public static final int TYPE_SIZE = Bytes.SIZEOF_BYTE;
116
117
118 public static final int ROW_LENGTH_SIZE = Bytes.SIZEOF_SHORT;
119
120
121 public static final int FAMILY_LENGTH_SIZE = Bytes.SIZEOF_BYTE;
122
123
124 public static final int TIMESTAMP_SIZE = Bytes.SIZEOF_LONG;
125
126
127 public static final int TIMESTAMP_TYPE_SIZE = TIMESTAMP_SIZE + TYPE_SIZE;
128
129
130 public static final int KEY_INFRASTRUCTURE_SIZE = ROW_LENGTH_SIZE
131 + FAMILY_LENGTH_SIZE + TIMESTAMP_TYPE_SIZE;
132
133
134
135 public static final int ROW_OFFSET =
136 Bytes.SIZEOF_INT
137 Bytes.SIZEOF_INT
138
139
140 public static final int KEYVALUE_INFRASTRUCTURE_SIZE = ROW_OFFSET;
141
142
143 public static final int TAGS_LENGTH_SIZE = Bytes.SIZEOF_SHORT;
144
145 public static final int KEYVALUE_WITH_TAGS_INFRASTRUCTURE_SIZE = ROW_OFFSET + TAGS_LENGTH_SIZE;
146
147 private static final int MAX_TAGS_LENGTH = (2 * Short.MAX_VALUE) + 1;
148
149
150
151
152
153
154
155
156
157
158
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
168
169
170
171
172
173
174
175
176
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
189
190
191
192
193
194
195
196
197 public static long getKeyValueDataStructureSize(int klength, int vlength, int tagsLength) {
198 if (tagsLength == 0) {
199 return KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE + klength + vlength;
200 }
201 return KeyValue.KEYVALUE_WITH_TAGS_INFRASTRUCTURE_SIZE + klength + vlength + tagsLength;
202 }
203
204
205
206
207
208
209
210
211
212
213
214 public static long getKeyDataStructureSize(int rlength, int flength, int qlength) {
215 return KeyValue.KEY_INFRASTRUCTURE_SIZE + rlength + flength + qlength;
216 }
217
218
219
220
221
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
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
246
247
248
249
250
251 public static Type codeToType(final byte b) {
252 for (Type t : Type.values()) {
253 if (t.getCode() == b) {
254 return t;
255 }
256 }
257 throw new RuntimeException("Unknown code " + b);
258 }
259 }
260
261
262
263
264
265
266 public static final KeyValue LOWESTKEY =
267 new KeyValue(HConstants.EMPTY_BYTE_ARRAY, HConstants.LATEST_TIMESTAMP);
268
269
270
271 private byte [] bytes = null;
272 private int offset = 0;
273 private int length = 0;
274
275
276
277
278
279
280 public static boolean isDelete(byte t) {
281 return Type.Delete.getCode() <= t && t <= Type.DeleteFamily.getCode();
282 }
283
284
285
286
287 @Override
288 public long getMvccVersion() {
289 return mvcc;
290 }
291
292 public void setMvccVersion(long mvccVersion){
293 this.mvcc = mvccVersion;
294 }
295
296
297 private long mvcc = 0;
298
299
300
301
302
303 public KeyValue() {}
304
305
306
307
308
309
310 public KeyValue(final byte [] bytes) {
311 this(bytes, 0);
312 }
313
314
315
316
317
318
319
320
321 public KeyValue(final byte [] bytes, final int offset) {
322 this(bytes, offset, getLength(bytes, offset));
323 }
324
325
326
327
328
329
330
331
332 public KeyValue(final byte [] bytes, final int offset, final int length) {
333 this.bytes = bytes;
334 this.offset = offset;
335 this.length = length;
336 }
337
338
339
340
341
342
343
344
345
346
347 public KeyValue(final byte[] bytes, final int offset, final int length, long ts) {
348 this(bytes, offset, length, null, 0, 0, null, 0, 0, ts, Type.Maximum, null, 0, 0, null);
349 }
350
351
352
353
354
355
356
357
358
359 public KeyValue(final byte [] row, final long timestamp) {
360 this(row, null, null, timestamp, Type.Maximum, null);
361 }
362
363
364
365
366
367
368 public KeyValue(final byte [] row, final long timestamp, Type type) {
369 this(row, null, null, timestamp, type, null);
370 }
371
372
373
374
375
376
377
378
379 public KeyValue(final byte [] row, final byte [] family,
380 final byte [] qualifier) {
381 this(row, family, qualifier, HConstants.LATEST_TIMESTAMP, Type.Maximum);
382 }
383
384
385
386
387
388
389
390
391 public KeyValue(final byte [] row, final byte [] family,
392 final byte [] qualifier, final byte [] value) {
393 this(row, family, qualifier, HConstants.LATEST_TIMESTAMP, Type.Put, value);
394 }
395
396
397
398
399
400
401
402
403
404
405 public KeyValue(final byte[] row, final byte[] family,
406 final byte[] qualifier, final long timestamp, Type type) {
407 this(row, family, qualifier, timestamp, type, null);
408 }
409
410
411
412
413
414
415
416
417
418
419 public KeyValue(final byte[] row, final byte[] family,
420 final byte[] qualifier, final long timestamp, final byte[] value) {
421 this(row, family, qualifier, timestamp, Type.Put, value);
422 }
423
424
425
426
427
428
429
430
431
432
433
434 public KeyValue(final byte[] row, final byte[] family,
435 final byte[] qualifier, final long timestamp, final byte[] value,
436 final Tag[] tags) {
437 this(row, family, qualifier, timestamp, value, tags != null ? Arrays.asList(tags) : null);
438 }
439
440
441
442
443
444
445
446
447
448
449
450 public KeyValue(final byte[] row, final byte[] family,
451 final byte[] qualifier, final long timestamp, final byte[] value,
452 final List<Tag> tags) {
453 this(row, 0, row==null ? 0 : row.length,
454 family, 0, family==null ? 0 : family.length,
455 qualifier, 0, qualifier==null ? 0 : qualifier.length,
456 timestamp, Type.Put,
457 value, 0, value==null ? 0 : value.length, tags);
458 }
459
460
461
462
463
464
465
466
467
468
469
470 public KeyValue(final byte[] row, final byte[] family,
471 final byte[] qualifier, final long timestamp, Type type,
472 final byte[] value) {
473 this(row, 0, len(row), family, 0, len(family), qualifier, 0, len(qualifier),
474 timestamp, type, value, 0, len(value));
475 }
476
477
478
479
480
481
482
483
484
485
486
487
488
489 public KeyValue(final byte[] row, final byte[] family,
490 final byte[] qualifier, final long timestamp, Type type,
491 final byte[] value, final List<Tag> tags) {
492 this(row, family, qualifier, 0, qualifier==null ? 0 : qualifier.length,
493 timestamp, type, value, 0, value==null ? 0 : value.length, tags);
494 }
495
496
497
498
499
500
501
502
503
504
505
506 public KeyValue(final byte[] row, final byte[] family,
507 final byte[] qualifier, final long timestamp, Type type,
508 final byte[] value, final byte[] tags) {
509 this(row, family, qualifier, 0, qualifier==null ? 0 : qualifier.length,
510 timestamp, type, value, 0, value==null ? 0 : value.length, tags);
511 }
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527 public KeyValue(byte [] row, byte [] family,
528 byte [] qualifier, int qoffset, int qlength, long timestamp, Type type,
529 byte [] value, int voffset, int vlength, List<Tag> tags) {
530 this(row, 0, row==null ? 0 : row.length,
531 family, 0, family==null ? 0 : family.length,
532 qualifier, qoffset, qlength, timestamp, type,
533 value, voffset, vlength, tags);
534 }
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549 public KeyValue(byte [] row, byte [] family,
550 byte [] qualifier, int qoffset, int qlength, long timestamp, Type type,
551 byte [] value, int voffset, int vlength, byte[] tags) {
552 this(row, 0, row==null ? 0 : row.length,
553 family, 0, family==null ? 0 : family.length,
554 qualifier, qoffset, qlength, timestamp, type,
555 value, voffset, vlength, tags, 0, tags==null ? 0 : tags.length);
556 }
557
558
559
560
561
562
563
564
565 public KeyValue(final byte [] row, final int roffset, final int rlength,
566 final byte [] family, final int foffset, final int flength,
567 final byte [] qualifier, final int qoffset, final int qlength,
568 final long timestamp, final Type type,
569 final byte [] value, final int voffset, final int vlength) {
570 this(row, roffset, rlength, family, foffset, flength, qualifier, qoffset,
571 qlength, timestamp, type, value, voffset, vlength, null);
572 }
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600 public KeyValue(byte [] buffer, final int boffset,
601 final byte [] row, final int roffset, final int rlength,
602 final byte [] family, final int foffset, final int flength,
603 final byte [] qualifier, final int qoffset, final int qlength,
604 final long timestamp, final Type type,
605 final byte [] value, final int voffset, final int vlength,
606 final Tag[] tags) {
607 this.bytes = buffer;
608 this.length = writeByteArray(buffer, boffset,
609 row, roffset, rlength,
610 family, foffset, flength, qualifier, qoffset, qlength,
611 timestamp, type, value, voffset, vlength, tags);
612 this.offset = boffset;
613 }
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636 public KeyValue(final byte [] row, final int roffset, final int rlength,
637 final byte [] family, final int foffset, final int flength,
638 final byte [] qualifier, final int qoffset, final int qlength,
639 final long timestamp, final Type type,
640 final byte [] value, final int voffset, final int vlength,
641 final List<Tag> tags) {
642 this.bytes = createByteArray(row, roffset, rlength,
643 family, foffset, flength, qualifier, qoffset, qlength,
644 timestamp, type, value, voffset, vlength, tags);
645 this.length = bytes.length;
646 this.offset = 0;
647 }
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666 public KeyValue(final byte [] row, final int roffset, final int rlength,
667 final byte [] family, final int foffset, final int flength,
668 final byte [] qualifier, final int qoffset, final int qlength,
669 final long timestamp, final Type type,
670 final byte [] value, final int voffset, final int vlength,
671 final byte[] tags, final int tagsOffset, final int tagsLength) {
672 this.bytes = createByteArray(row, roffset, rlength,
673 family, foffset, flength, qualifier, qoffset, qlength,
674 timestamp, type, value, voffset, vlength, tags, tagsOffset, tagsLength);
675 this.length = bytes.length;
676 this.offset = 0;
677 }
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692 public KeyValue(final int rlength,
693 final int flength,
694 final int qlength,
695 final long timestamp, final Type type,
696 final int vlength) {
697 this(rlength, flength, qlength, timestamp, type, vlength, 0);
698 }
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714 public KeyValue(final int rlength,
715 final int flength,
716 final int qlength,
717 final long timestamp, final Type type,
718 final int vlength, final int tagsLength) {
719 this.bytes = createEmptyByteArray(rlength, flength, qlength, timestamp, type, vlength,
720 tagsLength);
721 this.length = bytes.length;
722 this.offset = 0;
723 }
724
725
726 public KeyValue(byte[] row, int roffset, int rlength,
727 byte[] family, int foffset, int flength,
728 ByteBuffer qualifier, long ts, Type type, ByteBuffer value, List<Tag> tags) {
729 this.bytes = createByteArray(row, roffset, rlength, family, foffset, flength,
730 qualifier, 0, qualifier == null ? 0 : qualifier.remaining(), ts, type,
731 value, 0, value == null ? 0 : value.remaining(), tags);
732 this.length = bytes.length;
733 this.offset = 0;
734 }
735
736 public KeyValue(Cell c) {
737 this(c.getRowArray(), c.getRowOffset(), (int)c.getRowLength(),
738 c.getFamilyArray(), c.getFamilyOffset(), (int)c.getFamilyLength(),
739 c.getQualifierArray(), c.getQualifierOffset(), (int) c.getQualifierLength(),
740 c.getTimestamp(), Type.codeToType(c.getTypeByte()), c.getValueArray(), c.getValueOffset(),
741 c.getValueLength(), c.getTagsArray(), c.getTagsOffset(), c.getTagsLengthUnsigned());
742 }
743
744
745
746
747
748
749
750
751
752
753
754
755 private static byte[] createEmptyByteArray(final int rlength, int flength,
756 int qlength, final long timestamp, final Type type, int vlength, int tagsLength) {
757 if (rlength > Short.MAX_VALUE) {
758 throw new IllegalArgumentException("Row > " + Short.MAX_VALUE);
759 }
760 if (flength > Byte.MAX_VALUE) {
761 throw new IllegalArgumentException("Family > " + Byte.MAX_VALUE);
762 }
763
764 if (qlength > Integer.MAX_VALUE - rlength - flength) {
765 throw new IllegalArgumentException("Qualifier > " + Integer.MAX_VALUE);
766 }
767 checkForTagsLength(tagsLength);
768
769 long longkeylength = getKeyDataStructureSize(rlength, flength, qlength);
770 if (longkeylength > Integer.MAX_VALUE) {
771 throw new IllegalArgumentException("keylength " + longkeylength + " > " +
772 Integer.MAX_VALUE);
773 }
774 int keylength = (int)longkeylength;
775
776 if (vlength > HConstants.MAXIMUM_VALUE_LENGTH) {
777 throw new IllegalArgumentException("Valuer > " +
778 HConstants.MAXIMUM_VALUE_LENGTH);
779 }
780
781
782 byte[] bytes= new byte[(int) getKeyValueDataStructureSize(rlength, flength, qlength, vlength,
783 tagsLength)];
784
785 int pos = 0;
786 pos = Bytes.putInt(bytes, pos, keylength);
787 pos = Bytes.putInt(bytes, pos, vlength);
788 pos = Bytes.putShort(bytes, pos, (short)(rlength & 0x0000ffff));
789 pos += rlength;
790 pos = Bytes.putByte(bytes, pos, (byte)(flength & 0x0000ff));
791 pos += flength + qlength;
792 pos = Bytes.putLong(bytes, pos, timestamp);
793 pos = Bytes.putByte(bytes, pos, type.getCode());
794 pos += vlength;
795 if (tagsLength > 0) {
796 pos = Bytes.putAsShort(bytes, pos, tagsLength);
797 }
798 return bytes;
799 }
800
801
802
803
804
805
806
807
808
809
810
811
812
813 private static void checkParameters(final byte [] row, final int rlength,
814 final byte [] family, int flength, int qlength, int vlength)
815 throws IllegalArgumentException {
816 if (rlength > Short.MAX_VALUE) {
817 throw new IllegalArgumentException("Row > " + Short.MAX_VALUE);
818 }
819 if (row == null) {
820 throw new IllegalArgumentException("Row is null");
821 }
822
823 flength = family == null ? 0 : flength;
824 if (flength > Byte.MAX_VALUE) {
825 throw new IllegalArgumentException("Family > " + Byte.MAX_VALUE);
826 }
827
828 if (qlength > Integer.MAX_VALUE - rlength - flength) {
829 throw new IllegalArgumentException("Qualifier > " + Integer.MAX_VALUE);
830 }
831
832 long longKeyLength = getKeyDataStructureSize(rlength, flength, qlength);
833 if (longKeyLength > Integer.MAX_VALUE) {
834 throw new IllegalArgumentException("keylength " + longKeyLength + " > " +
835 Integer.MAX_VALUE);
836 }
837
838 if (vlength > HConstants.MAXIMUM_VALUE_LENGTH) {
839 throw new IllegalArgumentException("Value length " + vlength + " > " +
840 HConstants.MAXIMUM_VALUE_LENGTH);
841 }
842 }
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869 private static int writeByteArray(byte [] buffer, final int boffset,
870 final byte [] row, final int roffset, final int rlength,
871 final byte [] family, final int foffset, int flength,
872 final byte [] qualifier, final int qoffset, int qlength,
873 final long timestamp, final Type type,
874 final byte [] value, final int voffset, int vlength, Tag[] tags) {
875
876 checkParameters(row, rlength, family, flength, qlength, vlength);
877
878
879 int tagsLength = 0;
880 if (tags != null && tags.length > 0) {
881 for (Tag t: tags) {
882 tagsLength += t.getLength();
883 }
884 }
885 checkForTagsLength(tagsLength);
886 int keyLength = (int) getKeyDataStructureSize(rlength, flength, qlength);
887 int keyValueLength = (int) getKeyValueDataStructureSize(rlength, flength, qlength, vlength,
888 tagsLength);
889 if (keyValueLength > buffer.length - boffset) {
890 throw new IllegalArgumentException("Buffer size " + (buffer.length - boffset) + " < " +
891 keyValueLength);
892 }
893
894
895 int pos = boffset;
896 pos = Bytes.putInt(buffer, pos, keyLength);
897 pos = Bytes.putInt(buffer, pos, vlength);
898 pos = Bytes.putShort(buffer, pos, (short)(rlength & 0x0000ffff));
899 pos = Bytes.putBytes(buffer, pos, row, roffset, rlength);
900 pos = Bytes.putByte(buffer, pos, (byte) (flength & 0x0000ff));
901 if (flength != 0) {
902 pos = Bytes.putBytes(buffer, pos, family, foffset, flength);
903 }
904 if (qlength != 0) {
905 pos = Bytes.putBytes(buffer, pos, qualifier, qoffset, qlength);
906 }
907 pos = Bytes.putLong(buffer, pos, timestamp);
908 pos = Bytes.putByte(buffer, pos, type.getCode());
909 if (value != null && value.length > 0) {
910 pos = Bytes.putBytes(buffer, pos, value, voffset, vlength);
911 }
912
913 if (tagsLength > 0) {
914 pos = Bytes.putAsShort(buffer, pos, tagsLength);
915 for (Tag t : tags) {
916 pos = Bytes.putBytes(buffer, pos, t.getBuffer(), t.getOffset(), t.getLength());
917 }
918 }
919 return keyValueLength;
920 }
921
922 private static void checkForTagsLength(int tagsLength) {
923 if (tagsLength > MAX_TAGS_LENGTH) {
924 throw new IllegalArgumentException("tagslength "+ tagsLength + " > " + MAX_TAGS_LENGTH);
925 }
926 }
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946 private static byte [] createByteArray(final byte [] row, final int roffset,
947 final int rlength, final byte [] family, final int foffset, int flength,
948 final byte [] qualifier, final int qoffset, int qlength,
949 final long timestamp, final Type type,
950 final byte [] value, final int voffset,
951 int vlength, byte[] tags, int tagsOffset, int tagsLength) {
952
953 checkParameters(row, rlength, family, flength, qlength, vlength);
954 checkForTagsLength(tagsLength);
955
956 int keyLength = (int) getKeyDataStructureSize(rlength, flength, qlength);
957 byte [] bytes =
958 new byte[(int) getKeyValueDataStructureSize(rlength, flength, qlength, vlength, tagsLength)];
959
960 int pos = 0;
961 pos = Bytes.putInt(bytes, pos, keyLength);
962 pos = Bytes.putInt(bytes, pos, vlength);
963 pos = Bytes.putShort(bytes, pos, (short)(rlength & 0x0000ffff));
964 pos = Bytes.putBytes(bytes, pos, row, roffset, rlength);
965 pos = Bytes.putByte(bytes, pos, (byte)(flength & 0x0000ff));
966 if(flength != 0) {
967 pos = Bytes.putBytes(bytes, pos, family, foffset, flength);
968 }
969 if(qlength != 0) {
970 pos = Bytes.putBytes(bytes, pos, qualifier, qoffset, qlength);
971 }
972 pos = Bytes.putLong(bytes, pos, timestamp);
973 pos = Bytes.putByte(bytes, pos, type.getCode());
974 if (value != null && value.length > 0) {
975 pos = Bytes.putBytes(bytes, pos, value, voffset, vlength);
976 }
977
978 if (tagsLength > 0) {
979 pos = Bytes.putAsShort(bytes, pos, tagsLength);
980 pos = Bytes.putBytes(bytes, pos, tags, tagsOffset, tagsLength);
981 }
982 return bytes;
983 }
984
985
986
987
988
989 private static byte [] createByteArray(final byte [] row, final int roffset,
990 final int rlength, final byte [] family, final int foffset, int flength,
991 final Object qualifier, final int qoffset, int qlength,
992 final long timestamp, final Type type,
993 final Object value, final int voffset, int vlength, List<Tag> tags) {
994
995 checkParameters(row, rlength, family, flength, qlength, vlength);
996
997
998 int tagsLength = 0;
999 if (tags != null && !tags.isEmpty()) {
1000 for (Tag t : tags) {
1001 tagsLength += t.getLength();
1002 }
1003 }
1004 checkForTagsLength(tagsLength);
1005
1006 int keyLength = (int) getKeyDataStructureSize(rlength, flength, qlength);
1007 byte[] bytes = new byte[(int) getKeyValueDataStructureSize(rlength, flength, qlength, vlength,
1008 tagsLength)];
1009
1010
1011 int pos = 0;
1012 pos = Bytes.putInt(bytes, pos, keyLength);
1013
1014 pos = Bytes.putInt(bytes, pos, vlength);
1015 pos = Bytes.putShort(bytes, pos, (short)(rlength & 0x0000ffff));
1016 pos = Bytes.putBytes(bytes, pos, row, roffset, rlength);
1017 pos = Bytes.putByte(bytes, pos, (byte)(flength & 0x0000ff));
1018 if(flength != 0) {
1019 pos = Bytes.putBytes(bytes, pos, family, foffset, flength);
1020 }
1021 if (qlength > 0) {
1022 if (qualifier instanceof ByteBuffer) {
1023 pos = Bytes.putByteBuffer(bytes, pos, (ByteBuffer) qualifier);
1024 } else {
1025 pos = Bytes.putBytes(bytes, pos, (byte[]) qualifier, qoffset, qlength);
1026 }
1027 }
1028 pos = Bytes.putLong(bytes, pos, timestamp);
1029 pos = Bytes.putByte(bytes, pos, type.getCode());
1030 if (vlength > 0) {
1031 if (value instanceof ByteBuffer) {
1032 pos = Bytes.putByteBuffer(bytes, pos, (ByteBuffer) value);
1033 } else {
1034 pos = Bytes.putBytes(bytes, pos, (byte[]) value, voffset, vlength);
1035 }
1036 }
1037
1038 if (tagsLength > 0) {
1039 pos = Bytes.putAsShort(bytes, pos, tagsLength);
1040 for (Tag t : tags) {
1041 pos = Bytes.putBytes(bytes, pos, t.getBuffer(), t.getOffset(), t.getLength());
1042 }
1043 }
1044 return bytes;
1045 }
1046
1047
1048
1049
1050 @Override
1051 public boolean equals(Object other) {
1052 if (!(other instanceof Cell)) {
1053 return false;
1054 }
1055 return CellComparator.equals(this, (Cell)other);
1056 }
1057
1058
1059
1060
1061 @Override
1062 public int hashCode() {
1063 return CellComparator.hashCodeIgnoreMvcc(this);
1064 }
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077 @Override
1078 public KeyValue clone() throws CloneNotSupportedException {
1079 super.clone();
1080 byte [] b = new byte[this.length];
1081 System.arraycopy(this.bytes, this.offset, b, 0, this.length);
1082 KeyValue ret = new KeyValue(b, 0, b.length);
1083
1084
1085
1086 ret.setMvccVersion(mvcc);
1087 return ret;
1088 }
1089
1090
1091
1092
1093
1094
1095 public KeyValue shallowCopy() {
1096 KeyValue shallowCopy = new KeyValue(this.bytes, this.offset, this.length);
1097 shallowCopy.setMvccVersion(this.mvcc);
1098 return shallowCopy;
1099 }
1100
1101
1102
1103
1104
1105
1106
1107 public String toString() {
1108 if (this.bytes == null || this.bytes.length == 0) {
1109 return "empty";
1110 }
1111 return keyToString(this.bytes, this.offset + ROW_OFFSET, getKeyLength()) +
1112 "/vlen=" + getValueLength() + "/mvcc=" + mvcc;
1113 }
1114
1115
1116
1117
1118
1119 public static String keyToString(final byte [] k) {
1120 if (k == null) {
1121 return "";
1122 }
1123 return keyToString(k, 0, k.length);
1124 }
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134 public Map<String, Object> toStringMap() {
1135 Map<String, Object> stringMap = new HashMap<String, Object>();
1136 stringMap.put("row", Bytes.toStringBinary(getRow()));
1137 stringMap.put("family", Bytes.toStringBinary(getFamily()));
1138 stringMap.put("qualifier", Bytes.toStringBinary(getQualifier()));
1139 stringMap.put("timestamp", getTimestamp());
1140 stringMap.put("vlen", getValueLength());
1141 List<Tag> tags = getTags();
1142 if (tags != null) {
1143 List<String> tagsString = new ArrayList<String>();
1144 for (Tag t : tags) {
1145 tagsString.add((t.getType()) + ":" +Bytes.toStringBinary(t.getValue()));
1146 }
1147 stringMap.put("tag", tagsString);
1148 }
1149 return stringMap;
1150 }
1151
1152
1153
1154
1155
1156
1157
1158
1159 public static String keyToString(final byte [] b, final int o, final int l) {
1160 if (b == null) return "";
1161 int rowlength = Bytes.toShort(b, o);
1162 String row = Bytes.toStringBinary(b, o + Bytes.SIZEOF_SHORT, rowlength);
1163 int columnoffset = o + Bytes.SIZEOF_SHORT + 1 + rowlength;
1164 int familylength = b[columnoffset - 1];
1165 int columnlength = l - ((columnoffset - o) + TIMESTAMP_TYPE_SIZE);
1166 String family = familylength == 0? "":
1167 Bytes.toStringBinary(b, columnoffset, familylength);
1168 String qualifier = columnlength == 0? "":
1169 Bytes.toStringBinary(b, columnoffset + familylength,
1170 columnlength - familylength);
1171 long timestamp = Bytes.toLong(b, o + (l - TIMESTAMP_TYPE_SIZE));
1172 String timestampStr = humanReadableTimestamp(timestamp);
1173 byte type = b[o + l - 1];
1174 return row + "/" + family +
1175 (family != null && family.length() > 0? ":" :"") +
1176 qualifier + "/" + timestampStr + "/" + Type.codeToType(type);
1177 }
1178
1179 public static String humanReadableTimestamp(final long timestamp) {
1180 if (timestamp == HConstants.LATEST_TIMESTAMP) {
1181 return "LATEST_TIMESTAMP";
1182 }
1183 if (timestamp == HConstants.OLDEST_TIMESTAMP) {
1184 return "OLDEST_TIMESTAMP";
1185 }
1186 return String.valueOf(timestamp);
1187 }
1188
1189
1190
1191
1192
1193
1194
1195
1196
1197
1198
1199 @Deprecated
1200 public byte [] getBuffer() {
1201 return this.bytes;
1202 }
1203
1204
1205
1206
1207 public int getOffset() {
1208 return this.offset;
1209 }
1210
1211
1212
1213
1214 public int getLength() {
1215 return length;
1216 }
1217
1218
1219
1220
1221
1222
1223
1224
1225
1226
1227
1228
1229
1230
1231 private static int getLength(byte [] bytes, int offset) {
1232 int klength = ROW_OFFSET + Bytes.toInt(bytes, offset);
1233 int vlength = Bytes.toInt(bytes, offset + Bytes.SIZEOF_INT);
1234 return klength + vlength;
1235 }
1236
1237
1238
1239
1240 public int getKeyOffset() {
1241 return this.offset + ROW_OFFSET;
1242 }
1243
1244 public String getKeyString() {
1245 return Bytes.toStringBinary(getBuffer(), getKeyOffset(), getKeyLength());
1246 }
1247
1248
1249
1250
1251 public int getKeyLength() {
1252 return Bytes.toInt(this.bytes, this.offset);
1253 }
1254
1255
1256
1257
1258 @Override
1259 public byte[] getValueArray() {
1260 return bytes;
1261 }
1262
1263
1264
1265
1266 @Override
1267 public int getValueOffset() {
1268 int voffset = getKeyOffset() + getKeyLength();
1269 return voffset;
1270 }
1271
1272
1273
1274
1275 @Override
1276 public int getValueLength() {
1277 int vlength = Bytes.toInt(this.bytes, this.offset + Bytes.SIZEOF_INT);
1278 return vlength;
1279 }
1280
1281
1282
1283
1284 @Override
1285 public byte[] getRowArray() {
1286 return bytes;
1287 }
1288
1289
1290
1291
1292 @Override
1293 public int getRowOffset() {
1294 return getKeyOffset() + Bytes.SIZEOF_SHORT;
1295 }
1296
1297
1298
1299
1300 @Override
1301 public short getRowLength() {
1302 return Bytes.toShort(this.bytes, getKeyOffset());
1303 }
1304
1305
1306
1307
1308 @Override
1309 public byte[] getFamilyArray() {
1310 return bytes;
1311 }
1312
1313
1314
1315
1316 @Override
1317 public int getFamilyOffset() {
1318 return getFamilyOffset(getRowLength());
1319 }
1320
1321
1322
1323
1324 private int getFamilyOffset(int rlength) {
1325 return this.offset + ROW_OFFSET + Bytes.SIZEOF_SHORT + rlength + Bytes.SIZEOF_BYTE;
1326 }
1327
1328
1329
1330
1331 @Override
1332 public byte getFamilyLength() {
1333 return getFamilyLength(getFamilyOffset());
1334 }
1335
1336
1337
1338
1339 public byte getFamilyLength(int foffset) {
1340 return this.bytes[foffset-1];
1341 }
1342
1343
1344
1345
1346 @Override
1347 public byte[] getQualifierArray() {
1348 return bytes;
1349 }
1350
1351
1352
1353
1354 @Override
1355 public int getQualifierOffset() {
1356 return getQualifierOffset(getFamilyOffset());
1357 }
1358
1359
1360
1361
1362 private int getQualifierOffset(int foffset) {
1363 return foffset + getFamilyLength(foffset);
1364 }
1365
1366
1367
1368
1369 @Override
1370 public int getQualifierLength() {
1371 return getQualifierLength(getRowLength(),getFamilyLength());
1372 }
1373
1374
1375
1376
1377 private int getQualifierLength(int rlength, int flength) {
1378 return getKeyLength() - (int) getKeyDataStructureSize(rlength, flength, 0);
1379 }
1380
1381
1382
1383
1384 private int getTotalColumnLength(int rlength, int foffset) {
1385 int flength = getFamilyLength(foffset);
1386 int qlength = getQualifierLength(rlength,flength);
1387 return flength + qlength;
1388 }
1389
1390
1391
1392
1393 public int getTimestampOffset() {
1394 return getTimestampOffset(getKeyLength());
1395 }
1396
1397
1398
1399
1400
1401 private int getTimestampOffset(final int keylength) {
1402 return getKeyOffset() + keylength - TIMESTAMP_TYPE_SIZE;
1403 }
1404
1405
1406
1407
1408 public boolean isLatestTimestamp() {
1409 return Bytes.equals(getBuffer(), getTimestampOffset(), Bytes.SIZEOF_LONG,
1410 HConstants.LATEST_TIMESTAMP_BYTES, 0, Bytes.SIZEOF_LONG);
1411 }
1412
1413
1414
1415
1416
1417
1418 public boolean updateLatestStamp(final byte [] now) {
1419 if (this.isLatestTimestamp()) {
1420 int tsOffset = getTimestampOffset();
1421 System.arraycopy(now, 0, this.bytes, tsOffset, Bytes.SIZEOF_LONG);
1422
1423 return true;
1424 }
1425 return false;
1426 }
1427
1428
1429
1430
1431
1432
1433
1434
1435
1436
1437
1438
1439
1440
1441 public byte [] getKey() {
1442 int keylength = getKeyLength();
1443 byte [] key = new byte[keylength];
1444 System.arraycopy(getBuffer(), getKeyOffset(), key, 0, keylength);
1445 return key;
1446 }
1447
1448
1449
1450
1451
1452
1453
1454
1455 @Deprecated
1456 public byte [] getValue() {
1457 return CellUtil.cloneValue(this);
1458 }
1459
1460
1461
1462
1463
1464
1465
1466
1467
1468 @Deprecated
1469 public byte [] getRow() {
1470 return CellUtil.cloneRow(this);
1471 }
1472
1473
1474
1475
1476
1477 @Override
1478 public long getTimestamp() {
1479 return getTimestamp(getKeyLength());
1480 }
1481
1482
1483
1484
1485
1486 long getTimestamp(final int keylength) {
1487 int tsOffset = getTimestampOffset(keylength);
1488 return Bytes.toLong(this.bytes, tsOffset);
1489 }
1490
1491
1492
1493
1494 @Deprecated
1495 public byte getType() {
1496 return getTypeByte();
1497 }
1498
1499
1500
1501
1502 @Override
1503 public byte getTypeByte() {
1504 return this.bytes[this.offset + getKeyLength() - 1 + ROW_OFFSET];
1505 }
1506
1507
1508
1509
1510
1511
1512 @Deprecated
1513 public boolean isDelete() {
1514 return KeyValue.isDelete(getType());
1515 }
1516
1517
1518
1519
1520 public boolean isDeleteType() {
1521
1522 return getTypeByte() == Type.Delete.getCode();
1523 }
1524
1525
1526
1527
1528 public boolean isDeleteFamily() {
1529 return getTypeByte() == Type.DeleteFamily.getCode();
1530 }
1531
1532
1533
1534
1535 public boolean isDeleteFamilyVersion() {
1536 return getTypeByte() == Type.DeleteFamilyVersion.getCode();
1537 }
1538
1539
1540
1541
1542
1543 public boolean isDeleteColumnOrFamily() {
1544 int t = getTypeByte();
1545 return t == Type.DeleteColumn.getCode() || t == Type.DeleteFamily.getCode();
1546 }
1547
1548
1549
1550
1551
1552
1553
1554
1555
1556 @Deprecated
1557 public byte [] getFamily() {
1558 return CellUtil.cloneFamily(this);
1559 }
1560
1561
1562
1563
1564
1565
1566
1567
1568
1569
1570 @Deprecated
1571 public byte [] getQualifier() {
1572 return CellUtil.cloneQualifier(this);
1573 }
1574
1575
1576
1577
1578 @Override
1579 public int getTagsOffset() {
1580 int tagsLen = getTagsLengthUnsigned();
1581 if (tagsLen == 0) {
1582 return this.offset + this.length;
1583 }
1584 return this.offset + this.length - tagsLen;
1585 }
1586
1587
1588
1589
1590 @Override
1591 @Deprecated
1592 public int getTagsLengthUnsigned() {
1593 int tagsLen = this.length - (getKeyLength() + getValueLength() + KEYVALUE_INFRASTRUCTURE_SIZE);
1594 if (tagsLen > 0) {
1595
1596
1597 tagsLen -= TAGS_LENGTH_SIZE;
1598 }
1599 return tagsLen;
1600 }
1601
1602 @Override
1603 @Deprecated
1604 public short getTagsLength() {
1605 return (short) getTagsLengthUnsigned();
1606 }
1607
1608
1609
1610
1611
1612 public List<Tag> getTags() {
1613 int tagsLength = getTagsLengthUnsigned();
1614 if (tagsLength == 0) {
1615 return EMPTY_ARRAY_LIST;
1616 }
1617 return Tag.asList(getBuffer(), getTagsOffset(), tagsLength);
1618 }
1619
1620
1621
1622
1623 @Override
1624 public byte[] getTagsArray() {
1625 return bytes;
1626 }
1627
1628
1629
1630
1631
1632
1633
1634
1635
1636
1637
1638 public boolean matchingFamily(final byte [] family) {
1639 if (this.length == 0 || this.bytes.length == 0) {
1640 return false;
1641 }
1642 return Bytes.equals(family, 0, family.length,
1643 this.bytes, getFamilyOffset(), getFamilyLength());
1644 }
1645
1646
1647
1648
1649
1650 public boolean matchingQualifier(final byte [] qualifier) {
1651 return matchingQualifier(qualifier, 0, qualifier.length);
1652 }
1653
1654 public boolean matchingQualifier(final byte [] qualifier, int offset, int length) {
1655 return Bytes.equals(qualifier, offset, length,
1656 this.bytes, getQualifierOffset(), getQualifierLength());
1657 }
1658
1659 public boolean matchingQualifier(final KeyValue other) {
1660 return matchingQualifier(other.getBuffer(), other.getQualifierOffset(),
1661 other.getQualifierLength());
1662 }
1663
1664 public boolean matchingRow(final byte [] row) {
1665 return matchingRow(row, 0, row.length);
1666 }
1667
1668 public boolean matchingRow(final byte[] row, int offset, int length) {
1669 return Bytes.equals(row, offset, length,
1670 this.bytes, getRowOffset(), getRowLength());
1671 }
1672
1673 public boolean matchingRow(KeyValue other) {
1674 return matchingRow(other.getBuffer(), other.getRowOffset(),
1675 other.getRowLength());
1676 }
1677
1678
1679
1680
1681
1682
1683
1684 public boolean matchingColumn(final byte[] family, final byte[] qualifier) {
1685 return matchingColumn(family, 0, len(family), qualifier, 0, len(qualifier));
1686 }
1687
1688
1689
1690
1691
1692
1693
1694
1695
1696
1697
1698
1699
1700 public boolean matchingColumn(final byte [] family, final int foffset, final int flength,
1701 final byte [] qualifier, final int qoffset, final int qlength) {
1702 int rl = getRowLength();
1703 int o = getFamilyOffset(rl);
1704 int fl = getFamilyLength(o);
1705 if (!Bytes.equals(family, foffset, flength, this.bytes, o, fl)) {
1706 return false;
1707 }
1708
1709 int ql = getQualifierLength(rl, fl);
1710 if (qualifier == null || qlength == 0) {
1711 return (ql == 0);
1712 }
1713 return Bytes.equals(qualifier, qoffset, qlength, this.bytes, o + fl, ql);
1714 }
1715
1716
1717
1718
1719
1720
1721
1722
1723 public KeyValue createKeyOnly(boolean lenAsVal) {
1724
1725
1726 int dataLen = lenAsVal? Bytes.SIZEOF_INT : 0;
1727 byte [] newBuffer = new byte[getKeyLength() + ROW_OFFSET + dataLen];
1728 System.arraycopy(this.bytes, this.offset, newBuffer, 0,
1729 Math.min(newBuffer.length,this.length));
1730 Bytes.putInt(newBuffer, Bytes.SIZEOF_INT, dataLen);
1731 if (lenAsVal) {
1732 Bytes.putInt(newBuffer, newBuffer.length - dataLen, this.getValueLength());
1733 }
1734 return new KeyValue(newBuffer);
1735 }
1736
1737
1738
1739
1740
1741
1742
1743
1744
1745
1746
1747
1748
1749
1750 public static byte [][] parseColumn(byte [] c) {
1751 final int index = getDelimiter(c, 0, c.length, COLUMN_FAMILY_DELIMITER);
1752 if (index == -1) {
1753
1754 return new byte [][] { c };
1755 } else if(index == c.length - 1) {
1756
1757 byte [] family = new byte[c.length-1];
1758 System.arraycopy(c, 0, family, 0, family.length);
1759 return new byte [][] { family, HConstants.EMPTY_BYTE_ARRAY};
1760 }
1761
1762 final byte [][] result = new byte [2][];
1763 result[0] = new byte [index];
1764 System.arraycopy(c, 0, result[0], 0, index);
1765 final int len = c.length - (index + 1);
1766 result[1] = new byte[len];
1767 System.arraycopy(c, index + 1
1768 return result;
1769 }
1770
1771
1772
1773
1774
1775
1776
1777
1778
1779 public static byte [] makeColumn(byte [] family, byte [] qualifier) {
1780 return Bytes.add(family, COLUMN_FAMILY_DELIM_ARRAY, qualifier);
1781 }
1782
1783
1784
1785
1786
1787
1788
1789 public static int getDelimiter(final byte [] b, int offset, final int length,
1790 final int delimiter) {
1791 if (b == null) {
1792 throw new IllegalArgumentException("Passed buffer is null");
1793 }
1794 int result = -1;
1795 for (int i = offset; i < length + offset; i++) {
1796 if (b[i] == delimiter) {
1797 result = i;
1798 break;
1799 }
1800 }
1801 return result;
1802 }
1803
1804
1805
1806
1807
1808
1809
1810 public static int getDelimiterInReverse(final byte [] b, final int offset,
1811 final int length, final int delimiter) {
1812 if (b == null) {
1813 throw new IllegalArgumentException("Passed buffer is null");
1814 }
1815 int result = -1;
1816 for (int i = (offset + length) - 1; i >= offset; i--) {
1817 if (b[i] == delimiter) {
1818 result = i;
1819 break;
1820 }
1821 }
1822 return result;
1823 }
1824
1825
1826
1827
1828
1829 public static class MetaComparator extends KVComparator {
1830
1831
1832
1833
1834 @Override
1835 public int compareRows(byte [] left, int loffset, int llength,
1836 byte [] right, int roffset, int rlength) {
1837 int leftDelimiter = getDelimiter(left, loffset, llength,
1838 HConstants.DELIMITER);
1839 int rightDelimiter = getDelimiter(right, roffset, rlength,
1840 HConstants.DELIMITER);
1841
1842 int lpart = (leftDelimiter < 0 ? llength :leftDelimiter - loffset);
1843 int rpart = (rightDelimiter < 0 ? rlength :rightDelimiter - roffset);
1844 int result = Bytes.compareTo(left, loffset, lpart, right, roffset, rpart);
1845 if (result != 0) {
1846 return result;
1847 } else {
1848 if (leftDelimiter < 0 && rightDelimiter >= 0) {
1849 return -1;
1850 } else if (rightDelimiter < 0 && leftDelimiter >= 0) {
1851 return 1;
1852 } else if (leftDelimiter < 0 && rightDelimiter < 0) {
1853 return 0;
1854 }
1855 }
1856
1857
1858 leftDelimiter++;
1859 rightDelimiter++;
1860 int leftFarDelimiter = getDelimiterInReverse(left, leftDelimiter,
1861 llength - (leftDelimiter - loffset), HConstants.DELIMITER);
1862 int rightFarDelimiter = getDelimiterInReverse(right,
1863 rightDelimiter, rlength - (rightDelimiter - roffset),
1864 HConstants.DELIMITER);
1865
1866 lpart = (leftFarDelimiter < 0 ? llength + loffset: leftFarDelimiter) - leftDelimiter;
1867 rpart = (rightFarDelimiter < 0 ? rlength + roffset: rightFarDelimiter)- rightDelimiter;
1868 result = super.compareRows(left, leftDelimiter, lpart, right, rightDelimiter, rpart);
1869 if (result != 0) {
1870 return result;
1871 } else {
1872 if (leftDelimiter < 0 && rightDelimiter >= 0) {
1873 return -1;
1874 } else if (rightDelimiter < 0 && leftDelimiter >= 0) {
1875 return 1;
1876 } else if (leftDelimiter < 0 && rightDelimiter < 0) {
1877 return 0;
1878 }
1879 }
1880
1881 leftFarDelimiter++;
1882 rightFarDelimiter++;
1883 result = Bytes.compareTo(left, leftFarDelimiter, llength - (leftFarDelimiter - loffset),
1884 right, rightFarDelimiter, rlength - (rightFarDelimiter - roffset));
1885 return result;
1886 }
1887
1888
1889
1890
1891 @Override
1892 public byte[] getShortMidpointKey(final byte[] leftKey, final byte[] rightKey) {
1893 return Arrays.copyOf(rightKey, rightKey.length);
1894 }
1895
1896
1897
1898
1899
1900
1901
1902 @Override
1903 public String getLegacyKeyComparatorName() {
1904 return "org.apache.hadoop.hbase.KeyValue$MetaKeyComparator";
1905 }
1906
1907 @Override
1908 protected Object clone() throws CloneNotSupportedException {
1909 return new MetaComparator();
1910 }
1911
1912
1913
1914
1915 @Override
1916 protected int compareRowKey(final Cell l, final Cell r) {
1917 byte[] left = l.getRowArray();
1918 int loffset = l.getRowOffset();
1919 int llength = l.getRowLength();
1920 byte[] right = r.getRowArray();
1921 int roffset = r.getRowOffset();
1922 int rlength = r.getRowLength();
1923 return compareRows(left, loffset, llength, right, roffset, rlength);
1924 }
1925 }
1926
1927
1928
1929
1930
1931
1932 public static class KVComparator implements RawComparator<Cell>, SamePrefixComparator<byte[]> {
1933
1934
1935
1936
1937
1938
1939
1940 public String getLegacyKeyComparatorName() {
1941 return "org.apache.hadoop.hbase.KeyValue$KeyComparator";
1942 }
1943
1944 @Override
1945 public int compare(byte[] l, int loff, int llen, byte[] r, int roff, int rlen) {
1946 return compareFlatKey(l,loff,llen, r,roff,rlen);
1947 }
1948
1949
1950
1951
1952
1953
1954
1955
1956 protected int compareRowKey(final Cell left, final Cell right) {
1957 return Bytes.compareTo(
1958 left.getRowArray(), left.getRowOffset(), left.getRowLength(),
1959 right.getRowArray(), right.getRowOffset(), right.getRowLength());
1960 }
1961
1962
1963
1964
1965
1966
1967
1968
1969
1970
1971
1972
1973 public int compareFlatKey(byte[] left, int loffset, int llength,
1974 byte[] right, int roffset, int rlength) {
1975
1976 short lrowlength = Bytes.toShort(left, loffset);
1977 short rrowlength = Bytes.toShort(right, roffset);
1978 int compare = compareRows(left, loffset + Bytes.SIZEOF_SHORT,
1979 lrowlength, right, roffset + Bytes.SIZEOF_SHORT, rrowlength);
1980 if (compare != 0) {
1981 return compare;
1982 }
1983
1984
1985
1986
1987 return compareWithoutRow(0, left, loffset, llength, right, roffset,
1988 rlength, rrowlength);
1989 }
1990
1991 public int compareFlatKey(byte[] left, byte[] right) {
1992 return compareFlatKey(left, 0, left.length, right, 0, right.length);
1993 }
1994
1995
1996 public int compareKey(byte[] key, int koff, int klen, byte[] row, int roff, int rlen,
1997 byte[] fam, int foff, int flen, byte[] col, int coff, int clen, long ts, byte type) {
1998
1999 short lrowlength = Bytes.toShort(key, koff);
2000 int compare = compareRows(key, koff + Bytes.SIZEOF_SHORT, lrowlength, row, roff, rlen);
2001 if (compare != 0) {
2002 return compare;
2003 }
2004
2005
2006 int commonLength = ROW_LENGTH_SIZE + FAMILY_LENGTH_SIZE + rlen;
2007
2008
2009 int lcolumnlength = klen - TIMESTAMP_TYPE_SIZE + commonLength;
2010
2011 byte ltype = key[koff + (klen - 1)];
2012
2013
2014
2015
2016
2017
2018 if (lcolumnlength == 0 && ltype == Type.Minimum.getCode()) {
2019
2020 return 1;
2021 }
2022 if (flen + clen == 0 && type == Type.Minimum.getCode()) {
2023 return -1;
2024 }
2025
2026 int lfamilyoffset = commonLength + koff;
2027 int lfamilylength = key[lfamilyoffset - 1];
2028 compare = Bytes.compareTo(key, lfamilyoffset, lfamilylength, fam, foff, flen);
2029 if (compare != 0) {
2030 return compare;
2031 }
2032 int lColOffset = lfamilyoffset + lfamilylength;
2033 int lColLength = lcolumnlength - lfamilylength;
2034 compare = Bytes.compareTo(key, lColOffset, lColLength, col, coff, clen);
2035 if (compare != 0) {
2036 return compare;
2037 }
2038
2039 long ltimestamp = Bytes.toLong(key, koff + (klen - TIMESTAMP_TYPE_SIZE));
2040 compare = compareTimestamps(ltimestamp, ts);
2041 if (compare != 0) {
2042 return compare;
2043 }
2044
2045
2046
2047
2048
2049 return (0xff & type) - (0xff & ltype);
2050 }
2051
2052
2053
2054
2055
2056 public int compare(final Cell left, final Cell right) {
2057
2058 int compare = compareRowKey(left, right);
2059 if (compare != 0) {
2060 return compare;
2061 }
2062
2063
2064 byte ltype = left.getTypeByte();
2065 byte rtype = right.getTypeByte();
2066
2067
2068
2069
2070
2071 int lcfqLen = left.getFamilyLength() + left.getQualifierLength() ;
2072 int rcfqLen = right.getFamilyLength() + right.getQualifierLength() ;
2073 if (lcfqLen == 0 && ltype == Type.Minimum.getCode()) {
2074
2075 return 1;
2076 }
2077 if (rcfqLen == 0 && rtype == Type.Minimum.getCode()) {
2078 return -1;
2079 }
2080
2081
2082
2083
2084
2085 compare = Bytes.compareTo(
2086 left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
2087 right.getFamilyArray(), right.getFamilyOffset(), right.getFamilyLength());
2088 if (compare != 0) {
2089 return compare;
2090 }
2091
2092
2093 compare = Bytes.compareTo(
2094 left.getQualifierArray(), left.getQualifierOffset(), left.getQualifierLength(),
2095 right.getQualifierArray(), right.getQualifierOffset(), right.getQualifierLength());
2096 if (compare!= 0) {
2097 return compare;
2098 }
2099
2100
2101 long ltimestamp = left.getTimestamp();
2102 long rtimestamp = right.getTimestamp();
2103 compare = compareTimestamps(ltimestamp, rtimestamp);
2104 if (compare != 0) {
2105 return compare;
2106 }
2107
2108
2109
2110
2111
2112 compare = (0xff & rtype) - (0xff & ltype);
2113 if (compare != 0) {
2114 return compare;
2115 }
2116
2117
2118
2119
2120
2121
2122
2123
2124
2125 long leftChangeSeqNum = getReplaySeqNum(left);
2126 long RightChangeSeqNum = getReplaySeqNum(right);
2127 if (leftChangeSeqNum != Long.MAX_VALUE || RightChangeSeqNum != Long.MAX_VALUE) {
2128 return Longs.compare(RightChangeSeqNum, leftChangeSeqNum);
2129 }
2130
2131
2132 return Longs.compare(right.getMvccVersion(), left.getMvccVersion());
2133 }
2134
2135
2136
2137
2138
2139
2140 private long getReplaySeqNum(final Cell c) {
2141 Tag tag = Tag.getTag(c.getTagsArray(), c.getTagsOffset(), c.getTagsLengthUnsigned(),
2142 TagType.LOG_REPLAY_TAG_TYPE);
2143
2144 if(tag != null) {
2145 return Bytes.toLong(tag.getBuffer(), tag.getTagOffset(), tag.getTagLength());
2146 }
2147 return Long.MAX_VALUE;
2148 }
2149
2150 public int compareTimestamps(final KeyValue left, final KeyValue right) {
2151
2152 long ltimestamp = left.getTimestamp(left.getKeyLength());
2153 long rtimestamp = right.getTimestamp(right.getKeyLength());
2154 return compareTimestamps(ltimestamp, rtimestamp);
2155 }
2156
2157
2158
2159
2160
2161
2162 public int compareRows(final KeyValue left, final KeyValue right) {
2163 return compareRows(left.getBuffer(),left.getRowOffset(), left.getRowLength(),
2164 right.getBuffer(), right.getRowOffset(), right.getRowLength());
2165 }
2166
2167
2168
2169
2170
2171
2172
2173
2174
2175
2176
2177 public int compareRows(byte [] left, int loffset, int llength,
2178 byte [] right, int roffset, int rlength) {
2179 return Bytes.compareTo(left, loffset, llength, right, roffset, rlength);
2180 }
2181
2182 int compareColumns(final KeyValue left, final short lrowlength,
2183 final KeyValue right, final short rrowlength) {
2184 int lfoffset = left.getFamilyOffset(lrowlength);
2185 int rfoffset = right.getFamilyOffset(rrowlength);
2186 int lclength = left.getTotalColumnLength(lrowlength,lfoffset);
2187 int rclength = right.getTotalColumnLength(rrowlength, rfoffset);
2188 int lfamilylength = left.getFamilyLength(lfoffset);
2189 int rfamilylength = right.getFamilyLength(rfoffset);
2190 return compareColumns(left.getBuffer(), lfoffset,
2191 lclength, lfamilylength,
2192 right.getBuffer(), rfoffset, rclength, rfamilylength);
2193 }
2194
2195 protected int compareColumns(
2196 byte [] left, int loffset, int llength, final int lfamilylength,
2197 byte [] right, int roffset, int rlength, final int rfamilylength) {
2198
2199 int diff = Bytes.compareTo(left, loffset, lfamilylength,
2200 right, roffset, rfamilylength);
2201 if (diff != 0) {
2202 return diff;
2203 }
2204
2205 return Bytes.compareTo(left, loffset + lfamilylength,
2206 llength - lfamilylength,
2207 right, roffset + rfamilylength, rlength - rfamilylength);
2208 }
2209
2210 static int compareTimestamps(final long ltimestamp, final long rtimestamp) {
2211
2212
2213
2214
2215 if (ltimestamp < rtimestamp) {
2216 return 1;
2217 } else if (ltimestamp > rtimestamp) {
2218 return -1;
2219 }
2220 return 0;
2221 }
2222
2223
2224
2225
2226
2227
2228
2229
2230
2231
2232
2233
2234 @Override
2235 public int compareIgnoringPrefix(int commonPrefix, byte[] left,
2236 int loffset, int llength, byte[] right, int roffset, int rlength) {
2237
2238 short lrowlength = Bytes.toShort(left, loffset);
2239 short rrowlength;
2240
2241 int comparisonResult = 0;
2242 if (commonPrefix < ROW_LENGTH_SIZE) {
2243
2244 rrowlength = Bytes.toShort(right, roffset);
2245 comparisonResult = compareRows(left, loffset + ROW_LENGTH_SIZE,
2246 lrowlength, right, roffset + ROW_LENGTH_SIZE, rrowlength);
2247 } else {
2248 rrowlength = lrowlength;
2249 if (commonPrefix < ROW_LENGTH_SIZE + rrowlength) {
2250
2251
2252 int common = commonPrefix - ROW_LENGTH_SIZE;
2253 comparisonResult = compareRows(
2254 left, loffset + common + ROW_LENGTH_SIZE, lrowlength - common,
2255 right, roffset + common + ROW_LENGTH_SIZE, rrowlength - common);
2256 }
2257 }
2258 if (comparisonResult != 0) {
2259 return comparisonResult;
2260 }
2261
2262 assert lrowlength == rrowlength;
2263 return compareWithoutRow(commonPrefix, left, loffset, llength, right,
2264 roffset, rlength, lrowlength);
2265 }
2266
2267
2268
2269
2270
2271
2272
2273
2274
2275
2276
2277 private int compareWithoutRow(int commonPrefix, byte[] left, int loffset,
2278 int llength, byte[] right, int roffset, int rlength, short rowlength) {
2279
2280
2281
2282
2283
2284 int commonLength = ROW_LENGTH_SIZE + FAMILY_LENGTH_SIZE + rowlength;
2285
2286
2287 int commonLengthWithTSAndType = TIMESTAMP_TYPE_SIZE + commonLength;
2288
2289 int lcolumnlength = llength - commonLengthWithTSAndType;
2290 int rcolumnlength = rlength - commonLengthWithTSAndType;
2291
2292 byte ltype = left[loffset + (llength - 1)];
2293 byte rtype = right[roffset + (rlength - 1)];
2294
2295
2296
2297
2298
2299
2300 if (lcolumnlength == 0 && ltype == Type.Minimum.getCode()) {
2301
2302 return 1;
2303 }
2304 if (rcolumnlength == 0 && rtype == Type.Minimum.getCode()) {
2305 return -1;
2306 }
2307
2308 int lfamilyoffset = commonLength + loffset;
2309 int rfamilyoffset = commonLength + roffset;
2310
2311
2312 int lfamilylength = left[lfamilyoffset - 1];
2313 int rfamilylength = right[rfamilyoffset - 1];
2314
2315
2316 boolean sameFamilySize = (lfamilylength == rfamilylength);
2317 int common = 0;
2318 if (commonPrefix > 0) {
2319 common = Math.max(0, commonPrefix - commonLength);
2320 if (!sameFamilySize) {
2321
2322
2323 common = Math.min(common, Math.min(lfamilylength, rfamilylength));
2324 } else {
2325 common = Math.min(common, Math.min(lcolumnlength, rcolumnlength));
2326 }
2327 }
2328 if (!sameFamilySize) {
2329
2330 return Bytes.compareTo(left, lfamilyoffset + common, lfamilylength
2331 - common, right, rfamilyoffset + common, rfamilylength - common);
2332 }
2333
2334 final int comparison = Bytes.compareTo(left, lfamilyoffset + common,
2335 lcolumnlength - common, right, rfamilyoffset + common,
2336 rcolumnlength - common);
2337 if (comparison != 0) {
2338 return comparison;
2339 }
2340
2341
2342
2343 long ltimestamp = Bytes.toLong(left,
2344 loffset + (llength - TIMESTAMP_TYPE_SIZE));
2345 long rtimestamp = Bytes.toLong(right,
2346 roffset + (rlength - TIMESTAMP_TYPE_SIZE));
2347 int compare = compareTimestamps(ltimestamp, rtimestamp);
2348 if (compare != 0) {
2349 return compare;
2350 }
2351
2352
2353
2354
2355
2356 return (0xff & rtype) - (0xff & ltype);
2357 }
2358
2359
2360
2361
2362
2363
2364
2365 public boolean matchingRowColumn(final KeyValue left,
2366 final KeyValue right) {
2367 short lrowlength = left.getRowLength();
2368 short rrowlength = right.getRowLength();
2369
2370
2371 if ((left.getTimestampOffset() - left.getOffset()) !=
2372 (right.getTimestampOffset() - right.getOffset())) {
2373 return false;
2374 }
2375
2376 if (!matchingRows(left, lrowlength, right, rrowlength)) {
2377 return false;
2378 }
2379
2380 int lfoffset = left.getFamilyOffset(lrowlength);
2381 int rfoffset = right.getFamilyOffset(rrowlength);
2382 int lclength = left.getTotalColumnLength(lrowlength,lfoffset);
2383 int rclength = right.getTotalColumnLength(rrowlength, rfoffset);
2384 int lfamilylength = left.getFamilyLength(lfoffset);
2385 int rfamilylength = right.getFamilyLength(rfoffset);
2386 int ccRes = compareColumns(left.getBuffer(), lfoffset, lclength, lfamilylength,
2387 right.getBuffer(), rfoffset, rclength, rfamilylength);
2388 return ccRes == 0;
2389 }
2390
2391
2392
2393
2394
2395
2396
2397 public boolean matchingRows(final KeyValue left, final KeyValue right) {
2398 short lrowlength = left.getRowLength();
2399 short rrowlength = right.getRowLength();
2400 return matchingRows(left, lrowlength, right, rrowlength);
2401 }
2402
2403
2404
2405
2406
2407
2408
2409
2410 private boolean matchingRows(final KeyValue left, final short lrowlength,
2411 final KeyValue right, final short rrowlength) {
2412 return lrowlength == rrowlength &&
2413 matchingRows(left.getBuffer(), left.getRowOffset(), lrowlength,
2414 right.getBuffer(), right.getRowOffset(), rrowlength);
2415 }
2416
2417
2418
2419
2420
2421
2422
2423
2424
2425
2426
2427 public boolean matchingRows(final byte [] left, final int loffset, final int llength,
2428 final byte [] right, final int roffset, final int rlength) {
2429 return Bytes.equals(left, loffset, llength, right, roffset, rlength);
2430 }
2431
2432 public byte[] calcIndexKey(byte[] lastKeyOfPreviousBlock, byte[] firstKeyInBlock) {
2433 byte[] fakeKey = getShortMidpointKey(lastKeyOfPreviousBlock, firstKeyInBlock);
2434 if (compareFlatKey(fakeKey, firstKeyInBlock) > 0) {
2435 LOG.error("Unexpected getShortMidpointKey result, fakeKey:"
2436 + Bytes.toStringBinary(fakeKey) + ", firstKeyInBlock:"
2437 + Bytes.toStringBinary(firstKeyInBlock));
2438 return firstKeyInBlock;
2439 }
2440 if (lastKeyOfPreviousBlock != null && compareFlatKey(lastKeyOfPreviousBlock, fakeKey) >= 0) {
2441 LOG.error("Unexpected getShortMidpointKey result, lastKeyOfPreviousBlock:" +
2442 Bytes.toStringBinary(lastKeyOfPreviousBlock) + ", fakeKey:" +
2443 Bytes.toStringBinary(fakeKey));
2444 return firstKeyInBlock;
2445 }
2446 return fakeKey;
2447 }
2448
2449
2450
2451
2452
2453
2454
2455 public byte[] getShortMidpointKey(final byte[] leftKey, final byte[] rightKey) {
2456 if (rightKey == null) {
2457 throw new IllegalArgumentException("rightKey can not be null");
2458 }
2459 if (leftKey == null) {
2460 return Arrays.copyOf(rightKey, rightKey.length);
2461 }
2462 if (compareFlatKey(leftKey, rightKey) >= 0) {
2463 throw new IllegalArgumentException("Unexpected input, leftKey:" + Bytes.toString(leftKey)
2464 + ", rightKey:" + Bytes.toString(rightKey));
2465 }
2466
2467 short leftRowLength = Bytes.toShort(leftKey, 0);
2468 short rightRowLength = Bytes.toShort(rightKey, 0);
2469 int leftCommonLength = ROW_LENGTH_SIZE + FAMILY_LENGTH_SIZE + leftRowLength;
2470 int rightCommonLength = ROW_LENGTH_SIZE + FAMILY_LENGTH_SIZE + rightRowLength;
2471 int leftCommonLengthWithTSAndType = TIMESTAMP_TYPE_SIZE + leftCommonLength;
2472 int rightCommonLengthWithTSAndType = TIMESTAMP_TYPE_SIZE + rightCommonLength;
2473 int leftColumnLength = leftKey.length - leftCommonLengthWithTSAndType;
2474 int rightColumnLength = rightKey.length - rightCommonLengthWithTSAndType;
2475
2476 if (leftRowLength == rightRowLength && compareRows(leftKey, ROW_LENGTH_SIZE, leftRowLength,
2477 rightKey, ROW_LENGTH_SIZE, rightRowLength) == 0) {
2478
2479 int comparison = Bytes.compareTo(leftKey, leftCommonLength, leftColumnLength, rightKey,
2480 rightCommonLength, rightColumnLength);
2481
2482 if (comparison == 0) {
2483 return Arrays.copyOf(rightKey, rightKey.length);
2484 }
2485
2486 byte[] newKey = Arrays.copyOf(rightKey, rightKey.length);
2487 Bytes.putLong(newKey, rightKey.length - TIMESTAMP_TYPE_SIZE, HConstants.LATEST_TIMESTAMP);
2488 Bytes.putByte(newKey, rightKey.length - TYPE_SIZE, Type.Maximum.getCode());
2489 return newKey;
2490 }
2491
2492 short minLength = leftRowLength < rightRowLength ? leftRowLength : rightRowLength;
2493 short diffIdx = 0;
2494 while (diffIdx < minLength
2495 && leftKey[ROW_LENGTH_SIZE + diffIdx] == rightKey[ROW_LENGTH_SIZE + diffIdx]) {
2496 diffIdx++;
2497 }
2498 byte[] newRowKey = null;
2499 if (diffIdx >= minLength) {
2500
2501 newRowKey = new byte[diffIdx + 1];
2502 System.arraycopy(rightKey, ROW_LENGTH_SIZE, newRowKey, 0, diffIdx + 1);
2503 } else {
2504 int diffByte = leftKey[ROW_LENGTH_SIZE + diffIdx];
2505 if ((0xff & diffByte) < 0xff && (diffByte + 1) <
2506 (rightKey[ROW_LENGTH_SIZE + diffIdx] & 0xff)) {
2507 newRowKey = new byte[diffIdx + 1];
2508 System.arraycopy(leftKey, ROW_LENGTH_SIZE, newRowKey, 0, diffIdx);
2509 newRowKey[diffIdx] = (byte) (diffByte + 1);
2510 } else {
2511 newRowKey = new byte[diffIdx + 1];
2512 System.arraycopy(rightKey, ROW_LENGTH_SIZE, newRowKey, 0, diffIdx + 1);
2513 }
2514 }
2515 return new KeyValue(newRowKey, null, null, HConstants.LATEST_TIMESTAMP,
2516 Type.Maximum).getKey();
2517 }
2518
2519 @Override
2520 protected Object clone() throws CloneNotSupportedException {
2521 return new KVComparator();
2522 }
2523
2524 }
2525
2526
2527
2528
2529
2530
2531
2532
2533 public static KeyValue createLastOnRow(final byte[] row) {
2534 return new KeyValue(row, null, null, HConstants.LATEST_TIMESTAMP, Type.Minimum);
2535 }
2536
2537
2538
2539
2540
2541
2542
2543
2544
2545 public static KeyValue createFirstOnRow(final byte [] row) {
2546 return createFirstOnRow(row, HConstants.LATEST_TIMESTAMP);
2547 }
2548
2549
2550
2551
2552
2553
2554
2555
2556
2557 public static KeyValue createFirstOnRow(final byte [] row, int roffset, short rlength) {
2558 return new KeyValue(row, roffset, rlength,
2559 null, 0, 0, null, 0, 0, HConstants.LATEST_TIMESTAMP, Type.Maximum, null, 0, 0);
2560 }
2561
2562
2563
2564
2565
2566
2567
2568
2569 public static KeyValue createFirstOnRow(final byte [] row,
2570 final long ts) {
2571 return new KeyValue(row, null, null, ts, Type.Maximum);
2572 }
2573
2574
2575
2576
2577
2578
2579
2580
2581
2582
2583 public static KeyValue createFirstOnRow(final byte [] row, final byte [] family,
2584 final byte [] qualifier) {
2585 return new KeyValue(row, family, qualifier, HConstants.LATEST_TIMESTAMP, Type.Maximum);
2586 }
2587
2588
2589
2590
2591
2592
2593
2594
2595
2596
2597 public static KeyValue createFirstDeleteFamilyOnRow(final byte [] row,
2598 final byte [] family) {
2599 return new KeyValue(row, family, null, HConstants.LATEST_TIMESTAMP,
2600 Type.DeleteFamily);
2601 }
2602
2603
2604
2605
2606
2607
2608
2609
2610 public static KeyValue createFirstOnRow(final byte [] row, final byte [] f,
2611 final byte [] q, final long ts) {
2612 return new KeyValue(row, f, q, ts, Type.Maximum);
2613 }
2614
2615
2616
2617
2618
2619
2620
2621
2622
2623
2624
2625
2626
2627
2628
2629
2630
2631 public static KeyValue createFirstOnRow(final byte [] row,
2632 final int roffset, final int rlength, final byte [] family,
2633 final int foffset, final int flength, final byte [] qualifier,
2634 final int qoffset, final int qlength) {
2635 return new KeyValue(row, roffset, rlength, family,
2636 foffset, flength, qualifier, qoffset, qlength,
2637 HConstants.LATEST_TIMESTAMP, Type.Maximum, null, 0, 0);
2638 }
2639
2640
2641
2642
2643
2644
2645
2646
2647
2648
2649
2650
2651
2652
2653
2654
2655
2656 public static KeyValue createFirstOnRow(byte [] buffer, final byte [] row,
2657 final byte [] family, final byte [] qualifier)
2658 throws IllegalArgumentException {
2659
2660 return createFirstOnRow(buffer, 0, row, 0, row.length,
2661 family, 0, family.length,
2662 qualifier, 0, qualifier.length);
2663 }
2664
2665
2666
2667
2668
2669
2670
2671
2672
2673
2674
2675
2676
2677
2678
2679
2680
2681
2682
2683
2684
2685
2686
2687
2688 public static KeyValue createFirstOnRow(byte [] buffer, final int boffset,
2689 final byte [] row, final int roffset, final int rlength,
2690 final byte [] family, final int foffset, final int flength,
2691 final byte [] qualifier, final int qoffset, final int qlength)
2692 throws IllegalArgumentException {
2693
2694 long lLength = getKeyValueDataStructureSize(rlength, flength, qlength, 0);
2695
2696 if (lLength > Integer.MAX_VALUE) {
2697 throw new IllegalArgumentException("KeyValue length " + lLength + " > " + Integer.MAX_VALUE);
2698 }
2699 int iLength = (int) lLength;
2700 if (buffer.length - boffset < iLength) {
2701 throw new IllegalArgumentException("Buffer size " + (buffer.length - boffset) + " < " +
2702 iLength);
2703 }
2704
2705 int len = writeByteArray(buffer, boffset, row, roffset, rlength, family, foffset, flength,
2706 qualifier, qoffset, qlength, HConstants.LATEST_TIMESTAMP, KeyValue.Type.Maximum,
2707 null, 0, 0, null);
2708 return new KeyValue(buffer, boffset, len);
2709 }
2710
2711
2712
2713
2714
2715
2716
2717
2718
2719
2720
2721
2722
2723
2724
2725
2726
2727 public static KeyValue createLastOnRow(final byte [] row,
2728 final int roffset, final int rlength, final byte [] family,
2729 final int foffset, final int flength, final byte [] qualifier,
2730 final int qoffset, final int qlength) {
2731 return new KeyValue(row, roffset, rlength, family,
2732 foffset, flength, qualifier, qoffset, qlength,
2733 HConstants.OLDEST_TIMESTAMP, Type.Minimum, null, 0, 0);
2734 }
2735
2736
2737
2738
2739
2740
2741
2742
2743
2744 public KeyValue createLastOnRowCol() {
2745 return new KeyValue(
2746 bytes, getRowOffset(), getRowLength(),
2747 bytes, getFamilyOffset(), getFamilyLength(),
2748 bytes, getQualifierOffset(), getQualifierLength(),
2749 HConstants.OLDEST_TIMESTAMP, Type.Minimum, null, 0, 0);
2750 }
2751
2752
2753
2754
2755
2756
2757
2758
2759 public KeyValue createFirstOnRowColTS(long ts) {
2760 return new KeyValue(
2761 bytes, getRowOffset(), getRowLength(),
2762 bytes, getFamilyOffset(), getFamilyLength(),
2763 bytes, getQualifierOffset(), getQualifierLength(),
2764 ts, Type.Maximum, bytes, getValueOffset(), getValueLength());
2765 }
2766
2767
2768
2769
2770
2771
2772 public static KeyValue createKeyValueFromKey(final byte [] b) {
2773 return createKeyValueFromKey(b, 0, b.length);
2774 }
2775
2776
2777
2778
2779
2780
2781 public static KeyValue createKeyValueFromKey(final ByteBuffer bb) {
2782 return createKeyValueFromKey(bb.array(), bb.arrayOffset(), bb.limit());
2783 }
2784
2785
2786
2787
2788
2789
2790
2791
2792 public static KeyValue createKeyValueFromKey(final byte [] b, final int o,
2793 final int l) {
2794 byte [] newb = new byte[l + ROW_OFFSET];
2795 System.arraycopy(b, o, newb, ROW_OFFSET, l);
2796 Bytes.putInt(newb, 0, l);
2797 Bytes.putInt(newb, Bytes.SIZEOF_INT, 0);
2798 return new KeyValue(newb);
2799 }
2800
2801
2802
2803
2804
2805
2806
2807
2808 public static KeyValue create(final DataInput in) throws IOException {
2809 return create(in.readInt(), in);
2810 }
2811
2812
2813
2814
2815
2816
2817
2818
2819
2820 public static KeyValue create(int length, final DataInput in) throws IOException {
2821
2822 if (length <= 0) {
2823 if (length == 0) return null;
2824 throw new IOException("Failed read " + length + " bytes, stream corrupt?");
2825 }
2826
2827
2828 byte [] bytes = new byte[length];
2829 in.readFully(bytes);
2830 return new KeyValue(bytes, 0, length);
2831 }
2832
2833
2834
2835
2836
2837
2838
2839 public static KeyValue cloneAndAddTags(Cell c, List<Tag> newTags) {
2840 List<Tag> existingTags = null;
2841 if(c.getTagsLengthUnsigned() > 0) {
2842 existingTags = Tag.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLengthUnsigned());
2843 existingTags.addAll(newTags);
2844 } else {
2845 existingTags = newTags;
2846 }
2847 return new KeyValue(c.getRowArray(), c.getRowOffset(), (int)c.getRowLength(),
2848 c.getFamilyArray(), c.getFamilyOffset(), (int)c.getFamilyLength(),
2849 c.getQualifierArray(), c.getQualifierOffset(), (int) c.getQualifierLength(),
2850 c.getTimestamp(), Type.codeToType(c.getTypeByte()), c.getValueArray(), c.getValueOffset(),
2851 c.getValueLength(), existingTags);
2852 }
2853
2854
2855
2856
2857
2858
2859
2860
2861 public static KeyValue iscreate(final InputStream in) throws IOException {
2862 byte [] intBytes = new byte[Bytes.SIZEOF_INT];
2863 int bytesRead = 0;
2864 while (bytesRead < intBytes.length) {
2865 int n = in.read(intBytes, bytesRead, intBytes.length - bytesRead);
2866 if (n < 0) {
2867 if (bytesRead == 0) {
2868 throw new EOFException();
2869 }
2870 throw new IOException("Failed read of int, read " + bytesRead + " bytes");
2871 }
2872 bytesRead += n;
2873 }
2874
2875 byte [] bytes = new byte[Bytes.toInt(intBytes)];
2876 IOUtils.readFully(in, bytes, 0, bytes.length);
2877 return new KeyValue(bytes, 0, bytes.length);
2878 }
2879
2880
2881
2882
2883
2884
2885
2886
2887
2888 public static long write(final KeyValue kv, final DataOutput out) throws IOException {
2889
2890
2891 int length = kv.getLength();
2892 out.writeInt(length);
2893 out.write(kv.getBuffer(), kv.getOffset(), length);
2894 return length + Bytes.SIZEOF_INT;
2895 }
2896
2897
2898
2899
2900
2901
2902
2903
2904
2905
2906
2907
2908 @Deprecated
2909 public static long oswrite(final KeyValue kv, final OutputStream out)
2910 throws IOException {
2911 int length = kv.getLength();
2912
2913 out.write(Bytes.toBytes(length));
2914 out.write(kv.getBuffer(), kv.getOffset(), length);
2915 return length + Bytes.SIZEOF_INT;
2916 }
2917
2918
2919
2920
2921
2922
2923
2924
2925
2926
2927
2928
2929
2930 public static long oswrite(final KeyValue kv, final OutputStream out, final boolean withTags)
2931 throws IOException {
2932 int length = kv.getLength();
2933 if (!withTags) {
2934 length = kv.getKeyLength() + kv.getValueLength() + KEYVALUE_INFRASTRUCTURE_SIZE;
2935 }
2936
2937 out.write(Bytes.toBytes(length));
2938 out.write(kv.getBuffer(), kv.getOffset(), length);
2939 return length + Bytes.SIZEOF_INT;
2940 }
2941
2942
2943
2944
2945 public static class RowOnlyComparator implements Comparator<KeyValue> {
2946 final KVComparator comparator;
2947
2948 public RowOnlyComparator(final KVComparator c) {
2949 this.comparator = c;
2950 }
2951
2952 public int compare(KeyValue left, KeyValue right) {
2953 return comparator.compareRows(left, right);
2954 }
2955 }
2956
2957
2958
2959
2960
2961
2962
2963 public interface SamePrefixComparator<T> {
2964
2965
2966
2967
2968 int compareIgnoringPrefix(
2969 int commonPrefix, byte[] left, int loffset, int llength, byte[] right, int roffset, int rlength
2970 );
2971 }
2972
2973
2974
2975
2976 public static class RawBytesComparator extends KVComparator {
2977
2978
2979
2980
2981
2982
2983 public String getLegacyKeyComparatorName() {
2984 return "org.apache.hadoop.hbase.util.Bytes$ByteArrayComparator";
2985 }
2986
2987 public int compareFlatKey(byte[] left, int loffset, int llength, byte[] right,
2988 int roffset, int rlength) {
2989 return Bytes.BYTES_RAWCOMPARATOR.compare(left, loffset, llength, right, roffset, rlength);
2990 }
2991
2992 public byte[] calcIndexKey(byte[] lastKeyOfPreviousBlock, byte[] firstKeyInBlock) {
2993 return firstKeyInBlock;
2994 }
2995
2996 }
2997
2998
2999
3000
3001
3002
3003
3004 @Override
3005 public long heapSize() {
3006 int sum = 0;
3007 sum += ClassSize.OBJECT;
3008 sum += ClassSize.REFERENCE;
3009 sum += ClassSize.align(ClassSize.ARRAY);
3010 sum += ClassSize.align(length);
3011 sum += 2 * Bytes.SIZEOF_INT;
3012 sum += Bytes.SIZEOF_LONG;
3013 return ClassSize.align(sum);
3014 }
3015
3016
3017
3018
3019
3020
3021
3022
3023 @Deprecated
3024 public long heapSizeWithoutTags() {
3025 int sum = 0;
3026 sum += ClassSize.OBJECT;
3027 sum += ClassSize.REFERENCE;
3028 sum += ClassSize.align(ClassSize.ARRAY);
3029 sum += KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE;
3030 sum += getKeyLength();
3031 sum += getValueLength();
3032 sum += 2 * Bytes.SIZEOF_INT;
3033 sum += Bytes.SIZEOF_LONG;
3034 return ClassSize.align(sum);
3035 }
3036 }