|
@@ -146,7 +146,7 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
|
|
compInstance.containerStatusFuture =
|
|
compInstance.containerStatusFuture =
|
|
compInstance.scheduler.executorService.scheduleAtFixedRate(
|
|
compInstance.scheduler.executorService.scheduleAtFixedRate(
|
|
new ContainerStatusRetriever(compInstance.scheduler,
|
|
new ContainerStatusRetriever(compInstance.scheduler,
|
|
- compInstance.getContainerId(), compInstance), 0, 1,
|
|
|
|
|
|
+ event.getContainerId(), compInstance), 0, 1,
|
|
TimeUnit.SECONDS);
|
|
TimeUnit.SECONDS);
|
|
compInstance.component.incRunningContainers();
|
|
compInstance.component.incRunningContainers();
|
|
long containerStartTime = System.currentTimeMillis();
|
|
long containerStartTime = System.currentTimeMillis();
|
|
@@ -160,10 +160,10 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
|
|
}
|
|
}
|
|
org.apache.hadoop.yarn.service.api.records.Container container =
|
|
org.apache.hadoop.yarn.service.api.records.Container container =
|
|
new org.apache.hadoop.yarn.service.api.records.Container();
|
|
new org.apache.hadoop.yarn.service.api.records.Container();
|
|
- container.setId(compInstance.getContainerId().toString());
|
|
|
|
|
|
+ container.setId(event.getContainerId().toString());
|
|
container.setLaunchTime(new Date(containerStartTime));
|
|
container.setLaunchTime(new Date(containerStartTime));
|
|
container.setState(ContainerState.RUNNING_BUT_UNREADY);
|
|
container.setState(ContainerState.RUNNING_BUT_UNREADY);
|
|
- container.setBareHost(compInstance.container.getNodeId().getHost());
|
|
|
|
|
|
+ container.setBareHost(compInstance.getNodeId().getHost());
|
|
container.setComponentInstanceName(compInstance.getCompInstanceName());
|
|
container.setComponentInstanceName(compInstance.getCompInstanceName());
|
|
if (compInstance.containerSpec != null) {
|
|
if (compInstance.containerSpec != null) {
|
|
// remove the previous container.
|
|
// remove the previous container.
|
|
@@ -219,15 +219,11 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
|
|
// re-ask the failed container.
|
|
// re-ask the failed container.
|
|
Component comp = compInstance.component;
|
|
Component comp = compInstance.component;
|
|
comp.requestContainers(1);
|
|
comp.requestContainers(1);
|
|
- LOG.info(compInstance.getCompInstanceId()
|
|
|
|
- + ": Container completed. Requested a new container." + System
|
|
|
|
- .lineSeparator() + " exitStatus={}, diagnostics={}.",
|
|
|
|
- event.getStatus().getExitStatus(),
|
|
|
|
- event.getStatus().getDiagnostics());
|
|
|
|
String containerDiag =
|
|
String containerDiag =
|
|
compInstance.getCompInstanceId() + ": " + event.getStatus()
|
|
compInstance.getCompInstanceId() + ": " + event.getStatus()
|
|
.getDiagnostics();
|
|
.getDiagnostics();
|
|
compInstance.diagnostics.append(containerDiag + System.lineSeparator());
|
|
compInstance.diagnostics.append(containerDiag + System.lineSeparator());
|
|
|
|
+ compInstance.cancelContainerStatusRetriever();
|
|
|
|
|
|
if (compInstance.getState().equals(READY)) {
|
|
if (compInstance.getState().equals(READY)) {
|
|
compInstance.component.decContainersReady();
|
|
compInstance.component.decContainersReady();
|
|
@@ -255,11 +251,13 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
|
|
// hdfs dir content will be overwritten when a new container gets started,
|
|
// hdfs dir content will be overwritten when a new container gets started,
|
|
// so no need remove.
|
|
// so no need remove.
|
|
compInstance.scheduler.executorService
|
|
compInstance.scheduler.executorService
|
|
- .submit(compInstance::cleanupRegistry);
|
|
|
|
|
|
+ .submit(() -> compInstance.cleanupRegistry(event.getContainerId()));
|
|
|
|
+
|
|
if (compInstance.timelineServiceEnabled) {
|
|
if (compInstance.timelineServiceEnabled) {
|
|
// record in ATS
|
|
// record in ATS
|
|
- compInstance.serviceTimelinePublisher.componentInstanceFinished
|
|
|
|
- (compInstance, event.getStatus().getExitStatus(), containerDiag);
|
|
|
|
|
|
+ compInstance.serviceTimelinePublisher
|
|
|
|
+ .componentInstanceFinished(event.getContainerId(),
|
|
|
|
+ event.getStatus().getExitStatus(), containerDiag);
|
|
}
|
|
}
|
|
compInstance.containerSpec.setState(ContainerState.STOPPED);
|
|
compInstance.containerSpec.setState(ContainerState.STOPPED);
|
|
}
|
|
}
|
|
@@ -267,6 +265,14 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
|
|
// remove the failed ContainerId -> CompInstance mapping
|
|
// remove the failed ContainerId -> CompInstance mapping
|
|
comp.getScheduler().removeLiveCompInstance(event.getContainerId());
|
|
comp.getScheduler().removeLiveCompInstance(event.getContainerId());
|
|
|
|
|
|
|
|
+ comp.reInsertPendingInstance(compInstance);
|
|
|
|
+
|
|
|
|
+ LOG.info(compInstance.getCompInstanceId()
|
|
|
|
+ + ": {} completed. Reinsert back to pending list and requested " +
|
|
|
|
+ "a new container." + System.lineSeparator() +
|
|
|
|
+ " exitStatus={}, diagnostics={}.",
|
|
|
|
+ event.getContainerId(), event.getStatus().getExitStatus(),
|
|
|
|
+ event.getStatus().getDiagnostics());
|
|
if (shouldExit) {
|
|
if (shouldExit) {
|
|
// Sleep for 5 seconds in hope that the state can be recorded in ATS.
|
|
// Sleep for 5 seconds in hope that the state can be recorded in ATS.
|
|
// in case there's a client polling the comp state, it can be notified.
|
|
// in case there's a client polling the comp state, it can be notified.
|
|
@@ -277,8 +283,6 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
|
|
}
|
|
}
|
|
ExitUtil.terminate(-1);
|
|
ExitUtil.terminate(-1);
|
|
}
|
|
}
|
|
-
|
|
|
|
- compInstance.removeContainer();
|
|
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -312,15 +316,6 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- public boolean hasContainer() {
|
|
|
|
- return this.container != null;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- public void removeContainer() {
|
|
|
|
- this.container = null;
|
|
|
|
- this.compInstanceId.setContainerId(null);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
public void setContainer(Container container) {
|
|
public void setContainer(Container container) {
|
|
this.container = container;
|
|
this.container = container;
|
|
this.compInstanceId.setContainerId(container.getId());
|
|
this.compInstanceId.setContainerId(container.getId());
|
|
@@ -337,7 +332,7 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
|
|
public void updateContainerStatus(ContainerStatus status) {
|
|
public void updateContainerStatus(ContainerStatus status) {
|
|
this.status = status;
|
|
this.status = status;
|
|
org.apache.hadoop.yarn.service.api.records.Container container =
|
|
org.apache.hadoop.yarn.service.api.records.Container container =
|
|
- getCompSpec().getContainer(getContainerId().toString());
|
|
|
|
|
|
+ getCompSpec().getContainer(status.getContainerId().toString());
|
|
if (container != null) {
|
|
if (container != null) {
|
|
container.setIp(StringUtils.join(",", status.getIPs()));
|
|
container.setIp(StringUtils.join(",", status.getIPs()));
|
|
container.setHostname(status.getHost());
|
|
container.setHostname(status.getHost());
|
|
@@ -348,10 +343,6 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
|
|
updateServiceRecord(yarnRegistryOperations, status);
|
|
updateServiceRecord(yarnRegistryOperations, status);
|
|
}
|
|
}
|
|
|
|
|
|
- public ContainerId getContainerId() {
|
|
|
|
- return container.getId();
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
public String getCompName() {
|
|
public String getCompName() {
|
|
return compInstanceId.getCompName();
|
|
return compInstanceId.getCompName();
|
|
}
|
|
}
|
|
@@ -423,12 +414,7 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
|
|
public void destroy() {
|
|
public void destroy() {
|
|
LOG.info(getCompInstanceId() + ": Flexed down by user, destroying.");
|
|
LOG.info(getCompInstanceId() + ": Flexed down by user, destroying.");
|
|
diagnostics.append(getCompInstanceId() + ": Flexed down by user");
|
|
diagnostics.append(getCompInstanceId() + ": Flexed down by user");
|
|
- if (container != null) {
|
|
|
|
- scheduler.removeLiveCompInstance(container.getId());
|
|
|
|
- component.getScheduler().getAmRMClient()
|
|
|
|
- .releaseAssignedContainer(container.getId());
|
|
|
|
- getCompSpec().removeContainer(containerSpec);
|
|
|
|
- }
|
|
|
|
|
|
+
|
|
// update metrics
|
|
// update metrics
|
|
if (getState() == STARTED) {
|
|
if (getState() == STARTED) {
|
|
component.decRunningContainers();
|
|
component.decRunningContainers();
|
|
@@ -437,16 +423,29 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
|
|
component.decContainersReady();
|
|
component.decContainersReady();
|
|
component.decRunningContainers();
|
|
component.decRunningContainers();
|
|
}
|
|
}
|
|
|
|
+ getCompSpec().removeContainer(containerSpec);
|
|
|
|
+
|
|
|
|
+ if (container == null) {
|
|
|
|
+ LOG.info(getCompInstanceId() + " no container is assigned when " +
|
|
|
|
+ "destroying");
|
|
|
|
+ return;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ ContainerId containerId = container.getId();
|
|
|
|
+ scheduler.removeLiveCompInstance(containerId);
|
|
|
|
+ component.getScheduler().getAmRMClient()
|
|
|
|
+ .releaseAssignedContainer(containerId);
|
|
|
|
|
|
if (timelineServiceEnabled) {
|
|
if (timelineServiceEnabled) {
|
|
- serviceTimelinePublisher.componentInstanceFinished(this,
|
|
|
|
|
|
+ serviceTimelinePublisher.componentInstanceFinished(containerId,
|
|
KILLED_BY_APPMASTER, diagnostics.toString());
|
|
KILLED_BY_APPMASTER, diagnostics.toString());
|
|
}
|
|
}
|
|
- scheduler.executorService.submit(this::cleanupRegistryAndCompHdfsDir);
|
|
|
|
|
|
+ cancelContainerStatusRetriever();
|
|
|
|
+ scheduler.executorService.submit(() ->
|
|
|
|
+ cleanupRegistryAndCompHdfsDir(containerId));
|
|
}
|
|
}
|
|
|
|
|
|
- private void cleanupRegistry() {
|
|
|
|
- ContainerId containerId = getContainerId();
|
|
|
|
|
|
+ private void cleanupRegistry(ContainerId containerId) {
|
|
String cid = RegistryPathUtils.encodeYarnID(containerId.toString());
|
|
String cid = RegistryPathUtils.encodeYarnID(containerId.toString());
|
|
try {
|
|
try {
|
|
yarnRegistryOperations.deleteComponent(getCompInstanceId(), cid);
|
|
yarnRegistryOperations.deleteComponent(getCompInstanceId(), cid);
|
|
@@ -456,8 +455,8 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
|
|
}
|
|
}
|
|
|
|
|
|
//TODO Maybe have a dedicated cleanup service.
|
|
//TODO Maybe have a dedicated cleanup service.
|
|
- public void cleanupRegistryAndCompHdfsDir() {
|
|
|
|
- cleanupRegistry();
|
|
|
|
|
|
+ public void cleanupRegistryAndCompHdfsDir(ContainerId containerId) {
|
|
|
|
+ cleanupRegistry(containerId);
|
|
try {
|
|
try {
|
|
if (compInstanceDir != null && fs.exists(compInstanceDir)) {
|
|
if (compInstanceDir != null && fs.exists(compInstanceDir)) {
|
|
boolean deleted = fs.delete(compInstanceDir, true);
|
|
boolean deleted = fs.delete(compInstanceDir, true);
|
|
@@ -515,6 +514,12 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ private void cancelContainerStatusRetriever() {
|
|
|
|
+ if (containerStatusFuture != null && !containerStatusFuture.isDone()) {
|
|
|
|
+ containerStatusFuture.cancel(true);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
@Override
|
|
@Override
|
|
public int compareTo(ComponentInstance to) {
|
|
public int compareTo(ComponentInstance to) {
|
|
long delta = containerStartedTime - to.containerStartedTime;
|
|
long delta = containerStartedTime - to.containerStartedTime;
|