You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@accumulo.apache.org by GitBox <gi...@apache.org> on 2022/04/11 21:17:31 UTC

[GitHub] [accumulo] ctubbsii commented on a diff in pull request #2620: Removed Thrift service hierarchy in preparation for service changes

ctubbsii commented on code in PR #2620:
URL: https://github.com/apache/accumulo/pull/2620#discussion_r847722820


##########
core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/ManagerClientService.java:
##########
@@ -27,7 +27,7 @@
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
 public class ManagerClientService {
 
-  public interface Iface extends FateService.Iface {
+  public interface Iface {

Review Comment:
   I'm assuming all the changes to `**/thrift/**/*.java` are not manual changes, but ones from re-generating the Thrift classes.



##########
test/src/main/java/org/apache/accumulo/test/performance/NullTserver.java:
##########
@@ -92,14 +91,11 @@
  */
 public class NullTserver {
 
-  public static class ThriftClientHandler extends ClientServiceHandler
-      implements TabletClientService.Iface {
+  public static class ThriftClientHandler implements TabletClientService.Iface {
 
     private long updateSession = 1;
 
-    public ThriftClientHandler(ServerContext context, TransactionWatcher watcher) {
-      super(context, watcher);
-    }
+    public ThriftClientHandler(ServerContext context, TransactionWatcher watcher) {}

Review Comment:
   Do we need this constructor here?



##########
core/src/main/java/org/apache/accumulo/core/rpc/ThriftUtil.java:
##########
@@ -93,8 +98,19 @@ public static TTransportFactory transportFactory() {
    */
   public static <T extends TServiceClient> T createClient(TServiceClientFactory<T> factory,
       TTransport transport) {
-    return factory.getClient(protocolFactory.getProtocol(transport),
-        protocolFactory.getProtocol(transport));
+
+    TProtocol protocol = protocolFactory.getProtocol(transport);
+    if (factory instanceof ClientService.Client.Factory) {
+      protocol = new TMultiplexedProtocol(protocol, "ClientService");
+    } else if (factory instanceof TabletClientService.Client.Factory) {
+      protocol = new TMultiplexedProtocol(protocol, "TabletClientService");
+    } else if (factory instanceof FateService.Client.Factory) {
+      protocol = new TMultiplexedProtocol(protocol, "FateService");
+    } else if (factory instanceof ManagerClientService.Client.Factory) {
+      protocol = new TMultiplexedProtocol(protocol, "ManagerClientService");
+    }

Review Comment:
   Can you explain what's going on here with a brief comment above this block? Is this just naming? I feel like we could reliably derive the name of the protocol from the class name simply by stripping off 'Client.Factory', instead of this if - else if block.



##########
core/src/main/java/org/apache/accumulo/core/rpc/ThriftUtil.java:
##########
@@ -163,7 +179,35 @@ public static void returnClient(TServiceClient iface, ClientContext context) {
   }
 
   /**
-   * Create a TabletServer Thrift client
+   * Create a TabletServer ClientService client
+   *
+   * @param address
+   *          Server address for client to connect to
+   * @param context
+   *          RPC options
+   */
+  public static ClientService.Client getClientServiceClient(HostAndPort address,

Review Comment:
   ClientServiceClient is not friendly naming. Is this a TabletServer-specific client?



##########
core/src/main/java/org/apache/accumulo/core/clientImpl/FateClient.java:
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.accumulo.core.clientImpl;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
+
+import java.net.UnknownHostException;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException;
+import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException;
+import org.apache.accumulo.core.manager.thrift.FateService;
+import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.thrift.TServiceClient;
+import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class FateClient {

Review Comment:
   Fate is a feature specific to the Manager. Removing Manager from the name might make the logs and code navigation a bit less intuitive.



##########
server/manager/src/main/java/org/apache/accumulo/manager/ManagerClientServiceHandler.java:
##########
@@ -96,15 +96,15 @@
 
 import com.google.protobuf.InvalidProtocolBufferException;
 
-public class ManagerClientServiceHandler extends FateServiceHandler
-    implements ManagerClientService.Iface {
+public class ManagerClientServiceHandler implements ManagerClientService.Iface {
 
   private static final Logger log = Manager.log;
   private static final Logger drainLog =
       LoggerFactory.getLogger("org.apache.accumulo.manager.ManagerDrainImpl");
+  protected final Manager manager;

Review Comment:
   I don't think this needs to be protected anymore. Can probably be private now, or package-private. Same with the one in FateServiceHandler.java.



##########
core/src/main/java/org/apache/accumulo/core/clientImpl/ManagerClient.java:
##########
@@ -196,31 +194,4 @@ public static void executeVoid(ClientContext context,
     }
   }
 
-  public static boolean cancelFateOperation(ClientContext context, long txid)

Review Comment:
   I'm surprised this method was here and not already in the Fate service.



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

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