[ https://issues.apache.org/jira/browse/HBASE-25797?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17338497#comment-17338497 ]
Hochul Shin edited comment on HBASE-25797 at 5/3/21, 5:11 PM: -------------------------------------------------------------- Hi Toshihiro, Sorry I can't share the complete code but I think the code attached above may share almost all the information except the part to create connection. I updated the code in the description for better readability. The connection part looks like {code:java} public class HBaseAsyncConnectionProviderImpl implements HBaseAsyncConnectionProvider { @Override public AsyncConnection getHBaseConnection() throws Exception { org.apache.hadoop.conf.Configuration hbaseConfig = HBaseConfiguration.create(); final String zookeeperQuorum; final String zookeeperZnodeParent; try { XMLConfiguration config = new XMLConfiguration("zookeeper.xml"); zookeeperQuorum = config.getString("zookeeperQuorum"); zookeeperZnodeParent = config.getString("zookeeperZnodeParent"); } catch (ConfigurationException cex) { System.out.println(cex); throw cex; } hbaseConfig.set(HConstants.ZOOKEEPER_QUORUM, zookeeperQuorum); hbaseConfig.setInt(HConstants.ZOOKEEPER_CLIENT_PORT, 2181); hbaseConfig.set(HConstants.ZOOKEEPER_ZNODE_PARENT, zookeeperZnodeParent); return ConnectionFactory.createAsyncConnection(hbaseConfig).get(); } } {code} was (Author: bryson): Hi Toshihiro, Sorry I can't share the complete code but I think the code attached above may share almost all the information except the part to create connection. I updated the code in the description for better readability. The part looks like {code:java} public class HBaseAsyncConnectionProviderImpl implements HBaseAsyncConnectionProvider { @Override public AsyncConnection getHBaseConnection() throws Exception { org.apache.hadoop.conf.Configuration hbaseConfig = HBaseConfiguration.create(); final String zookeeperQuorum; final String zookeeperZnodeParent; try { XMLConfiguration config = new XMLConfiguration("zookeeper.xml"); zookeeperQuorum = config.getString("zookeeperQuorum"); zookeeperZnodeParent = config.getString("zookeeperZnodeParent"); } catch (ConfigurationException cex) { System.out.println(cex); throw cex; } hbaseConfig.set(HConstants.ZOOKEEPER_QUORUM, zookeeperQuorum); hbaseConfig.setInt(HConstants.ZOOKEEPER_CLIENT_PORT, 2181); hbaseConfig.set(HConstants.ZOOKEEPER_ZNODE_PARENT, zookeeperZnodeParent); return ConnectionFactory.createAsyncConnection(hbaseConfig).get(); } } {code} > Batch version of checkAndMutate on AsyncTable always returns false > ------------------------------------------------------------------ > > Key: HBASE-25797 > URL: https://issues.apache.org/jira/browse/HBASE-25797 > Project: HBase > Issue Type: Bug > Components: Client > Affects Versions: 2.4.1 > Reporter: Hochul Shin > Priority: Minor > Labels: bug > > Batch version of CheckAndMutate always returns false whereas non-batch > version works fine. > The code is like: > > {code:java} > AysncTable<?> table = connection.getTable(tableName, executorService); > ... > // add r1 > table.put(Arrays.asList( > new Put(Bytes.toBytes("r1")).addColumn(COL_FAMILY, Bytes.toBytes("q1"), > Bytes.toBytes("v1")))); > CheckAndMutate checkAndMutate1 = > CheckAndMutate.newBuilder(Bytes.toBytes("r1")) > .ifNotExists(COL_FAMILY, Bytes.toBytes("q1")) > .build(new Put(Bytes.toBytes("r1")).addColumn(COL_FAMILY, > Bytes.toBytes("q1"), Bytes.toBytes("v1"))); > CheckAndMutate checkAndMutate2 = > CheckAndMutate.newBuilder(Bytes.toBytes("r2")) > .ifNotExists(COL_FAMILY, Bytes.toBytes("q2")) > .build(new Put(Bytes.toBytes("r2")).addColumn(COL_FAMILY, > Bytes.toBytes("q2"), Bytes.toBytes("v2"))); > {code} > > With batch version of checkAndMutate > {code:java} > List<CompletableFuture<CheckAndMutateResult>> results = > table.checkAndMutate(Arrays.asList(checkAndMutate1, checkAndMutate2)); > System.out.println("first: " + results.get(0).join().isSuccess()); // false > System.out.println("second: " + results.get(1).join().isSuccess()); //false! > {code} > > > Even it returned false, the r1 was written to the table. (timestamp was > updated) > > With non-batch version of checkAndMutate > > {code:java} > CompletableFuture<CheckAndMutateResult> result1 = > table.checkAndMutate(checkAndMutate1); > CompletableFuture<CheckAndMutateResult> result2 = > table.checkAndMutate(checkAndMutate2); > System.out.println("first: " + result1.join().isSuccess()); // false > System.out.println("second: " + result2.join().isSuccess()); // true as > expected > {code} > > As expected r1 wasn't written to the table. -- This message was sent by Atlassian Jira (v8.3.4#803005)