HBASE-24302 Add an "ignoreTimestamps" option (defaulted to false) to … (#1623)

Signed-off-by: Jan Hentschel <jan.hentschel@ultratendency.com>
Signed-off-by: Josh Elser <elserj@apache.org>
This commit is contained in:
Wellington Ramos Chevreuil 2020-05-04 11:24:43 +01:00 committed by GitHub
parent 4f9eecbe61
commit 3d59e328c8
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
3 changed files with 148 additions and 61 deletions

View File

@ -73,6 +73,7 @@ public class HashTable extends Configured implements Tool {
final static String MANIFEST_FILE_NAME = "manifest";
final static String HASH_DATA_DIR = "hashes";
final static String OUTPUT_DATA_FILE_PREFIX = "part-r-";
final static String IGNORE_TIMESTAMPS = "ignoreTimestamps";
private final static String TMP_MANIFEST_FILE_NAME = "manifest.tmp";
TableHash tableHash = new TableHash();
@ -96,6 +97,7 @@ public class HashTable extends Configured implements Tool {
int versions = -1;
long startTime = 0;
long endTime = 0;
boolean ignoreTimestamps;
List<ImmutableBytesWritable> partitions;
@ -434,6 +436,7 @@ public class HashTable extends Configured implements Tool {
getConf().get("mapreduce.job.name", "hashTable_" + tableHash.tableName));
Configuration jobConf = job.getConfiguration();
jobConf.setLong(HASH_BATCH_SIZE_CONF_KEY, tableHash.batchSize);
jobConf.setBoolean(IGNORE_TIMESTAMPS, tableHash.ignoreTimestamps);
job.setJarByClass(HashTable.class);
TableMapReduceUtil.initTableMapperJob(tableHash.tableName, tableHash.initScan(),
@ -471,6 +474,7 @@ public class HashTable extends Configured implements Tool {
private ImmutableBytesWritable batchStartKey;
private ImmutableBytesWritable batchHash;
private long batchSize = 0;
boolean ignoreTimestamps;
public ResultHasher() {
@ -503,11 +507,14 @@ public class HashTable extends Configured implements Tool {
digest.update(cell.getRowArray(), cell.getRowOffset(), rowLength);
digest.update(cell.getFamilyArray(), cell.getFamilyOffset(), familyLength);
digest.update(cell.getQualifierArray(), cell.getQualifierOffset(), qualifierLength);
if (!ignoreTimestamps) {
long ts = cell.getTimestamp();
for (int i = 8; i > 0; i--) {
digest.update((byte) ts);
ts >>>= 8;
}
}
digest.update(cell.getValueArray(), cell.getValueOffset(), valueLength);
batchSize += rowLength + familyLength + qualifierLength + 8 + valueLength;
@ -552,7 +559,8 @@ public class HashTable extends Configured implements Tool {
targetBatchSize = context.getConfiguration()
.getLong(HASH_BATCH_SIZE_CONF_KEY, DEFAULT_BATCH_SIZE);
hasher = new ResultHasher();
hasher.ignoreTimestamps = context.getConfiguration().
getBoolean(IGNORE_TIMESTAMPS, false);
TableSplit split = (TableSplit) context.getInputSplit();
hasher.startBatch(new ImmutableBytesWritable(split.getStartRow()));
}
@ -614,10 +622,13 @@ public class HashTable extends Configured implements Tool {
System.err.println(" stoprow the stop row");
System.err.println(" starttime beginning of the time range (unixtime in millis)");
System.err.println(" without endtime means from starttime to forever");
System.err.println(" endtime end of the time range. Ignored if no starttime specified.");
System.err.println(" endtime end of the time range.");
System.err.println(" Ignored if no starttime specified.");
System.err.println(" scanbatch scanner batch size to support intra row scans");
System.err.println(" versions number of cell versions to include");
System.err.println(" families comma-separated list of families to include");
System.err.println(" ignoreTimestamps if true, ignores cell timestamps");
System.err.println(" when calculating hashes");
System.err.println();
System.err.println("Args:");
System.err.println(" tablename Name of the table to hash");
@ -702,6 +713,13 @@ public class HashTable extends Configured implements Tool {
continue;
}
final String ignoreTimestampsKey = "--ignoreTimestamps=";
if (cmd.startsWith(ignoreTimestampsKey)) {
tableHash.ignoreTimestamps = Boolean.
parseBoolean(cmd.substring(ignoreTimestampsKey.length()));
continue;
}
printUsage("Invalid argument '" + cmd + "'");
return false;
}

View File

@ -26,6 +26,8 @@ import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellBuilderFactory;
import org.apache.hadoop.hbase.CellBuilderType;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseConfiguration;
@ -67,6 +69,7 @@ public class SyncTable extends Configured implements Tool {
static final String DRY_RUN_CONF_KEY = "sync.table.dry.run";
static final String DO_DELETES_CONF_KEY = "sync.table.do.deletes";
static final String DO_PUTS_CONF_KEY = "sync.table.do.puts";
static final String IGNORE_TIMESTAMPS = "sync.table.ignore.timestamps";
Path sourceHashDir;
String sourceTableName;
@ -77,6 +80,7 @@ public class SyncTable extends Configured implements Tool {
boolean dryRun;
boolean doDeletes = true;
boolean doPuts = true;
boolean ignoreTimestamps;
Counters counters;
@ -150,6 +154,7 @@ public class SyncTable extends Configured implements Tool {
jobConf.setBoolean(DRY_RUN_CONF_KEY, dryRun);
jobConf.setBoolean(DO_DELETES_CONF_KEY, doDeletes);
jobConf.setBoolean(DO_PUTS_CONF_KEY, doPuts);
jobConf.setBoolean(IGNORE_TIMESTAMPS, ignoreTimestamps);
TableMapReduceUtil.initTableMapperJob(targetTableName, tableHash.initScan(),
SyncMapper.class, null, null, job);
@ -186,6 +191,7 @@ public class SyncTable extends Configured implements Tool {
boolean dryRun;
boolean doDeletes = true;
boolean doPuts = true;
boolean ignoreTimestamp;
HashTable.TableHash sourceTableHash;
HashTable.TableHash.Reader sourceHashReader;
@ -213,6 +219,7 @@ public class SyncTable extends Configured implements Tool {
dryRun = conf.getBoolean(DRY_RUN_CONF_KEY, false);
doDeletes = conf.getBoolean(DO_DELETES_CONF_KEY, true);
doPuts = conf.getBoolean(DO_PUTS_CONF_KEY, true);
ignoreTimestamp = conf.getBoolean(IGNORE_TIMESTAMPS, false);
sourceTableHash = HashTable.TableHash.read(conf, sourceHashDir);
LOG.info("Read source hash manifest: " + sourceTableHash);
@ -228,6 +235,7 @@ public class SyncTable extends Configured implements Tool {
// instead, find the first hash batch at or after the start row
// and skip any rows that come before. they will be caught by the previous task
targetHasher = new HashTable.ResultHasher();
targetHasher.ignoreTimestamps = ignoreTimestamp;
}
private static Connection openConnection(Configuration conf, String zkClusterConfKey,
@ -475,6 +483,23 @@ public class SyncTable extends Configured implements Tool {
}
}
private Cell checkAndResetTimestamp(Cell sourceCell){
if (ignoreTimestamp) {
sourceCell = CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY)
.setType(sourceCell.getType())
.setRow(sourceCell.getRowArray(),
sourceCell.getRowOffset(), sourceCell.getRowLength())
.setFamily(sourceCell.getFamilyArray(),
sourceCell.getFamilyOffset(), sourceCell.getFamilyLength())
.setQualifier(sourceCell.getQualifierArray(),
sourceCell.getQualifierOffset(), sourceCell.getQualifierLength())
.setTimestamp(System.currentTimeMillis())
.setValue(sourceCell.getValueArray(),
sourceCell.getValueOffset(), sourceCell.getValueLength()).build();
}
return sourceCell;
}
/**
* Compare the cells for the given row from the source and target tables.
* Count and log any differences.
@ -503,6 +528,7 @@ public class SyncTable extends Configured implements Tool {
if (put == null) {
put = new Put(rowKey);
}
sourceCell = checkAndResetTimestamp(sourceCell);
put.add(sourceCell);
}
@ -546,6 +572,7 @@ public class SyncTable extends Configured implements Tool {
if (put == null) {
put = new Put(rowKey);
}
sourceCell = checkAndResetTimestamp(sourceCell);
put.add(sourceCell);
}
}
@ -607,7 +634,7 @@ public class SyncTable extends Configured implements Tool {
* They are assumed to be of the same row.
* Nulls are after non-nulls.
*/
private static int compareCellKeysWithinRow(Cell c1, Cell c2) {
private int compareCellKeysWithinRow(Cell c1, Cell c2) {
if (c1 == null) {
return 1; // source missing cell
}
@ -625,9 +652,13 @@ public class SyncTable extends Configured implements Tool {
return result;
}
if (this.ignoreTimestamp) {
return 0;
} else {
// note timestamp comparison is inverted - more recent cells first
return CellComparator.getInstance().compareTimestamps(c1, c2);
}
}
@Override
protected void cleanup(Context context)
@ -726,6 +757,10 @@ public class SyncTable extends Configured implements Tool {
System.err.println(" (defaults to true)");
System.err.println(" doPuts if false, does not perform puts");
System.err.println(" (defaults to true)");
System.err.println(" ignoreTimestamps if true, ignores cells timestamps while comparing ");
System.err.println(" cell values. Any missing cell on target then gets");
System.err.println(" added with current time as timestamp ");
System.err.println(" (defaults to false)");
System.err.println();
System.err.println("Args:");
System.err.println(" sourcehashdir path to HashTable output dir for source table");
@ -789,6 +824,12 @@ public class SyncTable extends Configured implements Tool {
continue;
}
final String ignoreTimestampsKey = "--ignoreTimestamps=";
if (cmd.startsWith(ignoreTimestampsKey)) {
ignoreTimestamps = Boolean.parseBoolean(cmd.substring(ignoreTimestampsKey.length()));
continue;
}
printUsage("Invalid argument '" + cmd + "'");
return false;
}

View File

@ -20,6 +20,8 @@ package org.apache.hadoop.hbase.mapreduce;
import static org.junit.Assert.assertEquals;
import java.util.Arrays;
import org.apache.commons.lang3.ArrayUtils;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
@ -93,7 +95,7 @@ public class TestSyncTable {
writeTestData(sourceTableName, targetTableName);
hashSourceTable(sourceTableName, testDir);
Counters syncCounters = syncTables(sourceTableName, targetTableName, testDir);
assertEqualTables(90, sourceTableName, targetTableName);
assertEqualTables(90, sourceTableName, targetTableName, false);
assertEquals(60, syncCounters.findCounter(Counter.ROWSWITHDIFFS).getValue());
assertEquals(10, syncCounters.findCounter(Counter.SOURCEMISSINGROWS).getValue());
@ -152,8 +154,31 @@ public class TestSyncTable {
TEST_UTIL.deleteTable(targetTableName);
}
@Test
public void testSyncTableIgnoreTimestampsTrue() throws Exception {
final TableName sourceTableName = TableName.valueOf(name.getMethodName() + "_source");
final TableName targetTableName = TableName.valueOf(name.getMethodName() + "_target");
Path testDir = TEST_UTIL.getDataTestDirOnTestFS("testSyncTableIgnoreTimestampsTrue");
long current = System.currentTimeMillis();
writeTestData(sourceTableName, targetTableName, current - 1000, current);
hashSourceTable(sourceTableName, testDir, "--ignoreTimestamps=true");
Counters syncCounters = syncTables(sourceTableName, targetTableName,
testDir, "--ignoreTimestamps=true");
assertEqualTables(90, sourceTableName, targetTableName, true);
assertEquals(50, syncCounters.findCounter(Counter.ROWSWITHDIFFS).getValue());
assertEquals(10, syncCounters.findCounter(Counter.SOURCEMISSINGROWS).getValue());
assertEquals(10, syncCounters.findCounter(Counter.TARGETMISSINGROWS).getValue());
assertEquals(30, syncCounters.findCounter(Counter.SOURCEMISSINGCELLS).getValue());
assertEquals(30, syncCounters.findCounter(Counter.TARGETMISSINGCELLS).getValue());
assertEquals(20, syncCounters.findCounter(Counter.DIFFERENTCELLVALUES).getValue());
TEST_UTIL.deleteTable(sourceTableName);
TEST_UTIL.deleteTable(targetTableName);
}
private void assertEqualTables(int expectedRows, TableName sourceTableName,
TableName targetTableName) throws Exception {
TableName targetTableName, boolean ignoreTimestamps) throws Exception {
Table sourceTable = TEST_UTIL.getConnection().getTable(sourceTableName);
Table targetTable = TEST_UTIL.getConnection().getTable(targetTableName);
@ -200,7 +225,7 @@ public class TestSyncTable {
if (!CellUtil.matchingQualifier(sourceCell, targetCell)) {
Assert.fail("Qualifiers don't match");
}
if (!CellUtil.matchingTimestamp(sourceCell, targetCell)) {
if (!ignoreTimestamps && !CellUtil.matchingTimestamp(sourceCell, targetCell)) {
Assert.fail("Timestamps don't match");
}
if (!CellUtil.matchingValue(sourceCell, targetCell)) {
@ -426,18 +451,19 @@ public class TestSyncTable {
return syncTable.counters;
}
private void hashSourceTable(TableName sourceTableName, Path testDir) throws Exception {
private void hashSourceTable(TableName sourceTableName, Path testDir, String... options)
throws Exception {
int numHashFiles = 3;
long batchSize = 100; // should be 2 batches per region
int scanBatch = 1;
HashTable hashTable = new HashTable(TEST_UTIL.getConfiguration());
int code = hashTable.run(new String[] {
"--batchsize=" + batchSize,
"--numhashfiles=" + numHashFiles,
"--scanbatch=" + scanBatch,
sourceTableName.getNameAsString(),
testDir.toString()
});
String[] args = Arrays.copyOf(options, options.length+5);
args[options.length] = "--batchsize=" + batchSize;
args[options.length + 1] = "--numhashfiles=" + numHashFiles;
args[options.length + 2] = "--scanbatch=" + scanBatch;
args[options.length + 3] = sourceTableName.getNameAsString();
args[options.length + 4] = testDir.toString();
int code = hashTable.run(args);
assertEquals("hash table job failed", 0, code);
FileSystem fs = TEST_UTIL.getTestFileSystem();
@ -451,8 +477,8 @@ public class TestSyncTable {
LOG.info("Hash table completed");
}
private void writeTestData(TableName sourceTableName, TableName targetTableName)
throws Exception {
private void writeTestData(TableName sourceTableName, TableName targetTableName,
long... timestamps) throws Exception {
final byte[] family = Bytes.toBytes("family");
final byte[] column1 = Bytes.toBytes("c1");
final byte[] column2 = Bytes.toBytes("c2");
@ -463,6 +489,10 @@ public class TestSyncTable {
int numRows = 100;
int sourceRegions = 10;
int targetRegions = 6;
if (ArrayUtils.isEmpty(timestamps)) {
long current = System.currentTimeMillis();
timestamps = new long[]{current,current};
}
Table sourceTable = TEST_UTIL.createTable(sourceTableName,
family, generateSplits(numRows, sourceRegions));
@ -470,19 +500,17 @@ public class TestSyncTable {
Table targetTable = TEST_UTIL.createTable(targetTableName,
family, generateSplits(numRows, targetRegions));
long timestamp = 1430764183454L;
int rowIndex = 0;
// a bunch of identical rows
for (; rowIndex < 40; rowIndex++) {
Put sourcePut = new Put(Bytes.toBytes(rowIndex));
sourcePut.addColumn(family, column1, timestamp, value1);
sourcePut.addColumn(family, column2, timestamp, value2);
sourcePut.addColumn(family, column1, timestamps[0], value1);
sourcePut.addColumn(family, column2, timestamps[0], value2);
sourceTable.put(sourcePut);
Put targetPut = new Put(Bytes.toBytes(rowIndex));
targetPut.addColumn(family, column1, timestamp, value1);
targetPut.addColumn(family, column2, timestamp, value2);
targetPut.addColumn(family, column1, timestamps[1], value1);
targetPut.addColumn(family, column2, timestamps[1], value2);
targetTable.put(targetPut);
}
// some rows only in the source table
@ -491,8 +519,8 @@ public class TestSyncTable {
// TARGETMISSINGCELLS: 20
for (; rowIndex < 50; rowIndex++) {
Put put = new Put(Bytes.toBytes(rowIndex));
put.addColumn(family, column1, timestamp, value1);
put.addColumn(family, column2, timestamp, value2);
put.addColumn(family, column1, timestamps[0], value1);
put.addColumn(family, column2, timestamps[0], value2);
sourceTable.put(put);
}
// some rows only in the target table
@ -501,8 +529,8 @@ public class TestSyncTable {
// SOURCEMISSINGCELLS: 20
for (; rowIndex < 60; rowIndex++) {
Put put = new Put(Bytes.toBytes(rowIndex));
put.addColumn(family, column1, timestamp, value1);
put.addColumn(family, column2, timestamp, value2);
put.addColumn(family, column1, timestamps[1], value1);
put.addColumn(family, column2, timestamps[1], value2);
targetTable.put(put);
}
// some rows with 1 missing cell in target table
@ -510,12 +538,12 @@ public class TestSyncTable {
// TARGETMISSINGCELLS: 10
for (; rowIndex < 70; rowIndex++) {
Put sourcePut = new Put(Bytes.toBytes(rowIndex));
sourcePut.addColumn(family, column1, timestamp, value1);
sourcePut.addColumn(family, column2, timestamp, value2);
sourcePut.addColumn(family, column1, timestamps[0], value1);
sourcePut.addColumn(family, column2, timestamps[0], value2);
sourceTable.put(sourcePut);
Put targetPut = new Put(Bytes.toBytes(rowIndex));
targetPut.addColumn(family, column1, timestamp, value1);
targetPut.addColumn(family, column1, timestamps[1], value1);
targetTable.put(targetPut);
}
// some rows with 1 missing cell in source table
@ -523,12 +551,12 @@ public class TestSyncTable {
// SOURCEMISSINGCELLS: 10
for (; rowIndex < 80; rowIndex++) {
Put sourcePut = new Put(Bytes.toBytes(rowIndex));
sourcePut.addColumn(family, column1, timestamp, value1);
sourcePut.addColumn(family, column1, timestamps[0], value1);
sourceTable.put(sourcePut);
Put targetPut = new Put(Bytes.toBytes(rowIndex));
targetPut.addColumn(family, column1, timestamp, value1);
targetPut.addColumn(family, column2, timestamp, value2);
targetPut.addColumn(family, column1, timestamps[1], value1);
targetPut.addColumn(family, column2, timestamps[1], value2);
targetTable.put(targetPut);
}
// some rows differing only in timestamp
@ -537,13 +565,13 @@ public class TestSyncTable {
// TARGETMISSINGCELLS: 20
for (; rowIndex < 90; rowIndex++) {
Put sourcePut = new Put(Bytes.toBytes(rowIndex));
sourcePut.addColumn(family, column1, timestamp, column1);
sourcePut.addColumn(family, column2, timestamp, value2);
sourcePut.addColumn(family, column1, timestamps[0], column1);
sourcePut.addColumn(family, column2, timestamps[0], value2);
sourceTable.put(sourcePut);
Put targetPut = new Put(Bytes.toBytes(rowIndex));
targetPut.addColumn(family, column1, timestamp+1, column1);
targetPut.addColumn(family, column2, timestamp-1, value2);
targetPut.addColumn(family, column1, timestamps[1]+1, column1);
targetPut.addColumn(family, column2, timestamps[1]-1, value2);
targetTable.put(targetPut);
}
// some rows with different values
@ -551,13 +579,13 @@ public class TestSyncTable {
// DIFFERENTCELLVALUES: 20
for (; rowIndex < numRows; rowIndex++) {
Put sourcePut = new Put(Bytes.toBytes(rowIndex));
sourcePut.addColumn(family, column1, timestamp, value1);
sourcePut.addColumn(family, column2, timestamp, value2);
sourcePut.addColumn(family, column1, timestamps[0], value1);
sourcePut.addColumn(family, column2, timestamps[0], value2);
sourceTable.put(sourcePut);
Put targetPut = new Put(Bytes.toBytes(rowIndex));
targetPut.addColumn(family, column1, timestamp, value3);
targetPut.addColumn(family, column2, timestamp, value3);
targetPut.addColumn(family, column1, timestamps[1], value3);
targetPut.addColumn(family, column2, timestamps[1], value3);
targetTable.put(targetPut);
}