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;