|
@@ -18,6 +18,8 @@ package org.apache.hadoop.hdds.scm.container.replication;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
import java.util.ArrayList;
|
|
|
+import java.util.Arrays;
|
|
|
+import java.util.HashSet;
|
|
|
import java.util.Iterator;
|
|
|
import java.util.List;
|
|
|
import java.util.Objects;
|
|
@@ -26,27 +28,22 @@ import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
|
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
|
|
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
|
|
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
|
|
|
-import org.apache.hadoop.hdds.protocol.proto
|
|
|
- .StorageContainerDatanodeProtocolProtos.ReplicateContainerCommandProto;
|
|
|
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ReplicateContainerCommandProto;
|
|
|
import org.apache.hadoop.hdds.scm.TestUtils;
|
|
|
+import org.apache.hadoop.hdds.scm.container.ContainerID;
|
|
|
import org.apache.hadoop.hdds.scm.container.ContainerStateManager;
|
|
|
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
|
|
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
|
|
import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineID;
|
|
|
-import org.apache.hadoop.hdds.scm.container.placement.algorithms
|
|
|
- .ContainerPlacementPolicy;
|
|
|
-import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager
|
|
|
- .ReplicationRequestToRepeat;
|
|
|
+import org.apache.hadoop.hdds.scm.container.placement.algorithms.ContainerPlacementPolicy;
|
|
|
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager.ReplicationRequestToRepeat;
|
|
|
import org.apache.hadoop.hdds.scm.events.SCMEvents;
|
|
|
import org.apache.hadoop.hdds.server.events.EventQueue;
|
|
|
import org.apache.hadoop.ozone.lease.LeaseManager;
|
|
|
import org.apache.hadoop.ozone.protocol.commands.CommandForDatanode;
|
|
|
|
|
|
import com.google.common.base.Preconditions;
|
|
|
-import static org.apache.hadoop.hdds.scm.events.SCMEvents
|
|
|
- .TRACK_REPLICATE_COMMAND;
|
|
|
-import org.apache.log4j.Level;
|
|
|
-import org.apache.log4j.Logger;
|
|
|
+import static org.apache.hadoop.hdds.scm.events.SCMEvents.TRACK_REPLICATE_COMMAND;
|
|
|
import org.junit.Assert;
|
|
|
import org.junit.Before;
|
|
|
import org.junit.Test;
|
|
@@ -69,6 +66,8 @@ public class TestReplicationManager {
|
|
|
|
|
|
private ContainerPlacementPolicy containerPlacementPolicy;
|
|
|
private List<DatanodeDetails> listOfDatanodeDetails;
|
|
|
+ private LeaseManager<Long> leaseManager;
|
|
|
+ private ReplicationManager replicationManager;
|
|
|
|
|
|
@Before
|
|
|
public void initReplicationManager() throws IOException {
|
|
@@ -86,7 +85,6 @@ public class TestReplicationManager {
|
|
|
|
|
|
containerStateManager = Mockito.mock(ContainerStateManager.class);
|
|
|
|
|
|
- //container with 2 replicas
|
|
|
ContainerInfo containerInfo = new ContainerInfo.Builder()
|
|
|
.setState(LifeCycleState.CLOSED)
|
|
|
.build();
|
|
@@ -94,6 +92,16 @@ public class TestReplicationManager {
|
|
|
when(containerStateManager.getContainer(anyObject()))
|
|
|
.thenReturn(containerInfo);
|
|
|
|
|
|
+ when(containerStateManager.getContainerReplicas(new ContainerID(1L)))
|
|
|
+ .thenReturn(new HashSet<>(Arrays.asList(
|
|
|
+ listOfDatanodeDetails.get(0),
|
|
|
+ listOfDatanodeDetails.get(1)
|
|
|
+ )));
|
|
|
+
|
|
|
+
|
|
|
+ when(containerStateManager.getContainerReplicas(new ContainerID(3L)))
|
|
|
+ .thenReturn(new HashSet<>());
|
|
|
+
|
|
|
queue = new EventQueue();
|
|
|
|
|
|
trackReplicationEvents = new ArrayList<>();
|
|
@@ -104,32 +112,53 @@ public class TestReplicationManager {
|
|
|
queue.addHandler(SCMEvents.DATANODE_COMMAND,
|
|
|
(event, publisher) -> copyEvents.add(event));
|
|
|
|
|
|
+ leaseManager = new LeaseManager<>("Test", 100000L);
|
|
|
+
|
|
|
+ replicationManager = new ReplicationManager(containerPlacementPolicy,
|
|
|
+ containerStateManager, queue, leaseManager);
|
|
|
+
|
|
|
+
|
|
|
+
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Container should be replicated but no source replicas.
|
|
|
+ */
|
|
|
+ @Test()
|
|
|
+ public void testNoExistingReplicas() throws InterruptedException {
|
|
|
+ try {
|
|
|
+ leaseManager.start();
|
|
|
+ replicationManager.start();
|
|
|
+
|
|
|
+ //WHEN
|
|
|
+ queue.fireEvent(SCMEvents.REPLICATE_CONTAINER,
|
|
|
+ new ReplicationRequest(3L, (short) 2, System.currentTimeMillis(),
|
|
|
+ (short) 3));
|
|
|
+
|
|
|
+ Thread.sleep(500L);
|
|
|
+ queue.processAll(1000L);
|
|
|
+
|
|
|
+ //THEN
|
|
|
+ Assert.assertEquals(0, trackReplicationEvents.size());
|
|
|
+ Assert.assertEquals(0, copyEvents.size());
|
|
|
+
|
|
|
+ } finally {
|
|
|
+ if (leaseManager != null) {
|
|
|
+ leaseManager.shutdown();
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
public void testEventSending() throws InterruptedException, IOException {
|
|
|
|
|
|
-
|
|
|
//GIVEN
|
|
|
-
|
|
|
- LeaseManager<Long> leaseManager = new LeaseManager<>("Test", 100000L);
|
|
|
try {
|
|
|
leaseManager.start();
|
|
|
|
|
|
- ReplicationManager replicationManager =
|
|
|
- new ReplicationManager(containerPlacementPolicy,
|
|
|
- containerStateManager,
|
|
|
- queue, leaseManager) {
|
|
|
- @Override
|
|
|
- protected List<DatanodeDetails> getCurrentReplicas(
|
|
|
- ReplicationRequest request) throws IOException {
|
|
|
- return listOfDatanodeDetails.subList(0, 2);
|
|
|
- }
|
|
|
- };
|
|
|
replicationManager.start();
|
|
|
|
|
|
//WHEN
|
|
|
-
|
|
|
queue.fireEvent(SCMEvents.REPLICATE_CONTAINER,
|
|
|
new ReplicationRequest(1L, (short) 2, System.currentTimeMillis(),
|
|
|
(short) 3));
|
|
@@ -138,7 +167,6 @@ public class TestReplicationManager {
|
|
|
queue.processAll(1000L);
|
|
|
|
|
|
//THEN
|
|
|
-
|
|
|
Assert.assertEquals(1, trackReplicationEvents.size());
|
|
|
Assert.assertEquals(1, copyEvents.size());
|
|
|
} finally {
|
|
@@ -150,22 +178,14 @@ public class TestReplicationManager {
|
|
|
|
|
|
@Test
|
|
|
public void testCommandWatcher() throws InterruptedException, IOException {
|
|
|
+ LeaseManager<Long> rapidLeaseManager =
|
|
|
+ new LeaseManager<>("Test", 1000L);
|
|
|
|
|
|
- Logger.getRootLogger().setLevel(Level.DEBUG);
|
|
|
- LeaseManager<Long> leaseManager = new LeaseManager<>("Test", 1000L);
|
|
|
+ replicationManager = new ReplicationManager(containerPlacementPolicy,
|
|
|
+ containerStateManager, queue, rapidLeaseManager);
|
|
|
|
|
|
try {
|
|
|
- leaseManager.start();
|
|
|
-
|
|
|
- ReplicationManager replicationManager =
|
|
|
- new ReplicationManager(containerPlacementPolicy,
|
|
|
- containerStateManager, queue, leaseManager) {
|
|
|
- @Override
|
|
|
- protected List<DatanodeDetails> getCurrentReplicas(
|
|
|
- ReplicationRequest request) throws IOException {
|
|
|
- return listOfDatanodeDetails.subList(0, 2);
|
|
|
- }
|
|
|
- };
|
|
|
+ rapidLeaseManager.start();
|
|
|
replicationManager.start();
|
|
|
|
|
|
queue.fireEvent(SCMEvents.REPLICATE_CONTAINER,
|
|
@@ -192,8 +212,8 @@ public class TestReplicationManager {
|
|
|
Assert.assertEquals(2, copyEvents.size());
|
|
|
|
|
|
} finally {
|
|
|
- if (leaseManager != null) {
|
|
|
- leaseManager.shutdown();
|
|
|
+ if (rapidLeaseManager != null) {
|
|
|
+ rapidLeaseManager.shutdown();
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -209,7 +229,7 @@ public class TestReplicationManager {
|
|
|
ReplicationType.STAND_ALONE, ReplicationFactor.ONE,
|
|
|
PipelineID.randomId());
|
|
|
pipeline.addMember(leader);
|
|
|
- for (; i.hasNext(); ) {
|
|
|
+ while (i.hasNext()) {
|
|
|
pipeline.addMember(i.next());
|
|
|
}
|
|
|
return pipeline;
|