[GitHub] [hbase] brfrn169 commented on a change in pull request #1991: HBASE-24650 Change the return types of the new checkAndMutate methods…

2020-07-06 Thread GitBox


brfrn169 commented on a change in pull request #1991:
URL: https://github.com/apache/hbase/pull/1991#discussion_r450112215



##
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java
##
@@ -497,7 +497,7 @@ public void run(MultiResponse resp) {
 "Failed to mutate row: " + 
Bytes.toStringBinary(mutation.getRow()), ex));
   } else {
 future.complete(respConverter
-  .apply((Result) 
multiResp.getResults().get(regionName).result.get(0)));
+  .apply((RES) 
multiResp.getResults().get(regionName).result.get(0)));

Review comment:
   Is there any problem here?





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [hbase] brfrn169 commented on a change in pull request #1991: HBASE-24650 Change the return types of the new checkAndMutate methods…

2020-07-05 Thread GitBox


brfrn169 commented on a change in pull request #1991:
URL: https://github.com/apache/hbase/pull/1991#discussion_r449854459



##
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java
##
@@ -497,7 +497,7 @@ public void run(MultiResponse resp) {
 "Failed to mutate row: " + 
Bytes.toStringBinary(mutation.getRow()), ex));
   } else {
 future.complete(respConverter
-  .apply((Result) 
multiResp.getResults().get(regionName).result.get(0)));
+  .apply((RES) 
multiResp.getResults().get(regionName).result.get(0)));

Review comment:
   Yes, correct. Previously, we returned only Result type but after this 
fix, we will return Result (for normal mutateRow) and CheckAndMutateResult (for 
checkAndMutate). So I used a generic type here.

##
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableOverAsyncTable.java
##
@@ -300,13 +300,16 @@ public boolean thenMutate(RowMutations mutation) throws 
IOException {
   }
 
   @Override
-  public boolean checkAndMutate(CheckAndMutate checkAndMutate) throws 
IOException {
+  public CheckAndMutateResult checkAndMutate(CheckAndMutate checkAndMutate) 
throws IOException {
 return FutureUtils.get(table.checkAndMutate(checkAndMutate));
   }
 
   @Override
-  public boolean[] checkAndMutate(List checkAndMutates) throws 
IOException {
-return 
Booleans.toArray(FutureUtils.get(table.checkAndMutateAll(checkAndMutates)));
+  public CheckAndMutateResult[] checkAndMutate(List 
checkAndMutates)

Review comment:
   Yeah that's better. I will change it.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [hbase] brfrn169 commented on a change in pull request #1991: HBASE-24650 Change the return types of the new CheckAndMutate methods…

2020-06-27 Thread GitBox


brfrn169 commented on a change in pull request #1991:
URL: https://github.com/apache/hbase/pull/1991#discussion_r446588453



##
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java
##
@@ -349,7 +349,7 @@ private void preCheck() {
   loc, stub, mutation,
   (rn, rm) -> RequestConverter.buildMutateRequest(rn, row, family, 
qualifier, op, value,
 null, timeRange, rm),
-  resp -> resp.getExists()))
+  resp -> ((CheckAndMutateResult) resp).isSuccess()))

Review comment:
   I will modify it to make use of generic type. Thanks.

##
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java
##
@@ -431,49 +431,49 @@ public CheckAndMutateWithFilterBuilder 
checkAndMutate(byte[] row, Filter filter)
   if (mutation instanceof Put) {
 validatePut((Put) mutation, conn.connConf.getMaxKeyValueSize());
   }
-  return RawAsyncTableImpl.this. 
newCaller(checkAndMutate.getRow(),
+  return RawAsyncTableImpl.this. 
newCaller(checkAndMutate.getRow(),
 mutation.getPriority(), rpcTimeoutNs)
 .action((controller, loc, stub) -> RawAsyncTableImpl.mutate(controller,
   loc, stub, mutation,
   (rn, m) -> RequestConverter.buildMutateRequest(rn, 
checkAndMutate.getRow(),
 checkAndMutate.getFamily(), checkAndMutate.getQualifier(),
 checkAndMutate.getCompareOp(), checkAndMutate.getValue(), 
checkAndMutate.getFilter(),
 checkAndMutate.getTimeRange(), m),
-  (c, r) -> r.getProcessed()))
+  (c, r) -> ResponseConverter.getCheckAndMutateResult(r)))
 .call();
 } else if (checkAndMutate.getAction() instanceof RowMutations) {
   RowMutations rowMutations = (RowMutations) checkAndMutate.getAction();
-  return RawAsyncTableImpl.this. 
newCaller(checkAndMutate.getRow(),
+  return RawAsyncTableImpl.this. 
newCaller(checkAndMutate.getRow(),
 rowMutations.getMaxPriority(), rpcTimeoutNs)
 .action((controller, loc, stub) -> 
RawAsyncTableImpl.this.mutateRow(controller,
   loc, stub, rowMutations,
   (rn, rm) -> RequestConverter.buildMutateRequest(rn, 
checkAndMutate.getRow(),
 checkAndMutate.getFamily(), checkAndMutate.getQualifier(),
 checkAndMutate.getCompareOp(), checkAndMutate.getValue(), 
checkAndMutate.getFilter(),
 checkAndMutate.getTimeRange(), rm),
-  resp -> resp.getExists()))
+  resp -> (CheckAndMutateResult) resp))
 .call();
 } else {
-  CompletableFuture future = new CompletableFuture<>();
+  CompletableFuture future = new 
CompletableFuture<>();
   future.completeExceptionally(new DoNotRetryIOException(
 "CheckAndMutate doesn't support " + 
checkAndMutate.getAction().getClass().getName()));
   return future;
 }
   }
 
   @Override
-  public List> checkAndMutate(List 
checkAndMutates) {
+  public List> checkAndMutate(
+List checkAndMutates) {
 return batch(checkAndMutates, rpcTimeoutNs).stream()
-  .map(f -> f.thenApply(r -> ((Result)r).getExists()))
-  .collect(toList());
+  .map(f -> f.thenApply(r -> (CheckAndMutateResult) r)).collect(toList());
   }
 
   // We need the MultiRequest when constructing the 
org.apache.hadoop.hbase.client.MultiResponse,
   // so here I write a new method as I do not want to change the abstraction 
of call method.
   private  CompletableFuture mutateRow(HBaseRpcController 
controller,
   HRegionLocation loc, ClientService.Interface stub, RowMutations mutation,
   Converter reqConvert,
-  Function respConverter) {
+  Function respConverter) {

Review comment:
   The reason why I changed Result to Object here is that this method is 
used by both checkAndMutate() and mutateRow() and we need to use different 
types each case (respectively CheckAndMutateResult and Result). However as you 
mentioned, we can make use of generic type here. I will modify it. Thanks.
   
   

##
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/CheckAndMutateResult.java
##
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and