[
https://issues.apache.org/jira/browse/HDDS-230?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16568847#comment-16568847
]
Tsz Wo Nicholas Sze commented on HDDS-230:
------------------------------------------
- In readStateMachineData, we should use CompletableFuture.supplyAsync(..)
instead of CompletableFuture.completedFuture(..). Otherwise, it is not async.
- We should not to build the intermediate protos -- just pass the builders.
The code should look like below:
{code}
private LogEntryProto readStateMachineData(SMLogEntryProto smLogEntryProto,
ContainerCommandRequestProto requestProto) {
WriteChunkRequestProto writeChunkRequestProto =
requestProto.getWriteChunk();
// Assert that store log entry is for COMMIT_DATA, the WRITE_DATA is
// written through writeStateMachineData.
Preconditions.checkArgument(writeChunkRequestProto.getStage()
== Stage.COMMIT_DATA);
// prepare the chunk to be read
ReadChunkRequestProto.Builder readChunkRequestProto =
ReadChunkRequestProto.newBuilder()
.setBlockID(writeChunkRequestProto.getBlockID())
.setChunkData(writeChunkRequestProto.getChunkData());
ContainerCommandRequestProto dataContainerCommandProto =
ContainerCommandRequestProto.newBuilder(requestProto)
.setCmdType(Type.ReadChunk)
.setReadChunk(readChunkRequestProto)
.build();
// read the chunk
ContainerCommandResponseProto response =
dispatchCommand(dataContainerCommandProto);
ReadChunkResponseProto responseProto = response.getReadChunk();
// assert that the response has data in it.
Preconditions.checkNotNull(responseProto.getData());
// reconstruct the write chunk request
final WriteChunkRequestProto.Builder dataWriteChunkProto =
WriteChunkRequestProto.newBuilder(writeChunkRequestProto)
// adding the state machine data
.setData(responseProto.getData())
.setStage(Stage.WRITE_DATA);
ContainerCommandRequestProto.Builder newStateMachineProto =
ContainerCommandRequestProto.newBuilder(requestProto)
.setWriteChunk(dataWriteChunkProto);
// recreate the log entry
final SMLogEntryProto log =
SMLogEntryProto.newBuilder(smLogEntryProto)
.setStateMachineData(newStateMachineProto.build().toByteString())
.build();
return LogEntryProto.newBuilder().setSmLogEntry(log).build();
}
/*
* This api is used by the leader while appending logs to the follower
* This allows the leader to read the state machine data from the
* state machine implementation in case cached state machine data has been
* evicted.
*/
@Override
public CompletableFuture<LogEntryProto> readStateMachineData(
LogEntryProto entry) {
SMLogEntryProto smLogEntryProto = entry.getSmLogEntry();
if (!smLogEntryProto.getStateMachineData().isEmpty()) {
return CompletableFuture.completedFuture(entry);
}
final CompletableFuture<LogEntryProto> f = new CompletableFuture<>();
try {
final ContainerCommandRequestProto requestProto =
getRequestProto(entry.getSmLogEntry().getData());
// readStateMachineData should only be called for "write" to Ratis.
Preconditions.checkArgument(!HddsUtils.isReadOnly(requestProto));
if (requestProto.getCmdType() == Type.WriteChunk) {
return CompletableFuture.supplyAsync(() ->
readStateMachineData(smLogEntryProto, requestProto), writeChunkExecutor);
} else if (requestProto.getCmdType() == Type.CreateContainer) {
// recreate the log entry
final SMLogEntryProto log =
SMLogEntryProto.newBuilder(smLogEntryProto)
.setStateMachineData(requestProto.toByteString())
.build();
return CompletableFuture.completedFuture(
LogEntryProto.newBuilder().setSmLogEntry(log).build());
} else {
throw new IllegalStateException("Cmd type:" + requestProto.getCmdType()
+ " cannot have state machine data");
}
} catch (Exception e) {
LOG.error("unable to read stateMachineData:" + e);
return completeExceptionally(e);
}
}
{code}
> ContainerStateMachine should provide readStateMachineData api to read data if
> Containers with required during replication
> -------------------------------------------------------------------------------------------------------------------------
>
> Key: HDDS-230
> URL: https://issues.apache.org/jira/browse/HDDS-230
> Project: Hadoop Distributed Data Store
> Issue Type: Bug
> Components: Ozone Datanode
> Affects Versions: 0.2.1
> Reporter: Mukul Kumar Singh
> Assignee: Mukul Kumar Singh
> Priority: Critical
> Fix For: 0.2.1
>
> Attachments: HDDS-230.001.patch, HDDS-230.002.patch,
> HDDS-230.003.patch, HDDS-230.004.patch
>
>
> Ozone datanode exits during data write with the following exception.
> {code}
> 2018-07-05 14:10:01,605 INFO org.apache.ratis.server.storage.RaftLogWorker:
> Rolling segment:40356aa1-741f-499c-aad1-b500f2620a3d_9858-RaftLogWorker index
> to:4565
> 2018-07-05 14:10:01,607 ERROR
> org.apache.ratis.server.impl.StateMachineUpdater: Terminating with exit
> status 2: StateMachineUpdater-40356aa1-741f-499c-aad1-b500f2620a3d_9858: the
> StateMachineUpdater hits Throwable
> java.lang.NullPointerException
> at
> org.apache.hadoop.ozone.container.common.transport.server.ratis.ContainerStateMachine.applyTransaction(ContainerStateMachine.java:272)
> at
> org.apache.ratis.server.impl.RaftServerImpl.applyLogToStateMachine(RaftServerImpl.java:1058)
> at
> org.apache.ratis.server.impl.StateMachineUpdater.run(StateMachineUpdater.java:154)
> at java.lang.Thread.run(Thread.java:745)
> {code}
> This might be as a result of a ratis transaction which was not written
> through the "writeStateMachineData" phase, however it was added to the raft
> log. This implied that stateMachineUpdater now applies a transaction without
> the corresponding entry being added to the stateMachine.
> I am raising this jira to track the issue and will also raise a Ratis jira if
> required.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]