anoopsjohn commented on a change in pull request #2745:
URL: https://github.com/apache/hbase/pull/2745#discussion_r541510708
##########
File path:
hbase-client/src/test/java/org/apache/hadoop/hbase/shaded/protobuf/TestProtobufUtil.java
##########
@@ -479,4 +488,59 @@ public void testRegionLockInfo() {
+ "\"sharedLockCount\":0"
+ "}]", lockJson);
}
+
+ /**
+ * Test {@link ProtobufUtil#toCell(Cell, boolean)} and
+ * {@link ProtobufUtil#toCell(ExtendedCellBuilder, CellProtos.Cell,
boolean)} conversion
+ * methods when it contains tags and encode/decode tags is set to true.
+ */
+ @Test
+ public void testCellConversionWithTags() {
+
+ Cell cell = getCellWithTags();
+ CellProtos.Cell protoCell = ProtobufUtil.toCell(cell, true);
+ assertNotNull(protoCell);
+
+ Cell decodedCell = getCellFromProtoResult(protoCell, true);
+ List<Tag> decodedTags = PrivateCellUtil.getTags(decodedCell);
+ assertEquals(1, decodedTags.size());
+ Tag decodedTag = decodedTags.get(0);
+ assertEquals(TAG_TYPE, decodedTag.getType());
+ assertEquals(TAG_STR, Tag.getValueAsString(decodedTag));
+ }
+
+ private Cell getCellWithTags() {
+ Tag tag = new ArrayBackedTag(TAG_TYPE, TAG_STR);
+ ExtendedCellBuilder cellBuilder =
ExtendedCellBuilderFactory.create(CellBuilderType.DEEP_COPY);
+ cellBuilder.setRow(Bytes.toBytes("row1"));
+ cellBuilder.setFamily(Bytes.toBytes("f1"));
+ cellBuilder.setQualifier(Bytes.toBytes("q1"));
+ cellBuilder.setValue(Bytes.toBytes("value1"));
+ cellBuilder.setType(Cell.Type.Delete);
+ cellBuilder.setTags(Collections.singletonList(tag));
+ return cellBuilder.build();
+ }
+
+ private Cell getCellFromProtoResult(CellProtos.Cell protoCell, boolean
decodeTags) {
+ ExtendedCellBuilder decodedBuilder =
+ ExtendedCellBuilderFactory.create(CellBuilderType.DEEP_COPY);
+ return ProtobufUtil.toCell(decodedBuilder, protoCell, decodeTags);
+ }
+
+ /**
+ * Test {@link ProtobufUtil#toCell(Cell, boolean)} and
+ * {@link ProtobufUtil#toCell(ExtendedCellBuilder, CellProtos.Cell,
boolean)} conversion
+ * methods when it contains tags and encode/decode tags is set to false.
+ */
+ @Test
+ public void testCellConversionWithoutTags() {
+ Cell cell = getCellWithTags();
+ CellProtos.Cell protoCell =
+ ProtobufUtil.toCell(cell, false);
+ assertNotNull(protoCell);
+
+ Cell decodedCell = getCellFromProtoResult(protoCell, false);
Review comment:
Can you also add getCellFromProtoResult(protoCell, true ); and assert
not tags come back. This clearly assert when POJO is encoded to PB object no
tags been encoded.
##########
File path:
hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java
##########
@@ -801,4 +820,207 @@ public boolean isWALVisited() {
return isVisited;
}
}
+
+ /**
+ * Add cell tags to delete mutations, run export and import tool and
+ * verify that tags are present in import table also.
+ * @throws Throwable throws Throwable.
+ */
+ @Test
+ public void testTagsAddition() throws Throwable {
+ final TableName exportTable = TableName.valueOf(name.getMethodName());
+ TableDescriptor desc = TableDescriptorBuilder
+ .newBuilder(exportTable)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILYA)
+ .setMaxVersions(5)
+ .setKeepDeletedCells(KeepDeletedCells.TRUE)
+ .build())
+ .setCoprocessor(MetadataController.class.getName())
+ .build();
+ UTIL.getAdmin().createTable(desc);
+
+ Table exportT = UTIL.getConnection().getTable(exportTable);
+
+ //Add first version of QUAL
+ Put p = new Put(ROW1);
+ p.addColumn(FAMILYA, QUAL, now, QUAL);
+ exportT.put(p);
+
+ //Add Delete family marker
+ Delete d = new Delete(ROW1, now+3);
+ // Add test attribute to delete mutation.
+ d.setAttribute(TEST_ATTR, Bytes.toBytes(TEST_TAG));
+ exportT.delete(d);
+
+ // Run export too with KeyValueCodecWithTags as Codec. This will ensure
that export tool
+ // will use KeyValueCodecWithTags.
+ String[] args = new String[] {
+ "-D" + ExportUtils.RAW_SCAN + "=true",
+ // This will make sure that codec will encode and decode tags in rpc
call.
+
"-Dhbase.client.rpc.codec=org.apache.hadoop.hbase.codec.KeyValueCodecWithTags",
Review comment:
If we have entry in book regarding Export tool, we can add this detail
also there about changing Codec. Can be done in another issue too.
##########
File path:
hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java
##########
@@ -801,4 +820,207 @@ public boolean isWALVisited() {
return isVisited;
}
}
+
+ /**
+ * Add cell tags to delete mutations, run export and import tool and
+ * verify that tags are present in import table also.
+ * @throws Throwable throws Throwable.
+ */
+ @Test
+ public void testTagsAddition() throws Throwable {
+ final TableName exportTable = TableName.valueOf(name.getMethodName());
+ TableDescriptor desc = TableDescriptorBuilder
+ .newBuilder(exportTable)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILYA)
+ .setMaxVersions(5)
+ .setKeepDeletedCells(KeepDeletedCells.TRUE)
+ .build())
+ .setCoprocessor(MetadataController.class.getName())
+ .build();
+ UTIL.getAdmin().createTable(desc);
+
+ Table exportT = UTIL.getConnection().getTable(exportTable);
+
+ //Add first version of QUAL
+ Put p = new Put(ROW1);
+ p.addColumn(FAMILYA, QUAL, now, QUAL);
+ exportT.put(p);
+
+ //Add Delete family marker
+ Delete d = new Delete(ROW1, now+3);
+ // Add test attribute to delete mutation.
+ d.setAttribute(TEST_ATTR, Bytes.toBytes(TEST_TAG));
+ exportT.delete(d);
+
+ // Run export too with KeyValueCodecWithTags as Codec. This will ensure
that export tool
Review comment:
export tool
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]