HDDS-716. Update ozone to latest ratis snapshot build(0.3.0-aa38160-SNAPSHOT). Contributed by Mukul Kumar Singh.

This commit is contained in:
Shashikant Banerjee 2018-10-24 16:04:57 +05:30
parent 553044818d
commit 0891cdda79
2 changed files with 48 additions and 26 deletions

View File

@ -22,6 +22,7 @@
import org.apache.hadoop.hdds.HddsUtils; import org.apache.hadoop.hdds.HddsUtils;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.ozone.container.common.helpers.BlockData; import org.apache.hadoop.ozone.container.common.helpers.BlockData;
import org.apache.ratis.proto.RaftProtos.StateMachineEntryProto;
import org.apache.ratis.protocol.RaftGroup; import org.apache.ratis.protocol.RaftGroup;
import org.apache.ratis.protocol.RaftGroupId; import org.apache.ratis.protocol.RaftGroupId;
import org.apache.ratis.server.RaftServer; import org.apache.ratis.server.RaftServer;
@ -49,7 +50,7 @@
import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
import org.apache.ratis.proto.RaftProtos.RoleInfoProto; import org.apache.ratis.proto.RaftProtos.RoleInfoProto;
import org.apache.ratis.proto.RaftProtos.LogEntryProto; import org.apache.ratis.proto.RaftProtos.LogEntryProto;
import org.apache.ratis.proto.RaftProtos.SMLogEntryProto; import org.apache.ratis.proto.RaftProtos.StateMachineLogEntryProto;
import org.apache.ratis.statemachine.StateMachineStorage; import org.apache.ratis.statemachine.StateMachineStorage;
import org.apache.ratis.statemachine.TransactionContext; import org.apache.ratis.statemachine.TransactionContext;
import org.apache.ratis.statemachine.impl.BaseStateMachine; import org.apache.ratis.statemachine.impl.BaseStateMachine;
@ -207,7 +208,7 @@ public TransactionContext startTransaction(RaftClientRequest request)
final ContainerCommandRequestProto proto = final ContainerCommandRequestProto proto =
getRequestProto(request.getMessage().getContent()); getRequestProto(request.getMessage().getContent());
final SMLogEntryProto log; final StateMachineLogEntryProto log;
if (proto.getCmdType() == Type.WriteChunk) { if (proto.getCmdType() == Type.WriteChunk) {
final WriteChunkRequestProto write = proto.getWriteChunk(); final WriteChunkRequestProto write = proto.getWriteChunk();
// create the state machine data proto // create the state machine data proto
@ -237,23 +238,39 @@ public TransactionContext startTransaction(RaftClientRequest request)
.setWriteChunk(commitWriteChunkProto) .setWriteChunk(commitWriteChunkProto)
.build(); .build();
log = SMLogEntryProto.newBuilder() log = createSMLogEntryProto(request,
.setData(commitContainerCommandProto.toByteString()) commitContainerCommandProto.toByteString(),
.setStateMachineData(dataContainerCommandProto.toByteString()) dataContainerCommandProto.toByteString());
.build();
} else if (proto.getCmdType() == Type.CreateContainer) { } else if (proto.getCmdType() == Type.CreateContainer) {
log = SMLogEntryProto.newBuilder() log = createSMLogEntryProto(request,
.setData(request.getMessage().getContent()) request.getMessage().getContent(), request.getMessage().getContent());
.setStateMachineData(request.getMessage().getContent())
.build();
} else { } else {
log = SMLogEntryProto.newBuilder() log = createSMLogEntryProto(request, request.getMessage().getContent(),
.setData(request.getMessage().getContent()) null);
.build();
} }
return new TransactionContextImpl(this, request, log); return new TransactionContextImpl(this, request, log);
} }
private StateMachineLogEntryProto createSMLogEntryProto(RaftClientRequest r,
ByteString logData, ByteString smData) {
StateMachineLogEntryProto.Builder builder =
StateMachineLogEntryProto.newBuilder();
builder.setCallId(r.getCallId())
.setClientId(r.getClientId().toByteString())
.setLogData(logData);
if (smData != null) {
builder.setStateMachineEntry(StateMachineEntryProto.newBuilder()
.setStateMachineData(smData).build());
}
return builder.build();
}
private ByteString getStateMachineData(StateMachineLogEntryProto entryProto) {
return entryProto.getStateMachineEntry().getStateMachineData();
}
private ContainerCommandRequestProto getRequestProto(ByteString request) private ContainerCommandRequestProto getRequestProto(ByteString request)
throws InvalidProtocolBufferException { throws InvalidProtocolBufferException {
return ContainerCommandRequestProto.parseFrom(request); return ContainerCommandRequestProto.parseFrom(request);
@ -315,7 +332,7 @@ public CompletableFuture<Message> writeStateMachineData(LogEntryProto entry) {
try { try {
metrics.incNumWriteStateMachineOps(); metrics.incNumWriteStateMachineOps();
final ContainerCommandRequestProto requestProto = final ContainerCommandRequestProto requestProto =
getRequestProto(entry.getSmLogEntry().getStateMachineData()); getRequestProto(getStateMachineData(entry.getStateMachineLogEntry()));
Type cmdType = requestProto.getCmdType(); Type cmdType = requestProto.getCmdType();
switch (cmdType) { switch (cmdType) {
case CreateContainer: case CreateContainer:
@ -345,8 +362,8 @@ public CompletableFuture<Message> query(Message request) {
} }
} }
private ByteString readStateMachineData(LogEntryProto entry, private ByteString readStateMachineData(ContainerCommandRequestProto
ContainerCommandRequestProto requestProto) { requestProto) {
WriteChunkRequestProto writeChunkRequestProto = WriteChunkRequestProto writeChunkRequestProto =
requestProto.getWriteChunk(); requestProto.getWriteChunk();
// Assert that store log entry is for COMMIT_DATA, the WRITE_DATA is // Assert that store log entry is for COMMIT_DATA, the WRITE_DATA is
@ -361,7 +378,8 @@ private ByteString readStateMachineData(LogEntryProto entry,
.setChunkData(writeChunkRequestProto.getChunkData()); .setChunkData(writeChunkRequestProto.getChunkData());
ContainerCommandRequestProto dataContainerCommandProto = ContainerCommandRequestProto dataContainerCommandProto =
ContainerCommandRequestProto.newBuilder(requestProto) ContainerCommandRequestProto.newBuilder(requestProto)
.setCmdType(Type.ReadChunk).setReadChunk(readChunkRequestProto) .setCmdType(Type.ReadChunk)
.setReadChunk(readChunkRequestProto)
.build(); .build();
// read the chunk // read the chunk
@ -376,7 +394,8 @@ private ByteString readStateMachineData(LogEntryProto entry,
final WriteChunkRequestProto.Builder dataWriteChunkProto = final WriteChunkRequestProto.Builder dataWriteChunkProto =
WriteChunkRequestProto.newBuilder(writeChunkRequestProto) WriteChunkRequestProto.newBuilder(writeChunkRequestProto)
// adding the state machine data // adding the state machine data
.setData(responseProto.getData()).setStage(Stage.WRITE_DATA); .setData(responseProto.getData())
.setStage(Stage.WRITE_DATA);
ContainerCommandRequestProto.Builder newStateMachineProto = ContainerCommandRequestProto.Builder newStateMachineProto =
ContainerCommandRequestProto.newBuilder(requestProto) ContainerCommandRequestProto.newBuilder(requestProto)
@ -410,21 +429,20 @@ public CompletableFuture<Void> flushStateMachineData(long index) {
@Override @Override
public CompletableFuture<ByteString> readStateMachineData( public CompletableFuture<ByteString> readStateMachineData(
LogEntryProto entry) { LogEntryProto entry) {
SMLogEntryProto smLogEntryProto = entry.getSmLogEntry(); StateMachineLogEntryProto smLogEntryProto = entry.getStateMachineLogEntry();
if (!smLogEntryProto.getStateMachineData().isEmpty()) { if (!getStateMachineData(smLogEntryProto).isEmpty()) {
return CompletableFuture.completedFuture(ByteString.EMPTY); return CompletableFuture.completedFuture(ByteString.EMPTY);
} }
try { try {
final ContainerCommandRequestProto requestProto = final ContainerCommandRequestProto requestProto =
getRequestProto(entry.getSmLogEntry().getData()); getRequestProto(entry.getStateMachineLogEntry().getLogData());
// readStateMachineData should only be called for "write" to Ratis. // readStateMachineData should only be called for "write" to Ratis.
Preconditions.checkArgument(!HddsUtils.isReadOnly(requestProto)); Preconditions.checkArgument(!HddsUtils.isReadOnly(requestProto));
if (requestProto.getCmdType() == Type.WriteChunk) { if (requestProto.getCmdType() == Type.WriteChunk) {
return CompletableFuture.supplyAsync(() -> return CompletableFuture.supplyAsync(() ->
readStateMachineData(entry, requestProto), readStateMachineData(requestProto), chunkExecutor);
chunkExecutor);
} else if (requestProto.getCmdType() == Type.CreateContainer) { } else if (requestProto.getCmdType() == Type.CreateContainer) {
return CompletableFuture.completedFuture(requestProto.toByteString()); return CompletableFuture.completedFuture(requestProto.toByteString());
} else { } else {
@ -462,7 +480,7 @@ public CompletableFuture<Message> applyTransaction(TransactionContext trx) {
try { try {
metrics.incNumApplyTransactionsOps(); metrics.incNumApplyTransactionsOps();
ContainerCommandRequestProto requestProto = ContainerCommandRequestProto requestProto =
getRequestProto(trx.getSMLogEntry().getData()); getRequestProto(trx.getStateMachineLogEntry().getLogData());
Type cmdType = requestProto.getCmdType(); Type cmdType = requestProto.getCmdType();
CompletableFuture<Message> future; CompletableFuture<Message> future;
if (cmdType == Type.PutBlock) { if (cmdType == Type.PutBlock) {
@ -490,6 +508,11 @@ public CompletableFuture<Message> applyTransaction(TransactionContext trx) {
.supplyAsync(() -> runCommand(containerCommandRequestProto), .supplyAsync(() -> runCommand(containerCommandRequestProto),
getCommandExecutor(requestProto)); getCommandExecutor(requestProto));
} else { } else {
// Make sure that in write chunk, the user data is not set
if (cmdType == Type.WriteChunk) {
Preconditions.checkArgument(requestProto
.getWriteChunk().getData().isEmpty());
}
future = CompletableFuture.supplyAsync(() -> runCommand(requestProto), future = CompletableFuture.supplyAsync(() -> runCommand(requestProto),
getCommandExecutor(requestProto)); getCommandExecutor(requestProto));
} }
@ -534,7 +557,6 @@ public void notifyExtendedNoLeader(RaftGroup group,
@Override @Override
public void close() throws IOException { public void close() throws IOException {
takeSnapshot();
for (int i = 0; i < numExecutors; i++) { for (int i = 0; i < numExecutors; i++) {
executors[i].shutdown(); executors[i].shutdown();
} }

View File

@ -103,7 +103,7 @@
<ldap-api.version>1.0.0-M33</ldap-api.version> <ldap-api.version>1.0.0-M33</ldap-api.version>
<!-- Apache Ratis version --> <!-- Apache Ratis version -->
<ratis.version>0.3.0-9b2d7b6-SNAPSHOT</ratis.version> <ratis.version>0.3.0-aa38160-SNAPSHOT</ratis.version>
<jcache.version>1.0-alpha-1</jcache.version> <jcache.version>1.0-alpha-1</jcache.version>
<ehcache.version>3.3.1</ehcache.version> <ehcache.version>3.3.1</ehcache.version>
<hikari.version>2.4.12</hikari.version> <hikari.version>2.4.12</hikari.version>