hemantk-12 commented on code in PR #7988:
URL: https://github.com/apache/ozone/pull/7988#discussion_r2019299127
##########
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java:
##########
@@ -1009,14 +1009,29 @@ public void deleteSnapshot(String volumeName,
public OzoneSnapshot getSnapshotInfo(String volumeName,
String bucketName,
String snapshotName) throws IOException
{
+ return getSnapshotInfo(volumeName, bucketName, snapshotName, null);
+ }
+
+ /**
+ * {@inheritDoc}
+ */
+ @Override
+ public OzoneSnapshot getSnapshotInfo(String volumeName,
+ String bucketName,
+ String snapshotName,
+ String omNodeId) throws IOException {
Preconditions.checkArgument(StringUtils.isNotBlank(volumeName),
"volume can't be null or empty.");
Preconditions.checkArgument(StringUtils.isNotBlank(bucketName),
"bucket can't be null or empty.");
Preconditions.checkArgument(StringUtils.isNotBlank(snapshotName),
"snapshot name can't be null or empty.");
+ if (StringUtils.isNotBlank(omNodeId) &&
Review Comment:
nit: create a validation function and move this duplicate code into one
place.
##########
hadoop-ozone/ozonefs-hadoop2/src/main/java/org/apache/hadoop/fs/ozone/Hadoop27RpcTransport.java:
##########
@@ -101,18 +145,35 @@ public Text getDelegationTokenService() {
* network exception or if the current proxy is not the leader OM.
*/
private OzoneManagerProtocolPB createRetryProxy(
- HadoopRpcOMFailoverProxyProvider failoverProxyProvider,
+ HadoopRpcOMFailoverProxyProvider<OzoneManagerProtocolPB>
failoverProxyProvider,
int maxFailovers) {
- OzoneManagerProtocolPB proxy = (OzoneManagerProtocolPB) RetryProxy.create(
+ return (OzoneManagerProtocolPB) RetryProxy.create(
OzoneManagerProtocolPB.class, failoverProxyProvider,
failoverProxyProvider.getRetryPolicy(maxFailovers));
- return proxy;
+ }
+
+ /**
+ * Creates a {@link RetryProxy} encapsulating the
+ * {@link HadoopRpcSingleOMFailoverProxyProvider}. The retry proxy fails
over on
+ * network exception.
+ */
+ private OzoneManagerProtocolPB createRetryProxy(
+ HadoopRpcSingleOMFailoverProxyProvider<OzoneManagerProtocolPB>
singleOMFailoverProxyProvider,
+ int maxRetry) {
+
+ return (OzoneManagerProtocolPB) RetryProxy.create(
+ OzoneManagerProtocolPB.class, singleOMFailoverProxyProvider,
+ singleOMFailoverProxyProvider.getRetryPolicy(maxRetry));
}
@Override
public void close() throws IOException {
omFailoverProxyProvider.close();
+ for (HadoopRpcSingleOMFailoverProxyProvider<OzoneManagerProtocolPB>
failoverProxyProvider
+ : omFailoverProxyProviders.values()) {
+ failoverProxyProvider.close();
+ }
Review Comment:
```suggestion
IOUtils.closeQuietly(omFailoverProxyProviders.values());
```
##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerProtocolServerSideTranslatorPB.java:
##########
@@ -187,14 +188,50 @@ private OMResponse submitReadRequestToOM(OMRequest
request)
throws ServiceException {
// Check if this OM is the leader.
RaftServerStatus raftServerStatus = omRatisServer.checkLeaderStatus();
- if (raftServerStatus == LEADER_AND_READY ||
- request.getCmdType().equals(PrepareStatus)) {
+ boolean isNodeIdSpecified = validateNodeId(request);
+ if (raftServerStatus == LEADER_AND_READY ||
request.getCmdType().equals(PrepareStatus)) {
+ return handler.handleReadRequest(request);
+ } else if (raftServerStatus == NOT_LEADER && isNodeIdSpecified &&
allowSnapshotReadFromFollower(request)) {
return handler.handleReadRequest(request);
} else {
throw createLeaderErrorException(raftServerStatus);
}
}
+
+ /**
+ * Check whether the OM request specified an OM node ID.
+ * @param omRequest OM request.
+ * @return true if the request specified an OM node ID and it is the same as
the current OM, false otherwise.
+ * @throws ServiceException exception if there is a mismatch between the OM
request's node ID and the
+ * current OM Node ID.
+ */
+ private boolean validateNodeId(OMRequest omRequest) throws ServiceException {
+ if (!omRequest.hasOmNodeId()) {
+ return false;
+ }
+
+ if (!omRequest.getOmNodeId().equals(ozoneManager.getOMNodeId())) {
+ throw new ServiceException(new OMNodeIdMismatchException("OM request
node ID is " + omRequest.getOmNodeId() +
+ ", but the current OM node ID is " + ozoneManager.getOMNodeId()));
+ }
+
+ return true;
+ }
+
+ private boolean allowSnapshotReadFromFollower(OMRequest omRequest) {
Review Comment:
nit: should we just call it `allowReadFromFollower` if this feature is
extended to other read APIs like read key and list keys?
##########
hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto:
##########
@@ -179,6 +179,13 @@ message OMRequest {
optional LayoutVersion layoutVersion = 6;
+ // If omNodeID is specified, this means that this client request is
specifically
+ // made to the particular OM node, regardless whether the OM node is a
leader.
+ // OM needs to rejects the request if the omNodeID is not equal to its own
node ID.
Review Comment:
1. Should we reject the request or forward it to the correct node?
2. What if leadership changes in-between request processing, and a
non-leader node becomes a leader node? It can happen for long running async
requests like SnapDiff.
##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/ha/SingleOMFailoverProxyProviderBase.java:
##########
@@ -0,0 +1,252 @@
+/*
+ * 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 static
org.apache.hadoop.ozone.om.ha.OMFailoverProxyProviderBase.getLeaderNotReadyException;
+import static
org.apache.hadoop.ozone.om.ha.OMFailoverProxyProviderBase.getNotLeaderException;
+
+import com.google.common.base.Preconditions;
+import com.google.protobuf.ServiceException;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.HddsUtils;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.utils.LegacyHadoopConfigurationSource;
+import org.apache.hadoop.io.retry.FailoverProxyProvider;
+import org.apache.hadoop.io.retry.RetryPolicies;
+import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.hadoop.io.retry.RetryPolicy.RetryAction.RetryDecision;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.ozone.OmUtils;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.exceptions.OMLeaderNotReadyException;
+import org.apache.hadoop.ozone.om.exceptions.OMNodeIdMismatchException;
+import org.apache.hadoop.ozone.om.exceptions.OMNotLeaderException;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.SecretManager;
+import org.apache.ratis.protocol.exceptions.StateMachineException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * An implementation of {@link FailoverProxyProvider} which does nothing in the
+ * event of OM failover, and always returns the same proxy object. In case of
OM failover,
+ * client will keep retrying to connect to the same OM node.
Review Comment:
What if that particular OM is down for a long time? Will it stop all the
read operations?
##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerProtocolServerSideTranslatorPB.java:
##########
@@ -187,14 +188,50 @@ private OMResponse submitReadRequestToOM(OMRequest
request)
throws ServiceException {
// Check if this OM is the leader.
RaftServerStatus raftServerStatus = omRatisServer.checkLeaderStatus();
- if (raftServerStatus == LEADER_AND_READY ||
- request.getCmdType().equals(PrepareStatus)) {
+ boolean isNodeIdSpecified = validateNodeId(request);
Review Comment:
This is not doing what it says in comment
[here](https://github.com/apache/ozone/pull/7988/files#diff-31e34e282f9c79c316ed42775ccba5aba01cf10771bf8954e72ccc1e87d04a62R182).
Comments says "`If omNodeID is specified, this means that this client
request is specifically made to the particular OM node, regardless whether the
OM node is a leader. OM needs to rejects the request if the omNodeID is not
equal to its own node ID.`"
While code is let the request pass to the leader even if omNodeId is passed.
Correct me if I'm missing something.
##########
hadoop-ozone/cli-shell/src/main/java/org/apache/hadoop/ozone/shell/snapshot/SnapshotDiffHandler.java:
##########
@@ -91,6 +92,11 @@ public class SnapshotDiffHandler extends Handler {
description = "Format output as JSON")
private boolean json;
+ @CommandLine.Option(
+ names = {"-n", "--om-node-id"},
Review Comment:
Does a user need to pass `om-node-id` when they want to run against a
particular OM node? Same for other handlers.
--
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]