diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt index 27e657b353c..d8d412ab0e7 100644 --- a/hadoop-mapreduce-project/CHANGES.txt +++ b/hadoop-mapreduce-project/CHANGES.txt @@ -43,6 +43,10 @@ Release 2.0.5-beta - UNRELEASED MAPREDUCE-4985. Add compression option to TestDFSIO usage. (Plamen Jeliazkov via shv) + MAPREDUCE-5152. Make MR App to simply pass through the container from RM + instead of extracting and populating information itself to start any + container. (vinodkv) + OPTIMIZATIONS MAPREDUCE-4974. Optimising the LineRecordReader initialize() method diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java index ac3af4f0128..fae70742107 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java @@ -117,7 +117,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationAccessType; import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; -import org.apache.hadoop.yarn.api.records.ContainerToken; import org.apache.hadoop.yarn.api.records.LocalResource; import org.apache.hadoop.yarn.api.records.LocalResourceType; import org.apache.hadoop.yarn.api.records.LocalResourceVisibility; @@ -490,14 +489,10 @@ public abstract class TaskAttemptImpl implements stateMachine; - private ContainerId containerID; - private NodeId containerNodeId; - private String containerMgrAddress; - private String nodeHttpAddress; + @VisibleForTesting + public Container container; private String nodeRackName; private WrappedJvmID jvmID; - private ContainerToken containerToken; - private Resource assignedCapability; //this takes good amount of memory ~ 30KB. Instantiate it lazily //and make it null once task is launched. @@ -825,7 +820,7 @@ public abstract class TaskAttemptImpl implements public ContainerId getAssignedContainerID() { readLock.lock(); try { - return containerID; + return container == null ? null : container.getId(); } finally { readLock.unlock(); } @@ -835,7 +830,8 @@ public abstract class TaskAttemptImpl implements public String getAssignedContainerMgrAddress() { readLock.lock(); try { - return containerMgrAddress; + return container == null ? null : StringInterner.weakIntern(container + .getNodeId().toString()); } finally { readLock.unlock(); } @@ -895,7 +891,7 @@ public abstract class TaskAttemptImpl implements public NodeId getNodeId() { readLock.lock(); try { - return containerNodeId; + return container == null ? null : container.getNodeId(); } finally { readLock.unlock(); } @@ -907,7 +903,7 @@ public abstract class TaskAttemptImpl implements public String getNodeHttpAddress() { readLock.lock(); try { - return nodeHttpAddress; + return container == null ? null : container.getNodeHttpAddress(); } finally { readLock.unlock(); } @@ -967,8 +963,8 @@ public abstract class TaskAttemptImpl implements result.setContainerId(this.getAssignedContainerID()); result.setNodeManagerHost(trackerName); result.setNodeManagerHttpPort(httpPort); - if (this.containerNodeId != null) { - result.setNodeManagerPort(this.containerNodeId.getPort()); + if (this.container != null) { + result.setNodeManagerPort(this.container.getNodeId().getPort()); } return result; } finally { @@ -1093,13 +1089,17 @@ public abstract class TaskAttemptImpl implements @SuppressWarnings("unchecked") public TaskAttemptStateInternal recover(TaskAttemptInfo taInfo, OutputCommitter committer, boolean recoverOutput) { - containerID = taInfo.getContainerId(); - containerNodeId = ConverterUtils.toNodeId(taInfo.getHostname() + ":" + ContainerId containerId = taInfo.getContainerId(); + NodeId containerNodeId = ConverterUtils.toNodeId(taInfo.getHostname() + ":" + taInfo.getPort()); - containerMgrAddress = StringInterner.weakIntern( - containerNodeId.toString()); - nodeHttpAddress = StringInterner.weakIntern(taInfo.getHostname() + ":" + String nodeHttpAddress = StringInterner.weakIntern(taInfo.getHostname() + ":" + taInfo.getHttpPort()); + // Resource/Priority/Tokens are only needed while launching the + // container on an NM, these are already completed tasks, so setting them to + // null + container = + BuilderUtils.newContainer(containerId, containerNodeId, + nodeHttpAddress, null, null, null); computeRackAndLocality(); launchTime = taInfo.getStartTime(); finishTime = (taInfo.getFinishTime() != -1) ? @@ -1227,6 +1227,7 @@ public abstract class TaskAttemptImpl implements } private void computeRackAndLocality() { + NodeId containerNodeId = container.getNodeId(); nodeRackName = RackResolver.resolve( containerNodeId.getHost()).getNetworkLocation(); @@ -1331,10 +1332,10 @@ public abstract class TaskAttemptImpl implements TypeConverter.fromYarn(taskAttempt.attemptId.getTaskId() .getTaskType()), attemptState.toString(), taskAttempt.finishTime, - taskAttempt.containerNodeId == null ? "UNKNOWN" - : taskAttempt.containerNodeId.getHost(), - taskAttempt.containerNodeId == null ? -1 - : taskAttempt.containerNodeId.getPort(), + taskAttempt.container == null ? "UNKNOWN" + : taskAttempt.container.getNodeId().getHost(), + taskAttempt.container == null ? -1 + : taskAttempt.container.getNodeId().getPort(), taskAttempt.nodeRackName == null ? "UNKNOWN" : taskAttempt.nodeRackName, StringUtils.join( @@ -1353,12 +1354,12 @@ public abstract class TaskAttemptImpl implements eventHandler.handle(jce); LOG.info("TaskAttempt: [" + attemptId - + "] using containerId: [" + containerID + " on NM: [" - + containerMgrAddress + "]"); + + "] using containerId: [" + container.getId() + " on NM: [" + + StringInterner.weakIntern(container.getNodeId().toString()) + "]"); TaskAttemptStartedEvent tase = new TaskAttemptStartedEvent(TypeConverter.fromYarn(attemptId), TypeConverter.fromYarn(attemptId.getTaskId().getTaskType()), - launchTime, trackerName, httpPort, shufflePort, containerID, + launchTime, trackerName, httpPort, shufflePort, container.getId(), locality.toString(), avataar.toString()); eventHandler.handle( new JobHistoryEvent(attemptId.getTaskId().getJobId(), tase)); @@ -1490,19 +1491,14 @@ public abstract class TaskAttemptImpl implements TaskAttemptEvent event) { final TaskAttemptContainerAssignedEvent cEvent = (TaskAttemptContainerAssignedEvent) event; - taskAttempt.containerID = cEvent.getContainer().getId(); - taskAttempt.containerNodeId = cEvent.getContainer().getNodeId(); - taskAttempt.containerMgrAddress = StringInterner.weakIntern( - taskAttempt.containerNodeId.toString()); - taskAttempt.nodeHttpAddress = StringInterner.weakIntern( - cEvent.getContainer().getNodeHttpAddress()); - taskAttempt.containerToken = cEvent.getContainer().getContainerToken(); - taskAttempt.assignedCapability = cEvent.getContainer().getResource(); + Container container = cEvent.getContainer(); + taskAttempt.container = container; // this is a _real_ Task (classic Hadoop mapred flavor): taskAttempt.remoteTask = taskAttempt.createRemoteTask(); - taskAttempt.jvmID = new WrappedJvmID( - taskAttempt.remoteTask.getTaskID().getJobID(), - taskAttempt.remoteTask.isMapTask(), taskAttempt.containerID.getId()); + taskAttempt.jvmID = + new WrappedJvmID(taskAttempt.remoteTask.getTaskID().getJobID(), + taskAttempt.remoteTask.isMapTask(), taskAttempt.container.getId() + .getId()); taskAttempt.taskAttemptListener.registerPendingTask( taskAttempt.remoteTask, taskAttempt.jvmID); @@ -1514,10 +1510,9 @@ public abstract class TaskAttemptImpl implements cEvent.getApplicationACLs(), taskAttempt.conf, taskAttempt.jobToken, taskAttempt.remoteTask, taskAttempt.oldJobId, taskAttempt.jvmID, taskAttempt.taskAttemptListener, taskAttempt.credentials); - taskAttempt.eventHandler.handle(new ContainerRemoteLaunchEvent( - taskAttempt.attemptId, taskAttempt.containerID, - taskAttempt.containerMgrAddress, taskAttempt.containerToken, - launchContext, taskAttempt.assignedCapability, taskAttempt.remoteTask)); + taskAttempt.eventHandler + .handle(new ContainerRemoteLaunchEvent(taskAttempt.attemptId, + launchContext, container, taskAttempt.remoteTask)); // send event to speculator that our container needs are satisfied taskAttempt.eventHandler.handle @@ -1604,9 +1599,8 @@ public abstract class TaskAttemptImpl implements taskAttempt.taskAttemptListener .registerLaunchedTask(taskAttempt.attemptId, taskAttempt.jvmID); //TODO Resolve to host / IP in case of a local address. - InetSocketAddress nodeHttpInetAddr = - NetUtils.createSocketAddr(taskAttempt.nodeHttpAddress); // TODO: - // Costly? + InetSocketAddress nodeHttpInetAddr = // TODO: Costly to create sock-addr? + NetUtils.createSocketAddr(taskAttempt.container.getNodeHttpAddress()); taskAttempt.trackerName = nodeHttpInetAddr.getHostName(); taskAttempt.httpPort = nodeHttpInetAddr.getPort(); taskAttempt.sendLaunchedEvents(); @@ -1713,6 +1707,10 @@ public abstract class TaskAttemptImpl implements private void logAttemptFinishedEvent(TaskAttemptStateInternal state) { //Log finished events only if an attempt started. if (getLaunchTime() == 0) return; + String containerHostName = this.container == null ? "UNKNOWN" + : this.container.getNodeId().getHost(); + int containerNodePort = + this.container == null ? -1 : this.container.getNodeId().getPort(); if (attemptId.getTaskId().getTaskType() == TaskType.MAP) { MapAttemptFinishedEvent mfe = new MapAttemptFinishedEvent(TypeConverter.fromYarn(attemptId), @@ -1720,9 +1718,8 @@ public abstract class TaskAttemptImpl implements state.toString(), this.reportedStatus.mapFinishTime, finishTime, - this.containerNodeId == null ? "UNKNOWN" - : this.containerNodeId.getHost(), - this.containerNodeId == null ? -1 : this.containerNodeId.getPort(), + containerHostName, + containerNodePort, this.nodeRackName == null ? "UNKNOWN" : this.nodeRackName, this.reportedStatus.stateString, getCounters(), @@ -1737,9 +1734,8 @@ public abstract class TaskAttemptImpl implements this.reportedStatus.shuffleFinishTime, this.reportedStatus.sortFinishTime, finishTime, - this.containerNodeId == null ? "UNKNOWN" - : this.containerNodeId.getHost(), - this.containerNodeId == null ? -1 : this.containerNodeId.getPort(), + containerHostName, + containerNodePort, this.nodeRackName == null ? "UNKNOWN" : this.nodeRackName, this.reportedStatus.stateString, getCounters(), @@ -1864,8 +1860,9 @@ public abstract class TaskAttemptImpl implements //send the cleanup event to containerLauncher taskAttempt.eventHandler.handle(new ContainerLauncherEvent( taskAttempt.attemptId, - taskAttempt.containerID, taskAttempt.containerMgrAddress, - taskAttempt.containerToken, + taskAttempt.container.getId(), StringInterner + .weakIntern(taskAttempt.container.getNodeId().toString()), + taskAttempt.container.getContainerToken(), ContainerLauncher.EventType.CONTAINER_REMOTE_CLEANUP)); } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java index 86281f60b1b..76e00f83e93 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java @@ -59,7 +59,6 @@ import org.apache.hadoop.yarn.api.records.ContainerToken; import org.apache.hadoop.yarn.ipc.YarnRPC; import org.apache.hadoop.yarn.security.ContainerTokenIdentifier; import org.apache.hadoop.yarn.service.AbstractService; -import org.apache.hadoop.yarn.util.BuilderUtils; import org.apache.hadoop.yarn.util.ProtoUtils; import org.apache.hadoop.yarn.util.Records; @@ -149,16 +148,13 @@ public class ContainerLauncherImpl extends AbstractService implements // Construct the actual Container ContainerLaunchContext containerLaunchContext = - event.getContainer(); + event.getContainerLaunchContext(); - org.apache.hadoop.yarn.api.records.Container container = - BuilderUtils.newContainer(containerID, null, null, - event.getResource(), null, containerToken); // Now launch the actual container StartContainerRequest startRequest = Records .newRecord(StartContainerRequest.class); startRequest.setContainerLaunchContext(containerLaunchContext); - startRequest.setContainer(container); + startRequest.setContainer(event.getAllocatedContainer()); StartContainerResponse response = proxy.startContainer(startRequest); ByteBuffer portInfo = response diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerRemoteLaunchEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerRemoteLaunchEvent.java index eb95f3bb314..adb9c7ec7e2 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerRemoteLaunchEvent.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerRemoteLaunchEvent.java @@ -20,35 +20,34 @@ package org.apache.hadoop.mapreduce.v2.app.launcher; import org.apache.hadoop.mapred.Task; import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId; -import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.util.StringInterner; +import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; -import org.apache.hadoop.yarn.api.records.ContainerToken; -import org.apache.hadoop.yarn.api.records.Resource; public class ContainerRemoteLaunchEvent extends ContainerLauncherEvent { - private final ContainerLaunchContext container; + private final Container allocatedContainer; + private final ContainerLaunchContext containerLaunchContext; private final Task task; - private final Resource resource; public ContainerRemoteLaunchEvent(TaskAttemptId taskAttemptID, - ContainerId containerID, String containerMgrAddress, - ContainerToken containerToken, - ContainerLaunchContext containerLaunchContext, Resource resource, - Task remoteTask) { - super(taskAttemptID, containerID, containerMgrAddress, containerToken, - ContainerLauncher.EventType.CONTAINER_REMOTE_LAUNCH); - this.container = containerLaunchContext; + ContainerLaunchContext containerLaunchContext, + Container allocatedContainer, Task remoteTask) { + super(taskAttemptID, allocatedContainer.getId(), StringInterner + .weakIntern(allocatedContainer.getNodeId().toString()), + allocatedContainer.getContainerToken(), + ContainerLauncher.EventType.CONTAINER_REMOTE_LAUNCH); + this.allocatedContainer = allocatedContainer; + this.containerLaunchContext = containerLaunchContext; this.task = remoteTask; - this.resource = resource; } - public ContainerLaunchContext getContainer() { - return this.container; + public ContainerLaunchContext getContainerLaunchContext() { + return this.containerLaunchContext; } - public Resource getResource() { - return this.resource; + public Container getAllocatedContainer() { + return this.allocatedContainer; } public Task getRemoteTask() { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRApp.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRApp.java index 758a2427fba..48e7e7ef182 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRApp.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRApp.java @@ -23,6 +23,7 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import java.util.ArrayList; +import java.util.Collection; import java.util.Iterator; import junit.framework.Assert; @@ -46,6 +47,11 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType; import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEvent; import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEventType; import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl; +import org.apache.hadoop.mapreduce.v2.app.job.impl.TaskAttemptImpl; +import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncher; +import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncherEvent; +import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerRemoteLaunchEvent; +import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.NodeReport; import org.apache.hadoop.yarn.api.records.NodeState; @@ -411,7 +417,40 @@ public class TestMRApp { TypeConverter.fromYarn(state); } } - + + private Container containerObtainedByContainerLauncher; + @Test + public void testContainerPassThrough() throws Exception { + MRApp app = new MRApp(0, 1, true, this.getClass().getName(), true) { + @Override + protected ContainerLauncher createContainerLauncher(AppContext context) { + return new MockContainerLauncher() { + @Override + public void handle(ContainerLauncherEvent event) { + if (event instanceof ContainerRemoteLaunchEvent) { + containerObtainedByContainerLauncher = + ((ContainerRemoteLaunchEvent) event).getAllocatedContainer(); + } + super.handle(event); + } + }; + }; + }; + Job job = app.submit(new Configuration()); + app.waitForState(job, JobState.SUCCEEDED); + app.verifyCompleted(); + + Collection tasks = job.getTasks().values(); + Collection taskAttempts = + tasks.iterator().next().getAttempts().values(); + TaskAttemptImpl taskAttempt = + (TaskAttemptImpl) taskAttempts.iterator().next(); + // Container from RM should pass through to the launcher. Container object + // should be the same. + Assert.assertTrue(taskAttempt.container + == containerObtainedByContainerLauncher); + } + private final class MRAppWithHistory extends MRApp { public MRAppWithHistory(int maps, int reduces, boolean autoComplete, String testName, boolean cleanOnStart, int startCount) { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestMapReduceChildJVM.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestMapReduceChildJVM.java index ea0a342d623..8230cd26c50 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestMapReduceChildJVM.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestMapReduceChildJVM.java @@ -78,7 +78,8 @@ public class TestMapReduceChildJVM { public void handle(ContainerLauncherEvent event) { if (event.getType() == EventType.CONTAINER_REMOTE_LAUNCH) { ContainerRemoteLaunchEvent launchEvent = (ContainerRemoteLaunchEvent) event; - ContainerLaunchContext launchContext = launchEvent.getContainer(); + ContainerLaunchContext launchContext = + launchEvent.getContainerLaunchContext(); String cmdString = launchContext.getCommands().toString(); LOG.info("launchContext " + cmdString); myCommandLine = cmdString; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java index c5d0a885f4b..681d4daef27 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java @@ -37,7 +37,6 @@ import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.hadoop.mapreduce.v2.api.records.JobId; import org.apache.hadoop.mapreduce.v2.api.records.JobState; import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId; -import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptState; import org.apache.hadoop.mapreduce.v2.api.records.TaskId; import org.apache.hadoop.mapreduce.v2.api.records.TaskState; import org.apache.hadoop.mapreduce.v2.api.records.TaskType; @@ -224,10 +223,6 @@ public class TestContainerLauncher { @Test public void testSlowNM() throws Exception { - test(); - } - - private void test() throws Exception { conf = new Configuration(); int maxAttempts = 1; @@ -382,6 +377,15 @@ public class TestContainerLauncher { @Override public StartContainerResponse startContainer(StartContainerRequest request) throws YarnRemoteException { + + // Validate that the container is what RM is giving. + Assert.assertEquals(MRApp.NM_HOST, request.getContainer().getNodeId() + .getHost()); + Assert.assertEquals(MRApp.NM_PORT, request.getContainer().getNodeId() + .getPort()); + Assert.assertEquals(MRApp.NM_HOST + ":" + MRApp.NM_HTTP_PORT, request + .getContainer().getNodeHttpAddress()); + StartContainerResponse response = recordFactory .newRecordInstance(StartContainerResponse.class); status = recordFactory.newRecordInstance(ContainerStatus.class);