Skip to content
Merged
Show file tree
Hide file tree
Changes from 5 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 @@ -32,6 +32,7 @@
import org.apache.hadoop.hdds.security.x509.SecurityConfig;
import org.apache.hadoop.ozone.OzoneConfigKeys;

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

Expand Down Expand Up @@ -74,24 +75,16 @@ public CompletableFuture<Path> getContainerDataFromReplicas(long containerId,
for (DatanodeDetails datanode : sourceDatanodes) {
try {
if (result == null) {
GrpcReplicationClient grpcReplicationClient =
new GrpcReplicationClient(datanode.getIpAddress(),
datanode.getPort(Name.STANDALONE).getValue(),
workingDirectory, securityConfig, caCert);
result = grpcReplicationClient.download(containerId);
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);
return downloadContainer(containerId, datanode);
} catch (IOException e) {
LOG.error("Error on replicating container: " + containerId,
t);
e);
return null;
}
}).thenCompose(Function.identity());
Expand All @@ -107,6 +100,20 @@ public CompletableFuture<Path> getContainerDataFromReplicas(long containerId,

}

@VisibleForTesting
protected CompletableFuture<Path> downloadContainer(
long containerId,
DatanodeDetails datanode
) throws IOException {
CompletableFuture<Path> result;
GrpcReplicationClient grpcReplicationClient =
new GrpcReplicationClient(datanode.getIpAddress(),
datanode.getPort(Name.STANDALONE).getValue(),
workingDirectory, securityConfig, caCert);
result = grpcReplicationClient.download(containerId);
return result;
}

@Override
public void close() {
// noop
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,150 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

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

import java.io.IOException;
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.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 SimpleContainerDownloader.
*/
public class TestSimpleContainerDownloader {

@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());
}

/**
* 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) {

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

if (datanodes.contains(datanode)) {
if (directException) {
throw new IOException("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;
}
}