Fixed cleanup on job abort. Fixed profiling
git-svn-id: https://hyracks.googlecode.com/svn/branches/hyracks_scheduling@306 123451ca-8445-de46-9d55-352943316053
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/INodeController.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/INodeController.java
index 53cd531..d0269fc 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/INodeController.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/INodeController.java
@@ -41,7 +41,7 @@
public void commitJobletInitialization(UUID jobId, UUID stageId) throws Exception;
- public void abortJoblet(UUID jobId) throws Exception;
+ public void abortJoblet(UUID jobId, int attempt) throws Exception;
public void cleanUpJob(UUID jobId) throws Exception;
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/counters/ICounterContext.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/counters/ICounterContext.java
index 2992757..4be02d7 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/counters/ICounterContext.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/counters/ICounterContext.java
@@ -21,13 +21,6 @@
*/
public interface ICounterContext {
/**
- * Returns the fully-qualified context name
- *
- * @return fully-qualified context name.
- */
- public String getContextName();
-
- /**
* Get a counter with the specified name.
*
* @param name
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/JobLifecycleHelper.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/JobLifecycleHelper.java
index 8855ec1..be42e22 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/JobLifecycleHelper.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/JobLifecycleHelper.java
@@ -9,12 +9,12 @@
import edu.uci.ics.hyracks.control.cc.remote.ops.JobletAborter;
public class JobLifecycleHelper {
- public static void abortJob(ClusterControllerService ccs, UUID jobId, Set<String> targetNodes) {
+ public static void abortJob(ClusterControllerService ccs, UUID jobId, int attempt, Set<String> targetNodes) {
if (!targetNodes.isEmpty()) {
JobletAborter[] jas = new JobletAborter[targetNodes.size()];
int i = 0;
for (String nodeId : targetNodes) {
- jas[i++] = new JobletAborter(nodeId, jobId);
+ jas[i++] = new JobletAborter(nodeId, jobId, attempt);
}
try {
RemoteRunner.runRemote(ccs, jas, null);
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobAbortEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobAbortEvent.java
index 706ddb8..9a0eff1 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobAbortEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobAbortEvent.java
@@ -31,9 +31,12 @@
private final UUID jobId;
- public JobAbortEvent(ClusterControllerService ccs, UUID jobId) {
+ private final int attempt;
+
+ public JobAbortEvent(ClusterControllerService ccs, UUID jobId, int attempt) {
this.ccs = ccs;
this.jobId = jobId;
+ this.attempt = attempt;
}
@Override
@@ -43,9 +46,9 @@
final JobRun run = runMap.get(jobId);
final Set<String> targetNodes = new HashSet<String>();
if (run != null) {
- List<JobAttempt> attempts = run.getAttempts();
- JobAttempt attempt = attempts.get(attempts.size() - 1);
- for (String runningNodeId : attempt.getParticipatingNodeIds()) {
+ List<JobAttempt> jas = run.getAttempts();
+ JobAttempt ja = jas.get(attempt);
+ for (String runningNodeId : ja.getParticipatingNodeIds()) {
if (nodeMap.containsKey(runningNodeId)) {
targetNodes.add(runningNodeId);
nodeMap.get(runningNodeId).getActiveJobIds().remove(jobId);
@@ -53,6 +56,11 @@
}
}
- JobLifecycleHelper.abortJob(ccs, jobId, targetNodes);
+ ccs.getExecutor().execute(new Runnable() {
+ @Override
+ public void run() {
+ JobLifecycleHelper.abortJob(ccs, jobId, attempt, targetNodes);
+ }
+ });
}
}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RemoveDeadNodesEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RemoveDeadNodesEvent.java
index 11850ee..38fb7ae 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RemoveDeadNodesEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RemoveDeadNodesEvent.java
@@ -20,7 +20,6 @@
import java.util.UUID;
import java.util.logging.Logger;
-import edu.uci.ics.hyracks.api.job.JobStatus;
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
import edu.uci.ics.hyracks.control.cc.NodeControllerState;
import edu.uci.ics.hyracks.control.cc.job.JobRun;
@@ -48,8 +47,10 @@
for (String deadNode : deadNodes) {
NodeControllerState state = nodeMap.remove(deadNode);
for (final UUID jid : state.getActiveJobIds()) {
+ JobRun run = ccs.getRunMap().get(jid);
+ int lastAttempt = run.getAttempts().size() - 1;
LOGGER.info("Aborting: " + jid);
- ccs.getJobQueue().schedule(new JobAbortEvent(ccs, jid));
+ ccs.getJobQueue().schedule(new JobAbortEvent(ccs, jid, lastAttempt));
}
}
}
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ScheduleRunnableStagesEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ScheduleRunnableStagesEvent.java
index c4839da..ba1c711 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ScheduleRunnableStagesEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ScheduleRunnableStagesEvent.java
@@ -90,7 +90,7 @@
ccs.getScheduler().schedule(runnableStageAttempts);
} catch (HyracksException e) {
e.printStackTrace();
- ccs.getJobQueue().schedule(new JobAbortEvent(ccs, jobId));
+ ccs.getJobQueue().schedule(new JobAbortEvent(ccs, jobId, attempt));
return;
}
@@ -118,8 +118,10 @@
}
}
+ Set<String> participatingNodeIds = ja.getParticipatingNodeIds();
for (String nid : targetMap.keySet()) {
ccs.getNodeMap().get(nid).getActiveJobIds().add(jobId);
+ participatingNodeIds.add(nid);
}
ccs.getExecutor().execute(new Runnable() {
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/StageletFailureEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/StageletFailureEvent.java
index aa06c75..34dbeb4 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/StageletFailureEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/StageletFailureEvent.java
@@ -52,6 +52,11 @@
for (String nodeId : targetNodes) {
nodeMap.get(nodeId).getActiveJobIds().remove(jobId);
}
- JobLifecycleHelper.abortJob(ccs, jobId, targetNodes);
+ ccs.getExecutor().execute(new Runnable() {
+ @Override
+ public void run() {
+ JobLifecycleHelper.abortJob(ccs, jobId, attempt, targetNodes);
+ }
+ });
}
}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/JobletAborter.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/JobletAborter.java
index 4172031..b50aa8e 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/JobletAborter.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/JobletAborter.java
@@ -22,15 +22,17 @@
public class JobletAborter implements RemoteOp<Void> {
private String nodeId;
private UUID jobId;
+ private int attempt;
- public JobletAborter(String nodeId, UUID jobId) {
+ public JobletAborter(String nodeId, UUID jobId, int attempt) {
this.nodeId = nodeId;
this.jobId = jobId;
+ this.attempt = attempt;
}
@Override
public Void execute(INodeController node) throws Exception {
- node.abortJoblet(jobId);
+ node.abortJoblet(jobId, attempt);
return null;
}
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
index be8a76a..6cc7450 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
@@ -16,9 +16,7 @@
import java.nio.ByteBuffer;
import java.util.HashMap;
-import java.util.HashSet;
import java.util.Map;
-import java.util.Set;
import java.util.UUID;
import java.util.concurrent.Executor;
@@ -31,16 +29,17 @@
import edu.uci.ics.hyracks.api.io.IIOManager;
import edu.uci.ics.hyracks.api.io.IWorkspaceFileFactory;
import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
+import edu.uci.ics.hyracks.api.job.profiling.counters.ICounter;
import edu.uci.ics.hyracks.api.job.profiling.counters.ICounterContext;
import edu.uci.ics.hyracks.api.job.profiling.om.JobletProfile;
import edu.uci.ics.hyracks.api.job.profiling.om.StageletProfile;
import edu.uci.ics.hyracks.api.resources.IDeallocatable;
+import edu.uci.ics.hyracks.control.common.job.profiling.counters.Counter;
import edu.uci.ics.hyracks.control.nc.io.IOManager;
import edu.uci.ics.hyracks.control.nc.io.ManagedWorkspaceFileFactory;
-import edu.uci.ics.hyracks.control.nc.job.profiling.CounterContext;
import edu.uci.ics.hyracks.control.nc.resources.DefaultDeallocatableRegistry;
-public class Joblet implements IHyracksJobletContext {
+public class Joblet implements IHyracksJobletContext, ICounterContext {
private static final long serialVersionUID = 1L;
private final NodeControllerService nodeController;
@@ -55,7 +54,7 @@
private final Map<OperatorDescriptorId, Map<Integer, IOperatorEnvironment>> envMap;
- private final ICounterContext counterContext;
+ private final Map<String, Counter> counterMap;
private final DefaultDeallocatableRegistry deallocatableRegistry;
@@ -68,7 +67,7 @@
this.attempt = attempt;
stageletMap = new HashMap<UUID, Stagelet>();
envMap = new HashMap<OperatorDescriptorId, Map<Integer, IOperatorEnvironment>>();
- counterContext = new CounterContext(getJobId() + "." + nodeController.getId());
+ counterMap = new HashMap<String, Counter>();
deallocatableRegistry = new DefaultDeallocatableRegistry();
fileFactory = new ManagedWorkspaceFileFactory(this, (IOManager) appCtx.getRootContext().getIOManager());
}
@@ -133,24 +132,15 @@
return nodeController;
}
- public void dumpProfile(JobletProfile jProfile) {
- Set<UUID> stageIds;
- synchronized (this) {
- for (Stagelet si : stageletMap.values()) {
- StageletProfile sProfile = new StageletProfile(si.getStageId());
- si.dumpProfile(sProfile);
- }
- stageIds = new HashSet<UUID>(stageletMap.keySet());
+ public synchronized void dumpProfile(JobletProfile jProfile) {
+ Map<String, Long> counters = jProfile.getCounters();
+ for (Map.Entry<String, Counter> e : counterMap.entrySet()) {
+ counters.put(e.getKey(), e.getValue().get());
}
- for (UUID stageId : stageIds) {
- Stagelet si;
- synchronized (this) {
- si = stageletMap.get(stageId);
- }
- if (si != null) {
- StageletProfile sProfile = new StageletProfile(si.getStageId());
- si.dumpProfile(sProfile);
- }
+ for (Stagelet si : stageletMap.values()) {
+ StageletProfile sProfile = new StageletProfile(si.getStageId());
+ si.dumpProfile(sProfile);
+ jProfile.getStageletProfiles().put(si.getStageId(), sProfile);
}
}
@@ -166,7 +156,7 @@
@Override
public ICounterContext getCounterContext() {
- return counterContext;
+ return this;
}
@Override
@@ -201,4 +191,14 @@
public Map<UUID, Stagelet> getStageletMap() {
return stageletMap;
}
+
+ @Override
+ public synchronized ICounter getCounter(String name, boolean create) {
+ Counter counter = counterMap.get(name);
+ if (counter == null && create) {
+ counter = new Counter(name);
+ counterMap.put(name, counter);
+ }
+ return counter;
+ }
}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
index 2335c4e..2702386 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
@@ -446,7 +446,7 @@
private synchronized Joblet getOrCreateLocalJoblet(UUID jobId, int attempt, INCApplicationContext appCtx)
throws Exception {
Joblet ji = jobletMap.get(jobId);
- if (ji == null) {
+ if (ji == null || ji.getAttempt() != attempt) {
ji = new Joblet(this, jobId, attempt, appCtx);
jobletMap.put(jobId, ji);
}
@@ -538,6 +538,9 @@
List<JobProfile> profiles;
synchronized (NodeControllerService.this) {
profiles = new ArrayList<JobProfile>();
+ for (Joblet ji : jobletMap.values()) {
+ profiles.add(new JobProfile(ji.getJobId(), ji.getAttempt()));
+ }
}
for (JobProfile jProfile : profiles) {
Joblet ji;
@@ -560,9 +563,15 @@
}
@Override
- public synchronized void abortJoblet(UUID jobId) throws Exception {
- Joblet ji = jobletMap.remove(jobId);
+ public synchronized void abortJoblet(UUID jobId, int attempt) throws Exception {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Aborting Job: " + jobId + ":" + attempt);
+ }
+ Joblet ji = jobletMap.get(jobId);
if (ji != null) {
+ if (ji.getAttempt() == attempt) {
+ jobletMap.remove(jobId);
+ }
for (Stagelet stagelet : ji.getStageletMap().values()) {
stagelet.abort();
stagelet.close();
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Stagelet.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Stagelet.java
index c8afbc4..b0945be 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Stagelet.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Stagelet.java
@@ -34,16 +34,17 @@
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.io.IIOManager;
import edu.uci.ics.hyracks.api.io.IWorkspaceFileFactory;
+import edu.uci.ics.hyracks.api.job.profiling.counters.ICounter;
import edu.uci.ics.hyracks.api.job.profiling.counters.ICounterContext;
import edu.uci.ics.hyracks.api.job.profiling.om.StageletProfile;
import edu.uci.ics.hyracks.api.resources.IDeallocatable;
+import edu.uci.ics.hyracks.control.common.job.profiling.counters.Counter;
import edu.uci.ics.hyracks.control.nc.io.IOManager;
import edu.uci.ics.hyracks.control.nc.io.ManagedWorkspaceFileFactory;
-import edu.uci.ics.hyracks.control.nc.job.profiling.CounterContext;
import edu.uci.ics.hyracks.control.nc.resources.DefaultDeallocatableRegistry;
import edu.uci.ics.hyracks.control.nc.runtime.OperatorRunnable;
-public class Stagelet implements IHyracksStageletContext {
+public class Stagelet implements IHyracksStageletContext, ICounterContext {
private static final long serialVersionUID = 1L;
private static final Logger LOGGER = Logger.getLogger(Stagelet.class.getName());
@@ -56,7 +57,7 @@
private final Map<OperatorInstanceId, OperatorRunnable> honMap;
- private final CounterContext stageletCounterContext;
+ private final Map<String, Counter> counterMap;
private final IWorkspaceFileFactory fileFactory;
@@ -77,7 +78,7 @@
pendingOperators = new HashSet<OperatorInstanceId>();
started = false;
honMap = new HashMap<OperatorInstanceId, OperatorRunnable>();
- stageletCounterContext = new CounterContext(joblet.getJobId() + "." + stageId + "." + nodeId);
+ counterMap = new HashMap<String, Counter>();
deallocatableRegistry = new DefaultDeallocatableRegistry();
fileFactory = new ManagedWorkspaceFileFactory(this, (IOManager) joblet.getIOManager());
}
@@ -90,10 +91,6 @@
return honMap;
}
- public CounterContext getStageletCounterContext() {
- return stageletCounterContext;
- }
-
public void setEndpointList(List<Endpoint> endpointList) {
this.endpointList = endpointList;
}
@@ -183,8 +180,11 @@
}
}
- public void dumpProfile(StageletProfile sProfile) {
- stageletCounterContext.dump(sProfile.getCounters());
+ public synchronized void dumpProfile(StageletProfile sProfile) {
+ Map<String, Long> dumpMap = sProfile.getCounters();
+ for (Counter c : counterMap.values()) {
+ dumpMap.put(c.getName(), c.get());
+ }
}
@Override
@@ -199,7 +199,7 @@
@Override
public ICounterContext getCounterContext() {
- return stageletCounterContext;
+ return this;
}
@Override
@@ -230,4 +230,14 @@
public FileReference createWorkspaceFile(String prefix) throws HyracksDataException {
return fileFactory.createWorkspaceFile(prefix);
}
+
+ @Override
+ public ICounter getCounter(String name, boolean create) {
+ Counter counter = counterMap.get(name);
+ if (counter == null && create) {
+ counter = new Counter(name);
+ counterMap.put(name, counter);
+ }
+ return counter;
+ }
}
\ No newline at end of file
diff --git a/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java b/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
index b2eccdf..b1a1224 100644
--- a/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
+++ b/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
@@ -15,6 +15,7 @@
package edu.uci.ics.hyracks.examples.tpch.client;
import java.io.File;
+import java.util.EnumSet;
import java.util.UUID;
import org.kohsuke.args4j.CmdLineParser;
@@ -30,6 +31,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.job.JobFlag;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
import edu.uci.ics.hyracks.dataflow.common.data.hash.UTF8StringBinaryHashFunctionFactory;
@@ -74,6 +76,9 @@
@Option(name = "-num-join-partitions", usage = "Number of Join partitions to use (default: 1)", required = false)
public int numJoinPartitions = 1;
+
+ @Option(name = "-profile", usage = "Enable/Disable profiling. (default: enabled)")
+ public boolean profile = true;
}
public static void main(String[] args) throws Exception {
@@ -88,7 +93,8 @@
options.numJoinPartitions);
long start = System.currentTimeMillis();
- UUID jobId = hcc.createJob(options.app, job);
+ UUID jobId = hcc.createJob(options.app, job,
+ options.profile ? EnumSet.of(JobFlag.PROFILE_RUNTIME) : EnumSet.noneOf(JobFlag.class));
hcc.start(jobId);
hcc.waitForCompletion(jobId);
long end = System.currentTimeMillis();
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/job/profiling/CounterContext.java b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/CounterContext.java
similarity index 91%
rename from hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/job/profiling/CounterContext.java
rename to hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/CounterContext.java
index 401dd57..689a5241 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/job/profiling/CounterContext.java
+++ b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/CounterContext.java
@@ -12,7 +12,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.control.nc.job.profiling;
+package edu.uci.ics.hyracks.test.support;
import java.util.HashMap;
import java.util.Map;
@@ -40,11 +40,6 @@
return counter;
}
- @Override
- public String getContextName() {
- return contextName;
- }
-
public synchronized void dump(Map<String, Long> dumpMap) {
for (Counter c : counterMap.values()) {
dumpMap.put(c.getName(), c.get());
diff --git a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestJobletContext.java b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestJobletContext.java
index 921cb55..2fc43fc 100644
--- a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestJobletContext.java
+++ b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestJobletContext.java
@@ -27,7 +27,6 @@
import edu.uci.ics.hyracks.api.resources.IDeallocatable;
import edu.uci.ics.hyracks.control.nc.io.IOManager;
import edu.uci.ics.hyracks.control.nc.io.ManagedWorkspaceFileFactory;
-import edu.uci.ics.hyracks.control.nc.job.profiling.CounterContext;
public class TestJobletContext implements IHyracksJobletContext {
private final INCApplicationContext appContext;
diff --git a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStageletContext.java b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStageletContext.java
index d28911b..4d31326 100644
--- a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStageletContext.java
+++ b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStageletContext.java
@@ -27,7 +27,6 @@
import edu.uci.ics.hyracks.api.resources.IDeallocatable;
import edu.uci.ics.hyracks.control.nc.io.IOManager;
import edu.uci.ics.hyracks.control.nc.io.ManagedWorkspaceFileFactory;
-import edu.uci.ics.hyracks.control.nc.job.profiling.CounterContext;
public class TestStageletContext implements IHyracksStageletContext {
private final IHyracksJobletContext jobletContext;