You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by GitBox <gi...@apache.org> on 2022/01/20 13:05:32 UTC

[GitHub] [ozone] adoroszlai commented on a change in pull request #2901: HDDS-5544. Update GRPC OmTransport implementation for HA

adoroszlai commented on a change in pull request #2901:
URL: https://github.com/apache/ozone/pull/2901#discussion_r788400799



##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/ha/GrpcOMFailoverProxyProvider.java
##########
@@ -0,0 +1,144 @@
+/**
+ * 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.ozone.om.ha;
+
+import org.apache.hadoop.hdds.conf.ConfigurationException;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.HddsUtils;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.ozone.OmUtils;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.ha.ConfUtils;
+import org.apache.hadoop.ozone.om.OMConfigKeys;
+import org.apache.hadoop.ozone.om.protocolPB.GrpcOmTransport;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalInt;
+
+import static org.apache.hadoop.hdds.HddsUtils.getHostNameFromConfigKeys;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ADDRESS_KEY;
+
+/**
+ * The Grpc s3gateway om transport failover proxy provider implementation
+ * extending the ozone client OM failover proxy provider.  This implmentation
+ * allows the Grpc OMTransport reuse OM failover retry policies and
+ * getRetryAction methods.  In case of OM failover, client can try
+ * connecting to another OM node from the list of proxies.
+ */
+public class GrpcOMFailoverProxyProvider<T> extends
+    OMFailoverProxyProvider<T> {
+
+  private Map<String, String> omAddresses;
+
+  public GrpcOMFailoverProxyProvider(ConfigurationSource configuration,
+                                     UserGroupInformation ugi,
+                                     String omServiceId,
+                                     Class<T> protocol) throws IOException {
+    super(configuration, ugi, omServiceId, protocol);
+  }
+
+  @Override
+  protected void loadOMClientConfigs(ConfigurationSource config, String omSvcId)
+      throws IOException {
+    Map omProxiesNodeIdKeyset = new HashMap<>();
+    List omNodeIDList = new ArrayList<>();

Review comment:
       These variables should have generic parameters to enforce strong typing.

##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/GrpcOmTransport.java
##########
@@ -70,42 +77,71 @@
   private final AtomicBoolean isRunning = new AtomicBoolean(false);
 
   // gRPC specific
-  private ManagedChannel channel;
-
   private OzoneManagerServiceGrpc.OzoneManagerServiceBlockingStub client;
+  private Map<String,
+      OzoneManagerServiceGrpc.OzoneManagerServiceBlockingStub> clients;
+  private Map<String, ManagedChannel> channels;
+  private int lastVisited = -1;
+  private ConfigurationSource conf;
 
   private String host = "om";
-  private int port = 8981;
   private int maxSize;
 
+  private List<String> oms;
+  private RetryPolicy retryPolicy;
+  private int failoverCount = 0;
+  private GrpcOMFailoverProxyProvider omFailoverProxyProvider;

Review comment:
       `GrpcOMFailoverProxyProvider` is generic, please specify type.

##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/ha/GrpcOMFailoverProxyProvider.java
##########
@@ -0,0 +1,144 @@
+/**
+ * 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.ozone.om.ha;
+
+import org.apache.hadoop.hdds.conf.ConfigurationException;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.HddsUtils;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.ozone.OmUtils;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.ha.ConfUtils;
+import org.apache.hadoop.ozone.om.OMConfigKeys;
+import org.apache.hadoop.ozone.om.protocolPB.GrpcOmTransport;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalInt;
+
+import static org.apache.hadoop.hdds.HddsUtils.getHostNameFromConfigKeys;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ADDRESS_KEY;
+
+/**
+ * The Grpc s3gateway om transport failover proxy provider implementation
+ * extending the ozone client OM failover proxy provider.  This implmentation
+ * allows the Grpc OMTransport reuse OM failover retry policies and
+ * getRetryAction methods.  In case of OM failover, client can try
+ * connecting to another OM node from the list of proxies.
+ */
+public class GrpcOMFailoverProxyProvider<T> extends
+    OMFailoverProxyProvider<T> {
+
+  private Map<String, String> omAddresses;
+
+  public GrpcOMFailoverProxyProvider(ConfigurationSource configuration,
+                                     UserGroupInformation ugi,
+                                     String omServiceId,
+                                     Class<T> protocol) throws IOException {
+    super(configuration, ugi, omServiceId, protocol);
+  }
+
+  @Override
+  protected void loadOMClientConfigs(ConfigurationSource config, String omSvcId)
+      throws IOException {
+    Map omProxiesNodeIdKeyset = new HashMap<>();
+    List omNodeIDList = new ArrayList<>();
+    omAddresses = new HashMap<>();
+
+    Collection<String> omServiceIds = Collections.singletonList(omSvcId);
+
+    for (String serviceId : OmUtils.emptyAsSingletonNull(omServiceIds)) {
+      Collection<String> omNodeIds = OmUtils.getOMNodeIds(config, serviceId);
+
+      for (String nodeId : OmUtils.emptyAsSingletonNull(omNodeIds)) {
+
+        String rpcAddrKey = ConfUtils.addKeySuffixes(OZONE_OM_ADDRESS_KEY,
+            serviceId, nodeId);
+
+        Optional<String> hostaddr = getHostNameFromConfigKeys(config,
+            rpcAddrKey);
+
+        OptionalInt hostport = HddsUtils.getNumberFromConfigKeys(config,
+            ConfUtils.addKeySuffixes(OMConfigKeys.OZONE_OM_GRPC_PORT_KEY,
+                serviceId, nodeId),
+            OMConfigKeys.OZONE_OM_GRPC_PORT_KEY);
+        if (nodeId == null) {
+          nodeId = OzoneConsts.OM_DEFAULT_NODE_ID;
+        }
+        omProxiesNodeIdKeyset.put(nodeId, null);
+        if (hostaddr.isPresent()) {
+          omAddresses.put(nodeId,
+              hostaddr.get() + ":"
+                  + hostport.orElse(config
+                  .getObject(GrpcOmTransport
+                      .GrpcOmTransportConfig.class)
+                  .getPort()));
+        } else {
+          LOG.error("expected host address not defined: {}", rpcAddrKey);
+          throw new ConfigurationException(rpcAddrKey + "is not defined");
+        }
+        omNodeIDList.add(nodeId);
+      }
+    }
+
+    if (omProxiesNodeIdKeyset.isEmpty()) {
+      throw new IllegalArgumentException("Could not find any configured " +
+          "addresses for OM. Please configure the system with "
+          + OZONE_OM_ADDRESS_KEY);
+    }
+
+    // set base class omProxies, omProxyInfos, omNodeIDList
+
+    // omProxies needed in base class
+    // omProxies.size == number of om nodes
+    // omProxies key needs to be valid nodeid
+    setOmProxies(omProxiesNodeIdKeyset);
+    // omProxyInfos keyset needed in base class
+    setOmProxyInfos(omProxiesNodeIdKeyset);
+    setOmNodeIDList(omNodeIDList);

Review comment:
       How can both `omProxies` and `omProxyInfos` use the same map?  They have different value types.

##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/GrpcOmTransport.java
##########
@@ -138,31 +174,115 @@ public OMResponse submitRequest(OMRequest payload) throws IOException {
             .build();
       }
     }
+    LOG.debug("OMRequest {}", payload);
     OMResponse resp = null;
-    try {
-      resp = client.submitRequest(payload);
-    } catch (io.grpc.StatusRuntimeException e) {
-      ResultCodes resultCode = ResultCodes.INTERNAL_ERROR;
-      if (e.getStatus().getCode() == Status.Code.UNAVAILABLE) {
-        resultCode = ResultCodes.TIMEOUT;
+    boolean tryOtherHost = true;
+    ResultCodes resultCode = ResultCodes.INTERNAL_ERROR;
+    while (tryOtherHost) {
+      tryOtherHost = false;
+      try {
+        resp = clients.get(host).submitRequest(payload);
+      } catch (io.grpc.StatusRuntimeException e) {

Review comment:
       Can you please add import for `StatusRuntimeException`?

##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/ha/OMFailoverProxyProvider.java
##########
@@ -552,5 +552,22 @@ protected void setProxiesForTesting(
     this.omProxyInfos = testOMProxyInfos;
     this.omNodeIDList = testOMNodeIDList;
   }
+
+  protected void setOmProxies(Map omProxies) {
+    this.omProxies = omProxies;
+  }
+
+  protected void setOmProxyInfos(Map omProxyInfos) {
+    this.omProxyInfos = omProxyInfos;
+  }
+
+  protected void setOmNodeIDList(List omNodeIDList) {
+    this.omNodeIDList = omNodeIDList;
+  }

Review comment:
       If duplication of `loadOMClientConfigs` is addressed, these would be unnecessary.  (Otherwise we can use existing `setProxiesForTesting` (and rename it, since it would no longer be only for testing)).

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/GrpcOzoneManagerServer.java
##########
@@ -50,6 +53,17 @@ public GrpcOzoneManagerServer(OzoneConfiguration config,
     this.port = config.getObject(
         GrpcOzoneManagerServerConfig.class).
         getPort();

Review comment:
       Nit: please move inside `else` of the new `if (haPort.isPresent())`.

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManagerServiceGrpc.java
##########
@@ -78,7 +79,7 @@ public void submitRequest(OMRequest request,
         "processing s3g client submit request - for command {}",
         request.getCmdType().name());
     AtomicInteger callCount = new AtomicInteger(0);
-    OMResponse omResponse;
+    OMResponse omResponse = null;

Review comment:
       If `onNext` is called only inside `try`, we can also move this declaration there.

##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/GrpcOmTransport.java
##########
@@ -70,42 +77,71 @@
   private final AtomicBoolean isRunning = new AtomicBoolean(false);
 
   // gRPC specific
-  private ManagedChannel channel;
-
   private OzoneManagerServiceGrpc.OzoneManagerServiceBlockingStub client;
+  private Map<String,
+      OzoneManagerServiceGrpc.OzoneManagerServiceBlockingStub> clients;
+  private Map<String, ManagedChannel> channels;
+  private int lastVisited = -1;
+  private ConfigurationSource conf;
 
   private String host = "om";
-  private int port = 8981;
   private int maxSize;
 
+  private List<String> oms;
+  private RetryPolicy retryPolicy;
+  private int failoverCount = 0;
+  private GrpcOMFailoverProxyProvider omFailoverProxyProvider;
+
   public GrpcOmTransport(ConfigurationSource conf,
                           UserGroupInformation ugi, String omServiceId)
       throws IOException {
-    Optional<String> omHost = getHostNameFromConfigKeys(conf,
-        OZONE_OM_ADDRESS_KEY);
-    this.host = omHost.orElse("0.0.0.0");
 
-    port = conf.getObject(GrpcOmTransportConfig.class).getPort();
+    this.channels = new HashMap<>();
+    this.clients = new HashMap<>();
+    this.conf = conf;
 
     maxSize = conf.getInt(OZONE_OM_GRPC_MAXIMUM_RESPONSE_LENGTH,
         OZONE_OM_GRPC_MAXIMUM_RESPONSE_LENGTH_DEFAULT);
 
+    omFailoverProxyProvider = new GrpcOMFailoverProxyProvider(
+        conf,
+        ugi,
+        omServiceId,
+        OzoneManagerProtocolPB.class);
+
     start();
   }
 
-  public void start() {
+  public void start() throws IOException {
     if (!isRunning.compareAndSet(false, true)) {

Review comment:
       If the method throws due to address lookup, `isRunning` will be `true`, but it will not be really running.  I think address lookup should be done before setting the `isRunning` flag.

##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/GrpcOmTransport.java
##########
@@ -138,31 +174,115 @@ public OMResponse submitRequest(OMRequest payload) throws IOException {
             .build();
       }
     }
+    LOG.debug("OMRequest {}", payload);
     OMResponse resp = null;
-    try {
-      resp = client.submitRequest(payload);
-    } catch (io.grpc.StatusRuntimeException e) {
-      ResultCodes resultCode = ResultCodes.INTERNAL_ERROR;
-      if (e.getStatus().getCode() == Status.Code.UNAVAILABLE) {
-        resultCode = ResultCodes.TIMEOUT;
+    boolean tryOtherHost = true;
+    ResultCodes resultCode = ResultCodes.INTERNAL_ERROR;
+    while (tryOtherHost) {
+      tryOtherHost = false;
+      try {
+        resp = clients.get(host).submitRequest(payload);
+      } catch (io.grpc.StatusRuntimeException e) {
+        if (e.getStatus().getCode() == Status.Code.UNAVAILABLE) {
+          resultCode = ResultCodes.TIMEOUT;
+        }
+        Exception exp = new Exception(e);
+        tryOtherHost = shouldRetry(unwrapException(exp));
+        if (!tryOtherHost) {
+          throw new OMException(resultCode);
+        }
       }
-      throw new OMException(e.getCause(), resultCode);
     }
     return resp;
   }
 
+  private Exception unwrapException(Exception ex) {
+    Exception grpcException = null;
+    try {
+      io.grpc.StatusRuntimeException srexp =
+          (io.grpc.StatusRuntimeException)ex.getCause();
+      io.grpc.Status status = srexp.getStatus();
+      LOG.debug("GRPC exception wrapped: "+status.getDescription());

Review comment:
       ```suggestion
         LOG.debug("GRPC exception wrapped: {}", status.getDescription());
   ```

##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/GrpcOmTransport.java
##########
@@ -138,31 +174,115 @@ public OMResponse submitRequest(OMRequest payload) throws IOException {
             .build();
       }
     }
+    LOG.debug("OMRequest {}", payload);
     OMResponse resp = null;
-    try {
-      resp = client.submitRequest(payload);
-    } catch (io.grpc.StatusRuntimeException e) {
-      ResultCodes resultCode = ResultCodes.INTERNAL_ERROR;
-      if (e.getStatus().getCode() == Status.Code.UNAVAILABLE) {
-        resultCode = ResultCodes.TIMEOUT;
+    boolean tryOtherHost = true;
+    ResultCodes resultCode = ResultCodes.INTERNAL_ERROR;
+    while (tryOtherHost) {
+      tryOtherHost = false;
+      try {
+        resp = clients.get(host).submitRequest(payload);
+      } catch (io.grpc.StatusRuntimeException e) {
+        if (e.getStatus().getCode() == Status.Code.UNAVAILABLE) {
+          resultCode = ResultCodes.TIMEOUT;
+        }
+        Exception exp = new Exception(e);
+        tryOtherHost = shouldRetry(unwrapException(exp));
+        if (!tryOtherHost) {
+          throw new OMException(resultCode);
+        }
       }
-      throw new OMException(e.getCause(), resultCode);
     }
     return resp;
   }
 
+  private Exception unwrapException(Exception ex) {
+    Exception grpcException = null;
+    try {
+      io.grpc.StatusRuntimeException srexp =
+          (io.grpc.StatusRuntimeException)ex.getCause();
+      io.grpc.Status status = srexp.getStatus();

Review comment:
       ```suggestion
         Status status = srexp.getStatus();
   ```

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManagerServiceGrpc.java
##########
@@ -126,23 +127,15 @@ public void submitRequest(OMRequest request,
       omResponse =
           UserGroupInformation.getCurrentUser().doAs(
               (PrivilegedExceptionAction<OMResponse>) () -> {
-                try {
                   return this.omTranslator.
                       submitRequest(NULL_RPC_CONTROLLER, request);
-                } catch (Throwable se) {
-                  Throwable e = se.getCause();
-                  if (e == null) {
-                    throw new IOException(se);
-                  } else {
-                    throw e instanceof IOException ?
-                        (IOException) e : new IOException(se);
-                  }
-                }
               });
     } catch (Throwable e) {
-      omResponse = createErrorResponse(
-          request,
-          new IOException(e));
+      IOException ex = new IOException(e.getCause());
+      responseObserver.onError(Status
+          .INTERNAL
+          .withDescription(ex.getMessage())
+          .asRuntimeException());
     }
     responseObserver.onNext(omResponse);

Review comment:
       I guess `onNext` should be called only if there was no exception (i.e. inside the `try` block above)?

##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/ha/GrpcOMFailoverProxyProvider.java
##########
@@ -0,0 +1,144 @@
+/**
+ * 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.ozone.om.ha;
+
+import org.apache.hadoop.hdds.conf.ConfigurationException;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.HddsUtils;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.ozone.OmUtils;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.ha.ConfUtils;
+import org.apache.hadoop.ozone.om.OMConfigKeys;
+import org.apache.hadoop.ozone.om.protocolPB.GrpcOmTransport;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalInt;
+
+import static org.apache.hadoop.hdds.HddsUtils.getHostNameFromConfigKeys;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ADDRESS_KEY;
+
+/**
+ * The Grpc s3gateway om transport failover proxy provider implementation
+ * extending the ozone client OM failover proxy provider.  This implmentation
+ * allows the Grpc OMTransport reuse OM failover retry policies and
+ * getRetryAction methods.  In case of OM failover, client can try
+ * connecting to another OM node from the list of proxies.
+ */
+public class GrpcOMFailoverProxyProvider<T> extends
+    OMFailoverProxyProvider<T> {
+
+  private Map<String, String> omAddresses;
+
+  public GrpcOMFailoverProxyProvider(ConfigurationSource configuration,
+                                     UserGroupInformation ugi,
+                                     String omServiceId,
+                                     Class<T> protocol) throws IOException {
+    super(configuration, ugi, omServiceId, protocol);
+  }
+
+  @Override
+  protected void loadOMClientConfigs(ConfigurationSource config, String omSvcId)

Review comment:
       This method duplicates the one in the parent class with very minor variation.  I think it would be simpler and cleaner to extract the small parts that are different (hostname and port lookup logic).




-- 
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: issues-unsubscribe@ozone.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org