You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@tephra.apache.org by Micael Capitão <mi...@xpand-it.com> on 2017/11/13 14:08:54 UTC
Delete markers are returned from a regular Get/Scan
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
Re: Delete markers are returned from a regular Get/Scan
Posted by Andreas Neumann <an...@apache.org>.
I looked at the coprocessor code a little more and I think it is here:
https://github.com/apache/incubator-tephra/blob/master/tephra-hbase-compat-1.0/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionProcessor.java#L218
where we should also return immediately when (tx == null). What do you
think?
On Tue, Nov 14, 2017 at 2:26 PM, Andreas Neumann <an...@apache.org> wrote:
> Hi Micael,
>
> mixing transactional and non-transactional writes in the same table is
> risky, and you spotted one of the issues with that. Tephra inserts a
> "family delete marker" when you delete the entire row. It needs to do that
> for the same reason that it inserts a delete marker (an empty cell value)
> when you delete a cell: HBase's built-in delete markers cannot be rolled
> back.
>
> (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.
>
> Agreed that for a non-transactional family delete, the coprocessor should
> not get in the way, it could just let HBase handle it. Not sure why it is
> doing that, it might be a bug. The reason it is returned as part of a
> transactional read is most likely that its timestamp is so small that
> Tephra assumes it is a non-transactional write that was performed on the
> table before Tephra was enabled on it. Would you want to contribute a patch
> to fix this?
>
> (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?
>
> Yes, the timestamps in Tephra are significantly greater than HBase's
> (current time in milliseconds) timestamps. For Tephra to work consistently,
> it is essential that transaction ids are monotonic ascending over time.
> This is to guarantee visibility correctness. The way Tephra generates these
> monotonic transaction ids is that it takes the current time in millis,
> multiplies by 100000, and uses the lower bits to generate unique ids within
> the milli second (we could create more than one tx per ms).
>
> Cheers -Andreas.
>
>
> On Mon, Nov 13, 2017 at 6:08 AM, Micael Capitão <
> micael.capitao@xpand-it.com> wrote:
>
>> 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.createConne
>> ction(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
>>
>>
>>
>>
>>
>
Re: Delete markers are returned from a regular Get/Scan
Posted by Andreas Neumann <an...@apache.org>.
Hi Micael,
mixing transactional and non-transactional writes in the same table is
risky, and you spotted one of the issues with that. Tephra inserts a
"family delete marker" when you delete the entire row. It needs to do that
for the same reason that it inserts a delete marker (an empty cell value)
when you delete a cell: HBase's built-in delete markers cannot be rolled
back.
(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.
Agreed that for a non-transactional family delete, the coprocessor should
not get in the way, it could just let HBase handle it. Not sure why it is
doing that, it might be a bug. The reason it is returned as part of a
transactional read is most likely that its timestamp is so small that
Tephra assumes it is a non-transactional write that was performed on the
table before Tephra was enabled on it. Would you want to contribute a patch
to fix this?
(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?
Yes, the timestamps in Tephra are significantly greater than HBase's
(current time in milliseconds) timestamps. For Tephra to work consistently,
it is essential that transaction ids are monotonic ascending over time.
This is to guarantee visibility correctness. The way Tephra generates these
monotonic transaction ids is that it takes the current time in millis,
multiplies by 100000, and uses the lower bits to generate unique ids within
the milli second (we could create more than one tx per ms).
Cheers -Andreas.
On Mon, Nov 13, 2017 at 6:08 AM, Micael Capitão <micael.capitao@xpand-it.com
> wrote:
> 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.createConne
> ction(conf);
> TransactionManager transactionManager = new
> TransactionManager(conf);
> transactionManager.startAndWait();
> TransactionSystemClient client = new InMemoryTxSystemClient(transac
> tionManager);
>
> 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
>
>
>
>
>