Merge branch 'gerrit/neo' into 'gerrit/trinity'

Change-Id: Ibcbefe51fe91e800dbc218d5b7d65ca849c3db2f
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/RecoveryTask.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/RecoveryTask.java
index 34a54d1..e4e7b36 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/RecoveryTask.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/RecoveryTask.java
@@ -140,6 +140,7 @@
                         if (!cancelRecovery && listener.getState() == ActivityState.TEMPORARILY_FAILED) {
                             listener.setState(ActivityState.RECOVERING);
                             listener.doRecover(metadataProvider);
+                            listener.setRunning(metadataProvider, true);
                         }
                         LOGGER.log(level, "Recovery completed successfully");
                         return null;
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/executor/JobExecutor.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/executor/JobExecutor.java
index 1b0f377..b476993 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/executor/JobExecutor.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/executor/JobExecutor.java
@@ -539,7 +539,7 @@
 
     private void abortTaskCluster(TaskClusterAttempt tcAttempt,
             TaskClusterAttempt.TaskClusterStatus failedOrAbortedStatus) {
-        LOGGER.trace(() -> "Aborting task cluster: " + tcAttempt.getAttempt());
+        LOGGER.trace("Aborting task cluster: {}", tcAttempt.getAttempt());
         Set<TaskAttemptId> abortTaskIds = new HashSet<>();
         Map<String, List<TaskAttemptId>> abortTaskAttemptMap = new HashMap<>();
         for (TaskAttempt ta : tcAttempt.getTaskAttempts().values()) {
@@ -561,14 +561,12 @@
             }
         }
         final JobId jobId = jobRun.getJobId();
