diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java index 6a1da0886b4..0f17ddf7733 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java @@ -1506,6 +1506,21 @@ public final class ProtobufUtil { * @return the converted protocol buffer Result */ public static ClientProtos.Result toResult(final Result result) { + return toResult(result, false); + } + + /** + * Convert a client Result to a protocol buffer Result + * @param result the client Result to convert + * @param encodeTags whether to includeTags in converted protobuf result or not + * When @encodeTags is set to true, it will return all the tags in the response. + * These tags may contain some sensitive data like acl permissions, etc. + * Only the tools like Export, Import which needs to take backup needs to set + * it to true so that cell tags are persisted in backup. + * Refer to HBASE-25246 for more context. + * @return the converted protocol buffer Result + */ + public static ClientProtos.Result toResult(final Result result, boolean encodeTags) { if (result.getExists() != null) { return toResult(result.getExists(), result.isStale()); } @@ -1517,7 +1532,7 @@ public final class ProtobufUtil { ClientProtos.Result.Builder builder = ClientProtos.Result.newBuilder(); for (Cell c : cells) { - builder.addCell(toCell(c)); + builder.addCell(toCell(c, encodeTags)); } builder.setStale(result.isStale()); @@ -1564,6 +1579,22 @@ public final class ProtobufUtil { * @return the converted client Result */ public static Result toResult(final ClientProtos.Result proto) { + return toResult(proto, false); + } + + /** + * Convert a protocol buffer Result to a client Result + * + * @param proto the protocol buffer Result to convert + * @param decodeTags whether to decode tags into converted client Result + * When @decodeTags is set to true, it will decode all the tags from the + * response. These tags may contain some sensitive data like acl permissions, + * etc. Only the tools like Export, Import which needs to take backup needs to + * set it to true so that cell tags are persisted in backup. + * Refer to HBASE-25246 for more context. + * @return the converted client Result + */ + public static Result toResult(final ClientProtos.Result proto, boolean decodeTags) { if (proto.hasExists()) { if (proto.getStale()) { return proto.getExists() ? EMPTY_RESULT_EXISTS_TRUE_STALE :EMPTY_RESULT_EXISTS_FALSE_STALE; @@ -1578,7 +1609,7 @@ public final class ProtobufUtil { List cells = new ArrayList(values.size()); for (CellProtos.Cell c : values) { - cells.add(toCell(c)); + cells.add(toCell(c, decodeTags)); } return Result.create(cells, null, proto.getStale(), proto.getPartial()); } @@ -1620,7 +1651,7 @@ public final class ProtobufUtil { if (!values.isEmpty()){ if (cells == null) cells = new ArrayList(values.size()); for (CellProtos.Cell c: values) { - cells.add(toCell(c)); + cells.add(toCell(c, false)); } } @@ -2802,7 +2833,7 @@ public final class ProtobufUtil { throw new IOException(se); } - public static CellProtos.Cell toCell(final Cell kv) { + public static CellProtos.Cell toCell(final Cell kv, boolean encodeTags) { // Doing this is going to kill us if we do it for all data passed. // St.Ack 20121205 CellProtos.Cell.Builder kvbuilder = CellProtos.Cell.newBuilder(); @@ -2816,18 +2847,27 @@ public final class ProtobufUtil { kvbuilder.setTimestamp(kv.getTimestamp()); kvbuilder.setValue(ByteStringer.wrap(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength())); + if (encodeTags && kv.getTagsLength() > 0) { + kvbuilder.setTags(ByteStringer.wrap(kv.getTagsArray(), kv.getTagsOffset(), + kv.getTagsLength())); + } return kvbuilder.build(); } - public static Cell toCell(final CellProtos.Cell cell) { + public static Cell toCell(final CellProtos.Cell cell, boolean decodeTags) { // Doing this is going to kill us if we do it for all data passed. // St.Ack 20121205 + byte[] tags = null; + if (decodeTags && cell.hasTags()) { + tags = cell.getTags().toByteArray(); + } return CellUtil.createCell(cell.getRow().toByteArray(), cell.getFamily().toByteArray(), cell.getQualifier().toByteArray(), cell.getTimestamp(), - (byte)cell.getCellType().getNumber(), - cell.getValue().toByteArray()); + KeyValue.Type.codeToType((byte)(cell.getCellType().getNumber())), + cell.getValue().toByteArray(), + tags); } public static HBaseProtos.NamespaceDescriptor toProtoNamespaceDescriptor(NamespaceDescriptor ns) { diff --git a/hbase-examples/src/test/java/org/apache/hadoop/hbase/types/TestPBCell.java b/hbase-examples/src/test/java/org/apache/hadoop/hbase/types/TestPBCell.java index 952a31971ae..f939c1d66a7 100644 --- a/hbase-examples/src/test/java/org/apache/hadoop/hbase/types/TestPBCell.java +++ b/hbase-examples/src/test/java/org/apache/hadoop/hbase/types/TestPBCell.java @@ -41,13 +41,13 @@ public class TestPBCell { public void testRoundTrip() { final Cell cell = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("qual"), Bytes.toBytes("val")); - CellProtos.Cell c = ProtobufUtil.toCell(cell), decoded; + CellProtos.Cell c = ProtobufUtil.toCell(cell, false), decoded; PositionedByteRange pbr = new SimplePositionedByteRange(c.getSerializedSize()); pbr.setPosition(0); int encodedLength = CODEC.encode(pbr, c); pbr.setPosition(0); decoded = CODEC.decode(pbr); assertEquals(encodedLength, pbr.getPosition()); - assertTrue(CellComparator.equals(cell, ProtobufUtil.toCell(decoded))); + assertTrue(CellComparator.equals(cell, ProtobufUtil.toCell(decoded, false))); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java index 8a80d15a3f3..466aeb7c172 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java @@ -45,6 +45,7 @@ import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; @@ -495,6 +496,10 @@ public class Import { // If there's a rename mapping for this CF, create a new KeyValue byte[] newCfName = cfRenameMap.get(CellUtil.cloneFamily(kv)); if(newCfName != null) { + List tags = null; + if (kv.getTagsLength() > 0) { + tags = Tag.asList(kv.getTagsArray(), kv.getTagsOffset(), kv.getTagsLength()); + } kv = new KeyValue(kv.getRowArray(), // row buffer kv.getRowOffset(), // row offset kv.getRowLength(), // row length @@ -508,7 +513,8 @@ public class Import { KeyValue.Type.codeToType(kv.getTypeByte()), // KV Type kv.getValueArray(), // value buffer kv.getValueOffset(), // value offset - kv.getValueLength()); // value length + kv.getValueLength(), // value length + tags); } } return kv; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ResultSerialization.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ResultSerialization.java index a267ca94b4b..d27dcc3dcbf 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ResultSerialization.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ResultSerialization.java @@ -132,7 +132,7 @@ public class ResultSerialization extends Configured implements Serialization values = new ArrayList<>(); + for (HRegion region : UTIL.getHBaseCluster().getRegions(table)) { + Scan scan = new Scan(); + // Make sure to set rawScan to true so that we will get Delete Markers. + scan.setRaw(true); + scan.setMaxVersions(); + scan.withStartRow(ROW1); + // Need to use RegionScanner instead of table#getScanner since the latter will + // not return tags since it will go through rpc layer and remove tags intentionally. + RegionScanner scanner = region.getScanner(scan); + scanner.next(values); + if (!values.isEmpty()) { + break; + } + } + boolean deleteFound = false; + for (Cell cell: values) { + if (CellUtil.isDelete(cell)) { + deleteFound = true; + List tags = Tag.asList(cell.getTagsArray(), cell.getTagsOffset(), + cell.getTagsLength()); + // If tagExists flag is true then validate whether tag contents are as expected. + if (tagExists) { + Assert.assertEquals(1, tags.size()); + for (Tag tag : tags) { + Assert.assertEquals(TEST_TAG, Bytes.toStringBinary(tag.getValue())); + } + } else { + // If tagExists flag is disabled then check for 0 size tags. + assertEquals(0, tags.size()); + } + } + } + Assert.assertTrue(deleteFound); + } + + /* + This co-proc will add a cell tag to delete mutation. + */ + public static class MetadataController + extends BaseRegionObserver /*implements CoprocessorService*/ { + @Override + public void preBatchMutate(ObserverContext c, + MiniBatchOperationInProgress miniBatchOp) throws IOException { + if (c.getEnvironment().getRegion().getRegionInfo().getTable().isSystemTable()) { + return; + } + + for (int i = 0; i < miniBatchOp.size(); i++) { + Mutation m = miniBatchOp.getOperation(i); + if (!(m instanceof Delete)) { + continue; + } + byte[] sourceOpAttr = m.getAttribute(TEST_ATTR); + if (sourceOpAttr == null) { + continue; + } + Tag sourceOpTag = new Tag(TEST_TAG_TYPE, sourceOpAttr); + List updatedCells = new ArrayList<>(); + for (CellScanner cellScanner = m.cellScanner(); cellScanner.advance(); ) { + Cell cell = cellScanner.current(); + List tags = Tag.asList(cell.getTagsArray(), cell.getTagsOffset(), + cell.getTagsLength()); + tags.add(sourceOpTag); + Cell updatedCell = new TagRewriteCell(cell, Tag.fromList(tags)); + updatedCells.add(updatedCell); + } + m.getFamilyCellMap().clear(); + // Clear and add new Cells to the Mutation. + for (Cell cell : updatedCells) { + Delete d = (Delete) m; + d.addDeleteMarker(cell); + } + } + } + } + + /** + * Set hbase.client.rpc.codec and hbase.client.default.rpc.codec both to empty string + * This means it will use no Codec. Make sure that we don't return Tags in response. + * @throws Exception Exception + */ + @Test + public void testTagsWithEmptyCodec() throws Exception { + final TableName tableName = TableName.valueOf("testTagsWithEmptyCodec"); + HTableDescriptor desc = new HTableDescriptor(tableName) + .addCoprocessor(MetadataController.class.getName()); + desc.addFamily(new HColumnDescriptor(FAMILYA) + .setMaxVersions(5) + .setKeepDeletedCells(true)); + + UTIL.getHBaseAdmin().createTable(desc); + Configuration conf = new Configuration(UTIL.getConfiguration()); + conf.set(RPC_CODEC_CONF_KEY, ""); + conf.set(DEFAULT_CODEC_CLASS, ""); + try (Connection connection = ConnectionFactory.createConnection(conf); + Table table = connection.getTable(tableName)) { + //Add first version of QUAL + Put p = new Put(ROW1); + p.addColumn(FAMILYA, QUAL, now, QUAL); + table.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)); + table.delete(d); + + // Since RPC_CODEC_CONF_KEY and DEFAULT_CODEC_CLASS is set to empty, it will use + // empty Codec and it shouldn't encode/decode tags. + Scan scan = new Scan().withStartRow(ROW1).setRaw(true); + ResultScanner scanner = table.getScanner(scan); + int count = 0; + Result result; + while ((result = scanner.next()) != null) { + List cells = result.listCells(); + assertEquals(2, cells.size()); + Cell cell = cells.get(0); + assertTrue(CellUtil.isDelete(cell)); + List tags = Tag.asList(cell.getTagsArray(), cell.getTagsOffset(), + cell.getTagsLength()); + assertEquals(0, tags.size()); + count++; + } + assertEquals(1, count); + } finally { + UTIL.deleteTable(tableName); + } + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java index ab6b2758d71..d50f0bb057c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java @@ -19,6 +19,8 @@ package org.apache.hadoop.hbase.protobuf; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + import com.google.protobuf.ByteString; import java.io.IOException; import java.util.Collections; @@ -29,12 +31,14 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.client.Append; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Increment; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.master.RegionState; +import org.apache.hadoop.hbase.protobuf.generated.CellProtos; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Column; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto; @@ -55,6 +59,9 @@ import org.junit.experimental.categories.Category; */ @Category(SmallTests.class) public class TestProtobufUtil { + private static final String TAG_STR = "tag-1"; + private static final byte TAG_TYPE = (byte)10; + @Test public void testException() throws IOException { NameBytesPair.Builder builder = NameBytesPair.newBuilder(); @@ -379,4 +386,91 @@ public class TestProtobufUtil { assertEquals(serverName, rs.getServerName()); assertEquals(rs.getState(), RegionState.State.OPEN); } + + /** + * Test {@link ProtobufUtil#toCell(Cell, boolean)} and + * {@link ProtobufUtil#toCell(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 decodedTags = Tag.asList(decodedCell.getTagsArray(), decodedCell.getTagsOffset(), + decodedCell.getTagsLength()); + assertEquals(1, decodedTags.size()); + Tag decodedTag = decodedTags.get(0); + assertEquals(TAG_TYPE, decodedTag.getType()); + assertEquals(TAG_STR, Bytes.toStringBinary(decodedTag.getValue())); + } + + private Cell getCellWithTags() { + Tag tag = new Tag(TAG_TYPE, TAG_STR); + KeyValue kv = new KeyValue(Bytes.toBytes("row1"), Bytes.toBytes("f1"), + Bytes.toBytes("q1"), 10L, Bytes.toBytes("value1"), new Tag[] {tag}); + return kv; + } + + private Cell getCellFromProtoResult(CellProtos.Cell protoCell, boolean decodeTags) { + return ProtobufUtil.toCell(protoCell, decodeTags); + } + + /** + * Test {@link ProtobufUtil#toCell(Cell, boolean)} and + * {@link ProtobufUtil#toCell(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); + List decodedTags = Tag.asList(decodedCell.getTagsArray(), decodedCell.getTagsOffset(), + decodedCell.getTagsLength()); + assertEquals(0, decodedTags.size()); + } + + /** + * Test {@link ProtobufUtil#toCell(Cell, boolean)} and + * {@link ProtobufUtil#toCell(CellProtos.Cell, boolean)} conversion + * methods when it contains tags and encoding of tags is set to false + * and decoding of tags is set to true. + */ + @Test + public void testTagEncodeFalseDecodeTrue() { + Cell cell = getCellWithTags(); + CellProtos.Cell protoCell = + ProtobufUtil.toCell(cell, false); + assertNotNull(protoCell); + + Cell decodedCell = getCellFromProtoResult(protoCell, true); + List decodedTags = Tag.asList(decodedCell.getTagsArray(), decodedCell.getTagsOffset(), + decodedCell.getTagsLength()); + assertEquals(0, decodedTags.size()); + } + + /** + * Test {@link ProtobufUtil#toCell(Cell, boolean)} and + * {@link ProtobufUtil#toCell(CellProtos.Cell, boolean)} conversion + * methods when it contains tags and encoding of tags is set to true + * and decoding of tags is set to false. + */ + @Test + public void testTagEncodeTrueDecodeFalse() { + Cell cell = getCellWithTags(); + CellProtos.Cell protoCell = + ProtobufUtil.toCell(cell, true); + assertNotNull(protoCell); + + Cell decodedCell = getCellFromProtoResult(protoCell, false); + List decodedTags = Tag.asList(decodedCell.getTagsArray(), decodedCell.getTagsOffset(), + decodedCell.getTagsLength()); + assertEquals(0, decodedTags.size()); + } } diff --git a/src/main/asciidoc/_chapters/ops_mgt.adoc b/src/main/asciidoc/_chapters/ops_mgt.adoc index b08e2699a93..5bd60c6eac9 100644 --- a/src/main/asciidoc/_chapters/ops_mgt.adoc +++ b/src/main/asciidoc/_chapters/ops_mgt.adoc @@ -591,6 +591,9 @@ $ bin/hbase org.apache.hadoop.hbase.mapreduce.Export [_* with the desired number of versions. +For mapreduce based Export, if you want to export cell tags then set the following config property +`hbase.client.rpc.codec` to `org.apache.hadoop.hbase.codec.KeyValueCodecWithTags` + Note: caching for the input Scan is configured via `hbase.client.scanner.caching` in the job configuration. === Import @@ -608,6 +611,9 @@ To import 0.94 exported files in a 0.96 cluster or onwards, you need to set syst $ bin/hbase -Dhbase.import.version=0.94 org.apache.hadoop.hbase.mapreduce.Import ---- +If you want to import cell tags then set the following config property +`hbase.client.rpc.codec` to `org.apache.hadoop.hbase.codec.KeyValueCodecWithTags` + === ImportTsv ImportTsv is a utility that will load data in TSV format into HBase.