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 @Override
1057 public int hashCode() {
1058 byte[] b = getBuffer();
1059 int start = getOffset(), end = getOffset() + getLength();
1060 int h = b[start++];
1061 for (int i = start; i < end; i++) {
1062 h = (h * 13) ^ b[i];
1063 }
1064 return h;
1065 }
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077
1078 @Override
1079 public KeyValue clone() throws CloneNotSupportedException {
1080 super.clone();
1081 byte [] b = new byte[this.length];
1082 System.arraycopy(this.bytes, this.offset, b, 0, this.length);
1083 KeyValue ret = new KeyValue(b, 0, b.length);
1084
1085
1086
1087 ret.setMvccVersion(mvcc);
1088 return ret;
1089 }
1090
1091
1092
1093
1094
1095
1096 public KeyValue shallowCopy() {
1097 KeyValue shallowCopy = new KeyValue(this.bytes, this.offset, this.length);
1098 shallowCopy.setMvccVersion(this.mvcc);
1099 return shallowCopy;
1100 }
1101
1102
1103
1104
1105
1106
1107
1108 public String toString() {
1109 if (this.bytes == null || this.bytes.length == 0) {
1110 return "empty";
1111 }
1112 return keyToString(this.bytes, this.offset + ROW_OFFSET, getKeyLength()) +
1113 "/vlen=" + getValueLength() + "/mvcc=" + mvcc;
1114 }
1115
1116
1117
1118
1119
1120 public static String keyToString(final byte [] k) {
1121 if (k == null) {
1122 return "";
1123 }
1124 return keyToString(k, 0, k.length);
1125 }
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135 public Map<String, Object> toStringMap() {
1136 Map<String, Object> stringMap = new HashMap<String, Object>();
1137 stringMap.put("row", Bytes.toStringBinary(getRow()));
1138 stringMap.put("family", Bytes.toStringBinary(getFamily()));
1139 stringMap.put("qualifier", Bytes.toStringBinary(getQualifier()));
1140 stringMap.put("timestamp", getTimestamp());
1141 stringMap.put("vlen", getValueLength());
1142 List<Tag> tags = getTags();
1143 if (tags != null) {
1144 List<String> tagsString = new ArrayList<String>();
1145 for (Tag t : tags) {
1146 tagsString.add((t.getType()) + ":" +Bytes.toStringBinary(t.getValue()));
1147 }
1148 stringMap.put("tag", tagsString);
1149 }
1150 return stringMap;
1151 }
1152
1153
1154
1155
1156
1157
1158
1159
1160 public static String keyToString(final byte [] b, final int o, final int l) {
1161 if (b == null) return "";
1162 int rowlength = Bytes.toShort(b, o);
1163 String row = Bytes.toStringBinary(b, o + Bytes.SIZEOF_SHORT, rowlength);
1164 int columnoffset = o + Bytes.SIZEOF_SHORT + 1 + rowlength;
1165 int familylength = b[columnoffset - 1];
1166 int columnlength = l - ((columnoffset - o) + TIMESTAMP_TYPE_SIZE);
1167 String family = familylength == 0? "":
1168 Bytes.toStringBinary(b, columnoffset, familylength);
1169 String qualifier = columnlength == 0? "":
1170 Bytes.toStringBinary(b, columnoffset + familylength,
1171 columnlength - familylength);
1172 long timestamp = Bytes.toLong(b, o + (l - TIMESTAMP_TYPE_SIZE));
1173 String timestampStr = humanReadableTimestamp(timestamp);
1174 byte type = b[o + l - 1];
1175 return row + "/" + family +
1176 (family != null && family.length() > 0? ":" :"") +
1177 qualifier + "/" + timestampStr + "/" + Type.codeToType(type);
1178 }
1179
1180 public static String humanReadableTimestamp(final long timestamp) {
1181 if (timestamp == HConstants.LATEST_TIMESTAMP) {
1182 return "LATEST_TIMESTAMP";
1183 }
1184 if (timestamp == HConstants.OLDEST_TIMESTAMP) {
1185 return "OLDEST_TIMESTAMP";
1186 }
1187 return String.valueOf(timestamp);
1188 }
1189
1190
1191
1192
1193
1194
1195
1196
1197
1198
1199
1200 @Deprecated
1201 public byte [] getBuffer() {
1202 return this.bytes;
1203 }
1204
1205
1206
1207
1208 public int getOffset() {
1209 return this.offset;
1210 }
1211
1212
1213
1214
1215 public int getLength() {
1216 return length;
1217 }
1218
1219
1220
1221
1222
1223
1224
1225
1226
1227
1228
1229
1230
1231
1232 private static int getLength(byte [] bytes, int offset) {
1233 int klength = ROW_OFFSET + Bytes.toInt(bytes, offset);
1234 int vlength = Bytes.toInt(bytes, offset + Bytes.SIZEOF_INT);
1235 return klength + vlength;
1236 }
1237
1238
1239
1240
1241 public int getKeyOffset() {
1242 return this.offset + ROW_OFFSET;
1243 }
1244
1245 public String getKeyString() {
1246 return Bytes.toStringBinary(getBuffer(), getKeyOffset(), getKeyLength());
1247 }
1248
1249
1250
1251
1252 public int getKeyLength() {
1253 return Bytes.toInt(this.bytes, this.offset);
1254 }
1255
1256
1257
1258
1259 @Override
1260 public byte[] getValueArray() {
1261 return bytes;
1262 }
1263
1264
1265
1266
1267 @Override
1268 public int getValueOffset() {
1269 int voffset = getKeyOffset() + getKeyLength();
1270 return voffset;
1271 }
1272
1273
1274
1275
1276 @Override
1277 public int getValueLength() {
1278 int vlength = Bytes.toInt(this.bytes, this.offset + Bytes.SIZEOF_INT);
1279 return vlength;
1280 }
1281
1282
1283
1284
1285 @Override
1286 public byte[] getRowArray() {
1287 return bytes;
1288 }
1289
1290
1291
1292
1293 @Override
1294 public int getRowOffset() {
1295 return getKeyOffset() + Bytes.SIZEOF_SHORT;
1296 }
1297
1298
1299
1300
1301 @Override
1302 public short getRowLength() {
1303 return Bytes.toShort(this.bytes, getKeyOffset());
1304 }
1305
1306
1307
1308
1309 @Override
1310 public byte[] getFamilyArray() {
1311 return bytes;
1312 }
1313
1314
1315
1316
1317 @Override
1318 public int getFamilyOffset() {
1319 return getFamilyOffset(getRowLength());
1320 }
1321
1322
1323
1324
1325 private int getFamilyOffset(int rlength) {
1326 return this.offset + ROW_OFFSET + Bytes.SIZEOF_SHORT + rlength + Bytes.SIZEOF_BYTE;
1327 }
1328
1329
1330
1331
1332 @Override
1333 public byte getFamilyLength() {
1334 return getFamilyLength(getFamilyOffset());
1335 }
1336
1337
1338
1339
1340 public byte getFamilyLength(int foffset) {
1341 return this.bytes[foffset-1];
1342 }
1343
1344
1345
1346
1347 @Override
1348 public byte[] getQualifierArray() {
1349 return bytes;
1350 }
1351
1352
1353
1354
1355 @Override
1356 public int getQualifierOffset() {
1357 return getQualifierOffset(getFamilyOffset());
1358 }
1359
1360
1361
1362
1363 private int getQualifierOffset(int foffset) {
1364 return foffset + getFamilyLength(foffset);
1365 }
1366
1367
1368
1369
1370 @Override
1371 public int getQualifierLength() {
1372 return getQualifierLength(getRowLength(),getFamilyLength());
1373 }
1374
1375
1376
1377
1378 private int getQualifierLength(int rlength, int flength) {
1379 return getKeyLength() - (int) getKeyDataStructureSize(rlength, flength, 0);
1380 }
1381
1382
1383
1384
1385 private int getTotalColumnLength(int rlength, int foffset) {
1386 int flength = getFamilyLength(foffset);
1387 int qlength = getQualifierLength(rlength,flength);
1388 return flength + qlength;
1389 }
1390
1391
1392
1393
1394 public int getTimestampOffset() {
1395 return getTimestampOffset(getKeyLength());
1396 }
1397
1398
1399
1400
1401
1402 private int getTimestampOffset(final int keylength) {
1403 return getKeyOffset() + keylength - TIMESTAMP_TYPE_SIZE;
1404 }
1405
1406
1407
1408
1409 public boolean isLatestTimestamp() {
1410 return Bytes.equals(getBuffer(), getTimestampOffset(), Bytes.SIZEOF_LONG,
1411 HConstants.LATEST_TIMESTAMP_BYTES, 0, Bytes.SIZEOF_LONG);
1412 }
1413
1414
1415
1416
1417
1418
1419 public boolean updateLatestStamp(final byte [] now) {
1420 if (this.isLatestTimestamp()) {
1421 int tsOffset = getTimestampOffset();
1422 System.arraycopy(now, 0, this.bytes, tsOffset, Bytes.SIZEOF_LONG);
1423
1424 return true;
1425 }
1426 return false;
1427 }
1428
1429
1430
1431
1432
1433
1434
1435
1436
1437
1438
1439
1440
1441
1442 public byte [] getKey() {
1443 int keylength = getKeyLength();
1444 byte [] key = new byte[keylength];
1445 System.arraycopy(getBuffer(), getKeyOffset(), key, 0, keylength);
1446 return key;
1447 }
1448
1449
1450
1451
1452
1453
1454
1455
1456 @Deprecated
1457 public byte [] getValue() {
1458 return CellUtil.cloneValue(this);
1459 }
1460
1461
1462
1463
1464
1465
1466
1467
1468
1469 @Deprecated
1470 public byte [] getRow() {
1471 return CellUtil.cloneRow(this);
1472 }
1473
1474
1475
1476
1477
1478 @Override
1479 public long getTimestamp() {
1480 return getTimestamp(getKeyLength());
1481 }
1482
1483
1484
1485
1486
1487 long getTimestamp(final int keylength) {
1488 int tsOffset = getTimestampOffset(keylength);
1489 return Bytes.toLong(this.bytes, tsOffset);
1490 }
1491
1492
1493
1494
1495 @Deprecated
1496 public byte getType() {
1497 return getTypeByte();
1498 }
1499
1500
1501
1502
1503 @Override
1504 public byte getTypeByte() {
1505 return this.bytes[this.offset + getKeyLength() - 1 + ROW_OFFSET];
1506 }
1507
1508
1509
1510
1511
1512
1513 @Deprecated
1514 public boolean isDelete() {
1515 return KeyValue.isDelete(getType());
1516 }
1517
1518
1519
1520
1521 public boolean isDeleteType() {
1522
1523 return getTypeByte() == Type.Delete.getCode();
1524 }
1525
1526
1527
1528
1529 public boolean isDeleteFamily() {
1530 return getTypeByte() == Type.DeleteFamily.getCode();
1531 }
1532
1533
1534
1535
1536 public boolean isDeleteFamilyVersion() {
1537 return getTypeByte() == Type.DeleteFamilyVersion.getCode();
1538 }
1539
1540
1541
1542
1543
1544 public boolean isDeleteColumnOrFamily() {
1545 int t = getTypeByte();
1546 return t == Type.DeleteColumn.getCode() || t == Type.DeleteFamily.getCode();
1547 }
1548
1549
1550
1551
1552
1553
1554
1555
1556
1557 @Deprecated
1558 public byte [] getFamily() {
1559 return CellUtil.cloneFamily(this);
1560 }
1561
1562
1563
1564
1565
1566
1567
1568
1569
1570
1571 @Deprecated
1572 public byte [] getQualifier() {
1573 return CellUtil.cloneQualifier(this);
1574 }
1575
1576
1577
1578
1579 @Override
1580 public int getTagsOffset() {
1581 int tagsLen = getTagsLengthUnsigned();
1582 if (tagsLen == 0) {
1583 return this.offset + this.length;
1584 }
1585 return this.offset + this.length - tagsLen;
1586 }
1587
1588
1589
1590
1591 @Override
1592 @Deprecated
1593 public int getTagsLengthUnsigned() {
1594 int tagsLen = this.length - (getKeyLength() + getValueLength() + KEYVALUE_INFRASTRUCTURE_SIZE);
1595 if (tagsLen > 0) {
1596
1597
1598 tagsLen -= TAGS_LENGTH_SIZE;
1599 }
1600 return tagsLen;
1601 }
1602
1603 @Override
1604 @Deprecated
1605 public short getTagsLength() {
1606 return (short) getTagsLengthUnsigned();
1607 }
1608
1609
1610
1611
1612
1613 public List<Tag> getTags() {
1614 int tagsLength = getTagsLengthUnsigned();
1615 if (tagsLength == 0) {
1616 return EMPTY_ARRAY_LIST;
1617 }
1618 return Tag.asList(getBuffer(), getTagsOffset(), tagsLength);
1619 }
1620
1621
1622
1623
1624 @Override
1625 public byte[] getTagsArray() {
1626 return bytes;
1627 }
1628
1629
1630
1631
1632
1633
1634
1635
1636
1637
1638
1639 public boolean matchingFamily(final byte [] family) {
1640 if (this.length == 0 || this.bytes.length == 0) {
1641 return false;
1642 }
1643 return Bytes.equals(family, 0, family.length,
1644 this.bytes, getFamilyOffset(), getFamilyLength());
1645 }
1646
1647
1648
1649
1650
1651 public boolean matchingQualifier(final byte [] qualifier) {
1652 return matchingQualifier(qualifier, 0, qualifier.length);
1653 }
1654
1655 public boolean matchingQualifier(final byte [] qualifier, int offset, int length) {
1656 return Bytes.equals(qualifier, offset, length,
1657 this.bytes, getQualifierOffset(), getQualifierLength());
1658 }
1659
1660 public boolean matchingQualifier(final KeyValue other) {
1661 return matchingQualifier(other.getBuffer(), other.getQualifierOffset(),
1662 other.getQualifierLength());
1663 }
1664
1665 public boolean matchingRow(final byte [] row) {
1666 return matchingRow(row, 0, row.length);
1667 }
1668
1669 public boolean matchingRow(final byte[] row, int offset, int length) {
1670 return Bytes.equals(row, offset, length,
1671 this.bytes, getRowOffset(), getRowLength());
1672 }
1673
1674 public boolean matchingRow(KeyValue other) {
1675 return matchingRow(other.getBuffer(), other.getRowOffset(),
1676 other.getRowLength());
1677 }
1678
1679
1680
1681
1682
1683
1684
1685 public boolean matchingColumn(final byte[] family, final byte[] qualifier) {
1686 return matchingColumn(family, 0, len(family), qualifier, 0, len(qualifier));
1687 }
1688
1689
1690
1691
1692
1693
1694
1695
1696
1697
1698
1699
1700
1701 public boolean matchingColumn(final byte [] family, final int foffset, final int flength,
1702 final byte [] qualifier, final int qoffset, final int qlength) {
1703 int rl = getRowLength();
1704 int o = getFamilyOffset(rl);
1705 int fl = getFamilyLength(o);
1706 if (!Bytes.equals(family, foffset, flength, this.bytes, o, fl)) {
1707 return false;
1708 }
1709
1710 int ql = getQualifierLength(rl, fl);
1711 if (qualifier == null || qlength == 0) {
1712 return (ql == 0);
1713 }
1714 return Bytes.equals(qualifier, qoffset, qlength, this.bytes, o + fl, ql);
1715 }
1716
1717
1718
1719
1720
1721
1722
1723
1724 public KeyValue createKeyOnly(boolean lenAsVal) {
1725
1726
1727 int dataLen = lenAsVal? Bytes.SIZEOF_INT : 0;
1728 byte [] newBuffer = new byte[getKeyLength() + ROW_OFFSET + dataLen];
1729 System.arraycopy(this.bytes, this.offset, newBuffer, 0,
1730 Math.min(newBuffer.length,this.length));
1731 Bytes.putInt(newBuffer, Bytes.SIZEOF_INT, dataLen);
1732 if (lenAsVal) {
1733 Bytes.putInt(newBuffer, newBuffer.length - dataLen, this.getValueLength());
1734 }
1735 return new KeyValue(newBuffer);
1736 }
1737
1738
1739
1740
1741
1742
1743
1744
1745
1746
1747
1748
1749
1750
1751 public static byte [][] parseColumn(byte [] c) {
1752 final int index = getDelimiter(c, 0, c.length, COLUMN_FAMILY_DELIMITER);
1753 if (index == -1) {
1754
1755 return new byte [][] { c };
1756 } else if(index == c.length - 1) {
1757
1758 byte [] family = new byte[c.length-1];
1759 System.arraycopy(c, 0, family, 0, family.length);
1760 return new byte [][] { family, HConstants.EMPTY_BYTE_ARRAY};
1761 }
1762
1763 final byte [][] result = new byte [2][];
1764 result[0] = new byte [index];
1765 System.arraycopy(c, 0, result[0], 0, index);
1766 final int len = c.length - (index + 1);
1767 result[1] = new byte[len];
1768 System.arraycopy(c, index + 1
1769 return result;
1770 }
1771
1772
1773
1774
1775
1776
1777
1778
1779
1780 public static byte [] makeColumn(byte [] family, byte [] qualifier) {
1781 return Bytes.add(family, COLUMN_FAMILY_DELIM_ARRAY, qualifier);
1782 }
1783
1784
1785
1786
1787
1788
1789
1790 public static int getDelimiter(final byte [] b, int offset, final int length,
1791 final int delimiter) {
1792 if (b == null) {
1793 throw new IllegalArgumentException("Passed buffer is null");
1794 }
1795 int result = -1;
1796 for (int i = offset; i < length + offset; i++) {
1797 if (b[i] == delimiter) {
1798 result = i;
1799 break;
1800 }
1801 }
1802 return result;
1803 }
1804
1805
1806
1807
1808
1809
1810
1811 public static int getDelimiterInReverse(final byte [] b, final int offset,
1812 final int length, final int delimiter) {
1813 if (b == null) {
1814 throw new IllegalArgumentException("Passed buffer is null");
1815 }
1816 int result = -1;
1817 for (int i = (offset + length) - 1; i >= offset; i--) {
1818 if (b[i] == delimiter) {
1819 result = i;
1820 break;
1821 }
1822 }
1823 return result;
1824 }
1825
1826
1827
1828
1829
1830 public static class MetaComparator extends KVComparator {
1831
1832
1833
1834
1835 @Override
1836 public int compareRows(byte [] left, int loffset, int llength,
1837 byte [] right, int roffset, int rlength) {
1838 int leftDelimiter = getDelimiter(left, loffset, llength,
1839 HConstants.DELIMITER);
1840 int rightDelimiter = getDelimiter(right, roffset, rlength,
1841 HConstants.DELIMITER);
1842
1843 int lpart = (leftDelimiter < 0 ? llength :leftDelimiter - loffset);
1844 int rpart = (rightDelimiter < 0 ? rlength :rightDelimiter - roffset);
1845 int result = Bytes.compareTo(left, loffset, lpart, right, roffset, rpart);
1846 if (result != 0) {
1847 return result;
1848 } else {
1849 if (leftDelimiter < 0 && rightDelimiter >= 0) {
1850 return -1;
1851 } else if (rightDelimiter < 0 && leftDelimiter >= 0) {
1852 return 1;
1853 } else if (leftDelimiter < 0 && rightDelimiter < 0) {
1854 return 0;
1855 }
1856 }
1857
1858
1859 leftDelimiter++;
1860 rightDelimiter++;
1861 int leftFarDelimiter = getDelimiterInReverse(left, leftDelimiter,
1862 llength - (leftDelimiter - loffset), HConstants.DELIMITER);
1863 int rightFarDelimiter = getDelimiterInReverse(right,
1864 rightDelimiter, rlength - (rightDelimiter - roffset),
1865 HConstants.DELIMITER);
1866
1867 lpart = (leftFarDelimiter < 0 ? llength + loffset: leftFarDelimiter) - leftDelimiter;
1868 rpart = (rightFarDelimiter < 0 ? rlength + roffset: rightFarDelimiter)- rightDelimiter;
1869 result = super.compareRows(left, leftDelimiter, lpart, right, rightDelimiter, rpart);
1870 if (result != 0) {
1871 return result;
1872 } else {
1873 if (leftDelimiter < 0 && rightDelimiter >= 0) {
1874 return -1;
1875 } else if (rightDelimiter < 0 && leftDelimiter >= 0) {
1876 return 1;
1877 } else if (leftDelimiter < 0 && rightDelimiter < 0) {
1878 return 0;
1879 }
1880 }
1881
1882 leftFarDelimiter++;
1883 rightFarDelimiter++;
1884 result = Bytes.compareTo(left, leftFarDelimiter, llength - (leftFarDelimiter - loffset),
1885 right, rightFarDelimiter, rlength - (rightFarDelimiter - roffset));
1886 return result;
1887 }
1888
1889
1890
1891
1892 @Override
1893 public byte[] getShortMidpointKey(final byte[] leftKey, final byte[] rightKey) {
1894 return Arrays.copyOf(rightKey, rightKey.length);
1895 }
1896
1897
1898
1899
1900
1901
1902
1903 @Override
1904 public String getLegacyKeyComparatorName() {
1905 return "org.apache.hadoop.hbase.KeyValue$MetaKeyComparator";
1906 }
1907
1908 @Override
1909 protected Object clone() throws CloneNotSupportedException {
1910 return new MetaComparator();
1911 }
1912
1913
1914
1915
1916 @Override
1917 protected int compareRowKey(final Cell l, final Cell r) {
1918 byte[] left = l.getRowArray();
1919 int loffset = l.getRowOffset();
1920 int llength = l.getRowLength();
1921 byte[] right = r.getRowArray();
1922 int roffset = r.getRowOffset();
1923 int rlength = r.getRowLength();
1924 return compareRows(left, loffset, llength, right, roffset, rlength);
1925 }
1926 }
1927
1928
1929
1930
1931
1932
1933 public static class KVComparator implements RawComparator<Cell>, SamePrefixComparator<byte[]> {
1934
1935
1936
1937
1938
1939
1940
1941 public String getLegacyKeyComparatorName() {
1942 return "org.apache.hadoop.hbase.KeyValue$KeyComparator";
1943 }
1944
1945 @Override
1946 public int compare(byte[] l, int loff, int llen, byte[] r, int roff, int rlen) {
1947 return compareFlatKey(l,loff,llen, r,roff,rlen);
1948 }
1949
1950
1951
1952
1953
1954
1955
1956
1957 protected int compareRowKey(final Cell left, final Cell right) {
1958 return Bytes.compareTo(
1959 left.getRowArray(), left.getRowOffset(), left.getRowLength(),
1960 right.getRowArray(), right.getRowOffset(), right.getRowLength());
1961 }
1962
1963
1964
1965
1966
1967
1968
1969
1970
1971
1972
1973
1974 public int compareFlatKey(byte[] left, int loffset, int llength,
1975 byte[] right, int roffset, int rlength) {
1976
1977 short lrowlength = Bytes.toShort(left, loffset);
1978 short rrowlength = Bytes.toShort(right, roffset);
1979 int compare = compareRows(left, loffset + Bytes.SIZEOF_SHORT,
1980 lrowlength, right, roffset + Bytes.SIZEOF_SHORT, rrowlength);
1981 if (compare != 0) {
1982 return compare;
1983 }
1984
1985
1986
1987
1988 return compareWithoutRow(0, left, loffset, llength, right, roffset,
1989 rlength, rrowlength);
1990 }
1991
1992 public int compareFlatKey(byte[] left, byte[] right) {
1993 return compareFlatKey(left, 0, left.length, right, 0, right.length);
1994 }
1995
1996
1997
1998
1999
2000 public int compare(final Cell left, final Cell right) {
2001
2002 int compare = compareRowKey(left, right);
2003 if (compare != 0) {
2004 return compare;
2005 }
2006
2007
2008 byte ltype = left.getTypeByte();
2009 byte rtype = right.getTypeByte();
2010
2011
2012
2013
2014
2015 int lcfqLen = left.getFamilyLength() + left.getQualifierLength() ;
2016 int rcfqLen = right.getFamilyLength() + right.getQualifierLength() ;
2017 if (lcfqLen == 0 && ltype == Type.Minimum.getCode()) {
2018
2019 return 1;
2020 }
2021 if (rcfqLen == 0 && rtype == Type.Minimum.getCode()) {
2022 return -1;
2023 }
2024
2025
2026
2027
2028
2029 compare = Bytes.compareTo(
2030 left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
2031 right.getFamilyArray(), right.getFamilyOffset(), right.getFamilyLength());
2032 if (compare != 0) {
2033 return compare;
2034 }
2035
2036
2037 compare = Bytes.compareTo(
2038 left.getQualifierArray(), left.getQualifierOffset(), left.getQualifierLength(),
2039 right.getQualifierArray(), right.getQualifierOffset(), right.getQualifierLength());
2040 if (compare!= 0) {
2041 return compare;
2042 }
2043
2044
2045 long ltimestamp = left.getTimestamp();
2046 long rtimestamp = right.getTimestamp();
2047 compare = compareTimestamps(ltimestamp, rtimestamp);
2048 if (compare != 0) {
2049 return compare;
2050 }
2051
2052
2053
2054
2055
2056 compare = (0xff & rtype) - (0xff & ltype);
2057 if (compare != 0) {
2058 return compare;
2059 }
2060
2061
2062
2063
2064
2065
2066
2067
2068
2069 long leftChangeSeqNum = getReplaySeqNum(left);
2070 long RightChangeSeqNum = getReplaySeqNum(right);
2071 if (leftChangeSeqNum != Long.MAX_VALUE || RightChangeSeqNum != Long.MAX_VALUE) {
2072 return Longs.compare(RightChangeSeqNum, leftChangeSeqNum);
2073 }
2074
2075
2076 return Longs.compare(right.getMvccVersion(), left.getMvccVersion());
2077 }
2078
2079
2080
2081
2082
2083
2084 private long getReplaySeqNum(final Cell c) {
2085 Tag tag = Tag.getTag(c.getTagsArray(), c.getTagsOffset(), c.getTagsLengthUnsigned(),
2086 TagType.LOG_REPLAY_TAG_TYPE);
2087
2088 if(tag != null) {
2089 return Bytes.toLong(tag.getBuffer(), tag.getTagOffset(), tag.getTagLength());
2090 }
2091 return Long.MAX_VALUE;
2092 }
2093
2094 public int compareTimestamps(final KeyValue left, final KeyValue right) {
2095
2096 long ltimestamp = left.getTimestamp(left.getKeyLength());
2097 long rtimestamp = right.getTimestamp(right.getKeyLength());
2098 return compareTimestamps(ltimestamp, rtimestamp);
2099 }
2100
2101
2102
2103
2104
2105
2106 public int compareRows(final KeyValue left, final KeyValue right) {
2107 return compareRows(left.getBuffer(),left.getRowOffset(), left.getRowLength(),
2108 right.getBuffer(), right.getRowOffset(), right.getRowLength());
2109 }
2110
2111
2112
2113
2114
2115
2116
2117
2118
2119
2120
2121 public int compareRows(byte [] left, int loffset, int llength,
2122 byte [] right, int roffset, int rlength) {
2123 return Bytes.compareTo(left, loffset, llength, right, roffset, rlength);
2124 }
2125
2126 int compareColumns(final KeyValue left, final short lrowlength,
2127 final KeyValue right, final short rrowlength) {
2128 int lfoffset = left.getFamilyOffset(lrowlength);
2129 int rfoffset = right.getFamilyOffset(rrowlength);
2130 int lclength = left.getTotalColumnLength(lrowlength,lfoffset);
2131 int rclength = right.getTotalColumnLength(rrowlength, rfoffset);
2132 int lfamilylength = left.getFamilyLength(lfoffset);
2133 int rfamilylength = right.getFamilyLength(rfoffset);
2134 return compareColumns(left.getBuffer(), lfoffset,
2135 lclength, lfamilylength,
2136 right.getBuffer(), rfoffset, rclength, rfamilylength);
2137 }
2138
2139 protected int compareColumns(
2140 byte [] left, int loffset, int llength, final int lfamilylength,
2141 byte [] right, int roffset, int rlength, final int rfamilylength) {
2142
2143 int diff = Bytes.compareTo(left, loffset, lfamilylength,
2144 right, roffset, rfamilylength);
2145 if (diff != 0) {
2146 return diff;
2147 }
2148
2149 return Bytes.compareTo(left, loffset + lfamilylength,
2150 llength - lfamilylength,
2151 right, roffset + rfamilylength, rlength - rfamilylength);
2152 }
2153
2154 static int compareTimestamps(final long ltimestamp, final long rtimestamp) {
2155
2156
2157
2158
2159 if (ltimestamp < rtimestamp) {
2160 return 1;
2161 } else if (ltimestamp > rtimestamp) {
2162 return -1;
2163 }
2164 return 0;
2165 }
2166
2167
2168
2169
2170
2171
2172
2173
2174
2175
2176
2177
2178 @Override
2179 public int compareIgnoringPrefix(int commonPrefix, byte[] left,
2180 int loffset, int llength, byte[] right, int roffset, int rlength) {
2181
2182 short lrowlength = Bytes.toShort(left, loffset);
2183 short rrowlength;
2184
2185 int comparisonResult = 0;
2186 if (commonPrefix < ROW_LENGTH_SIZE) {
2187
2188 rrowlength = Bytes.toShort(right, roffset);
2189 comparisonResult = compareRows(left, loffset + ROW_LENGTH_SIZE,
2190 lrowlength, right, roffset + ROW_LENGTH_SIZE, rrowlength);
2191 } else {
2192 rrowlength = lrowlength;
2193 if (commonPrefix < ROW_LENGTH_SIZE + rrowlength) {
2194
2195
2196 int common = commonPrefix - ROW_LENGTH_SIZE;
2197 comparisonResult = compareRows(
2198 left, loffset + common + ROW_LENGTH_SIZE, lrowlength - common,
2199 right, roffset + common + ROW_LENGTH_SIZE, rrowlength - common);
2200 }
2201 }
2202 if (comparisonResult != 0) {
2203 return comparisonResult;
2204 }
2205
2206 assert lrowlength == rrowlength;
2207 return compareWithoutRow(commonPrefix, left, loffset, llength, right,
2208 roffset, rlength, lrowlength);
2209 }
2210
2211
2212
2213
2214
2215
2216
2217
2218
2219
2220
2221 private int compareWithoutRow(int commonPrefix, byte[] left, int loffset,
2222 int llength, byte[] right, int roffset, int rlength, short rowlength) {
2223
2224
2225
2226
2227
2228 int commonLength = ROW_LENGTH_SIZE + FAMILY_LENGTH_SIZE + rowlength;
2229
2230
2231 int commonLengthWithTSAndType = TIMESTAMP_TYPE_SIZE + commonLength;
2232
2233 int lcolumnlength = llength - commonLengthWithTSAndType;
2234 int rcolumnlength = rlength - commonLengthWithTSAndType;
2235
2236 byte ltype = left[loffset + (llength - 1)];
2237 byte rtype = right[roffset + (rlength - 1)];
2238
2239
2240
2241
2242
2243
2244 if (lcolumnlength == 0 && ltype == Type.Minimum.getCode()) {
2245
2246 return 1;
2247 }
2248 if (rcolumnlength == 0 && rtype == Type.Minimum.getCode()) {
2249 return -1;
2250 }
2251
2252 int lfamilyoffset = commonLength + loffset;
2253 int rfamilyoffset = commonLength + roffset;
2254
2255
2256 int lfamilylength = left[lfamilyoffset - 1];
2257 int rfamilylength = right[rfamilyoffset - 1];
2258
2259
2260 boolean sameFamilySize = (lfamilylength == rfamilylength);
2261 int common = 0;
2262 if (commonPrefix > 0) {
2263 common = Math.max(0, commonPrefix - commonLength);
2264 if (!sameFamilySize) {
2265
2266
2267 common = Math.min(common, Math.min(lfamilylength, rfamilylength));
2268 } else {
2269 common = Math.min(common, Math.min(lcolumnlength, rcolumnlength));
2270 }
2271 }
2272 if (!sameFamilySize) {
2273
2274 return Bytes.compareTo(left, lfamilyoffset + common, lfamilylength
2275 - common, right, rfamilyoffset + common, rfamilylength - common);
2276 }
2277
2278 final int comparison = Bytes.compareTo(left, lfamilyoffset + common,
2279 lcolumnlength - common, right, rfamilyoffset + common,
2280 rcolumnlength - common);
2281 if (comparison != 0) {
2282 return comparison;
2283 }
2284
2285
2286
2287 long ltimestamp = Bytes.toLong(left,
2288 loffset + (llength - TIMESTAMP_TYPE_SIZE));
2289 long rtimestamp = Bytes.toLong(right,
2290 roffset + (rlength - TIMESTAMP_TYPE_SIZE));
2291 int compare = compareTimestamps(ltimestamp, rtimestamp);
2292 if (compare != 0) {
2293 return compare;
2294 }
2295
2296
2297
2298
2299
2300 return (0xff & rtype) - (0xff & ltype);
2301 }
2302
2303
2304
2305
2306
2307
2308
2309 public boolean matchingRowColumn(final KeyValue left,
2310 final KeyValue right) {
2311 short lrowlength = left.getRowLength();
2312 short rrowlength = right.getRowLength();
2313
2314
2315 if ((left.getTimestampOffset() - left.getOffset()) !=
2316 (right.getTimestampOffset() - right.getOffset())) {
2317 return false;
2318 }
2319
2320 if (!matchingRows(left, lrowlength, right, rrowlength)) {
2321 return false;
2322 }
2323
2324 int lfoffset = left.getFamilyOffset(lrowlength);
2325 int rfoffset = right.getFamilyOffset(rrowlength);
2326 int lclength = left.getTotalColumnLength(lrowlength,lfoffset);
2327 int rclength = right.getTotalColumnLength(rrowlength, rfoffset);
2328 int lfamilylength = left.getFamilyLength(lfoffset);
2329 int rfamilylength = right.getFamilyLength(rfoffset);
2330 int ccRes = compareColumns(left.getBuffer(), lfoffset, lclength, lfamilylength,
2331 right.getBuffer(), rfoffset, rclength, rfamilylength);
2332 return ccRes == 0;
2333 }
2334
2335
2336
2337
2338
2339
2340
2341 public boolean matchingRows(final KeyValue left, final KeyValue right) {
2342 short lrowlength = left.getRowLength();
2343 short rrowlength = right.getRowLength();
2344 return matchingRows(left, lrowlength, right, rrowlength);
2345 }
2346
2347
2348
2349
2350
2351
2352
2353
2354 private boolean matchingRows(final KeyValue left, final short lrowlength,
2355 final KeyValue right, final short rrowlength) {
2356 return lrowlength == rrowlength &&
2357 matchingRows(left.getBuffer(), left.getRowOffset(), lrowlength,
2358 right.getBuffer(), right.getRowOffset(), rrowlength);
2359 }
2360
2361
2362
2363
2364
2365
2366
2367
2368
2369
2370
2371 public boolean matchingRows(final byte [] left, final int loffset, final int llength,
2372 final byte [] right, final int roffset, final int rlength) {
2373 return Bytes.equals(left, loffset, llength, right, roffset, rlength);
2374 }
2375
2376 public byte[] calcIndexKey(byte[] lastKeyOfPreviousBlock, byte[] firstKeyInBlock) {
2377 byte[] fakeKey = getShortMidpointKey(lastKeyOfPreviousBlock, firstKeyInBlock);
2378 if (compareFlatKey(fakeKey, firstKeyInBlock) > 0) {
2379 LOG.error("Unexpected getShortMidpointKey result, fakeKey:"
2380 + Bytes.toStringBinary(fakeKey) + ", firstKeyInBlock:"
2381 + Bytes.toStringBinary(firstKeyInBlock));
2382 return firstKeyInBlock;
2383 }
2384 if (lastKeyOfPreviousBlock != null && compareFlatKey(lastKeyOfPreviousBlock, fakeKey) >= 0) {
2385 LOG.error("Unexpected getShortMidpointKey result, lastKeyOfPreviousBlock:" +
2386 Bytes.toStringBinary(lastKeyOfPreviousBlock) + ", fakeKey:" +
2387 Bytes.toStringBinary(fakeKey));
2388 return firstKeyInBlock;
2389 }
2390 return fakeKey;
2391 }
2392
2393
2394
2395
2396
2397
2398
2399 public byte[] getShortMidpointKey(final byte[] leftKey, final byte[] rightKey) {
2400 if (rightKey == null) {
2401 throw new IllegalArgumentException("rightKey can not be null");
2402 }
2403 if (leftKey == null) {
2404 return Arrays.copyOf(rightKey, rightKey.length);
2405 }
2406 if (compareFlatKey(leftKey, rightKey) >= 0) {
2407 throw new IllegalArgumentException("Unexpected input, leftKey:" + Bytes.toString(leftKey)
2408 + ", rightKey:" + Bytes.toString(rightKey));
2409 }
2410
2411 short leftRowLength = Bytes.toShort(leftKey, 0);
2412 short rightRowLength = Bytes.toShort(rightKey, 0);
2413 int leftCommonLength = ROW_LENGTH_SIZE + FAMILY_LENGTH_SIZE + leftRowLength;
2414 int rightCommonLength = ROW_LENGTH_SIZE + FAMILY_LENGTH_SIZE + rightRowLength;
2415 int leftCommonLengthWithTSAndType = TIMESTAMP_TYPE_SIZE + leftCommonLength;
2416 int rightCommonLengthWithTSAndType = TIMESTAMP_TYPE_SIZE + rightCommonLength;
2417 int leftColumnLength = leftKey.length - leftCommonLengthWithTSAndType;
2418 int rightColumnLength = rightKey.length - rightCommonLengthWithTSAndType;
2419
2420 if (leftRowLength == rightRowLength && compareRows(leftKey, ROW_LENGTH_SIZE, leftRowLength,
2421 rightKey, ROW_LENGTH_SIZE, rightRowLength) == 0) {
2422
2423 int comparison = Bytes.compareTo(leftKey, leftCommonLength, leftColumnLength, rightKey,
2424 rightCommonLength, rightColumnLength);
2425
2426 if (comparison == 0) {
2427 return Arrays.copyOf(rightKey, rightKey.length);
2428 }
2429
2430 byte[] newKey = Arrays.copyOf(rightKey, rightKey.length);
2431 Bytes.putLong(newKey, rightKey.length - TIMESTAMP_TYPE_SIZE, HConstants.LATEST_TIMESTAMP);
2432 Bytes.putByte(newKey, rightKey.length - TYPE_SIZE, Type.Maximum.getCode());
2433 return newKey;
2434 }
2435
2436 short minLength = leftRowLength < rightRowLength ? leftRowLength : rightRowLength;
2437 short diffIdx = 0;
2438 while (diffIdx < minLength
2439 && leftKey[ROW_LENGTH_SIZE + diffIdx] == rightKey[ROW_LENGTH_SIZE + diffIdx]) {
2440 diffIdx++;
2441 }
2442 byte[] newRowKey = null;
2443 if (diffIdx >= minLength) {
2444
2445 newRowKey = new byte[diffIdx + 1];
2446 System.arraycopy(rightKey, ROW_LENGTH_SIZE, newRowKey, 0, diffIdx + 1);
2447 } else {
2448 int diffByte = leftKey[ROW_LENGTH_SIZE + diffIdx];
2449 if ((0xff & diffByte) < 0xff && (diffByte + 1) <
2450 (rightKey[ROW_LENGTH_SIZE + diffIdx] & 0xff)) {
2451 newRowKey = new byte[diffIdx + 1];
2452 System.arraycopy(leftKey, ROW_LENGTH_SIZE, newRowKey, 0, diffIdx);
2453 newRowKey[diffIdx] = (byte) (diffByte + 1);
2454 } else {
2455 newRowKey = new byte[diffIdx + 1];
2456 System.arraycopy(rightKey, ROW_LENGTH_SIZE, newRowKey, 0, diffIdx + 1);
2457 }
2458 }
2459 return new KeyValue(newRowKey, null, null, HConstants.LATEST_TIMESTAMP,
2460 Type.Maximum).getKey();
2461 }
2462
2463 @Override
2464 protected Object clone() throws CloneNotSupportedException {
2465 return new KVComparator();
2466 }
2467
2468 }
2469
2470
2471
2472
2473
2474
2475
2476
2477 public static KeyValue createLastOnRow(final byte[] row) {
2478 return new KeyValue(row, null, null, HConstants.LATEST_TIMESTAMP, Type.Minimum);
2479 }
2480
2481
2482
2483
2484
2485
2486
2487
2488
2489 public static KeyValue createFirstOnRow(final byte [] row) {
2490 return createFirstOnRow(row, HConstants.LATEST_TIMESTAMP);
2491 }
2492
2493
2494
2495
2496
2497
2498
2499
2500
2501 public static KeyValue createFirstOnRow(final byte [] row, int roffset, short rlength) {
2502 return new KeyValue(row, roffset, rlength,
2503 null, 0, 0, null, 0, 0, HConstants.LATEST_TIMESTAMP, Type.Maximum, null, 0, 0);
2504 }
2505
2506
2507
2508
2509
2510
2511
2512
2513 public static KeyValue createFirstOnRow(final byte [] row,
2514 final long ts) {
2515 return new KeyValue(row, null, null, ts, Type.Maximum);
2516 }
2517
2518
2519
2520
2521
2522
2523
2524
2525
2526
2527 public static KeyValue createFirstOnRow(final byte [] row, final byte [] family,
2528 final byte [] qualifier) {
2529 return new KeyValue(row, family, qualifier, HConstants.LATEST_TIMESTAMP, Type.Maximum);
2530 }
2531
2532
2533
2534
2535
2536
2537
2538
2539
2540
2541 public static KeyValue createFirstDeleteFamilyOnRow(final byte [] row,
2542 final byte [] family) {
2543 return new KeyValue(row, family, null, HConstants.LATEST_TIMESTAMP,
2544 Type.DeleteFamily);
2545 }
2546
2547
2548
2549
2550
2551
2552
2553
2554 public static KeyValue createFirstOnRow(final byte [] row, final byte [] f,
2555 final byte [] q, final long ts) {
2556 return new KeyValue(row, f, q, ts, Type.Maximum);
2557 }
2558
2559
2560
2561
2562
2563
2564
2565
2566
2567
2568
2569
2570
2571
2572
2573
2574
2575 public static KeyValue createFirstOnRow(final byte [] row,
2576 final int roffset, final int rlength, final byte [] family,
2577 final int foffset, final int flength, final byte [] qualifier,
2578 final int qoffset, final int qlength) {
2579 return new KeyValue(row, roffset, rlength, family,
2580 foffset, flength, qualifier, qoffset, qlength,
2581 HConstants.LATEST_TIMESTAMP, Type.Maximum, null, 0, 0);
2582 }
2583
2584
2585
2586
2587
2588
2589
2590
2591
2592
2593
2594
2595
2596
2597
2598
2599
2600 public static KeyValue createFirstOnRow(byte [] buffer, final byte [] row,
2601 final byte [] family, final byte [] qualifier)
2602 throws IllegalArgumentException {
2603
2604 return createFirstOnRow(buffer, 0, row, 0, row.length,
2605 family, 0, family.length,
2606 qualifier, 0, qualifier.length);
2607 }
2608
2609
2610
2611
2612
2613
2614
2615
2616
2617
2618
2619
2620
2621
2622
2623
2624
2625
2626
2627
2628
2629
2630
2631
2632 public static KeyValue createFirstOnRow(byte [] buffer, final int boffset,
2633 final byte [] row, final int roffset, final int rlength,
2634 final byte [] family, final int foffset, final int flength,
2635 final byte [] qualifier, final int qoffset, final int qlength)
2636 throws IllegalArgumentException {
2637
2638 long lLength = getKeyValueDataStructureSize(rlength, flength, qlength, 0);
2639
2640 if (lLength > Integer.MAX_VALUE) {
2641 throw new IllegalArgumentException("KeyValue length " + lLength + " > " + Integer.MAX_VALUE);
2642 }
2643 int iLength = (int) lLength;
2644 if (buffer.length - boffset < iLength) {
2645 throw new IllegalArgumentException("Buffer size " + (buffer.length - boffset) + " < " +
2646 iLength);
2647 }
2648
2649 int len = writeByteArray(buffer, boffset, row, roffset, rlength, family, foffset, flength,
2650 qualifier, qoffset, qlength, HConstants.LATEST_TIMESTAMP, KeyValue.Type.Maximum,
2651 null, 0, 0, null);
2652 return new KeyValue(buffer, boffset, len);
2653 }
2654
2655
2656
2657
2658
2659
2660
2661
2662
2663
2664
2665
2666
2667
2668
2669
2670
2671 public static KeyValue createLastOnRow(final byte [] row,
2672 final int roffset, final int rlength, final byte [] family,
2673 final int foffset, final int flength, final byte [] qualifier,
2674 final int qoffset, final int qlength) {
2675 return new KeyValue(row, roffset, rlength, family,
2676 foffset, flength, qualifier, qoffset, qlength,
2677 HConstants.OLDEST_TIMESTAMP, Type.Minimum, null, 0, 0);
2678 }
2679
2680
2681
2682
2683
2684
2685
2686
2687
2688 public KeyValue createLastOnRowCol() {
2689 return new KeyValue(
2690 bytes, getRowOffset(), getRowLength(),
2691 bytes, getFamilyOffset(), getFamilyLength(),
2692 bytes, getQualifierOffset(), getQualifierLength(),
2693 HConstants.OLDEST_TIMESTAMP, Type.Minimum, null, 0, 0);
2694 }
2695
2696
2697
2698
2699
2700
2701
2702
2703 public KeyValue createFirstOnRowColTS(long ts) {
2704 return new KeyValue(
2705 bytes, getRowOffset(), getRowLength(),
2706 bytes, getFamilyOffset(), getFamilyLength(),
2707 bytes, getQualifierOffset(), getQualifierLength(),
2708 ts, Type.Maximum, bytes, getValueOffset(), getValueLength());
2709 }
2710
2711
2712
2713
2714
2715
2716 public static KeyValue createKeyValueFromKey(final byte [] b) {
2717 return createKeyValueFromKey(b, 0, b.length);
2718 }
2719
2720
2721
2722
2723
2724
2725 public static KeyValue createKeyValueFromKey(final ByteBuffer bb) {
2726 return createKeyValueFromKey(bb.array(), bb.arrayOffset(), bb.limit());
2727 }
2728
2729
2730
2731
2732
2733
2734
2735
2736 public static KeyValue createKeyValueFromKey(final byte [] b, final int o,
2737 final int l) {
2738 byte [] newb = new byte[l + ROW_OFFSET];
2739 System.arraycopy(b, o, newb, ROW_OFFSET, l);
2740 Bytes.putInt(newb, 0, l);
2741 Bytes.putInt(newb, Bytes.SIZEOF_INT, 0);
2742 return new KeyValue(newb);
2743 }
2744
2745
2746
2747
2748
2749
2750
2751
2752 public static KeyValue create(final DataInput in) throws IOException {
2753 return create(in.readInt(), in);
2754 }
2755
2756
2757
2758
2759
2760
2761
2762
2763
2764 public static KeyValue create(int length, final DataInput in) throws IOException {
2765
2766 if (length <= 0) {
2767 if (length == 0) return null;
2768 throw new IOException("Failed read " + length + " bytes, stream corrupt?");
2769 }
2770
2771
2772 byte [] bytes = new byte[length];
2773 in.readFully(bytes);
2774 return new KeyValue(bytes, 0, length);
2775 }
2776
2777
2778
2779
2780
2781
2782
2783 public static KeyValue cloneAndAddTags(Cell c, List<Tag> newTags) {
2784 List<Tag> existingTags = null;
2785 if(c.getTagsLengthUnsigned() > 0) {
2786 existingTags = Tag.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLengthUnsigned());
2787 existingTags.addAll(newTags);
2788 } else {
2789 existingTags = newTags;
2790 }
2791 return new KeyValue(c.getRowArray(), c.getRowOffset(), (int)c.getRowLength(),
2792 c.getFamilyArray(), c.getFamilyOffset(), (int)c.getFamilyLength(),
2793 c.getQualifierArray(), c.getQualifierOffset(), (int) c.getQualifierLength(),
2794 c.getTimestamp(), Type.codeToType(c.getTypeByte()), c.getValueArray(), c.getValueOffset(),
2795 c.getValueLength(), existingTags);
2796 }
2797
2798
2799
2800
2801
2802
2803
2804
2805
2806 public static KeyValue iscreate(final InputStream in) throws IOException {
2807 byte [] intBytes = new byte[Bytes.SIZEOF_INT];
2808 int bytesRead = 0;
2809 while (bytesRead < intBytes.length) {
2810 int n = in.read(intBytes, bytesRead, intBytes.length - bytesRead);
2811 if (n < 0) {
2812 if (bytesRead == 0) return null;
2813 throw new IOException("Failed read of int, read " + bytesRead + " bytes");
2814 }
2815 bytesRead += n;
2816 }
2817
2818 byte [] bytes = new byte[Bytes.toInt(intBytes)];
2819 IOUtils.readFully(in, bytes, 0, bytes.length);
2820 return new KeyValue(bytes, 0, bytes.length);
2821 }
2822
2823
2824
2825
2826
2827
2828
2829
2830
2831 public static long write(final KeyValue kv, final DataOutput out) throws IOException {
2832
2833
2834 int length = kv.getLength();
2835 out.writeInt(length);
2836 out.write(kv.getBuffer(), kv.getOffset(), length);
2837 return length + Bytes.SIZEOF_INT;
2838 }
2839
2840
2841
2842
2843
2844
2845
2846
2847
2848
2849
2850
2851 @Deprecated
2852 public static long oswrite(final KeyValue kv, final OutputStream out)
2853 throws IOException {
2854 int length = kv.getLength();
2855
2856 out.write(Bytes.toBytes(length));
2857 out.write(kv.getBuffer(), kv.getOffset(), length);
2858 return length + Bytes.SIZEOF_INT;
2859 }
2860
2861
2862
2863
2864
2865
2866
2867
2868
2869
2870
2871
2872
2873 public static long oswrite(final KeyValue kv, final OutputStream out, final boolean withTags)
2874 throws IOException {
2875 int length = kv.getLength();
2876 if (!withTags) {
2877 length = kv.getKeyLength() + kv.getValueLength() + KEYVALUE_INFRASTRUCTURE_SIZE;
2878 }
2879
2880 out.write(Bytes.toBytes(length));
2881 out.write(kv.getBuffer(), kv.getOffset(), length);
2882 return length + Bytes.SIZEOF_INT;
2883 }
2884
2885
2886
2887
2888 public static class RowOnlyComparator implements Comparator<KeyValue> {
2889 final KVComparator comparator;
2890
2891 public RowOnlyComparator(final KVComparator c) {
2892 this.comparator = c;
2893 }
2894
2895 public int compare(KeyValue left, KeyValue right) {
2896 return comparator.compareRows(left, right);
2897 }
2898 }
2899
2900
2901
2902
2903
2904
2905
2906 public interface SamePrefixComparator<T> {
2907
2908
2909
2910
2911 int compareIgnoringPrefix(
2912 int commonPrefix, byte[] left, int loffset, int llength, byte[] right, int roffset, int rlength
2913 );
2914 }
2915
2916
2917
2918
2919 public static class RawBytesComparator extends KVComparator {
2920
2921
2922
2923
2924
2925
2926 public String getLegacyKeyComparatorName() {
2927 return "org.apache.hadoop.hbase.util.Bytes$ByteArrayComparator";
2928 }
2929
2930 public int compareFlatKey(byte[] left, int loffset, int llength, byte[] right,
2931 int roffset, int rlength) {
2932 return Bytes.BYTES_RAWCOMPARATOR.compare(left, loffset, llength, right, roffset, rlength);
2933 }
2934
2935 public byte[] calcIndexKey(byte[] lastKeyOfPreviousBlock, byte[] firstKeyInBlock) {
2936 return firstKeyInBlock;
2937 }
2938
2939 }
2940
2941
2942
2943
2944
2945
2946
2947 @Override
2948 public long heapSize() {
2949 int sum = 0;
2950 sum += ClassSize.OBJECT;
2951 sum += ClassSize.REFERENCE;
2952 sum += ClassSize.align(ClassSize.ARRAY);
2953 sum += ClassSize.align(length);
2954 sum += 2 * Bytes.SIZEOF_INT;
2955 sum += Bytes.SIZEOF_LONG;
2956 return ClassSize.align(sum);
2957 }
2958 }