HDDS-866. Handle RaftRetryFailureException in OzoneClient. Contributed by Shashikant Banerjee.
This commit is contained in:
parent
176bb3f812
commit
ee44b069c6
@ -20,6 +20,7 @@
|
||||
|
||||
import org.apache.hadoop.hdds.HddsUtils;
|
||||
import org.apache.ratis.proto.RaftProtos;
|
||||
import org.apache.ratis.protocol.RaftRetryFailureException;
|
||||
import org.apache.ratis.retry.RetryPolicy;
|
||||
import org.apache.ratis.thirdparty.com.google.protobuf
|
||||
.InvalidProtocolBufferException;
|
||||
@ -196,10 +197,16 @@ public XceiverClientAsyncReply sendCommandAsync(
|
||||
new ArrayList<>();
|
||||
CompletableFuture<ContainerCommandResponseProto> containerCommandResponse =
|
||||
raftClientReply.whenComplete((reply, e) -> LOG
|
||||
.debug("received reply {} for request: {} exception: {}", request,
|
||||
.info("received reply {} for request: {} exception: {}", request,
|
||||
reply, e))
|
||||
.thenApply(reply -> {
|
||||
try {
|
||||
// we need to handle RaftRetryFailure Exception
|
||||
RaftRetryFailureException raftRetryFailureException =
|
||||
reply.getRetryFailureException();
|
||||
if (raftRetryFailureException != null) {
|
||||
throw new CompletionException(raftRetryFailureException);
|
||||
}
|
||||
ContainerCommandResponseProto response =
|
||||
ContainerCommandResponseProto
|
||||
.parseFrom(reply.getMessage().getContent());
|
||||
|
@ -284,7 +284,12 @@ private void createContainer(ContainerCommandRequestProto containerRequest) {
|
||||
@Override
|
||||
public void validateContainerCommand(
|
||||
ContainerCommandRequestProto msg) throws StorageContainerException {
|
||||
ContainerType containerType = msg.getCreateContainer().getContainerType();
|
||||
long containerID = msg.getContainerID();
|
||||
Container container = getContainer(containerID);
|
||||
if (container == null) {
|
||||
return;
|
||||
}
|
||||
ContainerType containerType = container.getContainerType();
|
||||
ContainerProtos.Type cmdType = msg.getCmdType();
|
||||
AuditAction action =
|
||||
ContainerCommandRequestPBHelper.getAuditAction(cmdType);
|
||||
@ -299,35 +304,30 @@ public void validateContainerCommand(
|
||||
audit(action, eventType, params, AuditEventStatus.FAILURE, ex);
|
||||
throw ex;
|
||||
}
|
||||
long containerID = msg.getContainerID();
|
||||
Container container;
|
||||
container = getContainer(containerID);
|
||||
|
||||
if (container != null) {
|
||||
State containerState = container.getContainerState();
|
||||
if (!HddsUtils.isReadOnly(msg) && containerState != State.OPEN) {
|
||||
switch (cmdType) {
|
||||
case CreateContainer:
|
||||
// Create Container is idempotent. There is nothing to validate.
|
||||
break;
|
||||
case CloseContainer:
|
||||
// If the container is unhealthy, closeContainer will be rejected
|
||||
// while execution. Nothing to validate here.
|
||||
break;
|
||||
default:
|
||||
// if the container is not open, no updates can happen. Just throw
|
||||
// an exception
|
||||
ContainerNotOpenException cex = new ContainerNotOpenException(
|
||||
"Container " + containerID + " in " + containerState + " state");
|
||||
audit(action, eventType, params, AuditEventStatus.FAILURE, cex);
|
||||
throw cex;
|
||||
}
|
||||
} else if (HddsUtils.isReadOnly(msg) && containerState == State.INVALID) {
|
||||
InvalidContainerStateException iex = new InvalidContainerStateException(
|
||||
State containerState = container.getContainerState();
|
||||
if (!HddsUtils.isReadOnly(msg) && containerState != State.OPEN) {
|
||||
switch (cmdType) {
|
||||
case CreateContainer:
|
||||
// Create Container is idempotent. There is nothing to validate.
|
||||
break;
|
||||
case CloseContainer:
|
||||
// If the container is unhealthy, closeContainer will be rejected
|
||||
// while execution. Nothing to validate here.
|
||||
break;
|
||||
default:
|
||||
// if the container is not open, no updates can happen. Just throw
|
||||
// an exception
|
||||
ContainerNotOpenException cex = new ContainerNotOpenException(
|
||||
"Container " + containerID + " in " + containerState + " state");
|
||||
audit(action, eventType, params, AuditEventStatus.FAILURE, iex);
|
||||
throw iex;
|
||||
audit(action, eventType, params, AuditEventStatus.FAILURE, cex);
|
||||
throw cex;
|
||||
}
|
||||
} else if (HddsUtils.isReadOnly(msg) && containerState == State.INVALID) {
|
||||
InvalidContainerStateException iex = new InvalidContainerStateException(
|
||||
"Container " + containerID + " in " + containerState + " state");
|
||||
audit(action, eventType, params, AuditEventStatus.FAILURE, iex);
|
||||
throw iex;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -464,17 +464,13 @@ private void handlePipelineFailure(RaftGroupId groupId,
|
||||
|
||||
@Override
|
||||
public boolean isExist(HddsProtos.PipelineID pipelineId) {
|
||||
try {
|
||||
for (RaftGroupId groupId : server.getGroupIds()) {
|
||||
if (PipelineID.valueOf(
|
||||
groupId.getUuid()).getProtobuf().equals(pipelineId)) {
|
||||
return true;
|
||||
}
|
||||
for (RaftGroupId groupId : server.getGroupIds()) {
|
||||
if (PipelineID.valueOf(groupId.getUuid()).getProtobuf()
|
||||
.equals(pipelineId)) {
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
} catch (IOException e) {
|
||||
return false;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -313,6 +313,9 @@ private void compactDB() throws StorageContainerException {
|
||||
try {
|
||||
MetadataStore db = BlockUtils.getDB(containerData, config);
|
||||
db.compactDB();
|
||||
LOG.info("Container {} is closed with bcsId {}.",
|
||||
containerData.getContainerID(),
|
||||
containerData.getBlockCommitSequenceId());
|
||||
} catch (StorageContainerException ex) {
|
||||
throw ex;
|
||||
} catch (IOException ex) {
|
||||
|
@ -46,7 +46,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||
<hdds.version>0.4.0-SNAPSHOT</hdds.version>
|
||||
|
||||
<!-- Apache Ratis version -->
|
||||
<ratis.version>0.3.0-6f3419a-SNAPSHOT</ratis.version>
|
||||
<ratis.version>0.4.0-b600fc2-SNAPSHOT</ratis.version>
|
||||
|
||||
<bouncycastle.version>1.60</bouncycastle.version>
|
||||
|
||||
|
@ -39,6 +39,7 @@
|
||||
import org.apache.hadoop.hdds.scm.protocolPB
|
||||
.StorageContainerLocationProtocolClientSideTranslatorPB;
|
||||
import org.apache.hadoop.hdds.scm.storage.ChunkOutputStream;
|
||||
import org.apache.ratis.protocol.RaftRetryFailureException;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
@ -393,7 +394,7 @@ private void handleException(ChunkOutputStreamEntry streamEntry,
|
||||
|
||||
private boolean checkIfContainerIsClosed(IOException ioe) {
|
||||
if (ioe.getCause() != null) {
|
||||
return checkIfContainerNotOpenException(ioe) || Optional
|
||||
return checkIfContainerNotOpenOrRaftRetryFailureException(ioe) || Optional
|
||||
.of(ioe.getCause())
|
||||
.filter(e -> e instanceof StorageContainerException)
|
||||
.map(e -> (StorageContainerException) e)
|
||||
@ -403,10 +404,12 @@ private boolean checkIfContainerIsClosed(IOException ioe) {
|
||||
return false;
|
||||
}
|
||||
|
||||
private boolean checkIfContainerNotOpenException(IOException ioe) {
|
||||
private boolean checkIfContainerNotOpenOrRaftRetryFailureException(
|
||||
IOException ioe) {
|
||||
Throwable t = ioe.getCause();
|
||||
while (t != null) {
|
||||
if (t instanceof ContainerNotOpenException) {
|
||||
if (t instanceof ContainerNotOpenException
|
||||
|| t instanceof RaftRetryFailureException) {
|
||||
return true;
|
||||
}
|
||||
t = t.getCause();
|
||||
|
@ -110,10 +110,6 @@ public void shutdown() {
|
||||
}
|
||||
}
|
||||
|
||||
// TODO: currently, shutting down 2 datanodes in Ratis leads to
|
||||
// watchForCommit Api in RaftClient to hand=g forever. Once that gets
|
||||
// fixed, we need to execute the tets with 2 node failures.
|
||||
|
||||
@Test
|
||||
public void testBlockWritesWithDnFailures() throws Exception {
|
||||
String keyName = "ratis3";
|
||||
@ -139,7 +135,7 @@ public void testBlockWritesWithDnFailures() throws Exception {
|
||||
.getPipeline(container.getPipelineID());
|
||||
List<DatanodeDetails> datanodes = pipeline.getNodes();
|
||||
cluster.shutdownHddsDatanode(datanodes.get(0));
|
||||
// cluster.shutdownHddsDatanode(datanodes.get(1));
|
||||
cluster.shutdownHddsDatanode(datanodes.get(1));
|
||||
// The write will fail but exception will be handled and length will be
|
||||
// updated correctly in OzoneManager once the steam is closed
|
||||
key.close();
|
||||
@ -151,7 +147,6 @@ public void testBlockWritesWithDnFailures() throws Exception {
|
||||
OmKeyInfo keyInfo = cluster.getOzoneManager().lookupKey(keyArgs);
|
||||
Assert.assertEquals(data.length, keyInfo.getDataSize());
|
||||
validateData(keyName, data);
|
||||
cluster.restartHddsDatanode(datanodes.get(0), true);
|
||||
}
|
||||
|
||||
@Test
|
||||
@ -179,8 +174,8 @@ public void testMultiBlockWritesWithDnFailures() throws Exception {
|
||||
.getPipeline(container.getPipelineID());
|
||||
List<DatanodeDetails> datanodes = pipeline.getNodes();
|
||||
cluster.shutdownHddsDatanode(datanodes.get(0));
|
||||
cluster.shutdownHddsDatanode(datanodes.get(1));
|
||||
|
||||
// cluster.shutdownHddsDatanode(datanodes.get(1));
|
||||
// The write will fail but exception will be handled and length will be
|
||||
// updated correctly in OzoneManager once the steam is closed
|
||||
key.write(data.getBytes());
|
||||
@ -192,7 +187,6 @@ public void testMultiBlockWritesWithDnFailures() throws Exception {
|
||||
OmKeyInfo keyInfo = cluster.getOzoneManager().lookupKey(keyArgs);
|
||||
Assert.assertEquals(2 * data.getBytes().length, keyInfo.getDataSize());
|
||||
validateData(keyName, data.concat(data).getBytes());
|
||||
cluster.restartHddsDatanode(datanodes.get(0), true);
|
||||
}
|
||||
|
||||
private OzoneOutputStream createKey(String keyName, ReplicationType type,
|
||||
|
@ -33,7 +33,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||
<hadoop.version>3.2.1-SNAPSHOT</hadoop.version>
|
||||
<hdds.version>0.4.0-SNAPSHOT</hdds.version>
|
||||
<ozone.version>0.4.0-SNAPSHOT</ozone.version>
|
||||
<ratis.version>0.3.0-6f3419a-SNAPSHOT</ratis.version>
|
||||
<ratis.version>0.4.0-b600fc2-SNAPSHOT</ratis.version>
|
||||
<bouncycastle.version>1.60</bouncycastle.version>
|
||||
<ozone.release>Badlands</ozone.release>
|
||||
<declared.ozone.version>${ozone.version}</declared.ozone.version>
|
||||
|
Loading…
Reference in New Issue
Block a user