bharatviswa504 commented on a change in pull request #2901:
URL: https://github.com/apache/ozone/pull/2901#discussion_r779098390
##########
File path:
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/OzoneServiceProvider.java
##########
@@ -73,8 +73,13 @@ public void init() {
throw new IllegalArgumentException(OZONE_OM_NODES_KEY
+ "." + serviceId + " is not defined");
}
- omServiceAddr = new Text(OzoneS3Util.buildServiceNameForToken(conf,
- serviceId, omNodeIds));
+ // CODE NEEDS TO BE REMOVED, NOT USING TOKENS
Review comment:
Same here once after rebase, this can be removed. This is fixed by
https://issues.apache.org/jira/browse/HDDS-6066
##########
File path:
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/ha/GrpcOMFailoverProxyProvider.java
##########
@@ -0,0 +1,139 @@
+/**
+ * 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.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 omProxies = new HashMap<>();
+ Map omProxyInfos = 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;
+ }
+ omProxies.put(nodeId, null);
+ if (hostaddr.isPresent()) {
+ omAddresses.put(nodeId,
+ hostaddr.get() + ":"
+ + hostport.orElse(config
+ .getObject(GrpcOmTransport
+ .GrpcOmTransportConfig.class)
+ .getPort()));
+ } else {
+ omAddresses.put(nodeId,
+ "0.0.0.0:" + HddsUtils.getNumberFromConfigKeys(config,
+ OMConfigKeys.OZONE_OM_GRPC_PORT_KEY));
+ }
+ omNodeIDList.add(nodeId);
+ }
+ }
+
+ if (omProxies.isEmpty()) {
+ throw new IllegalArgumentException("Could not find any configured " +
+ "addresses for OM. Please configure the system with "
+ + OZONE_OM_ADDRESS_KEY);
+ }
+
+ setOmProxies(omProxies);
Review comment:
Question:
For proxies we set null, and for proxyInfo we are not even initialized any
thing missing here or we don't need them here?
##########
File path:
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManagerServiceGrpc.java
##########
@@ -126,20 +127,19 @@ public void submitRequest(OMRequest request,
omResponse =
UserGroupInformation.getCurrentUser().doAs(
Review comment:
Should nt be this with actual user passed from protobuf?
And also even now rpc also used to pass request with s3 credentials still do
we need it?
##########
File path:
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/ha/GrpcOMFailoverProxyProvider.java
##########
@@ -0,0 +1,139 @@
+/**
+ * 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.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 omProxies = new HashMap<>();
+ Map omProxyInfos = 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;
+ }
+ omProxies.put(nodeId, null);
+ if (hostaddr.isPresent()) {
+ omAddresses.put(nodeId,
+ hostaddr.get() + ":"
+ + hostport.orElse(config
+ .getObject(GrpcOmTransport
+ .GrpcOmTransportConfig.class)
+ .getPort()));
+ } else {
+ omAddresses.put(nodeId,
Review comment:
If OMAddress is not configured it looks wrong configuration, and I think
we should fail here, instead of using 0.0.0.0?
##########
File path:
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManagerServiceGrpc.java
##########
@@ -126,20 +127,19 @@ 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) {
+ IOException ex = new IOException(e.getCause());
+ responseObserver.onError(Status
+ .INTERNAL
+ .withDescription(ex.getMessage())
+ .asRuntimeException());
+
+ // setting the omResponse, unused since we are generating
+ // standard model gRPC error through grpc Status and
Review comment:
So ServiceException with LeaderNotReady/NotLeader will be thru Grpc
errors .
And for others like business logic errors we donot throw, so i think we can
remove setting the below line correct?
##########
File path:
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/UgiFilter.java
##########
@@ -129,7 +129,8 @@ public void doFilter(ServletRequest servletRequest,
Token<OzoneTokenIdentifier> token = new Token(identifier.getBytes(),
identifier.getSignature().getBytes(StandardCharsets.UTF_8),
identifier.getKind(),
- omService);
+ null);
Review comment:
I believe we don't need this filter class, once after we rebase with
apache main branch
##########
File path:
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/ha/GrpcOMFailoverProxyProvider.java
##########
@@ -0,0 +1,139 @@
+/**
+ * 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.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 omProxies = new HashMap<>();
Review comment:
NIT: Can we define their types here for better readability also
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]