HBASE-25827 Per Cell TTL tags get duplicated with increments causing tags length overflow (#3210)

Signed-off-by: Viraj Jasani <vjasani@apache.org>
Signed-off-by: Aman Poonia <apoonia@salesforce.com>
This commit is contained in:
Rushabh Shah 2021-05-21 13:26:11 -04:00 committed by GitHub
parent a1177b3e91
commit dfa88e1ffe
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
2 changed files with 55 additions and 1 deletions

View File

@ -4106,9 +4106,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
Function<Cell, byte[]> supplier) throws IOException {
// Forward any tags found on the delta.
List<Tag> tags = TagUtil.carryForwardTags(delta);
tags = TagUtil.carryForwardTTLTag(tags, mutation.getTTL());
if (currentCell != null) {
tags = TagUtil.carryForwardTags(tags, currentCell);
tags = TagUtil.carryForwardTTLTag(tags, mutation.getTTL());
byte[] newValue = supplier.apply(currentCell);
return ExtendedCellBuilderFactory.create(CellBuilderType.SHALLOW_COPY)
.setRow(mutation.getRow(), 0, mutation.getRow().length)
@ -4121,6 +4121,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
.setTags(TagUtil.fromList(tags))
.build();
} else {
tags = TagUtil.carryForwardTTLTag(tags, mutation.getTTL());
PrivateCellUtil.updateLatestStamp(delta, now);
return CollectionUtils.isEmpty(tags) ? delta : PrivateCellUtil.createCell(delta, tags);
}

View File

@ -17,6 +17,8 @@
*/
package org.apache.hadoop.hbase.client;
import static org.apache.hadoop.hbase.HConstants.RPC_CODEC_CONF_KEY;
import static org.apache.hadoop.hbase.ipc.RpcClient.DEFAULT_CODEC_CLASS;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertNotNull;
@ -38,7 +40,11 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.PrivateCellUtil;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Tag;
import org.apache.hadoop.hbase.TagType;
import org.apache.hadoop.hbase.codec.KeyValueCodecWithTags;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint;
import org.apache.hadoop.hbase.testclassification.LargeTests;
@ -547,4 +553,51 @@ public class TestIncrementsFromClientSide {
public static String filterStringSoTableNameSafe(final String str) {
return str.replaceAll("\\[fast\\=(.*)\\]", ".FAST.is.$1");
}
/*
Test that we have only 1 ttl tag with increment mutation.
*/
@Test
public void testIncrementWithTtlTags() throws Exception {
LOG.info("Starting " + this.name.getMethodName());
final TableName tableName =
TableName.valueOf(filterStringSoTableNameSafe(this.name.getMethodName()));
Table ht = TEST_UTIL.createTable(tableName, FAMILY);
final byte[] COLUMN = Bytes.toBytes("column");
Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
// Set RPC_CODEC_CONF_KEY to KeyValueCodecWithTags so that scan will return tags.
conf.set(RPC_CODEC_CONF_KEY, KeyValueCodecWithTags.class.getName());
conf.set(DEFAULT_CODEC_CLASS, "");
try (Connection connection = ConnectionFactory.createConnection(conf);
Table table = connection.getTable(tableName)) {
for (int i = 0; i < 10; i++) {
Increment inc = new Increment(ROW);
inc.addColumn(FAMILY, COLUMN, 1);
long ttl = i + 3600000 ;
inc.setTTL(ttl);
ht.increment(inc);
Scan scan = new Scan().withStartRow(ROW);
ResultScanner scanner = table.getScanner(scan);
int count = 0;
Result result;
while ((result = scanner.next()) != null) {
Cell[] cells = result.rawCells();
for (Cell cell: cells) {
List<Tag> tags = PrivateCellUtil.getTags(cell);
// Make sure there is only 1 tag.
assertEquals(1, tags.size());
Tag tag = tags.get(0);
assertEquals(TagType.TTL_TAG_TYPE, tag.getType());
long ttlTagValue = Bytes.toLong(tag.getValueArray(), tag.getValueOffset());
assertEquals(ttl, ttlTagValue);
}
count++;
}
// Make sure there is only 1 result.
assertEquals(1, count);
}
}
}
}