-        LOGGER.trace(() -> "Abort map for job: " + jobId + ": " + abortTaskAttemptMap);
+        LOGGER.trace("Abort map for job: {}: {}", jobId, abortTaskAttemptMap);
         INodeManager nodeManager = ccs.getNodeManager();
         abortTaskAttemptMap.forEach((key, abortTaskAttempts) -> {
             final NodeControllerState node = nodeManager.getNodeControllerState(key);
             if (node != null) {
-                if (LOGGER.isTraceEnabled()) {
-                    LOGGER.trace("Aborting: " + abortTaskAttempts + " at " + key);
-                }
+                LOGGER.trace("Aborting: {} at {}", abortTaskAttempts, key);
                 try {
                     node.getNodeController().abortTasks(jobId, abortTaskAttempts);
                 } catch (Exception e) {
@@ -579,8 +577,8 @@
         inProgressTaskClusters.remove(tcAttempt.getTaskCluster());
         TaskCluster tc = tcAttempt.getTaskCluster();
         PartitionMatchMaker pmm = jobRun.getPartitionMatchMaker();
-        pmm.removeUncommittedPartitions(tc.getProducedPartitions(), abortTaskIds);
-        pmm.removePartitionRequests(tc.getRequiredPartitions(), abortTaskIds);
+        pmm.removeUncommittedPartitions(tc.getProducedPartitions(), abortTaskIds, jobId);
+        pmm.removePartitionRequests(tc.getRequiredPartitions(), abortTaskIds, jobId);
 
         tcAttempt.setStatus(failedOrAbortedStatus);
         tcAttempt.setEndTime(System.currentTimeMillis());
@@ -683,7 +681,6 @@
      */
     public void notifyTaskFailure(TaskAttempt ta, List<Exception> exceptions) {
         try {
-            LOGGER.debug("Received failure notification for TaskAttempt " + ta.getTaskAttemptId());
             TaskAttemptId taId = ta.getTaskAttemptId();
             TaskCluster tc = ta.getTask().getTaskCluster();
             TaskClusterAttempt lastAttempt = findLastTaskClusterAttempt(tc);
@@ -696,7 +693,7 @@
                 LOGGER.trace(() -> "Marking TaskAttempt " + ta.getTaskAttemptId()
                         + " as failed and the number of max re-attempts = " + maxReattempts);
                 if (lastAttempt.getAttempt() >= maxReattempts || isCancelled()) {
-                    LOGGER.debug(() -> "Aborting the job of " + ta.getTaskAttemptId());
+                    LOGGER.debug("Aborting the job:{} of {}", jobRun.getJobId(), ta.getTaskAttemptId());
                     abortJob(exceptions, NoOpCallback.INSTANCE);
                     return;
                 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobManager.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobManager.java
index 9165953..3a954f4 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobManager.java
@@ -145,7 +145,7 @@
         if (activeRunMap.containsKey(jobId)) {
             JobRun jobRun = activeRunMap.get(jobId);
             // The following call will abort all ongoing tasks and then consequently
-            // trigger JobCleanupWork and JobCleanupNotificationWork which will update the lifecyle of the job.
+            // trigger JobCleanupWork and JobCleanupNotificationWork which will update the lifecycle of the job.
             // Therefore, we do not remove the job out of activeRunMap here.
             jobRun.getExecutor().cancelJob(callback);
             incrementCancelledJobs();
@@ -157,7 +157,7 @@
             incrementCancelledJobs();
             List<Exception> exceptions =
                     Collections.singletonList(HyracksException.create(ErrorCode.JOB_CANCELED, jobId));
-            // Since the job has not been executed, we only need to update its status and lifecyle here.
+            // Since the job has not been executed, we only need to update its status and lifecycle here.
             jobRun.setStatus(JobStatus.FAILURE_BEFORE_EXECUTION, exceptions);
             runMapArchive.put(jobId, jobRun);
             runMapHistory.put(jobId, exceptions);
@@ -189,7 +189,6 @@
             return;
         }
         if (run.getPendingStatus() != null) {
-            LOGGER.warn("Ignoring duplicate cleanup for JobRun with id: {}", run::getJobId);
             return;
         }
         Set<String> targetNodes = run.getParticipatingNodeIds();
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/partitions/PartitionMatchMaker.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/partitions/PartitionMatchMaker.java
index ac29b53..8f91944 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/partitions/PartitionMatchMaker.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/partitions/PartitionMatchMaker.java
@@ -28,6 +28,7 @@
 
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hyracks.api.dataflow.TaskAttemptId;
+import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.partitions.PartitionId;
 import org.apache.hyracks.control.common.job.PartitionDescriptor;
 import org.apache.hyracks.control.common.job.PartitionRequest;
@@ -43,14 +44,13 @@
     private final Map<PartitionId, List<PartitionRequest>> partitionRequests;
 
     public PartitionMatchMaker() {
-        partitionDescriptors = new HashMap<PartitionId, List<PartitionDescriptor>>();
-        partitionRequests = new HashMap<PartitionId, List<PartitionRequest>>();
+        partitionDescriptors = new HashMap<>();
+        partitionRequests = new HashMap<>();
     }
 
     public List<Pair<PartitionDescriptor, PartitionRequest>> registerPartitionDescriptor(
             PartitionDescriptor partitionDescriptor) {
-        List<Pair<PartitionDescriptor, PartitionRequest>> matches =
-                new ArrayList<Pair<PartitionDescriptor, PartitionRequest>>();
+        List<Pair<PartitionDescriptor, PartitionRequest>> matches = new ArrayList<>();
         PartitionId pid = partitionDescriptor.getPartitionId();
         boolean matched = false;
         List<PartitionRequest> requests = partitionRequests.get(pid);
@@ -73,11 +73,7 @@
         }
 
         if (!matched) {
-            List<PartitionDescriptor> descriptors = partitionDescriptors.get(pid);
-            if (descriptors == null) {
-                descriptors = new ArrayList<PartitionDescriptor>();
-                partitionDescriptors.put(pid, descriptors);
-            }
+            List<PartitionDescriptor> descriptors = partitionDescriptors.computeIfAbsent(pid, k -> new ArrayList<>());
             descriptors.add(partitionDescriptor);
         }
 
@@ -108,11 +104,7 @@
         }
 
         if (match == null) {
-            List<PartitionRequest> requests = partitionRequests.get(pid);
-            if (requests == null) {
-                requests = new ArrayList<PartitionRequest>();
-                partitionRequests.put(pid, requests);
-            }
+            List<PartitionRequest> requests = partitionRequests.computeIfAbsent(pid, k -> new ArrayList<>());
             requests.add(partitionRequest);
         }
 
@@ -133,17 +125,11 @@
     }
 
     private interface IEntryFilter<T> {
-        public boolean matches(T o);
+        boolean matches(T o);
     }
 
     private static <T> void removeEntries(List<T> list, IEntryFilter<T> filter) {
-        Iterator<T> j = list.iterator();
-        while (j.hasNext()) {
-            T o = j.next();
-            if (filter.matches(o)) {
-                j.remove();
-            }
-        }
+        list.removeIf(filter::matches);
     }
 
     private static <T> void removeEntries(Map<PartitionId, List<T>> map, IEntryFilter<T> filter) {
@@ -159,30 +145,16 @@
     }
 
     public void notifyNodeFailures(final Collection<String> deadNodes) {
-        removeEntries(partitionDescriptors, new IEntryFilter<PartitionDescriptor>() {
-            @Override
-            public boolean matches(PartitionDescriptor o) {
-                return deadNodes.contains(o.getNodeId());
-            }
-        });
-        removeEntries(partitionRequests, new IEntryFilter<PartitionRequest>() {
-            @Override
-            public boolean matches(PartitionRequest o) {
-                return deadNodes.contains(o.getNodeId());
-            }
-        });
+        removeEntries(partitionDescriptors, o -> deadNodes.contains(o.getNodeId()));
+        removeEntries(partitionRequests, o -> deadNodes.contains(o.getNodeId()));
     }
 
-    public void removeUncommittedPartitions(Set<PartitionId> partitionIds, final Set<TaskAttemptId> taIds) {
-        if (LOGGER.isDebugEnabled()) {
-            LOGGER.debug("Removing uncommitted partitions: " + partitionIds);
+    public void removeUncommittedPartitions(Set<PartitionId> partitionIds, Set<TaskAttemptId> taIds, JobId jobId) {
+        if (!partitionIds.isEmpty()) {
+            LOGGER.debug("Removing uncommitted partitions {}: {}", jobId, partitionIds);
         }
-        IEntryFilter<PartitionDescriptor> filter = new IEntryFilter<PartitionDescriptor>() {
-            @Override
-            public boolean matches(PartitionDescriptor o) {
-                return o.getState() != PartitionState.COMMITTED && taIds.contains(o.getProducingTaskAttemptId());
-            }
-        };
+        IEntryFilter<PartitionDescriptor> filter =
+                o -> o.getState() != PartitionState.COMMITTED && taIds.contains(o.getProducingTaskAttemptId());
         for (PartitionId pid : partitionIds) {
             List<PartitionDescriptor> descriptors = partitionDescriptors.get(pid);
             if (descriptors != null) {
@@ -194,16 +166,11 @@
         }
     }
 
-    public void removePartitionRequests(Set<PartitionId> partitionIds, final Set<TaskAttemptId> taIds) {
-        if (LOGGER.isDebugEnabled()) {
-            LOGGER.debug("Removing partition requests: " + partitionIds);
+    public void removePartitionRequests(Set<PartitionId> partitionIds, Set<TaskAttemptId> taIds, JobId jobId) {
+        if (!partitionIds.isEmpty()) {
+            LOGGER.debug("Removing partition requests {}: {}", jobId, partitionIds);
         }
-        IEntryFilter<PartitionRequest> filter = new IEntryFilter<PartitionRequest>() {
-            @Override
-            public boolean matches(PartitionRequest o) {
-                return taIds.contains(o.getRequestingTaskAttemptId());
-            }
-        };
+        IEntryFilter<PartitionRequest> filter = o -> taIds.contains(o.getRequestingTaskAttemptId());
         for (PartitionId pid : partitionIds) {
             List<PartitionRequest> requests = partitionRequests.get(pid);
             if (requests != null) {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/result/ResultDirectoryService.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/result/ResultDirectoryService.java
index a0c2ce4..f56ec33 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/result/ResultDirectoryService.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/result/ResultDirectoryService.java
@@ -46,7 +46,6 @@
 import org.apache.hyracks.control.common.result.AbstractResultManager;
 import org.apache.hyracks.control.common.result.ResultStateSweeper;
 import org.apache.hyracks.control.common.work.IResultCallback;
-import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -81,9 +80,7 @@
     @Override
     public synchronized void notifyJobCreation(JobId jobId, JobSpecification spec,
             IJobCapacityController.JobSubmissionStatus status) throws HyracksException {
-        if (LOGGER.isDebugEnabled()) {
-            LOGGER.debug(getClass().getSimpleName() + " notified of new job " + jobId);
-        }
+        LOGGER.debug("{} notified of new job {}", getClass().getSimpleName(), jobId);
         if (jobResultLocations.get(jobId) != null) {
             throw HyracksDataException.create(ErrorCode.MORE_THAN_ONE_RESULT, jobId);
         }
@@ -160,15 +157,14 @@
 
     @Override
     public synchronized void reportJobFailure(JobId jobId, List<Exception> exceptions) {
-        Exception ex = exceptions.isEmpty() ? null : exceptions.get(0);
-        Level logLevel = Level.DEBUG;
-        if (LOGGER.isEnabled(logLevel)) {
-            LOGGER.log(logLevel, "job " + jobId + " failed and is being reported to " + getClass().getSimpleName(), ex);
-        }
         ResultJobRecord rjr = getResultJobRecord(jobId);
+        if (logFailure(rjr)) {
+            LOGGER.debug("job {} failed and is being reported to {}", jobId, getClass().getSimpleName());
+        }
         if (rjr != null) {
             rjr.fail(exceptions);
         }
+        Exception ex = exceptions.isEmpty() ? null : exceptions.get(0);
         final JobResultInfo jobResultInfo = jobResultLocations.get(jobId);
         if (jobResultInfo != null) {
             jobResultInfo.setException(ex);
@@ -214,6 +210,15 @@
         }
     }
 
+    private static boolean logFailure(ResultJobRecord rjr) {
+        if (rjr == null) {
+            return true;
+        }
+        // don't re-log if the state is already failed
+        ResultJobRecord.Status status = rjr.getStatus();
+        return status == null || status.getState() != State.FAILED;
+    }
+
     /**
      * Compares the records already known by the client for the given job's result set id with the records that the
      * result directory service knows and if there are any newly discovered records returns a whole array with the
@@ -267,7 +272,7 @@
 
 class JobResultInfo {
 
-    private ResultJobRecord record;
+    private final ResultJobRecord record;
     private Waiters waiters;
     private Exception exception;
 
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobCleanupWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobCleanupWork.java
index 6262c47..f065940 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobCleanupWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobCleanupWork.java
@@ -52,7 +52,6 @@
 
     @Override
     public void run() {
-        LOGGER.info("cleaning up {} on NCs, status={}", jobId, status);
         final JobRun jobRun = jobManager.get(jobId);
         if (jobRun == null) {
             LOGGER.debug("ignoring cleanup for unknown {}", jobId);
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/TaskFailureWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/TaskFailureWork.java
index 833066e..33d391f 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/TaskFailureWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/TaskFailureWork.java
@@ -22,17 +22,13 @@
 
 import org.apache.hyracks.api.dataflow.TaskAttemptId;
 import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.api.util.ExceptionUtils;
 import org.apache.hyracks.control.cc.ClusterControllerService;
 import org.apache.hyracks.control.cc.job.IJobManager;
 import org.apache.hyracks.control.cc.job.JobRun;
 import org.apache.hyracks.control.cc.job.TaskAttempt;
-import org.apache.logging.log4j.Level;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
 
 public class TaskFailureWork extends AbstractTaskLifecycleWork {
-    private static final Logger LOGGER = LogManager.getLogger();
+
     private final List<Exception> exceptions;
 
     public TaskFailureWork(ClusterControllerService ccs, JobId jobId, TaskAttemptId taId, String nodeId,
@@ -43,9 +39,6 @@
 
     @Override
     protected void performEvent(TaskAttempt ta) {
-        Exception ex = exceptions.get(0);
-        LOGGER.log(ExceptionUtils.causedByInterrupt(ex) ? Level.DEBUG : Level.WARN,
-                "Executing task failure work for " + this, ex);
         IJobManager jobManager = ccs.getJobManager();
         JobRun run = jobManager.get(jobId);
         if (run == null) {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/config/ConfigManager.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/config/ConfigManager.java
index 85661fe..9c5a9fa 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/config/ConfigManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/config/ConfigManager.java
@@ -148,7 +148,7 @@
             if (configured) {
                 throw new IllegalStateException("configuration already processed");
             }
-            LOGGER.debug("registering option: " + option.toIniString());
+            LOGGER.trace("registering option: {}", option::toIniString);
             Map<String, IOption> optionMap = sectionMap.computeIfAbsent(option.section(), section -> new HashMap<>());
             IOption prev = optionMap.put(option.ini(), option);
             if (prev != null) {
@@ -160,8 +160,13 @@
                 registeredOptions.add(option);
                 optionSetters.put(option, (node, value, isDefault) -> correctedMap(node, isDefault).put(option, value));
                 if (LOGGER.isDebugEnabled()) {
-                    optionSetters.put(option, (node, value, isDefault) -> LOGGER.debug("{} {} to {} for node {}",
-                            isDefault ? "defaulting" : "setting", option.toIniString(), value, node));
+                    optionSetters.put(option, (node, value, isDefault) -> {
+                        if (isDefault) {
+                            LOGGER.trace("defaulting {} to {} for node {}", option.toIniString(), value, node);
+                        } else {
+                            LOGGER.debug("setting {} to {} for node {}", option.toIniString(), value, node);
+                        }
+                    });
                 }
             }
         }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/CcConnection.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/CcConnection.java
index d1f7d5a..c99898d 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/CcConnection.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/CcConnection.java
@@ -70,6 +70,8 @@
             InvokeUtil.runWithTimeout(() -> {
                 this.wait(REGISTRATION_RESPONSE_POLL_PERIOD); // NOSONAR while loop in timeout call
             }, () -> !registrationPending, 1, TimeUnit.MINUTES);
+        } catch (InterruptedException e) {
+            throw e;
         } catch (Exception e) {
             registrationException = e;
         }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/NotifyTaskFailureWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/NotifyTaskFailureWork.java
index b0c60aa..6dd4307 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/NotifyTaskFailureWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/NotifyTaskFailureWork.java
@@ -23,7 +23,6 @@
 import org.apache.hyracks.api.dataflow.TaskAttemptId;
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.result.IResultPartitionManager;
-import org.apache.hyracks.api.util.ExceptionUtils;
 import org.apache.hyracks.control.common.work.AbstractWork;
 import org.apache.hyracks.control.nc.NodeControllerService;
 import org.apache.hyracks.control.nc.Task;
@@ -50,9 +49,6 @@
 
     @Override
     public void run() {
-        Exception ex = exceptions.get(0);
-        LOGGER.log(ExceptionUtils.causedByInterrupt(ex) ? Level.DEBUG : Level.WARN, "task " + taskId + " has failed",
-                ex);
         try {
             IResultPartitionManager resultPartitionManager = ncs.getResultPartitionManager();
             if (resultPartitionManager != null) {
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TestJobLifecycleListener.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TestJobLifecycleListener.java
index 4d4635a..19fdcfe 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TestJobLifecycleListener.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TestJobLifecycleListener.java
@@ -48,7 +48,8 @@
     private final Set<JobId> finishWithoutStart = new HashSet<>();
 
     @Override
-    public void notifyJobCreation(JobId jobId, JobSpecification spec, IJobCapacityController.JobSubmissionStatus status) throws HyracksException {
+    public void notifyJobCreation(JobId jobId, JobSpecification spec, IJobCapacityController.JobSubmissionStatus status)
+            throws HyracksException {
         if (created.containsKey(jobId)) {
             LOGGER.log(Level.WARN, "Job " + jobId + "has been created before");
             increment(doubleCreated, jobId);
@@ -76,7 +77,8 @@
     }
 
     @Override
-    public void notifyJobFinish(JobId jobId, JobSpecification spec, JobStatus jobStatus, List<Exception> exceptions) throws HyracksException {
+    public void notifyJobFinish(JobId jobId, JobSpecification spec, JobStatus jobStatus, List<Exception> exceptions)
+            throws HyracksException {
         if (!started.contains(jobId)) {
             LOGGER.log(Level.WARN, "Job " + jobId + "has not been started");
             finishWithoutStart.add(jobId);