Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id E4DC2200D3D for ; Mon, 13 Nov 2017 18:17:08 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id E338A160BF3; Mon, 13 Nov 2017 17:17:08 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 11022160BF0 for ; Mon, 13 Nov 2017 18:17:07 +0100 (CET) Received: (qmail 9329 invoked by uid 500); 13 Nov 2017 17:17:07 -0000 Mailing-List: contact dev-help@tephra.incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@tephra.incubator.apache.org Delivered-To: mailing list dev@tephra.incubator.apache.org Delivered-To: moderator for dev@tephra.incubator.apache.org Received: (qmail 78764 invoked by uid 99); 13 Nov 2017 14:09:07 -0000 X-Virus-Scanned: Debian amavisd-new at spamd3-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -0.022 X-Spam-Level: X-Spam-Status: No, score=-0.022 tagged_above=-999 required=6.31 tests=[RCVD_IN_MSPIKE_H4=-0.01, RCVD_IN_MSPIKE_WL=-0.01, RP_MATCHES_RCVD=-0.001, SPF_PASS=-0.001] autolearn=disabled X-Virus-Scanned: amavisd-new at mail.xpand-it.com To: dev@tephra.incubator.apache.org From: =?UTF-8?Q?Micael_Capit=c3=a3o?= Subject: Delete markers are returned from a regular Get/Scan Message-ID: <3af996ba-9f99-8582-2b13-c1e0c01759e5@xpand-it.com> Date: Mon, 13 Nov 2017 14:08:54 +0000 User-Agent: Mozilla/5.0 (X11; Linux x86_64; rv:52.0) Gecko/20100101 Thunderbird/52.3.0 MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8; format=flowed Content-Language: pt-PT Content-Transfer-Encoding: quoted-printable archived-at: Mon, 13 Nov 2017 17:17:09 -0000 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=20 column "f:a") inside a tx context and then delete it (the whole row)=20 outside of a tx context. What happens is that the TransactionProcessor=20 replaces the delete with a delete marker instead and uses a somehow=20 weird timestamp for that delete marker (as we are not in a tx). As such=20 we get something like this on HBase: scan 'mcapitao_tests:transaction', {VERSIONS =3D> 3} ROW=C2=A0 COLUMN+CELL =C2=A0key00002 column=3Df:, timestamp=3D1510579373222, value=3D =C2=A0key00002 column=3Df:a, timestamp=3D1510579373107000000, value=3D\x= FF 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=20 "f:a" column but also the "f:" delete marker (which I'm not sure is the=20 intended behaviour). If I do a similar test but deleting only the "f:a" column I get=20 something like this on HBase: scan 'mcapitao_tests:transaction', {VERSIONS =3D> 3} ROW=C2=A0 COLUMN+CELL =C2=A0key00002 column=3Df:a, timestamp=3D1510580964444000000, value=3D\x= FF =C2=A0key00002 column=3Df:a, timestamp=3D1510580964573, value=3D 1 row(s) in 0.3150 seconds Doing a transactional get on it I retrieve the "f:a" value before the=20 non-tx delete which is the expected behaviour given the timestamps. I know this behaviour is on Tephra misusage but that was exactly my=20 point with the testing. My questions are: =C2=A0=C2=A0=C2=A0 (1) Should we be getting the delete marker on the cas= e we delete=20 all the columns outside of a transaction? Depending on the use case, the=20 appearance of that empty CQ may cause problems on client code. =C2=A0=C2=A0=C2=A0 (2) Are the timestamps used inside of transactional c= ontexts bigger=20 that the ones outside them just because of extra precision needed by the=20 tx machinery or are they bigger also to take into account this kind of=20 misusage? Because this way what is done inside a tx has priority over=20 what is done outside? Bellow you can find the test code I was using: =C2=A0=C2=A0=C2=A0 @Test =C2=A0=C2=A0=C2=A0 public void testD1() throws IOException { =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 final TxConstants.ConflictDet= ection conflictDetection =3D=20 TxConstants.ConflictDetection.ROW; =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 final String table =3D "mcapi= tao_tests:transaction"; =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 final Configuration conf =3D = HBaseConfiguration.create(); =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 final HConnection conn =3D HC= onnectionManager.createConnection(conf); =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 TransactionManager transactio= nManager =3D new=20 TransactionManager(conf); =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 transactionManager.startAndWa= it(); =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 TransactionSystemClient clien= t =3D new=20 InMemoryTxSystemClient(transactionManager); =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 Table nonTxTable =3D conn.get= Table(table); =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 TransactionAwareHTable txTabl= e =3D new=20 TransactionAwareHTable(conn.getTable(table), conflictDetection, false); =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 TransactionContext context =3D= new TransactionContext(client,=20 txTable); =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 try { =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 conte= xt.start(); =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 txTab= le.put( =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0= =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 new Put(Bytes.toBytes("key0000= 2")) =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0= =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2= =A0=C2=A0=C2=A0 .addColumn(Bytes.toBytes("f"),=20 Bytes.toBytes("a"), Bytes.toBytes(true)) =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 ); =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 conte= xt.finish(); =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 } catch (Exception e) { =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 LOGGE= R.error("Transaction has failed executing.", e); =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 try { =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0= =C2=A0=C2=A0=C2=A0 context.abort(); =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 } cat= ch (TransactionFailureException e1) { =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0= =C2=A0=C2=A0=C2=A0 throw new RuntimeException("could not rollback the=20 transaction", e1); =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 } =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 } =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 // Delete the row. =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 context =3D new TransactionCo= ntext(client, txTable); =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 try { =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 conte= xt.start(); =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 // [1= ] transactional delete; [2] non transactional delete.=20 Leave only one uncommented. =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 //txT= able.delete(new Delete(Bytes.toBytes("key00002"))); =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 nonTx= Table.delete(new=20 Delete(Bytes.toBytes("key00002")));//.addColumn(Bytes.toBytes("f"),=20 Bytes.toBytes("a"))); =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 conte= xt.finish(); =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 } catch (Exception e) { =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 throw= new RuntimeException("don't care about exceptions=20 here", e); =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 } =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 // Read the just deleted row. =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 context =3D new TransactionCo= ntext(client, txTable); =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 Result result; =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 try { =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 conte= xt.start(); =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 resul= t =3D txTable.get(new Get(Bytes.toBytes("key00002"))); =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 conte= xt.finish(); =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 } catch (Exception e) { =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 throw= new RuntimeException("don't care about exceptions=20 here", e); =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 } =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 nonTxTable.close(); =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 txTable.close(); =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 conn.close(); =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 Assert.assertTrue("expecting = result to be empty",=20 result.isEmpty()); =C2=A0=C2=A0=C2=A0 } Regards, Micael Capit=C3=A3o =09