1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase;
20
21 import java.io.DataOutputStream;
22 import java.io.IOException;
23 import java.nio.ByteBuffer;
24 import java.util.Iterator;
25 import java.util.List;
26 import java.util.Map.Entry;
27 import java.util.NavigableMap;
28
29 import org.apache.hadoop.hbase.classification.InterfaceAudience;
30 import org.apache.hadoop.hbase.classification.InterfaceStability;
31 import org.apache.hadoop.hbase.KeyValue.Type;
32 import org.apache.hadoop.hbase.io.HeapSize;
33 import org.apache.hadoop.hbase.util.ByteBufferUtils;
34 import org.apache.hadoop.hbase.util.ByteRange;
35 import org.apache.hadoop.hbase.util.Bytes;
36
37
38
39
40
41
42 @InterfaceAudience.Public
43 @InterfaceStability.Evolving
44 public final class CellUtil {
45
46
47
48 public static ByteRange fillRowRange(Cell cell, ByteRange range) {
49 return range.set(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
50 }
51
52 public static ByteRange fillFamilyRange(Cell cell, ByteRange range) {
53 return range.set(cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength());
54 }
55
56 public static ByteRange fillQualifierRange(Cell cell, ByteRange range) {
57 return range.set(cell.getQualifierArray(), cell.getQualifierOffset(),
58 cell.getQualifierLength());
59 }
60
61 public static ByteRange fillValueRange(Cell cell, ByteRange range) {
62 return range.set(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
63 }
64
65 public static ByteRange fillTagRange(Cell cell, ByteRange range) {
66 return range.set(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength());
67 }
68
69
70
71 public static byte[] cloneRow(Cell cell){
72 byte[] output = new byte[cell.getRowLength()];
73 copyRowTo(cell, output, 0);
74 return output;
75 }
76
77 public static byte[] cloneFamily(Cell cell){
78 byte[] output = new byte[cell.getFamilyLength()];
79 copyFamilyTo(cell, output, 0);
80 return output;
81 }
82
83 public static byte[] cloneQualifier(Cell cell){
84 byte[] output = new byte[cell.getQualifierLength()];
85 copyQualifierTo(cell, output, 0);
86 return output;
87 }
88
89 public static byte[] cloneValue(Cell cell){
90 byte[] output = new byte[cell.getValueLength()];
91 copyValueTo(cell, output, 0);
92 return output;
93 }
94
95
96
97
98
99
100
101
102 public static byte[] getTagArray(Cell cell){
103 byte[] output = new byte[cell.getTagsLength()];
104 copyTagTo(cell, output, 0);
105 return output;
106 }
107
108
109
110
111 public static int copyRowTo(Cell cell, byte[] destination, int destinationOffset) {
112 System.arraycopy(cell.getRowArray(), cell.getRowOffset(), destination, destinationOffset,
113 cell.getRowLength());
114 return destinationOffset + cell.getRowLength();
115 }
116
117 public static int copyFamilyTo(Cell cell, byte[] destination, int destinationOffset) {
118 System.arraycopy(cell.getFamilyArray(), cell.getFamilyOffset(), destination, destinationOffset,
119 cell.getFamilyLength());
120 return destinationOffset + cell.getFamilyLength();
121 }
122
123 public static int copyQualifierTo(Cell cell, byte[] destination, int destinationOffset) {
124 System.arraycopy(cell.getQualifierArray(), cell.getQualifierOffset(), destination,
125 destinationOffset, cell.getQualifierLength());
126 return destinationOffset + cell.getQualifierLength();
127 }
128
129 public static int copyValueTo(Cell cell, byte[] destination, int destinationOffset) {
130 System.arraycopy(cell.getValueArray(), cell.getValueOffset(), destination, destinationOffset,
131 cell.getValueLength());
132 return destinationOffset + cell.getValueLength();
133 }
134
135
136
137
138
139
140
141
142 public static int copyTagTo(Cell cell, byte[] destination, int destinationOffset) {
143 System.arraycopy(cell.getTagsArray(), cell.getTagsOffset(), destination, destinationOffset,
144 cell.getTagsLength());
145 return destinationOffset + cell.getTagsLength();
146 }
147
148
149
150 public static byte getRowByte(Cell cell, int index) {
151 return cell.getRowArray()[cell.getRowOffset() + index];
152 }
153
154 public static ByteBuffer getValueBufferShallowCopy(Cell cell) {
155 ByteBuffer buffer = ByteBuffer.wrap(cell.getValueArray(), cell.getValueOffset(),
156 cell.getValueLength());
157 return buffer;
158 }
159
160 public static ByteBuffer getQualifierBufferShallowCopy(Cell cell) {
161 ByteBuffer buffer = ByteBuffer.wrap(cell.getQualifierArray(), cell.getQualifierOffset(),
162 cell.getQualifierLength());
163 return buffer;
164 }
165
166 public static Cell createCell(final byte [] row, final byte [] family, final byte [] qualifier,
167 final long timestamp, final byte type, final byte [] value) {
168
169
170
171
172 return new KeyValue(row, family, qualifier, timestamp, KeyValue.Type.codeToType(type), value);
173 }
174
175 public static Cell createCell(final byte [] rowArray, final int rowOffset, final int rowLength,
176 final byte [] familyArray, final int familyOffset, final int familyLength,
177 final byte [] qualifierArray, final int qualifierOffset, final int qualifierLength) {
178
179 return new KeyValue(rowArray, rowOffset, rowLength,
180 familyArray, familyOffset, familyLength,
181 qualifierArray, qualifierOffset, qualifierLength,
182 HConstants.LATEST_TIMESTAMP,
183 KeyValue.Type.Maximum,
184 HConstants.EMPTY_BYTE_ARRAY, 0, HConstants.EMPTY_BYTE_ARRAY.length);
185 }
186
187
188
189
190
191
192 @InterfaceAudience.Private
193 public static Cell createCell(final byte[] row, final byte[] family, final byte[] qualifier,
194 final long timestamp, final byte type, final byte[] value, final long memstoreTS) {
195 KeyValue keyValue = new KeyValue(row, family, qualifier, timestamp,
196 KeyValue.Type.codeToType(type), value);
197 keyValue.setSequenceId(memstoreTS);
198 return keyValue;
199 }
200
201
202
203
204
205
206 @InterfaceAudience.Private
207 public static Cell createCell(final byte[] row, final byte[] family, final byte[] qualifier,
208 final long timestamp, final byte type, final byte[] value, byte[] tags, final long memstoreTS) {
209 KeyValue keyValue = new KeyValue(row, family, qualifier, timestamp,
210 KeyValue.Type.codeToType(type), value, tags);
211 keyValue.setSequenceId(memstoreTS);
212 return keyValue;
213 }
214
215
216
217
218
219
220 @InterfaceAudience.Private
221 public static Cell createCell(final byte[] row, final byte[] family, final byte[] qualifier,
222 final long timestamp, Type type, final byte[] value, byte[] tags) {
223 KeyValue keyValue = new KeyValue(row, family, qualifier, timestamp, type, value, tags);
224 return keyValue;
225 }
226
227
228
229
230
231
232 public static Cell createCell(final byte [] row) {
233 return createCell(row, HConstants.EMPTY_BYTE_ARRAY);
234 }
235
236
237
238
239
240
241
242 public static Cell createCell(final byte [] row, final byte [] value) {
243
244
245
246
247 return createCell(row, HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY,
248 HConstants.LATEST_TIMESTAMP, KeyValue.Type.Maximum.getCode(), value);
249 }
250
251
252
253
254
255
256
257
258 public static Cell createCell(final byte [] row, final byte [] family, final byte [] qualifier) {
259
260 return createCell(row, family, qualifier,
261 HConstants.LATEST_TIMESTAMP, KeyValue.Type.Maximum.getCode(), HConstants.EMPTY_BYTE_ARRAY);
262 }
263
264
265
266
267
268 public static CellScanner createCellScanner(final List<? extends CellScannable> cellScannerables) {
269 return new CellScanner() {
270 private final Iterator<? extends CellScannable> iterator = cellScannerables.iterator();
271 private CellScanner cellScanner = null;
272
273 @Override
274 public Cell current() {
275 return this.cellScanner != null? this.cellScanner.current(): null;
276 }
277
278 @Override
279 public boolean advance() throws IOException {
280 while (true) {
281 if (this.cellScanner == null) {
282 if (!this.iterator.hasNext()) return false;
283 this.cellScanner = this.iterator.next().cellScanner();
284 }
285 if (this.cellScanner.advance()) return true;
286 this.cellScanner = null;
287 }
288 }
289 };
290 }
291
292
293
294
295
296 public static CellScanner createCellScanner(final Iterable<Cell> cellIterable) {
297 if (cellIterable == null) return null;
298 return createCellScanner(cellIterable.iterator());
299 }
300
301
302
303
304
305
306 public static CellScanner createCellScanner(final Iterator<Cell> cells) {
307 if (cells == null) return null;
308 return new CellScanner() {
309 private final Iterator<Cell> iterator = cells;
310 private Cell current = null;
311
312 @Override
313 public Cell current() {
314 return this.current;
315 }
316
317 @Override
318 public boolean advance() {
319 boolean hasNext = this.iterator.hasNext();
320 this.current = hasNext? this.iterator.next(): null;
321 return hasNext;
322 }
323 };
324 }
325
326
327
328
329
330 public static CellScanner createCellScanner(final Cell[] cellArray) {
331 return new CellScanner() {
332 private final Cell [] cells = cellArray;
333 private int index = -1;
334
335 @Override
336 public Cell current() {
337 if (cells == null) return null;
338 return (index < 0)? null: this.cells[index];
339 }
340
341 @Override
342 public boolean advance() {
343 if (cells == null) return false;
344 return ++index < this.cells.length;
345 }
346 };
347 }
348
349
350
351
352
353
354
355 public static CellScanner createCellScanner(final NavigableMap<byte [], List<Cell>> map) {
356 return new CellScanner() {
357 private final Iterator<Entry<byte[], List<Cell>>> entries = map.entrySet().iterator();
358 private Iterator<Cell> currentIterator = null;
359 private Cell currentCell;
360
361 @Override
362 public Cell current() {
363 return this.currentCell;
364 }
365
366 @Override
367 public boolean advance() {
368 while(true) {
369 if (this.currentIterator == null) {
370 if (!this.entries.hasNext()) return false;
371 this.currentIterator = this.entries.next().getValue().iterator();
372 }
373 if (this.currentIterator.hasNext()) {
374 this.currentCell = this.currentIterator.next();
375 return true;
376 }
377 this.currentCell = null;
378 this.currentIterator = null;
379 }
380 }
381 };
382 }
383
384
385
386
387
388
389 public static boolean matchingRow(final Cell left, final Cell right) {
390 return Bytes.equals(left.getRowArray(), left.getRowOffset(), left.getRowLength(),
391 right.getRowArray(), right.getRowOffset(), right.getRowLength());
392 }
393
394 public static boolean matchingRow(final Cell left, final byte[] buf) {
395 return Bytes.equals(left.getRowArray(), left.getRowOffset(), left.getRowLength(), buf, 0,
396 buf.length);
397 }
398
399 public static boolean matchingRow(final Cell left, final byte[] buf, final int offset,
400 final int length) {
401 return Bytes.equals(left.getRowArray(), left.getRowOffset(), left.getRowLength(), buf, offset,
402 length);
403 }
404
405 public static boolean matchingFamily(final Cell left, final Cell right) {
406 return Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
407 right.getFamilyArray(), right.getFamilyOffset(), right.getFamilyLength());
408 }
409
410 public static boolean matchingFamily(final Cell left, final byte[] buf) {
411 return Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(), buf,
412 0, buf.length);
413 }
414
415 public static boolean matchingFamily(final Cell left, final byte[] buf, final int offset,
416 final int length) {
417 return Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(), buf,
418 offset, length);
419 }
420
421 public static boolean matchingQualifier(final Cell left, final Cell right) {
422 return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(),
423 left.getQualifierLength(), right.getQualifierArray(), right.getQualifierOffset(),
424 right.getQualifierLength());
425 }
426
427 public static boolean matchingQualifier(final Cell left, final byte[] buf) {
428 if (buf == null) {
429 return left.getQualifierLength() == 0;
430 }
431 return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(),
432 left.getQualifierLength(), buf, 0, buf.length);
433 }
434
435 public static boolean matchingQualifier(final Cell left, final byte[] buf, final int offset,
436 final int length) {
437 if (buf == null) {
438 return left.getQualifierLength() == 0;
439 }
440 return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(),
441 left.getQualifierLength(), buf, offset, length);
442 }
443
444 public static boolean matchingColumn(final Cell left, final byte[] fam, final byte[] qual) {
445 if (!matchingFamily(left, fam))
446 return false;
447 return matchingQualifier(left, qual);
448 }
449
450 public static boolean matchingColumn(final Cell left, final byte[] fam, final int foffset,
451 final int flength, final byte[] qual, final int qoffset, final int qlength) {
452 if (!matchingFamily(left, fam, foffset, flength))
453 return false;
454 return matchingQualifier(left, qual, qoffset, qlength);
455 }
456
457 public static boolean matchingColumn(final Cell left, final Cell right) {
458 if (!matchingFamily(left, right))
459 return false;
460 return matchingQualifier(left, right);
461 }
462
463 public static boolean matchingValue(final Cell left, final Cell right) {
464 return Bytes.equals(left.getValueArray(), left.getValueOffset(), left.getValueLength(),
465 right.getValueArray(), right.getValueOffset(), right.getValueLength());
466 }
467
468 public static boolean matchingValue(final Cell left, final byte[] buf) {
469 return Bytes.equals(left.getValueArray(), left.getValueOffset(), left.getValueLength(), buf, 0,
470 buf.length);
471 }
472
473 public static boolean matchingTimestamp(Cell a, Cell b) {
474 return CellComparator.compareTimestamps(a, b) == 0;
475 }
476
477
478
479
480
481
482 public static boolean isDelete(final Cell cell) {
483 return isDelete(cell.getTypeByte());
484 }
485
486
487
488
489
490
491 public static boolean isDelete(final byte type) {
492 return Type.Delete.getCode() <= type
493 && type <= Type.DeleteFamily.getCode();
494 }
495
496
497
498
499 public static boolean isDeleteType(Cell cell) {
500 return cell.getTypeByte() == Type.Delete.getCode();
501 }
502
503 public static boolean isDeleteFamily(final Cell cell) {
504 return cell.getTypeByte() == Type.DeleteFamily.getCode();
505 }
506
507 public static boolean isDeleteFamilyVersion(final Cell cell) {
508 return cell.getTypeByte() == Type.DeleteFamilyVersion.getCode();
509 }
510
511 public static boolean isDeleteColumns(final Cell cell) {
512 return cell.getTypeByte() == Type.DeleteColumn.getCode();
513 }
514
515 public static boolean isDeleteColumnVersion(final Cell cell) {
516 return cell.getTypeByte() == Type.Delete.getCode();
517 }
518
519
520
521
522
523 public static boolean isDeleteColumnOrFamily(Cell cell) {
524 int t = cell.getTypeByte();
525 return t == Type.DeleteColumn.getCode() || t == Type.DeleteFamily.getCode();
526 }
527
528
529
530
531
532
533 @Deprecated
534 public static int estimatedSizeOf(final Cell cell) {
535 return estimatedSerializedSizeOf(cell);
536 }
537
538
539
540
541
542 public static int estimatedSerializedSizeOf(final Cell cell) {
543
544 if (cell instanceof KeyValue) {
545 return ((KeyValue)cell).getLength() + Bytes.SIZEOF_INT;
546 }
547
548
549 return getSumOfCellElementLengths(cell) +
550
551
552 KeyValue.KEY_INFRASTRUCTURE_SIZE +
553
554 Bytes.SIZEOF_INT;
555 }
556
557
558
559
560
561 private static int getSumOfCellElementLengths(final Cell cell) {
562 return getSumOfCellKeyElementLengths(cell) + cell.getValueLength() + cell.getTagsLength();
563 }
564
565
566
567
568
569
570 private static int getSumOfCellKeyElementLengths(final Cell cell) {
571 return cell.getRowLength() + cell.getFamilyLength() +
572 cell.getQualifierLength() +
573 KeyValue.TIMESTAMP_TYPE_SIZE;
574 }
575
576 public static int estimatedSerializedSizeOfKey(final Cell cell) {
577 if (cell instanceof KeyValue) return ((KeyValue)cell).getKeyLength();
578
579 return getSumOfCellKeyElementLengths(cell);
580 }
581
582
583
584
585
586
587
588
589
590 public static long estimatedHeapSizeOf(final Cell cell) {
591 if (cell instanceof HeapSize) {
592 return ((HeapSize) cell).heapSize();
593 }
594
595 return estimatedSerializedSizeOf(cell);
596 }
597
598
599
600
601
602
603
604
605 @Deprecated
606 public static long estimatedHeapSizeOfWithoutTags(final Cell cell) {
607 if (cell instanceof KeyValue) {
608 return ((KeyValue)cell).heapSizeWithoutTags();
609 }
610 return getSumOfCellKeyElementLengths(cell) + cell.getValueLength();
611 }
612
613
614
615
616
617
618
619
620
621
622 public static Iterator<Tag> tagsIterator(final byte[] tags, final int offset, final int length) {
623 return new Iterator<Tag>() {
624 private int pos = offset;
625 private int endOffset = offset + length - 1;
626
627 @Override
628 public boolean hasNext() {
629 return this.pos < endOffset;
630 }
631
632 @Override
633 public Tag next() {
634 if (hasNext()) {
635 int curTagLen = Bytes.readAsInt(tags, this.pos, Tag.TAG_LENGTH_SIZE);
636 Tag tag = new Tag(tags, pos, curTagLen + Tag.TAG_LENGTH_SIZE);
637 this.pos += Bytes.SIZEOF_SHORT + curTagLen;
638 return tag;
639 }
640 return null;
641 }
642
643 @Override
644 public void remove() {
645 throw new UnsupportedOperationException();
646 }
647 };
648 }
649
650
651
652
653
654 public static boolean overlappingKeys(final byte[] start1, final byte[] end1,
655 final byte[] start2, final byte[] end2) {
656 return (end2.length == 0 || start1.length == 0 || Bytes.compareTo(start1,
657 end2) < 0)
658 && (end1.length == 0 || start2.length == 0 || Bytes.compareTo(start2,
659 end1) < 0);
660 }
661
662
663
664
665
666
667
668
669
670 @InterfaceAudience.Private
671 public static void setSequenceId(Cell cell, long seqId) throws IOException {
672 if (cell instanceof SettableSequenceId) {
673 ((SettableSequenceId) cell).setSequenceId(seqId);
674 } else {
675 throw new IOException(new UnsupportedOperationException("Cell is not of type "
676 + SettableSequenceId.class.getName()));
677 }
678 }
679
680
681
682
683
684
685
686 public static void setTimestamp(Cell cell, long ts) throws IOException {
687 if (cell instanceof SettableTimestamp) {
688 ((SettableTimestamp) cell).setTimestamp(ts);
689 } else {
690 throw new IOException(new UnsupportedOperationException("Cell is not of type "
691 + SettableTimestamp.class.getName()));
692 }
693 }
694
695
696
697
698
699
700
701
702 public static void setTimestamp(Cell cell, byte[] ts, int tsOffset) throws IOException {
703 if (cell instanceof SettableTimestamp) {
704 ((SettableTimestamp) cell).setTimestamp(ts, tsOffset);
705 } else {
706 throw new IOException(new UnsupportedOperationException("Cell is not of type "
707 + SettableTimestamp.class.getName()));
708 }
709 }
710
711
712
713
714
715
716
717
718
719 public static boolean updateLatestStamp(Cell cell, long ts) throws IOException {
720 if (cell.getTimestamp() == HConstants.LATEST_TIMESTAMP) {
721 setTimestamp(cell, ts);
722 return true;
723 }
724 return false;
725 }
726
727
728
729
730
731
732
733
734
735
736 public static boolean updateLatestStamp(Cell cell, byte[] ts, int tsOffset) throws IOException {
737 if (cell.getTimestamp() == HConstants.LATEST_TIMESTAMP) {
738 setTimestamp(cell, ts, tsOffset);
739 return true;
740 }
741 return false;
742 }
743
744
745
746
747
748
749
750
751
752 public static void writeFlatKey(Cell cell, DataOutputStream out) throws IOException {
753 short rowLen = cell.getRowLength();
754 out.writeShort(rowLen);
755 out.write(cell.getRowArray(), cell.getRowOffset(), rowLen);
756 byte fLen = cell.getFamilyLength();
757 out.writeByte(fLen);
758 out.write(cell.getFamilyArray(), cell.getFamilyOffset(), fLen);
759 out.write(cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength());
760 out.writeLong(cell.getTimestamp());
761 out.writeByte(cell.getTypeByte());
762 }
763
764
765
766
767
768 public static String getCellKeyAsString(Cell cell) {
769 StringBuilder sb = new StringBuilder(Bytes.toStringBinary(
770 cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()));
771 sb.append('/');
772 sb.append(cell.getFamilyLength() == 0? "":
773 Bytes.toStringBinary(cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength()));
774
775 if (cell.getFamilyLength() > 0) sb.append(':');
776 sb.append(cell.getQualifierLength() == 0? "":
777 Bytes.toStringBinary(cell.getQualifierArray(), cell.getQualifierOffset(),
778 cell.getQualifierLength()));
779 sb.append('/');
780 sb.append(KeyValue.humanReadableTimestamp(cell.getTimestamp()));
781 sb.append('/');
782 sb.append(Type.codeToType(cell.getTypeByte()));
783 sb.append("/vlen=");
784 sb.append(cell.getValueLength());
785 sb.append("/seqid=");
786 sb.append(cell.getSequenceId());
787 return sb.toString();
788 }
789
790
791
792
793
794
795
796
797
798 public static byte [] getCellKeySerializedAsKeyValueKey(final Cell cell) {
799 if (cell == null) return null;
800 byte [] b = new byte[KeyValueUtil.keyLength(cell)];
801 KeyValueUtil.appendKeyTo(cell, b, 0);
802 return b;
803 }
804
805
806
807
808
809
810
811
812
813 public static void writeRowKeyExcludingCommon(Cell cell, short rLen, int commonPrefix,
814 DataOutputStream out) throws IOException {
815 if (commonPrefix == 0) {
816 out.writeShort(rLen);
817 } else if (commonPrefix == 1) {
818 out.writeByte((byte) rLen);
819 commonPrefix--;
820 } else {
821 commonPrefix -= KeyValue.ROW_LENGTH_SIZE;
822 }
823 if (rLen > commonPrefix) {
824 out.write(cell.getRowArray(), cell.getRowOffset() + commonPrefix, rLen - commonPrefix);
825 }
826 }
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843 public static int findCommonPrefixInFlatKey(Cell c1, Cell c2, boolean bypassFamilyCheck,
844 boolean withTsType) {
845
846 short rLen1 = c1.getRowLength();
847 short rLen2 = c2.getRowLength();
848 int commonPrefix = KeyValue.ROW_LENGTH_SIZE;
849 if (rLen1 != rLen2) {
850
851 return ByteBufferUtils.findCommonPrefix(Bytes.toBytes(rLen1), 0, KeyValue.ROW_LENGTH_SIZE,
852 Bytes.toBytes(rLen2), 0, KeyValue.ROW_LENGTH_SIZE);
853 }
854
855 int rkCommonPrefix = ByteBufferUtils.findCommonPrefix(c1.getRowArray(), c1.getRowOffset(),
856 rLen1, c2.getRowArray(), c2.getRowOffset(), rLen2);
857 commonPrefix += rkCommonPrefix;
858 if (rkCommonPrefix != rLen1) {
859
860 return commonPrefix;
861 }
862
863 byte fLen1 = c1.getFamilyLength();
864 if (bypassFamilyCheck) {
865
866
867 commonPrefix += KeyValue.FAMILY_LENGTH_SIZE + fLen1;
868 } else {
869 byte fLen2 = c2.getFamilyLength();
870 if (fLen1 != fLen2) {
871
872 return commonPrefix;
873 }
874
875 commonPrefix += KeyValue.FAMILY_LENGTH_SIZE;
876
877 int fCommonPrefix = ByteBufferUtils.findCommonPrefix(c1.getFamilyArray(),
878 c1.getFamilyOffset(), fLen1, c2.getFamilyArray(), c2.getFamilyOffset(), fLen2);
879 commonPrefix += fCommonPrefix;
880 if (fCommonPrefix != fLen1) {
881 return commonPrefix;
882 }
883 }
884
885 int qLen1 = c1.getQualifierLength();
886 int qLen2 = c2.getQualifierLength();
887 int qCommon = ByteBufferUtils.findCommonPrefix(c1.getQualifierArray(), c1.getQualifierOffset(),
888 qLen1, c2.getQualifierArray(), c2.getQualifierOffset(), qLen2);
889 commonPrefix += qCommon;
890 if (!withTsType || Math.max(qLen1, qLen2) != qCommon) {
891 return commonPrefix;
892 }
893
894 int tsCommonPrefix = ByteBufferUtils.findCommonPrefix(Bytes.toBytes(c1.getTimestamp()), 0,
895 KeyValue.TIMESTAMP_SIZE, Bytes.toBytes(c2.getTimestamp()), 0, KeyValue.TIMESTAMP_SIZE);
896 commonPrefix += tsCommonPrefix;
897 if (tsCommonPrefix != KeyValue.TIMESTAMP_SIZE) {
898 return commonPrefix;
899 }
900
901 if (c1.getTypeByte() == c2.getTypeByte()) {
902 commonPrefix += KeyValue.TYPE_SIZE;
903 }
904 return commonPrefix;
905 }
906
907
908 public static String toString(Cell cell, boolean verbose) {
909 if (cell == null) {
910 return "";
911 }
912 StringBuilder builder = new StringBuilder();
913 String keyStr = getCellKeyAsString(cell);
914
915 String tag = null;
916 String value = null;
917 if (verbose) {
918
919 tag = Bytes.toStringBinary(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength());
920 value = Bytes.toStringBinary(cell.getValueArray(), cell.getValueOffset(),
921 cell.getValueLength());
922 }
923
924 builder
925 .append(keyStr);
926 if (tag != null && !tag.isEmpty()) {
927 builder.append("/").append(tag);
928 }
929 if (value != null) {
930 builder.append("/").append(value);
931 }
932
933 return builder.toString();
934 }
935
936
937
938 public static boolean equals(Cell a, Cell b) {
939 return matchingRow(a, b) && matchingFamily(a, b) && matchingQualifier(a, b)
940 && matchingTimestamp(a, b) && matchingType(a, b);
941 }
942
943 public static boolean matchingType(Cell a, Cell b) {
944 return a.getTypeByte() == b.getTypeByte();
945 }
946 }