Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

HDDS-5212. Create implementation of OmTransport which uses Grpc #2580

Merged
merged 6 commits into from Sep 15, 2021

Conversation

neils-dev
Copy link
Contributor

What changes were proposed in this pull request?

S3 gateway to issue s3 commands through OzoneManagerProtocol over gRPC OmTransport. This PR includes the implementation of the gRPC OmTransport provided by GrpcOmTransport created through a GrpcOmTransportFactory.

The GrpcOmTransport provides a persistent channel connection between the s3gateway and the ozone manager. The port used for the connection (8981) is configurable through a defined configuration class. Similarly, the host is configurable through configuration file through OZONE_OM_ADDRESS_KEY.

In addition, the connection is persistent and recoverable though internal gRPC channel implementation retry as recommended by gRPC dev community.

What is the link to the Apache JIRA

https://issues.apache.org/jira/browse/HDDS-5212

How was this patch tested?

Patch tested with new unit tests testing OmTransport Grpc service provider selection, OmTransport Grpc Factory creating GrpcOmTransport gRPC s3g om client for processing s3 requests through OzoneManagerProtocol.

i.) hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/protocolPB/TestGrpcOmTransport.java
Tests gRPC selection for rpc, starting and stopping s3g gRPC client, GrpcOmTransport through GrpcOmTransportFactory.

ii.) hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/protocolPB/TestS3GrpcOmTransport.java
Tests gRPC client server messaging channel by configuring GrpcOmTransport channel connection, sending OMRequest through channel and receiving OMResponse from mock OzoneManagerServiceGrpc server.

$ cd hadoop-ozone/s3gateway
$ mvn -Dtest=TestGrpcOmTransport test

[INFO] -------------------------------------------------------
[INFO]  T E S T S
[INFO] -------------------------------------------------------
[INFO] Running org.apache.hadoop.ozone.protocolPB.TestGrpcOmTransport
[INFO] Tests run: 2, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 0.488 s - in org.apache.hadoop.ozone.protocolPB.TestGrpcOmTransport
[INFO] 
[INFO] Results:
[INFO] 
[INFO] Tests run: 2, Failures: 0, Errors: 0, Skipped: 0
[INFO] 
[INFO] ------------------------------------------------------------------------
[INFO] BUILD SUCCESS
[INFO] ------------------------------------------------------------------------
[INFO] Total time:  8.321 s

$ cd hadoop-ozone/common
$ mvn -Dtest=TestS3GrpcOmTransport test

[INFO] -------------------------------------------------------
[INFO]  T E S T S
[INFO] -------------------------------------------------------
[INFO] Running org.apache.hadoop.ozone.om.protocolPB.TestS3GrpcOmTransport
[INFO] Tests run: 1, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 1.076 s - in org.apache.hadoop.ozone.om.protocolPB.TestS3GrpcOmTransport
[INFO] 
[INFO] Results:
[INFO] 
[INFO] Tests run: 1, Failures: 0, Errors: 0, Skipped: 0
[INFO] 
[INFO] ------------------------------------------------------------------------
[INFO] BUILD SUCCESS
[INFO] ------------------------------------------------------------------------
[INFO] Total time:  7.438 s

public GrpcOmTransport(ConfigurationSource conf,
UserGroupInformation ugi, String omServiceId)
throws IOException {
Optional<String> omHost = getHostNameFromConfigKeys(conf,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In HA we have 3 OMS, we considered only one OM here

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

And also failover to find leader logic I don't see

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @bharatviswa504 for reviewing. Thanks for your comments. HA feature support is upcoming in subsequent PR assoc with HDDS-5544. There we will support multiple OMs (3) for both grpc client and server with hosts and ports obtained respectively from conf through OZONE_OM_ADDRESS_KEY and OZONE_OM_GRPC_PORT_KEY uniquely identified by serviceid (federation N/A) and nodeid. The logic for failover (simplified) is also found in upcoming HDDS-5544. Let me know if you would like to see something added here related to HA support prior to it.

NettyChannelBuilder channelBuilder =
NettyChannelBuilder.forAddress(host, port)
.usePlaintext()
.maxInboundMessageSize(OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Max Inbound message should be large value I believe, list output can be large.
Should this have own config like how hadoop rpc has?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Added ozone.om.grpc.maximum.response.length to om config keys similar to hadoop rpc ipc.maximum.response.length with default size 128 * 1024 * 1024. Used in channel configuration for maxInboundMessageSize. In updated commit and push.

…ys for s3g om OmRequest OmResponses over gRPC.
try {
channel.awaitTermination(5, TimeUnit.SECONDS);
} catch (Exception e) {
LOG.error("failed to shutdown OzoneManagerServiceGrpc channel", e);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Does this need to be an error log?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @kerneltime for reviewing. Logged here at error level to reflect log at exception level (log indicative of raised exception). Can update if not of value.

Comment on lines +58 to +59
private final OMResponse omResponse = OMResponse.newBuilder()
.setSuccess(true)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Minor Nit: Good to set custom values that are checked.

Suggested change
private final OMResponse omResponse = OMResponse.newBuilder()
.setSuccess(true)
private final OMResponse omResponse = OMResponse.newBuilder()
.setLeaderOMNodeId("testOM")
.setSuccess(true)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Added custom value as noted to latest commit. Same as suggested, using a constant to reflect OMResponse leaderOMNodeId.

private final String leaderOMNodeId = "TestOM";

Comment on lines 127 to 128
Assert.assertEquals(resp.getStatus(), OK);
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Minor Nit: Good to set custom values that are checked.

Suggested change
Assert.assertEquals(resp.getStatus(), OK);
}
Assert.assertEquals(resp.getStatus(), OK);
Assert.assertEquals(resp.getLeaderOMNodeId(),"testOM");
}

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Added custom value as noted to latest commit. Same as suggested, using a constant to reflect OMResponse leaderOMNodeId.

private final String leaderOMNodeId = "TestOM";

.OzoneManagerProtocolProtos
.OMResponse>
responseObserver) {
LOG.info("GrpcOzoneManagerServer: "
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nit: Unclear why we need to log this and why at info

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Removed unnecessary log at info level. Update in latest commit.

private static final Logger LOG =
LoggerFactory.getLogger(TestGrpcOmTransport.class);
@Rule
public Timeout timeout = Timeout.seconds(300);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nit: This test should never take more than 30 seconds.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Updated timeout to 30s in latest commit.

Copy link
Contributor

@kerneltime kerneltime left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Some minor review comments. LGTM

… test timeout to appropriate test timeout value from 5min to 30s and using custom values (to reflect server response) in OMResponse to validate gRPC transport of Ozone Manager Protocol.
@neils-dev
Copy link
Contributor Author

Thanks @kerneltime for your comments. Latest push has updates as noted.

Copy link
Contributor

@bharatviswa504 bharatviswa504 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1 LGTM

@kerneltime
Copy link
Contributor

@neils-dev can you please resolve the conflicting files.

@neils-dev
Copy link
Contributor Author

Thanks @kerneltime for your comments. Latest push has updates as noted.

@neils-dev can you please resolve the conflicting files.

@kerneltime - should be resolved with latest commit. Thanks. Let me know if anything is missing.

@swagle
Copy link
Contributor

swagle commented Sep 15, 2021

Thanks @bharatviswa504 and @kerneltime for reviews, merging this.

@swagle swagle merged commit a115ebf into apache:HDDS-4440-s3-performance Sep 15, 2021
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
4 participants