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