[
https://issues.apache.org/jira/browse/HDDS-2169?focusedWorklogId=323612&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-323612
]
ASF GitHub Bot logged work on HDDS-2169:
----------------------------------------
Author: ASF GitHub Bot
Created on: 04/Oct/19 17:45
Start Date: 04/Oct/19 17:45
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-538495734
Thanks @szetszwo for updating the patch. I tried to run the tests in
TestDataValidateWithUnsafeByteOperations and i see the following exception
being thrown:
`2019-10-04 23:13:25,556
[ce18dfb1-da4d-401f-9614-bec32477b5f3@group-0099BCD205B6-SegmentedRaftLogWorker]
INFO segmented.SegmentedRaftLogWorker
(SegmentedRaftLogWorker.java:execute(574)) -
ce18dfb1-da4d-401f-9614-bec32477b5f3@group-0099BCD205B6-SegmentedRaftLogWorker:
created new log segment
/Users/sbanerjee/github_hadoop/hadoop-ozone/tools/target/test-dir/MiniOzoneClusterImpl-cd3ca672-68cd-49fd-bdb3-a7fc97d18c23/datanode-1/data/ratis/ace05abb-b740-47f7-95d4-0099bcd205b6/current/log_inprogress_0
2019-10-04 23:13:25,557
[ee7f2721-1de4-4264-8bf3-d340e83f8791@group-0099BCD205B6-SegmentedRaftLogWorker]
INFO segmented.SegmentedRaftLogWorker
(SegmentedRaftLogWorker.java:execute(574)) -
ee7f2721-1de4-4264-8bf3-d340e83f8791@group-0099BCD205B6-SegmentedRaftLogWorker:
created new log segment
/Users/sbanerjee/github_hadoop/hadoop-ozone/tools/target/test-dir/MiniOzoneClusterImpl-cd3ca672-68cd-49fd-bdb3-a7fc97d18c23/datanode-2/data/ratis/ace05abb-b740-47f7-95d4-0099bcd205b6/current/log_inprogress_0
2019-10-04 23:13:25,874 [pool-56-thread-1] ERROR impl.ChunkManagerImpl
(ChunkUtils.java:writeData(89)) - data array does not match the length
specified. DataLen: 1048576 Byte Array: 1048749
2019-10-04 23:13:25,874 [pool-56-thread-1] INFO keyvalue.KeyValueHandler
(ContainerUtils.java:logAndReturnError(146)) - Operation: WriteChunk : Trace
ID: cab5af5eafbad5ed:6a87e816d7e0ce20:e3ff42a900c31035:0 : Message: data array
does not match the length specified. DataLen: 1048576 Byte Array: 1048749 :
Result: INVALID_WRITE_SIZE
2019-10-04 23:13:25,881
[EventQueue-IncrementalContainerReportForIncrementalContainerReportHandler]
WARN container.IncrementalContainerReportHandler
(AbstractContainerReportHandler.java:updateContainerState(143)) - Container #1
is in OPEN state, but the datanode eb79af53-823f-485d-8402-ff71443cc79f{ip:
192.168.0.64, host: 192.168.0.64, networkLocation: /default-rack, certSerialId:
null} reports an UNHEALTHY replica.
23:13:25.886 [pool-56-thread-1] ERROR DNAudit - user=null | ip=null |
op=WRITE_CHUNK {blockData=conID: 1 locID: 102905348118937600 bcsId: 0} |
ret=FAILURE
java.lang.Exception: data array does not match the length specified.
DataLen: 1048576 Byte Array: 1048749
at
org.apache.hadoop.ozone.container.common.impl.HddsDispatcher.dispatchRequest(HddsDispatcher.java:330)
~[classes/:?]
at
org.apache.hadoop.ozone.container.common.impl.HddsDispatcher.dispatch(HddsDispatcher.java:150)
~[classes/:?]
at
org.apache.hadoop.ozone.container.common.transport.server.ratis.ContainerStateMachine.dispatchCommand(ContainerStateMachine.java:411)
~[classes/:?]
at
org.apache.hadoop.ozone.container.common.transport.server.ratis.ContainerStateMachine.runCommand(ContainerStateMachine.java:419)
~[classes/:?]
at
org.apache.hadoop.ozone.container.common.transport.server.ratis.ContainerStateMachine.lambda$handleWriteChunk$1(ContainerStateMachine.java:454)
~[classes/:?]
at
java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1590)
[?:1.8.0_181]
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
[?:1.8.0_181]
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
[?:1.8.0_181]
at java.lang.Thread.run(Thread.java:748) [?:1.8.0_181]
2019-10-04 23:13:25,896 [pool-56-thread-1] ERROR ratis.ContainerStateMachine
(ContainerStateMachine.java:lambda$handleWriteChunk$2(474)) -
group-0099BCD205B6: writeChunk writeStateMachineData failed:
blockIdcontainerID: 1
localID: 102905348118937600
blockCommitSequenceId: 0
logIndex 1 chunkName 102905348118937600_chunk_1 Error message: data array
does not match the length specified. DataLen: 1048576 Byte Array: 1048749
Container Result: INVALID_WRITE_SIZE
`
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:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 323612)
Time Spent: 3h 40m (was: 3.5h)
> 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: 3h 40m
> 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]