[
https://issues.apache.org/jira/browse/HDDS-2169?focusedWorklogId=324140&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-324140
]
ASF GitHub Bot logged work on HDDS-2169:
----------------------------------------
Author: ASF GitHub Bot
Created on: 07/Oct/19 04:08
Start Date: 07/Oct/19 04:08
Worklog Time Spent: 10m
Work Description: bshashikant commented on issue #1517: HDDS-2169. Avoid
buffer copies while submitting client requests in Ratis
URL: https://github.com/apache/hadoop/pull/1517#issuecomment-538832263
Thanks @szetszwo for working on this. I have added the relavent change
below as well while submitting the close container request.
`index 179547b8444..80e91cdf55d 100644
---
a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java
+++
b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java
@@ -27,6 +27,7 @@
import
org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.PipelineReport;
import
org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ClosePipelineInfo;
import
org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.PipelineAction;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
import org.apache.hadoop.hdds.scm.HddsServerUtil;
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
import org.apache.hadoop.hdds.security.x509.SecurityConfig;
@@ -516,8 +517,8 @@ private RaftClientRequest createRaftClientRequest(
RaftClientRequest.Type type) {
return new RaftClientRequest(clientId, server.getId(),
RaftGroupId.valueOf(PipelineID.getFromProtobuf(pipelineID).getId()),
- nextCallId(), Message.valueOf(request.toByteString()), type,
- null);
+ nextCallId(), ContainerCommandRequestMessage.toMessage(request,
null),
+ type, null);
}
`
I have verified the unit tests and committed this.
----------------------------------------------------------------
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.
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 324140)
Time Spent: 4h 20m (was: 4h 10m)
> Avoid buffer copies while submitting client requests in Ratis
> -------------------------------------------------------------
>
> Key: HDDS-2169
> URL: https://issues.apache.org/jira/browse/HDDS-2169
> Project: Hadoop Distributed Data Store
> Issue Type: Improvement
> Reporter: Shashikant Banerjee
> Assignee: Tsz-wo Sze
> Priority: Major
> Labels: pull-request-available
> Time Spent: 4h 20m
> Remaining Estimate: 0h
>
> Currently, while sending write requests to Ratis from ozone, a protobuf
> object containing data encoded and then resultant protobuf is again
> converted to a byteString which internally does a copy of the buffer embedded
> inside the protobuf again so that it can be submitted over to Ratis client.
> Again, while sending the appendRequest as well while building up the
> appendRequestProto, it might be again copying the data. The idea here is to
> provide client so pass the raw data(stateMachine data) separately to ratis
> client without copying overhead.
>
> {code:java}
> private CompletableFuture<RaftClientReply> sendRequestAsync(
> ContainerCommandRequestProto request) {
> try (Scope scope = GlobalTracer.get()
> .buildSpan("XceiverClientRatis." + request.getCmdType().name())
> .startActive(true)) {
> ContainerCommandRequestProto finalPayload =
> ContainerCommandRequestProto.newBuilder(request)
> .setTraceID(TracingUtil.exportCurrentSpan())
> .build();
> boolean isReadOnlyRequest = HddsUtils.isReadOnly(finalPayload);
> // finalPayload already has the byteString data embedded.
> ByteString byteString = finalPayload.toByteString(); -----> It involves a
> copy again.
> if (LOG.isDebugEnabled()) {
> LOG.debug("sendCommandAsync {} {}", isReadOnlyRequest,
> sanitizeForDebug(finalPayload));
> }
> return isReadOnlyRequest ?
> getClient().sendReadOnlyAsync(() -> byteString) :
> getClient().sendAsync(() -> byteString);
> }
> }
> {code}
--
This message was sent by Atlassian Jira
(v8.3.4#803005)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]