HDDS-721. NullPointerException thrown while trying to read a file when datanode restarted. Contributed by Shashikant Banerjee.
This commit is contained in:
parent
da32f6537b
commit
5e48a70474
@ -84,7 +84,7 @@ public XceiverClientGrpc(Pipeline pipeline, Configuration config) {
|
|||||||
public void connect() throws Exception {
|
public void connect() throws Exception {
|
||||||
|
|
||||||
// leader by default is the 1st datanode in the datanode list of pipleline
|
// leader by default is the 1st datanode in the datanode list of pipleline
|
||||||
DatanodeDetails leader = this.pipeline.getLeader();
|
DatanodeDetails leader = this.pipeline.getMachines().get(0);
|
||||||
// just make a connection to the 1st datanode at the beginning
|
// just make a connection to the 1st datanode at the beginning
|
||||||
connectToDatanode(leader);
|
connectToDatanode(leader);
|
||||||
}
|
}
|
||||||
@ -201,7 +201,7 @@ public ContainerCommandResponseProto sendCommandWithRetry(
|
|||||||
public CompletableFuture<ContainerCommandResponseProto> sendCommandAsync(
|
public CompletableFuture<ContainerCommandResponseProto> sendCommandAsync(
|
||||||
ContainerCommandRequestProto request)
|
ContainerCommandRequestProto request)
|
||||||
throws IOException, ExecutionException, InterruptedException {
|
throws IOException, ExecutionException, InterruptedException {
|
||||||
return sendCommandAsync(request, pipeline.getLeader());
|
return sendCommandAsync(request, pipeline.getMachines().get(0));
|
||||||
}
|
}
|
||||||
|
|
||||||
private CompletableFuture<ContainerCommandResponseProto> sendCommandAsync(
|
private CompletableFuture<ContainerCommandResponseProto> sendCommandAsync(
|
||||||
|
@ -279,6 +279,10 @@ public static LengthInputStream getFromOmKeyInfo(
|
|||||||
ContainerWithPipeline containerWithPipeline =
|
ContainerWithPipeline containerWithPipeline =
|
||||||
storageContainerLocationClient.getContainerWithPipeline(containerID);
|
storageContainerLocationClient.getContainerWithPipeline(containerID);
|
||||||
Pipeline pipeline = containerWithPipeline.getPipeline();
|
Pipeline pipeline = containerWithPipeline.getPipeline();
|
||||||
|
if (pipeline.getMachines().isEmpty()) {
|
||||||
|
throw new IOException(
|
||||||
|
"No datanodes found in the pipeline " + pipeline.getId());
|
||||||
|
}
|
||||||
|
|
||||||
// irrespective of the container state, we will always read via Standalone
|
// irrespective of the container state, we will always read via Standalone
|
||||||
// protocol.
|
// protocol.
|
||||||
|
@ -24,6 +24,7 @@
|
|||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
||||||
import org.apache.hadoop.hdds.client.BlockID;
|
import org.apache.hadoop.hdds.client.BlockID;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
||||||
|
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
||||||
import org.apache.hadoop.io.retry.RetryPolicy;
|
import org.apache.hadoop.io.retry.RetryPolicy;
|
||||||
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
|
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
|
||||||
@ -191,9 +192,14 @@ private void checkKeyLocationInfo(OmKeyLocationInfo subKeyInfo)
|
|||||||
ContainerWithPipeline containerWithPipeline = scmClient
|
ContainerWithPipeline containerWithPipeline = scmClient
|
||||||
.getContainerWithPipeline(subKeyInfo.getContainerID());
|
.getContainerWithPipeline(subKeyInfo.getContainerID());
|
||||||
ContainerInfo container = containerWithPipeline.getContainerInfo();
|
ContainerInfo container = containerWithPipeline.getContainerInfo();
|
||||||
|
Pipeline pipeline = containerWithPipeline.getPipeline();
|
||||||
|
if (pipeline.getMachines().isEmpty()) {
|
||||||
|
throw new IOException(
|
||||||
|
"No datanodes found in the pipeline " + pipeline.getId());
|
||||||
|
}
|
||||||
|
|
||||||
XceiverClientSpi xceiverClient =
|
XceiverClientSpi xceiverClient =
|
||||||
xceiverClientManager.acquireClient(containerWithPipeline.getPipeline());
|
xceiverClientManager.acquireClient(pipeline);
|
||||||
// create container if needed
|
// create container if needed
|
||||||
if (subKeyInfo.getShouldCreateContainer()) {
|
if (subKeyInfo.getShouldCreateContainer()) {
|
||||||
try {
|
try {
|
||||||
|
Loading…
x
Reference in New Issue
Block a user