You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by GitBox <gi...@apache.org> on 2020/06/28 01:29:10 UTC

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

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



##########
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
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
+/**
+ * Represents a result of a CheckAndMutate operation
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving

Review comment:
       For IA.Public, we do not use the IS annotation since it should always be Stable.

##########
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.<Boolean> newCaller(checkAndMutate.getRow(),
+      return RawAsyncTableImpl.this.<CheckAndMutateResult> 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.<Boolean> newCaller(checkAndMutate.getRow(),
+      return RawAsyncTableImpl.this.<CheckAndMutateResult> 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<Boolean> future = new CompletableFuture<>();
+      CompletableFuture<CheckAndMutateResult> future = new CompletableFuture<>();
       future.completeExceptionally(new DoNotRetryIOException(
         "CheckAndMutate doesn't support " + checkAndMutate.getAction().getClass().getName()));
       return future;
     }
   }
 
   @Override
-  public List<CompletableFuture<Boolean>> checkAndMutate(List<CheckAndMutate> checkAndMutates) {
+  public List<CompletableFuture<CheckAndMutateResult>> checkAndMutate(
+    List<CheckAndMutate> 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 <RESP> CompletableFuture<RESP> mutateRow(HBaseRpcController controller,
       HRegionLocation loc, ClientService.Interface stub, RowMutations mutation,
       Converter<MultiRequest, byte[], RowMutations> reqConvert,
-      Function<Result, RESP> respConverter) {
+      Function<Object, RESP> respConverter) {

Review comment:
       Object is too generic. Please explain why we need to do this change?

##########
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:
       We have already make use of generic type so why here we still need to cast?




----------------------------------------------------------------
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:
users@infra.apache.org