Hi everyone,
I'm not sure whether this can be considered a Tephra issue or not.
I was doing some testing on Tephra and tried inserting a row (with a
column "f:a") inside a tx context and then delete it (the whole row)
outside of a tx context. What happens is that the TransactionProcessor
replaces the delete with a delete marker instead and uses a somehow
weird timestamp for that delete marker (as we are not in a tx). As such
we get something like this on HBase:
scan 'mcapitao_tests:transaction', {VERSIONS => 3}
ROW COLUMN+CELL
key00002 column=f:, timestamp=1510579373222, value=
key00002 column=f:a, timestamp=1510579373107000000, value=\xFF
1 row(s) in 0.3170 seconds
When I do a get of that row inside of a tx context I get not only the
"f:a" column but also the "f:" delete marker (which I'm not sure is the
intended behaviour).
If I do a similar test but deleting only the "f:a" column I get
something like this on HBase:
scan 'mcapitao_tests:transaction', {VERSIONS => 3}
ROW COLUMN+CELL
key00002 column=f:a, timestamp=1510580964444000000, value=\xFF
key00002 column=f:a, timestamp=1510580964573, value=
1 row(s) in 0.3150 seconds
Doing a transactional get on it I retrieve the "f:a" value before the
non-tx delete which is the expected behaviour given the timestamps.
I know this behaviour is on Tephra misusage but that was exactly my
point with the testing.
My questions are:
(1) Should we be getting the delete marker on the case we delete
all the columns outside of a transaction? Depending on the use case, the
appearance of that empty CQ may cause problems on client code.
(2) Are the timestamps used inside of transactional contexts bigger
that the ones outside them just because of extra precision needed by the
tx machinery or are they bigger also to take into account this kind of
misusage? Because this way what is done inside a tx has priority over
what is done outside?
Bellow you can find the test code I was using:
@Test
public void testD1() throws IOException {
final TxConstants.ConflictDetection conflictDetection =
TxConstants.ConflictDetection.ROW;
final String table = "mcapitao_tests:transaction";
final Configuration conf = HBaseConfiguration.create();
final HConnection conn = HConnectionManager.createConnection(conf);
TransactionManager transactionManager = new
TransactionManager(conf);
transactionManager.startAndWait();
TransactionSystemClient client = new
InMemoryTxSystemClient(transactionManager);
Table nonTxTable = conn.getTable(table);
TransactionAwareHTable txTable = new
TransactionAwareHTable(conn.getTable(table), conflictDetection, false);
TransactionContext context = new TransactionContext(client,
txTable);
try {
context.start();
txTable.put(
new Put(Bytes.toBytes("key00002"))
.addColumn(Bytes.toBytes("f"),
Bytes.toBytes("a"), Bytes.toBytes(true))
);
context.finish();
} catch (Exception e) {
LOGGER.error("Transaction has failed executing.", e);
try {
context.abort();
} catch (TransactionFailureException e1) {
throw new RuntimeException("could not rollback the
transaction", e1);
}
}
// Delete the row.
context = new TransactionContext(client, txTable);
try {
context.start();
// [1] transactional delete; [2] non transactional delete.
Leave only one uncommented.
//txTable.delete(new Delete(Bytes.toBytes("key00002")));
nonTxTable.delete(new
Delete(Bytes.toBytes("key00002")));//.addColumn(Bytes.toBytes("f"),
Bytes.toBytes("a")));
context.finish();
} catch (Exception e) {
throw new RuntimeException("don't care about exceptions
here", e);
}
// Read the just deleted row.
context = new TransactionContext(client, txTable);
Result result;
try {
context.start();
result = txTable.get(new Get(Bytes.toBytes("key00002")));
context.finish();
} catch (Exception e) {
throw new RuntimeException("don't care about exceptions
here", e);
}
nonTxTable.close();
txTable.close();
conn.close();
Assert.assertTrue("expecting result to be empty",
result.isEmpty());
}
Regards,
Micael Capitão