Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -18,15 +18,13 @@

package org.apache.hadoop.ozone.container.replication;

import java.io.IOException;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.security.cert.X509Certificate;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.function.Function;

import org.apache.hadoop.hdds.conf.ConfigurationSource;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
Expand All @@ -35,6 +33,7 @@
import org.apache.hadoop.ozone.OzoneConfigKeys;

import com.google.common.annotations.VisibleForTesting;
import org.jetbrains.annotations.NotNull;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand All @@ -54,8 +53,10 @@ public class SimpleContainerDownloader implements ContainerDownloader {
private final SecurityConfig securityConfig;
private final X509Certificate caCert;

public SimpleContainerDownloader(ConfigurationSource conf,
X509Certificate caCert) {
public SimpleContainerDownloader(
ConfigurationSource conf,
X509Certificate caCert
) {

String workDirString =
conf.get(OzoneConfigKeys.OZONE_CONTAINER_COPY_WORKDIR);
Expand All @@ -71,38 +72,31 @@ public SimpleContainerDownloader(ConfigurationSource conf,
}

@Override
public CompletableFuture<Path> getContainerDataFromReplicas(long containerId,
List<DatanodeDetails> sourceDatanodes) {
public CompletableFuture<Path> getContainerDataFromReplicas(
long containerId,
List<DatanodeDetails> sourceDatanodes
) {

CompletableFuture<Path> result = null;

//There is a chance for the download is successful but import is failed,
//due to data corruption. We need a random selected datanode to have a
//chance to succeed next time.
final ArrayList<DatanodeDetails> shuffledDatanodes =
new ArrayList<>(sourceDatanodes);
Collections.shuffle(shuffledDatanodes);
final List<DatanodeDetails> shuffledDatanodes =
shuffleDatanodes(sourceDatanodes);

for (DatanodeDetails datanode : shuffledDatanodes) {
try {
if (result == null) {
result = downloadContainer(containerId, datanode);
} else {
result = result.thenApply(CompletableFuture::completedFuture)
.exceptionally(t -> {
LOG.error("Error on replicating container: " + containerId, t);
try {
GrpcReplicationClient grpcReplicationClient =
new GrpcReplicationClient(datanode.getIpAddress(),
datanode.getPort(Name.STANDALONE).getValue(),
workingDirectory, securityConfig, caCert);
return grpcReplicationClient.download(containerId);
} catch (IOException e) {
LOG.error("Error on replicating container: " + containerId,
t);
return null;
}
}).thenCompose(Function.identity());
result = result.exceptionally(t -> {
LOG.error("Error on replicating container: " + containerId, t);
try {
return downloadContainer(containerId, datanode).join();
} catch (Exception e) {
LOG.error("Error on replicating container: " + containerId,
e);
return null;
}
});
}
} catch (Exception ex) {
LOG.error(String.format(
Expand All @@ -114,18 +108,42 @@ public CompletableFuture<Path> getContainerDataFromReplicas(long containerId,

}

//There is a chance for the download is successful but import is failed,
//due to data corruption. We need a random selected datanode to have a
//chance to succeed next time.
@NotNull
protected List<DatanodeDetails> shuffleDatanodes(
List<DatanodeDetails> sourceDatanodes
) {

final ArrayList<DatanodeDetails> shuffledDatanodes =
new ArrayList<>(sourceDatanodes);

Collections.shuffle(shuffledDatanodes);

return shuffledDatanodes;
}

@VisibleForTesting
protected CompletableFuture<Path> downloadContainer(
long containerId,
DatanodeDetails datanode
) throws Exception {
CompletableFuture<Path> result;
try (GrpcReplicationClient grpcReplicationClient =
GrpcReplicationClient grpcReplicationClient =
new GrpcReplicationClient(datanode.getIpAddress(),
datanode.getPort(Name.STANDALONE).getValue(),
workingDirectory, securityConfig, caCert)) {
result = grpcReplicationClient.download(containerId);
}
workingDirectory, securityConfig, caCert);
result = grpcReplicationClient.download(containerId)
.thenApply(r -> {
try {
grpcReplicationClient.close();
Comment on lines +137 to +140
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

So if replication was slow, the previous version may have closed the client prematurely?

} catch (Exception e) {
LOG.error("Couldn't close Grpc replication client", e);
}
return r;
});

return result;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,28 +21,88 @@
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;

import org.apache.hadoop.hdds.conf.ConfigurationSource;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;

import org.junit.Assert;
import org.junit.Test;

/**
* Test container downloader.
/*
* Test SimpleContainerDownloader.
*/
public class TestSimpleContainerDownloader {

private static final String SUCCESS_PATH = "downloaded";

@Test
public void testGetContainerDataFromReplicasHappyPath() throws Exception {

//GIVEN
List<DatanodeDetails> datanodes = createDatanodes();

SimpleContainerDownloader downloader =
createDownloaderWithPredefinedFailures(true);

//WHEN
final Path result =
downloader.getContainerDataFromReplicas(1L, datanodes)
.get(1L, TimeUnit.SECONDS);

//THEN
Assert.assertEquals(datanodes.get(0).getUuidString(), result.toString());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actually, these checks are no longer valid after 5e8aaee since SimpleContainerDownloader shuffles the datanodes.

//There is a chance for the download is successful but import is failed,
//due to data corruption. We need a random selected datanode to have a
//chance to succeed next time.
final ArrayList<DatanodeDetails> shuffledDatanodes =
new ArrayList<>(sourceDatanodes);
Collections.shuffle(shuffledDatanodes);

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looking into it. Seems I stepped in my own trap ;-)

}

@Test
public void testGetContainerDataFromReplicasDirectFailure()
throws Exception {

//GIVEN
List<DatanodeDetails> datanodes = createDatanodes();

SimpleContainerDownloader downloader =
createDownloaderWithPredefinedFailures(true, datanodes.get(0));

//WHEN
final Path result =
downloader.getContainerDataFromReplicas(1L, datanodes)
.get(1L, TimeUnit.SECONDS);

//THEN
//first datanode is failed, second worked
Assert.assertEquals(datanodes.get(1).getUuidString(), result.toString());
}

@Test
public void testGetContainerDataFromReplicasAsyncFailure() throws Exception {

//GIVEN
List<DatanodeDetails> datanodes = createDatanodes();

SimpleContainerDownloader downloader =
createDownloaderWithPredefinedFailures(false, datanodes.get(0));

//WHEN
final Path result =
downloader.getContainerDataFromReplicas(1L, datanodes)
.get(1L, TimeUnit.SECONDS);

//THEN
//first datanode is failed, second worked
Assert.assertEquals(datanodes.get(1).getUuidString(), result.toString());
}

/**
* Test if different datanode is used for each download attempt.
*/
@Test(timeout = 1000L)
@Test(timeout = 10_000L)
public void testRandomSelection()
throws ExecutionException, InterruptedException {

Expand All @@ -62,24 +122,79 @@ protected CompletableFuture<Path> downloadContainer(
}
};

//WHEN executed, THEN at least once the second datanode should be returned.
//WHEN executed, THEN at least once the second datanode should be
//returned.
for (int i = 0; i < 10000; i++) {
Path path = downloader.getContainerDataFromReplicas(1L, datanodes).get();
Path path =
downloader.getContainerDataFromReplicas(1L, datanodes).get();
if (path.toString().equals(datanodes.get(1).getUuidString())) {
return;
}
}

//there is 1/2^10_000 chance for false positive, which is practically 0.
//there is 1/3^10_000 chance for false positive, which is practically 0.
Assert.fail(
"Datanodes are selected 10000 times but second datanode was never "
+ "used.");
}

/**
* Creates downloader which fails with datanodes in the arguments.
*
* @param directException if false the exception will be wrapped in the
* returning future.
*/
private SimpleContainerDownloader createDownloaderWithPredefinedFailures(
boolean directException,
DatanodeDetails... failedDatanodes
) {

ConfigurationSource conf = new OzoneConfiguration();

final List<DatanodeDetails> datanodes =
Arrays.asList(failedDatanodes);

return new SimpleContainerDownloader(conf, null) {

//for retry testing we use predictable list of datanodes.
@Override
protected List<DatanodeDetails> shuffleDatanodes(
List<DatanodeDetails> sourceDatanodes
) {
//turn off randomization
return sourceDatanodes;
}

@Override
protected CompletableFuture<Path> downloadContainer(
long containerId,
DatanodeDetails datanode
) throws Exception {

if (datanodes.contains(datanode)) {
if (directException) {
throw new RuntimeException("Unavailable datanode");
} else {
return CompletableFuture.supplyAsync(() -> {
throw new RuntimeException("Unavailable datanode");
});
}
} else {

//path includes the dn id to make it possible to assert.
return CompletableFuture.completedFuture(
Paths.get(datanode.getUuidString()));
}

}
};
}

private List<DatanodeDetails> createDatanodes() {
List<DatanodeDetails> datanodes = new ArrayList<>();
datanodes.add(MockDatanodeDetails.randomDatanodeDetails());
datanodes.add(MockDatanodeDetails.randomDatanodeDetails());
datanodes.add(MockDatanodeDetails.randomDatanodeDetails());
return datanodes;
}
}