[ 
https://issues.apache.org/jira/browse/HDDS-2169?focusedWorklogId=320632&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-320632
 ]

ASF GitHub Bot logged work on HDDS-2169:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 30/Sep/19 16:32
            Start Date: 30/Sep/19 16:32
    Worklog Time Spent: 10m 
      Work Description: bshashikant commented on issue #1517: HDDS-2169
URL: https://github.com/apache/hadoop/pull/1517#issuecomment-536642924
 
 
   Thanks @szetszwo for working on this. With the patch, while running the 
tests in TestDataValidateWithUnsafeByteOperations, the below issue is observed. 
   
   `2019-09-30 21:58:02,745 [grpc-default-executor-2] ERROR 
segmented.SegmentedRaftLogWorker (SegmentedRaftLogWorker.java:<init>(449)) - 
e4ab8454-30fe-420c-a1cf-40d223cb4898@group-D0335C23E8DA-SegmentedRaftLogWorker: 
writeStateMachineData failed for index 1, entry=(t:1, i:1), 
STATEMACHINELOGENTRY, client-6C45A0D09519, cid=8
   java.lang.IndexOutOfBoundsException: End index: 135008824 >= 207
        at 
org.apache.ratis.thirdparty.com.google.protobuf.ByteString.checkRange(ByteString.java:1233)
        at 
org.apache.ratis.thirdparty.com.google.protobuf.ByteString$LiteralByteString.substring(ByteString.java:1288)
        at 
org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage.toProto(ContainerCommandRequestMessage.java:66)
        at 
org.apache.hadoop.ozone.container.common.transport.server.ratis.ContainerStateMachine.getContainerCommandRequestProto(ContainerStateMachine.java:375)
        at 
org.apache.hadoop.ozone.container.common.transport.server.ratis.ContainerStateMachine.writeStateMachineData(ContainerStateMachine.java:494)
        at 
org.apache.ratis.server.raftlog.segmented.SegmentedRaftLogWorker$WriteLog.<init>(SegmentedRaftLogWorker.java:447)
        at 
org.apache.ratis.server.raftlog.segmented.SegmentedRaftLogWorker.writeLogEntry(SegmentedRaftLogWorker.java:397)
        at 
org.apache.ratis.server.raftlog.segmented.SegmentedRaftLog.appendEntryImpl(SegmentedRaftLog.java:411)
        at 
org.apache.ratis.server.raftlog.RaftLog.lambda$appendEntry$10(RaftLog.java:359)
        at 
org.apache.ratis.server.raftlog.RaftLogSequentialOps$Runner.runSequentially(RaftLogSequentialOps.java:77)
        at org.apache.ratis.server.raftlog.RaftLog.appendEntry(RaftLog.java:359)
        at org.apache.ratis.server.raftlog.RaftLog.appendImpl(RaftLog.java:183)
        at 
org.apache.ratis.server.raftlog.RaftLog.lambda$append$2(RaftLog.java:159)
        at 
org.apache.ratis.server.raftlog.RaftLogSequentialOps$Runner.runSequentially(RaftLogSequentialOps.java:68)
        at org.apache.ratis.server.raftlog.RaftLog.append(RaftLog.java:159)
        at 
org.apache.ratis.server.impl.ServerState.appendLog(ServerState.java:282)
        at 
org.apache.ratis.server.impl.RaftServerImpl.appendTransaction(RaftServerImpl.java:505)
        at 
org.apache.ratis.server.impl.RaftServerImpl.submitClientRequestAsync(RaftServerImpl.java:576)
        at 
org.apache.ratis.server.impl.RaftServerProxy.lambda$submitClientRequestAsync$7(RaftServerProxy.java:333)
        at 
org.apache.ratis.server.impl.RaftServerProxy.lambda$null$5(RaftServerProxy.java:328)
        at org.apache.ratis.util.JavaUtils.callAsUnchecked(JavaUtils.java:109)
        at 
org.apache.ratis.server.impl.RaftServerProxy.lambda$submitRequest$6(RaftServerProxy.java:328)
        at 
java.util.concurrent.CompletableFuture.uniComposeStage(CompletableFuture.java:981)
        at 
java.util.concurrent.CompletableFuture.thenCompose(CompletableFuture.java:2124)
        at 
org.apache.ratis.server.impl.RaftServerProxy.submitRequest(RaftServerProxy.java:327)
        at 
org.apache.ratis.server.impl.RaftServerProxy.submitClientRequestAsync(RaftServerProxy.java:333)
        at 
org.apache.ratis.grpc.client.GrpcClientProtocolService$RequestStreamObserver.processClientRequest(GrpcClientProtocolService.java:220)
        at 
org.apache.ratis.grpc.client.GrpcClientProtocolService$OrderedRequestStreamObserver.processClientRequest(GrpcClientProtocolService.java:326)
        at 
org.apache.ratis.util.SlidingWindow$Server.processRequestsFromHead(SlidingWindow.java:429)
        at 
org.apache.ratis.util.SlidingWindow$Server.receivedRequest(SlidingWindow.java:421)
        at 
org.apache.ratis.grpc.client.GrpcClientProtocolService$OrderedRequestStreamObserver.processClientRequest(GrpcClientProtocolService.java:345)
        at 
org.apache.ratis.grpc.client.GrpcClientProtocolService$RequestStreamObserver.onNext(GrpcClientProtocolService.java:240)
        at 
org.apache.ratis.grpc.client.GrpcClientProtocolService$RequestStreamObserver.onNext(GrpcClientProtocolService.java:168)
        at 
org.apache.ratis.thirdparty.io.grpc.stub.ServerCalls$StreamingServerCallHandler$StreamingServerCallListener.onMessage(ServerCalls.java:248)
        at 
org.apache.ratis.thirdparty.io.grpc.internal.ServerCallImpl$ServerStreamListenerImpl.messagesAvailable(ServerCallImpl.java:263)
        at 
org.apache.ratis.thirdparty.io.grpc.internal.ServerImpl$JumpToApplicationThreadServerStreamListener$1MessagesAvailable.runInContext(ServerImpl.java:686)
        at 
org.apache.ratis.thirdparty.io.grpc.internal.ContextRunnable.run(ContextRunnable.java:37)
        at 
org.apache.ratis.thirdparty.io.grpc.internal.SerializingExecutor.run(SerializingExecutor.java:123)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)`
   
   
   We need to change to code in ContainerStateMachine to parse the correctly in 
writeStateMachineData. Can you please check?
 
----------------------------------------------------------------
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:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 320632)
    Time Spent: 1h 20m  (was: 1h 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
>         Attachments: o2169_20190923.patch
>
>          Time Spent: 1h 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: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org

Reply via email to