[ 
https://issues.apache.org/jira/browse/HBASE-25797?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17339316#comment-17339316
 ] 

Hochul Shin commented on HBASE-25797:
-------------------------------------

Hi Toshihiro, sorry for confusing you. The put operation was just to let you 
guys know there were already existing items. 

 

This would be clearer: 

 
{code:java}
package hbase.java.client.example;

import com.google.common.util.concurrent.ThreadFactoryBuilder;
import java.util.Arrays;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.stream.Collectors;

import org.apache.commons.configuration.ConfigurationException;
import org.apache.commons.configuration.XMLConfiguration;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.AsyncConnection;
import org.apache.hadoop.hbase.client.AsyncTable;
import org.apache.hadoop.hbase.client.CheckAndMutate;
import org.apache.hadoop.hbase.client.CheckAndMutateResult;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Threads;

public class BatchCheckAndMutateTestProvider implements ClientTestProvider {

    private static final byte[] COL_FAMILY = Bytes.toBytes("cf1");

    private final AsyncTable<?> table;
    private final ExecutorService executorService;

    /**
     * Constructor.
     *
     * @param connection Connection to the hbase cluster
     */
    public BatchCheckAndMutateTestProvider(AsyncConnection connection) throws 
ConfigurationException {
        final String tableName;
        try {
            XMLConfiguration config = new XMLConfiguration("table.xml");
            tableName = config.getString("tableName");
            System.out.println("tableName :" + tableName);
        } catch (ConfigurationException cex) {
            System.out.println(cex);
            throw cex;
        }
        this.executorService = Executors.newFixedThreadPool(3,
                new ThreadFactoryBuilder()
                        .setNameFormat("HBase-" + 
this.getClass().getSimpleName() + "-Pool-%d").setDaemon(true)
                        
.setUncaughtExceptionHandler(Threads.LOGGING_EXCEPTION_HANDLER).build());
        this.table = connection.getTable(TableName.valueOf(tableName), 
executorService);
    }

    private List<CompletableFuture<CheckAndMutateResult>> asyncPuts(
            List<List<String>> input) {
        List<CheckAndMutate> checkAndMutates = input.stream().map(item -> {
            byte[] rowKey = Bytes.toBytes(item.get(0));
            byte[] qualifier = Bytes.toBytes(item.get(1));
            byte[] value = Bytes.toBytes(item.get(2));
            Put put = new Put(rowKey).addColumn(COL_FAMILY, qualifier, value);
            return CheckAndMutate.newBuilder(rowKey).ifNotExists(COL_FAMILY, 
qualifier).build(put);
        }).collect(Collectors.toList());
        return this.table.checkAndMutate(checkAndMutates);
    }

    @Override
    public void run() {
        List<String> first = Arrays.asList("r1", "q1", "v1");
        List<String> second = Arrays.asList("r1", "q2", "v2");
        List<List<String>> firstPutsInput = Arrays.asList(first, second);

        try {
            List<CompletableFuture<CheckAndMutateResult>> firstPuts = 
asyncPuts(firstPutsInput);
            firstPuts.stream().map(CompletableFuture::join).forEach(r -> {
                assert r.isSuccess() : "isSuccess() should be true";    // 
Failed. I doublechecked the table was empty before running this script. Also I 
checked the items were written. 
            });
        } catch (Exception e) {
            System.out.println(e);
        }
    }
}

{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
>            Assignee: Toshihiro Suzuki
>            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)

Reply via email to