|
@@ -19,8 +19,13 @@
|
|
|
package org.apache.hadoop.yarn.service;
|
|
|
|
|
|
import com.google.common.base.Supplier;
|
|
|
+import com.google.common.collect.Lists;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.registry.client.api.RegistryOperations;
|
|
|
+import org.apache.hadoop.registry.client.binding.RegistryPathUtils;
|
|
|
+import org.apache.hadoop.registry.client.types.ServiceRecord;
|
|
|
+import org.apache.hadoop.registry.client.types.yarn.PersistencePolicies;
|
|
|
+import org.apache.hadoop.registry.client.types.yarn.YarnRegistryAttributes;
|
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
|
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
|
|
|
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
|
|
@@ -42,15 +47,24 @@ import org.apache.hadoop.yarn.service.exceptions.BadClusterStateException;
|
|
|
import org.apache.hadoop.yarn.service.registry.YarnRegistryViewForProviders;
|
|
|
import org.apache.hadoop.yarn.service.utils.SliderFileSystem;
|
|
|
import org.apache.hadoop.yarn.util.Records;
|
|
|
+import org.slf4j.Logger;
|
|
|
+import org.slf4j.LoggerFactory;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
import java.util.*;
|
|
|
+import java.util.concurrent.ConcurrentHashMap;
|
|
|
import java.util.concurrent.TimeoutException;
|
|
|
|
|
|
+import static org.mockito.Matchers.anyObject;
|
|
|
+import static org.mockito.Matchers.anyString;
|
|
|
import static org.mockito.Mockito.mock;
|
|
|
+import static org.mockito.Mockito.when;
|
|
|
|
|
|
public class MockServiceAM extends ServiceMaster {
|
|
|
|
|
|
+ private static final Logger LOG =
|
|
|
+ LoggerFactory.getLogger(MockServiceAM.class);
|
|
|
+
|
|
|
Service service;
|
|
|
// The list of containers fed by tests to be returned on
|
|
|
// AMRMClientCallBackHandler#onContainersAllocated
|
|
@@ -59,6 +73,16 @@ public class MockServiceAM extends ServiceMaster {
|
|
|
|
|
|
final List<ContainerStatus> failedContainers =
|
|
|
Collections.synchronizedList(new LinkedList<>());
|
|
|
+
|
|
|
+ private final List<Container> recoveredContainers =
|
|
|
+ Collections.synchronizedList(new LinkedList<>());
|
|
|
+
|
|
|
+ private final Map<String, ServiceRecord> registryComponents =
|
|
|
+ new ConcurrentHashMap<>();
|
|
|
+
|
|
|
+ private Map<ContainerId, ContainerStatus> containerStatuses =
|
|
|
+ new ConcurrentHashMap<>();
|
|
|
+
|
|
|
public MockServiceAM(Service service) {
|
|
|
super(service.getName());
|
|
|
this.service = service;
|
|
@@ -75,7 +99,7 @@ public class MockServiceAM extends ServiceMaster {
|
|
|
@Override
|
|
|
protected Path getAppDir() {
|
|
|
Path path = new Path(new Path("target", "apps"), service.getName());
|
|
|
- System.out.println("Service path: " + path);
|
|
|
+ LOG.info("Service path: {}", path);
|
|
|
return path;
|
|
|
}
|
|
|
|
|
@@ -84,10 +108,24 @@ public class MockServiceAM extends ServiceMaster {
|
|
|
throws IOException, YarnException {
|
|
|
return new ServiceScheduler(context) {
|
|
|
|
|
|
+ @SuppressWarnings("SuspiciousMethodCalls")
|
|
|
@Override
|
|
|
protected YarnRegistryViewForProviders createYarnRegistryOperations(
|
|
|
ServiceContext context, RegistryOperations registryClient) {
|
|
|
- return mock(YarnRegistryViewForProviders.class);
|
|
|
+ YarnRegistryViewForProviders yarnRegistryView = mock(
|
|
|
+ YarnRegistryViewForProviders.class);
|
|
|
+ if (!registryComponents.isEmpty()) {
|
|
|
+ try {
|
|
|
+ when(yarnRegistryView.listComponents())
|
|
|
+ .thenReturn(new LinkedList<>(registryComponents.keySet()));
|
|
|
+ when(yarnRegistryView.getComponent(anyString())).thenAnswer(
|
|
|
+ invocation ->
|
|
|
+ registryComponents.get(invocation.getArguments()[0]));
|
|
|
+ } catch (IOException e) {
|
|
|
+ throw new RuntimeException(e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return yarnRegistryView;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -101,7 +139,7 @@ public class MockServiceAM extends ServiceMaster {
|
|
|
// add new containers if any
|
|
|
synchronized (feedContainers) {
|
|
|
if (feedContainers.isEmpty()) {
|
|
|
- System.out.println("Allocating........ no containers");
|
|
|
+ LOG.info("Allocating........ no containers");
|
|
|
} else {
|
|
|
// The AMRMClient will return containers for compoenent that are
|
|
|
// at FLEXING state
|
|
@@ -112,7 +150,7 @@ public class MockServiceAM extends ServiceMaster {
|
|
|
org.apache.hadoop.yarn.service.component.Component component =
|
|
|
componentsById.get(c.getAllocationRequestId());
|
|
|
if (component.getState() == ComponentState.FLEXING) {
|
|
|
- System.out.println("Allocated container " + c.getId());
|
|
|
+ LOG.info("Allocated container {} ", c.getId());
|
|
|
allocatedContainers.add(c);
|
|
|
itor.remove();
|
|
|
}
|
|
@@ -121,6 +159,17 @@ public class MockServiceAM extends ServiceMaster {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ // add recovered containers if any
|
|
|
+ synchronized (recoveredContainers) {
|
|
|
+ if (!recoveredContainers.isEmpty()) {
|
|
|
+ List<Container> containersFromPrevAttempt = new LinkedList<>();
|
|
|
+ containersFromPrevAttempt.addAll(recoveredContainers);
|
|
|
+ recoveredContainers.clear();
|
|
|
+ builder.containersFromPreviousAttempt(
|
|
|
+ containersFromPrevAttempt);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
// add failed containers if any
|
|
|
synchronized (failedContainers) {
|
|
|
if (!failedContainers.isEmpty()) {
|
|
@@ -146,15 +195,23 @@ public class MockServiceAM extends ServiceMaster {
|
|
|
}
|
|
|
};
|
|
|
|
|
|
- return AMRMClientAsync
|
|
|
- .createAMRMClientAsync(client1, 1000,
|
|
|
+ return AMRMClientAsync.createAMRMClientAsync(client1, 1000,
|
|
|
this.new AMRMClientCallback());
|
|
|
}
|
|
|
|
|
|
+ @SuppressWarnings("SuspiciousMethodCalls")
|
|
|
@Override
|
|
|
public NMClientAsync createNMClient() {
|
|
|
NMClientAsync nmClientAsync = super.createNMClient();
|
|
|
- nmClientAsync.setClient(mock(NMClient.class));
|
|
|
+ NMClient nmClient = mock(NMClient.class);
|
|
|
+ try {
|
|
|
+ when(nmClient.getContainerStatus(anyObject(), anyObject()))
|
|
|
+ .thenAnswer(invocation ->
|
|
|
+ containerStatuses.get(invocation.getArguments()[0]));
|
|
|
+ } catch (YarnException | IOException e) {
|
|
|
+ throw new RuntimeException(e);
|
|
|
+ }
|
|
|
+ nmClientAsync.setClient(nmClient);
|
|
|
return nmClientAsync;
|
|
|
}
|
|
|
};
|
|
@@ -165,6 +222,33 @@ public class MockServiceAM extends ServiceMaster {
|
|
|
context.service = service;
|
|
|
}
|
|
|
|
|
|
+ public void feedRegistryComponent(ContainerId containerId, String compName,
|
|
|
+ String compInstName) {
|
|
|
+ ServiceRecord record = new ServiceRecord();
|
|
|
+ record.set(YarnRegistryAttributes.YARN_ID, containerId.toString());
|
|
|
+ record.description = compInstName;
|
|
|
+ record.set(YarnRegistryAttributes.YARN_PERSISTENCE,
|
|
|
+ PersistencePolicies.CONTAINER);
|
|
|
+ record.set(YarnRegistryAttributes.YARN_IP, "localhost");
|
|
|
+ record.set(YarnRegistryAttributes.YARN_HOSTNAME, "localhost");
|
|
|
+ record.set(YarnRegistryAttributes.YARN_COMPONENT, compName);
|
|
|
+ registryComponents.put(RegistryPathUtils.encodeYarnID(
|
|
|
+ containerId.toString()), record);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Simulates a recovered container that is sent to the AM in the heartbeat
|
|
|
+ * response.
|
|
|
+ *
|
|
|
+ * @param containerId The ID for the container
|
|
|
+ * @param compName The component to which the recovered container is fed.
|
|
|
+ */
|
|
|
+ public void feedRecoveredContainer(ContainerId containerId, String compName) {
|
|
|
+ Container container = createContainer(containerId, compName);
|
|
|
+ recoveredContainers.add(container);
|
|
|
+ addContainerStatus(container, ContainerState.RUNNING);
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
*
|
|
|
* @param service The service for the component
|
|
@@ -174,20 +258,12 @@ public class MockServiceAM extends ServiceMaster {
|
|
|
*/
|
|
|
public Container feedContainerToComp(Service service, int id,
|
|
|
String compName) {
|
|
|
- ApplicationId applicationId = ApplicationId.fromString(service.getId());
|
|
|
- ContainerId containerId = ContainerId
|
|
|
- .newContainerId(ApplicationAttemptId.newInstance(applicationId, 1), id);
|
|
|
- NodeId nodeId = NodeId.newInstance("localhost", 1234);
|
|
|
- Container container = Container
|
|
|
- .newInstance(containerId, nodeId, "localhost",
|
|
|
- Resource.newInstance(100, 1), Priority.newInstance(0), null);
|
|
|
-
|
|
|
- long allocateId =
|
|
|
- context.scheduler.getAllComponents().get(compName).getAllocateId();
|
|
|
- container.setAllocationRequestId(allocateId);
|
|
|
+ ContainerId containerId = createContainerId(id);
|
|
|
+ Container container = createContainer(containerId, compName);
|
|
|
synchronized (feedContainers) {
|
|
|
feedContainers.add(container);
|
|
|
}
|
|
|
+ addContainerStatus(container, ContainerState.RUNNING);
|
|
|
return container;
|
|
|
}
|
|
|
|
|
@@ -196,13 +272,30 @@ public class MockServiceAM extends ServiceMaster {
|
|
|
ApplicationId applicationId = ApplicationId.fromString(service.getId());
|
|
|
ContainerId containerId = ContainerId
|
|
|
.newContainerId(ApplicationAttemptId.newInstance(applicationId, 1), id);
|
|
|
- ContainerStatus containerStatus = Records.newRecord(ContainerStatus.class);
|
|
|
- containerStatus.setContainerId(containerId);
|
|
|
+ ContainerStatus status = Records.newRecord(ContainerStatus.class);
|
|
|
+ status.setContainerId(containerId);
|
|
|
synchronized (failedContainers) {
|
|
|
- failedContainers.add(containerStatus);
|
|
|
+ failedContainers.add(status);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ public ContainerId createContainerId(int id) {
|
|
|
+ ApplicationId applicationId = ApplicationId.fromString(service.getId());
|
|
|
+ return ContainerId.newContainerId(
|
|
|
+ ApplicationAttemptId.newInstance(applicationId, 1), id);
|
|
|
+ }
|
|
|
+
|
|
|
+ private Container createContainer(ContainerId containerId, String compName) {
|
|
|
+ NodeId nodeId = NodeId.newInstance("localhost", 1234);
|
|
|
+ Container container = Container.newInstance(
|
|
|
+ containerId, nodeId, "localhost",
|
|
|
+ Resource.newInstance(100, 1),
|
|
|
+ Priority.newInstance(0), null);
|
|
|
+ long allocateId =
|
|
|
+ context.scheduler.getAllComponents().get(compName).getAllocateId();
|
|
|
+ container.setAllocationRequestId(allocateId);
|
|
|
+ return container;
|
|
|
+ }
|
|
|
|
|
|
public void flexComponent(String compName, long numberOfContainers)
|
|
|
throws IOException {
|
|
@@ -256,4 +349,13 @@ public class MockServiceAM extends ServiceMaster {
|
|
|
}
|
|
|
}, 1000, 20000);
|
|
|
}
|
|
|
+
|
|
|
+ private void addContainerStatus(Container container, ContainerState state) {
|
|
|
+ ContainerStatus status = ContainerStatus.newInstance(container.getId(),
|
|
|
+ state, "", 0);
|
|
|
+ status.setHost(container.getNodeId().getHost());
|
|
|
+ status.setIPs(Lists.newArrayList(container.getNodeId().getHost()));
|
|
|
+ containerStatuses.put(container.getId(), status);
|
|
|
+ }
|
|
|
+
|
|
|
}
|