adoroszlai commented on a change in pull request #1551: URL: https://github.com/apache/ozone/pull/1551#discussion_r521327400
########## File path: hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestSimpleContainerDownloader.java ########## @@ -0,0 +1,120 @@ +/* + * 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(); + + //WHEN + final Path result = + downloader.getContainerDataFromReplicas(1L, datanodes) + .get(1L, TimeUnit.SECONDS); + + //THEN + Assert.assertEquals(datanodes.get(0).getUuidString(), result.toString()); + } + + @Test + public void testGetContainerDataFromReplicasOneFailure() throws Exception { + + //GIVEN + List<DatanodeDetails> datanodes = createDatanodes(); + + SimpleContainerDownloader downloader = + createDownloaderWithPredefinedFailures(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. + */ + private SimpleContainerDownloader createDownloaderWithPredefinedFailures( + 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)) { + throw new IOException("Unavailable datanode"); Review comment: `downloadContainer` may fail in two ways: `IOException` may be thrown immediately or the returned `CompletableFuture` may be completed exceptionally. These hit different code paths in `getContainerDataFromReplicas`. I think we should cover both cases. ########## File path: hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SimpleContainerDownloader.java ########## @@ -74,21 +75,13 @@ public SimpleContainerDownloader(ConfigurationSource conf, 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); Review comment: Not introduced in this patch, but I think this should be: ```suggestion } catch (IOException e) { LOG.error("Error on replicating container: " + containerId, e); ``` ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
