Merge branch 'gerrit/stabilization-40cfb8705b' into 'gerrit/neo'

Change-Id: Ia8359d2d0d8e1b39b1f2c15e5f367332671f9df2
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveManager.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveManager.java
index b99e4f2..ed481bc 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveManager.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveManager.java
@@ -104,9 +104,9 @@
     }
 
     public void submit(ActiveManagerMessage message) throws HyracksDataException {
-        LOGGER.debug("Message of type {} received in {}", message.getKind(), nodeId);
         switch (message.getKind()) {
             case STOP_ACTIVITY:
+                LOGGER.debug("Message of type {} received in {}", message.getKind(), nodeId);
                 stopRuntime(message);
                 break;
             case REQUEST_STATS:
@@ -151,7 +151,6 @@
                 return;
             }
             String stats = runtime.getStats();
-            LOGGER.debug("Sending stats response for {} ", runtimeId);
             ActiveStatsResponse response = new ActiveStatsResponse(reqId, stats, null);
             ((NodeControllerService) serviceCtx.getControllerService()).sendRealTimeApplicationMessageToCC(
                     message.getCcId(), JavaSerializationUtils.serialize(response), null);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AsterixOptimizationContext.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AsterixOptimizationContext.java
index 05e7f7b..733da2d 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AsterixOptimizationContext.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AsterixOptimizationContext.java
@@ -24,6 +24,7 @@
 import java.util.function.Predicate;
 
 import org.apache.asterix.metadata.declared.DataSource;
+import org.apache.asterix.metadata.declared.FunctionDataSource;
 import org.apache.asterix.metadata.utils.MetadataConstants;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IConflictingTypeResolver;
@@ -83,10 +84,10 @@
         }
         for (Int2ObjectMap.Entry<Set<DataSource>> me : dataSourceMap.int2ObjectEntrySet()) {
             int dataSourceType = me.getIntKey();
-            if (dataSourceType != DataSource.Type.INTERNAL_DATASET) {
+            if (dataSourceType != DataSource.Type.INTERNAL_DATASET && dataSourceType != DataSource.Type.FUNCTION) {
                 return false;
             }
-            Predicate<DataSource> dataSourceTest = AsterixOptimizationContext::isMetadata;
+            Predicate<DataSource> dataSourceTest = AsterixOptimizationContext::skipJobCapacityAssignment;
             if (!me.getValue().stream().allMatch(dataSourceTest)) {
                 return false;
             }
@@ -94,7 +95,11 @@
         return true;
     }
 
-    private static boolean isMetadata(DataSource ds) {
-        return MetadataConstants.METADATA_DATAVERSE_NAME.equals(ds.getId().getDataverseName());
+    private static boolean skipJobCapacityAssignment(DataSource ds) {
+        return MetadataConstants.METADATA_DATAVERSE_NAME.equals(ds.getId().getDataverseName()) || isSkipping(ds);
+    }
+
+    private static boolean isSkipping(DataSource ds) {
+        return (ds instanceof FunctionDataSource) && ((FunctionDataSource) ds).skipJobCapacityAssignment();
     }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveEntityEventsListener.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveEntityEventsListener.java
index 63d8846..7b253f1 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveEntityEventsListener.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveEntityEventsListener.java
@@ -138,7 +138,7 @@
     }
 
     protected synchronized void setState(ActivityState newState) {
-        LOGGER.log(level, "State of {} is being set to {} from {}", getEntityId(), newState, state);
+        LOGGER.log(level, "state of {} is being set from {} to {}", getEntityId(), state, newState);
         this.prevState = state;
         this.state = newState;
         if (newState == ActivityState.STARTING || newState == ActivityState.RECOVERING
@@ -471,7 +471,7 @@
         // Note: once we start sending stop messages, we can't go back until the entity is stopped
         final String nameBefore = Thread.currentThread().getName();
         try {
-            Thread.currentThread().setName(nameBefore + " : WaitForCompletionForJobId: " + jobId);
+            Thread.currentThread().setName(nameBefore + " : wait-for-ingestion-completion: " + jobId);
             sendStopMessages(metadataProvider, timeout, unit);
             LOGGER.debug("waiting for {} to become {}", jobId, waitFor);
             subscriber.sync();
@@ -501,7 +501,7 @@
         ICCMessageBroker messageBroker = (ICCMessageBroker) applicationCtx.getServiceContext().getMessageBroker();
         AlgebricksAbsolutePartitionConstraint runtimeLocations = getLocations();
         int partition = 0;
-        LOGGER.log(Level.INFO, "Sending stop messages to {}", runtimeLocations);
+        LOGGER.log(Level.INFO, "sending stop messages to {}", runtimeLocations);
         for (String location : runtimeLocations.getLocations()) {
             ActiveRuntimeId runtimeId = getActiveRuntimeId(partition++);
             messageBroker.sendApplicationMessageToNC(new ActiveManagerMessage(ActiveManagerMessage.Kind.STOP_ACTIVITY,
@@ -564,10 +564,10 @@
         WaitForStateSubscriber subscriber;
         Future<Void> suspendTask;
         synchronized (this) {
-            LOGGER.log(level, "{} Suspending entity {}", jobId, entityId);
-            LOGGER.log(level, "{} Waiting for ongoing activities", jobId);
+            LOGGER.log(level, "{} suspending entity {}", jobId, entityId);
+            LOGGER.log(level, "{} waiting for ongoing activities", jobId);
             waitForNonTransitionState();
-            LOGGER.log(level, "{} Proceeding with suspension. Current state is {}", jobId, state);
+            LOGGER.log(level, "{} proceeding with suspension. current state is {}", jobId, state);
             if (state == ActivityState.STOPPED) {
                 suspended = true;
                 return;
@@ -588,12 +588,12 @@
                         doSuspend(metadataProvider);
                         return null;
                     });
-            LOGGER.log(level, "{} Suspension task has been submitted", jobId);
+            LOGGER.log(level, "{} suspension task has been submitted", jobId);
         }
         try {
-            LOGGER.log(level, "{} Waiting for suspension task to complete", jobId);
+            LOGGER.log(level, "{} waiting for suspension task to complete", jobId);
             suspendTask.get();
-            LOGGER.log(level, "{} Waiting for state to become SUSPENDED or TEMPORARILY_FAILED", jobId);
+            LOGGER.log(level, "{} waiting for state to become SUSPENDED or TEMPORARILY_FAILED", jobId);
             subscriber.sync();
             suspended = true;
         } catch (Exception e) {
@@ -715,7 +715,7 @@
 
     @Override
     public String toString() {
-        return "{\"class\":\"" + getClass().getSimpleName() + "\"," + "\"entityId\":\"" + entityId + "\","
-                + "\"state\":\"" + state + "\"" + "}";
+        return "{\"class\":\"" + getClass().getSimpleName() + "\", \"entityId\":\"" + entityId + "\", \"state\":\""
+                + state + "\", \"prev state\":\"" + prevState + "\", \"suspended\":" + suspended + "}";
     }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveNotificationHandler.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveNotificationHandler.java
index 1f31698..662884d 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveNotificationHandler.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveNotificationHandler.java
@@ -72,16 +72,14 @@
         EntityId entityId = jobId2EntityId.get(jobId);
         if (entityId != null) {
             IActiveEntityEventsListener listener = entityEventListeners.get(entityId);
-            if (LOGGER.isEnabled(level)) {
-                LOGGER.log(level, "Next event is {} for job {}", eventKind, jobId);
-            }
             if (eventKind == Kind.JOB_FINISHED) {
-                LOGGER.log(level, "Removing job {}", jobId);
+                LOGGER.debug("removing ingestion job {}", jobId);
                 jobId2EntityId.remove(jobId);
             }
             if (listener != null) {
-                LOGGER.log(level, "Notifying the listener");
                 listener.notify(event);
+            } else {
+                LOGGER.debug("listener not found for entity {} on event={}", entityId, event);
             }
         } else {
             LOGGER.log(Level.ERROR, "Entity not found for event {} for job {}", eventKind, jobId);
@@ -95,40 +93,26 @@
         Object property = jobSpecification.getProperty(ACTIVE_ENTITY_PROPERTY_NAME);
         if (!(property instanceof EntityId)) {
             if (property != null) {
-                LOGGER.debug("{} is not an active job. job property={}", jobId, property);
+                LOGGER.debug("{} is not an ingestion job. job property={}", jobId, property);
             }
             return;
         }
         LOGGER.debug("notified of ingestion job creation {}", jobId);
         EntityId entityId = (EntityId) property;
         monitorJob(jobId, entityId);
-        boolean found = jobId2EntityId.get(jobId) != null;
-        LOGGER.log(level, "Job {} was found to be {}", jobId, (found ? "Active" : "Inactive"));
         add(new ActiveEvent(jobId, Kind.JOB_CREATED, entityId, jobSpecification));
     }
 
     private synchronized void monitorJob(JobId jobId, EntityId entityId) {
-        if (LOGGER.isEnabled(level)) {
-            LOGGER.log(level, "monitorJob was called for job {}", jobId);
-        }
-        boolean found = jobId2EntityId.get(jobId) != null;
-        if (LOGGER.isEnabled(level)) {
-            LOGGER.log(level, "Job {} was found to be {}", jobId, (found ? "Active" : "Inactive"));
-        }
+        boolean found = jobId2EntityId.containsKey(jobId);
+        LOGGER.debug("{} was {}", jobId, (found ? "active" : "inactive"));
         if (entityEventListeners.containsKey(entityId)) {
-            if (jobId2EntityId.containsKey(jobId)) {
-                if (LOGGER.isErrorEnabled()) {
-                    LOGGER.error("Job {} is already being monitored", jobId);
-                }
+            if (found) {
+                LOGGER.error("{} is already being monitored", jobId);
                 return;
             }
-            if (LOGGER.isEnabled(level)) {
-                LOGGER.log(level, "Monitoring started for job {}", jobId);
-            }
         } else {
-            if (LOGGER.isEnabled(level)) {
-                LOGGER.info("No listener was found for the entity {} for job {}", entityId, jobId);
-            }
+            LOGGER.debug("no listener found for entity {}; {}", entityId, jobId);
         }
         jobId2EntityId.put(jobId, entityId);
     }
@@ -139,22 +123,18 @@
         if (entityId != null) {
             add(new ActiveEvent(jobId, Kind.JOB_STARTED, entityId, null));
         }
+        // else must be non-active job, e.g. a job for a query
     }
 
     @Override
     public synchronized void notifyJobFinish(JobId jobId, JobStatus jobStatus, List<Exception> exceptions)
             throws HyracksException {
-        if (LOGGER.isEnabled(level)) {
-            LOGGER.log(level, "Getting notified of job finish for job {}", jobId);
-        }
         EntityId entityId = jobId2EntityId.get(jobId);
         if (entityId != null) {
+            LOGGER.debug("notified of ingestion job finish {}", jobId);
             add(new ActiveEvent(jobId, Kind.JOB_FINISHED, entityId, Pair.of(jobStatus, exceptions)));
-        } else {
-            if (LOGGER.isEnabled(level)) {
-                LOGGER.log(level, "No need to notify JOB_FINISHED for job {}", jobId);
-            }
         }
+        // else must be non-active job, e.g. a job for a query
     }
 
     // *** IActiveNotificationHandler
@@ -167,13 +147,6 @@
 
     @Override
     public IActiveEntityEventsListener getListener(EntityId entityId) {
-        if (LOGGER.isEnabled(level)) {
-            LOGGER.log(level, "getActiveEntityListener was called with entity {}", entityId);
-        }
-        IActiveEntityEventsListener listener = entityEventListeners.get(entityId);
-        if (LOGGER.isEnabled(level)) {
-            LOGGER.log(level, "Listener found: {}", listener);
-        }
         return entityEventListeners.get(entityId);
     }
 
@@ -195,9 +168,7 @@
         if (suspended) {
             throw new RuntimeDataException(ErrorCode.ACTIVE_NOTIFICATION_HANDLER_IS_SUSPENDED);
         }
-        if (LOGGER.isEnabled(level)) {
-            LOGGER.log(level, "registerListener was called for the entity {}", listener.getEntityId());
-        }
+        LOGGER.debug("register listener for entity {}, state={}", listener.getEntityId(), listener.getState());
         if (entityEventListeners.containsKey(listener.getEntityId())) {
             throw new RuntimeDataException(ErrorCode.ACTIVE_ENTITY_IS_ALREADY_REGISTERED, listener.getEntityId());
         }
@@ -209,9 +180,7 @@
         if (suspended) {
             throw new RuntimeDataException(ErrorCode.ACTIVE_NOTIFICATION_HANDLER_IS_SUSPENDED);
         }
-        if (LOGGER.isEnabled(level)) {
-            LOGGER.log(level, "unregisterListener was called for the entity {}", listener.getEntityId());
-        }
+        LOGGER.debug("unregister listener for entity {}, state={}", listener.getEntityId(), listener.getState());
         IActiveEntityEventsListener registeredListener = entityEventListeners.remove(listener.getEntityId());
         if (registeredListener == null) {
             throw new RuntimeDataException(ErrorCode.ACTIVE_ENTITY_LISTENER_IS_NOT_REGISTERED, listener.getEntityId());
@@ -227,9 +196,8 @@
         LOGGER.info("Starting active recovery");
         for (IActiveEntityEventsListener listener : getEventListeners()) {
             synchronized (listener) {
-                if (LOGGER.isEnabled(level)) {
-                    LOGGER.log(level, "Entity {} is {}", listener.getEntityId(), listener.getState());
-                }
+                LOGGER.debug("entity {} is {}, active={}, suspended={}", listener.getEntityId(), listener.getState(),
+                        listener.isActive(), listener.isSuspended());
                 listener.notifyAll();
             }
         }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/ActiveRequestsDatasource.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/ActiveRequestsDatasource.java
index 24a127a..5239d01 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/ActiveRequestsDatasource.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/ActiveRequestsDatasource.java
@@ -43,4 +43,9 @@
                 AlgebricksAbsolutePartitionConstraint.randomLocation(locations.getLocations());
         return new ClientRequestsFunction(randomLocation, ClientRequestsRequest.RequestType.RUNNING);
     }
+
+    @Override
+    public boolean skipJobCapacityAssignment() {
+        return true;
+    }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/CompletedRequestsDatasource.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/CompletedRequestsDatasource.java
index 0bfbff1..f02af21 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/CompletedRequestsDatasource.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/CompletedRequestsDatasource.java
@@ -43,4 +43,9 @@
                 AlgebricksAbsolutePartitionConstraint.randomLocation(locations.getLocations());
         return new ClientRequestsFunction(randomLocation, ClientRequestsRequest.RequestType.COMPLETED);
     }
+
+    @Override
+    public boolean skipJobCapacityAssignment() {
+        return true;
+    }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CancelQueryRequest.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CancelQueryRequest.java
index 04e661b..6570041 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CancelQueryRequest.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CancelQueryRequest.java
@@ -69,10 +69,8 @@
                 }
             }
         }
-        if (LOGGER.isDebugEnabled()) {
-            LOGGER.debug("Sending CancelQueryResponse to {}. requestId:{}, uuid:{}, contextId:{}, status:{}", nodeId,
-                    requestId, uuid, contextId, status);
-        }
+        LOGGER.debug("sending CancelQueryResponse to {}. reqId:{}, uuid:{}, contextId:{}, status:{}", nodeId, requestId,
+                uuid, contextId, status);
         CancelQueryResponse response = new CancelQueryResponse(reqId, status);
         CCMessageBroker messageBroker = (CCMessageBroker) appCtx.getServiceContext().getMessageBroker();
         try {
@@ -84,7 +82,7 @@
 
     @Override
     public String toString() {
-        return String.format("%s(id=%s, uuid=%s, contextId=%s, node=%s)", getClass().getSimpleName(), reqId, uuid,
-                contextId, nodeId);
+        return "CancelQueryRequest{from='" + nodeId + "', reqId=" + reqId + ", uuid='" + uuid + "', contextId='"
+                + contextId + "'}";
     }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CancelQueryResponse.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CancelQueryResponse.java
index a711b73..68d3430 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CancelQueryResponse.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CancelQueryResponse.java
@@ -51,6 +51,6 @@
 
     @Override
     public String toString() {
-        return String.format("%s(id=%s, status=%s)", getClass().getSimpleName(), reqId, status);
+        return "CancelQueryResponse{reqId=" + reqId + ", status=" + status + '}';
     }
 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/active_requests/active_requests.4.regex b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/active_requests/active_requests.4.regex
index e2d166b..695472c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/active_requests/active_requests.4.regex
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/active_requests/active_requests.4.regex
@@ -1 +1 @@
-/memory\D+229376/
\ No newline at end of file
+/memory\D+0/
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/completed_requests/completed_requests.3.regex b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/completed_requests/completed_requests.3.regex
index e2d166b..695472c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/completed_requests/completed_requests.3.regex
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/completed_requests/completed_requests.3.regex
@@ -1 +1 @@
-/memory\D+229376/
\ No newline at end of file
+/memory\D+0/
\ No newline at end of file
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java
index 97c4c5d..3b6bd9d 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java
@@ -106,6 +106,10 @@
                 tupleFilterFactory, outputLimit);
     }
 
+    public boolean skipJobCapacityAssignment() {
+        return false;
+    }
+
     protected abstract IDatasourceFunction createFunction(MetadataProvider metadataProvider,
             AlgebricksAbsolutePartitionConstraint locations);
 
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 2347449..ad97188 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
@@ -314,7 +314,7 @@
         run.setStartTime(System.currentTimeMillis());
         run.setStartTimeZoneId(ZoneId.systemDefault().getId());
         JobId jobId = run.getJobId();
-        logJobCapacity(run, "running");
+        logJobCapacity(run, "running", Level.DEBUG);
         activeRunMap.put(jobId, run);
         run.setStatus(JobStatus.RUNNING, null);
         executeJobInternal(run);
@@ -322,7 +322,7 @@
 
     // Queue a job when the required capacity for the job is not met.
     private void queueJob(JobRun jobRun) throws HyracksException {
-        logJobCapacity(jobRun, "queueing");
+        logJobCapacity(jobRun, "queueing", Level.INFO);
         jobRun.setStatus(JobStatus.PENDING, null);
         jobQueue.add(jobRun);
     }
@@ -358,10 +358,10 @@
     private void releaseJobCapacity(JobRun jobRun) {
         final JobSpecification job = jobRun.getJobSpecification();
         jobCapacityController.release(job);
-        logJobCapacity(jobRun, "released");
+        logJobCapacity(jobRun, "released", Level.DEBUG);
     }
 
-    private void logJobCapacity(JobRun jobRun, String jobStateDesc) {
+    private void logJobCapacity(JobRun jobRun, String jobStateDesc, Level lvl) {
         IClusterCapacity requiredResources = jobRun.getJobSpecification().getRequiredClusterCapacity();
         if (requiredResources == null) {
             return;
@@ -372,7 +372,7 @@
             return;
         }
         IReadOnlyClusterCapacity clusterCapacity = jobCapacityController.getClusterCapacity();
-        LOGGER.info("{} {}, memory={}, cpu={}, (new) cluster memory={}, cpu={}, currently running={}, queued={}",
+        LOGGER.log(lvl, "{} {}, memory={}, cpu={}, (new) cluster memory={}, cpu={}, currently running={}, queued={}",
                 jobStateDesc, jobRun.getJobId(), requiredMemory, requiredCPUs,
                 clusterCapacity.getAggregatedMemoryByteSize(), clusterCapacity.getAggregatedCores(),
                 getRunningJobsCount(), jobQueue.size());
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 8f91944..6278693 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
@@ -150,7 +150,7 @@
     }
 
     public void removeUncommittedPartitions(Set<PartitionId> partitionIds, Set<TaskAttemptId> taIds, JobId jobId) {
-        if (!partitionIds.isEmpty()) {
+        if (partitionIds != null && !partitionIds.isEmpty()) {
             LOGGER.debug("Removing uncommitted partitions {}: {}", jobId, partitionIds);
         }
         IEntryFilter<PartitionDescriptor> filter =
@@ -167,7 +167,7 @@
     }
 
     public void removePartitionRequests(Set<PartitionId> partitionIds, Set<TaskAttemptId> taIds, JobId jobId) {
-        if (!partitionIds.isEmpty()) {
+        if (partitionIds != null && !partitionIds.isEmpty()) {
             LOGGER.debug("Removing partition requests {}: {}", jobId, partitionIds);
         }
         IEntryFilter<PartitionRequest> filter = o -> taIds.contains(o.getRequestingTaskAttemptId());
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/ApplicationMessageWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/ApplicationMessageWork.java
index 4f77914..6630ba7 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/ApplicationMessageWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/ApplicationMessageWork.java
@@ -58,13 +58,13 @@
     }
 
     @Override
-    public String toString() {
-        return getName() + ": nodeID: " + nodeId;
+    public Level logLevel() {
+        return Level.TRACE;
     }
 
     @Override
-    public Level logLevel() {
-        return Level.TRACE;
+    public String toString() {
+        return getName() + ": nodeID: " + nodeId;
     }
 
     private static void notifyMessageBroker(ICCServiceContext ctx, IMessage msg, String nodeId) {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetNodeControllersInfoWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetNodeControllersInfoWork.java
index 73c0c7d..f08e209 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetNodeControllersInfoWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetNodeControllersInfoWork.java
@@ -27,6 +27,7 @@
 import org.apache.logging.log4j.Level;
 
 public class GetNodeControllersInfoWork extends AbstractWork {
+
     private final INodeManager nodeManager;
     private final IResultCallback<Map<String, NodeControllerInfo>> callback;
 
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetResultDirectoryAddressWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetResultDirectoryAddressWork.java
index bf95ff2..b7dbd75 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetResultDirectoryAddressWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetResultDirectoryAddressWork.java
@@ -22,8 +22,10 @@
 import org.apache.hyracks.control.cc.ClusterControllerService;
 import org.apache.hyracks.control.common.work.IResultCallback;
 import org.apache.hyracks.control.common.work.SynchronizableWork;
+import org.apache.logging.log4j.Level;
 
 public class GetResultDirectoryAddressWork extends SynchronizableWork {
+
     private final ClusterControllerService ccs;
 
     private final IResultCallback<NetworkAddress> callback;
@@ -42,4 +44,9 @@
             callback.setException(e);
         }
     }
+
+    @Override
+    public Level logLevel() {
+        return Level.TRACE;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetResultPartitionLocationsWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetResultPartitionLocationsWork.java
index d1d2269..1e34b96 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetResultPartitionLocationsWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetResultPartitionLocationsWork.java
@@ -28,6 +28,7 @@
 import org.apache.hyracks.control.cc.result.IResultDirectoryService;
 import org.apache.hyracks.control.common.work.IResultCallback;
 import org.apache.hyracks.control.common.work.SynchronizableWork;
+import org.apache.logging.log4j.Level;
 
 public class GetResultPartitionLocationsWork extends SynchronizableWork {
     private final ClusterControllerService ccs;
@@ -68,4 +69,9 @@
     public String toString() {
         return getName() + ": JobId@" + jobId + " ResultSetId@" + rsId + " Known@" + Arrays.toString(knownRecords);
     }
+
+    @Override
+    public Level logLevel() {
+        return Level.TRACE;
+    }
 }
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 6454804..6fe9909 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
@@ -28,10 +28,12 @@
 import org.apache.hyracks.control.cc.job.JobRun;
 import org.apache.hyracks.control.common.work.AbstractWork;
 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;
 
 public class JobCleanupWork extends AbstractWork {
+
     private static final Logger LOGGER = LogManager.getLogger();
 
     private final IJobManager jobManager;
@@ -51,9 +53,10 @@
 
     @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 job: {}", jobId);
+            LOGGER.debug("ignoring cleanup for unknown {}", jobId);
             return;
         }
         try {
@@ -77,4 +80,9 @@
         return getName() + ": JobId@" + jobId + " Status@" + status
                 + (exceptions == null ? "" : " Exceptions@" + exceptions);
     }
+
+    @Override
+    public Level logLevel() {
+        return Level.TRACE;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobStartWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobStartWork.java
index cfedfc9..7606dc9 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobStartWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobStartWork.java
@@ -35,6 +35,7 @@
 import org.apache.hyracks.control.common.deployment.DeploymentUtils;
 import org.apache.hyracks.control.common.work.IResultCallback;
 import org.apache.hyracks.control.common.work.SynchronizableWork;
+import org.apache.logging.log4j.Level;
 
 public class JobStartWork extends SynchronizableWork {
     private final ClusterControllerService ccs;
@@ -85,4 +86,9 @@
             callback.setException(e);
         }
     }
+
+    @Override
+    public Level logLevel() {
+        return Level.TRACE;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobletCleanupNotificationWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobletCleanupNotificationWork.java
index c3a09f9..76a72c0 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobletCleanupNotificationWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobletCleanupNotificationWork.java
@@ -35,6 +35,7 @@
 import org.apache.logging.log4j.Logger;
 
 public class JobletCleanupNotificationWork extends AbstractHeartbeatWork {
+
     private static final Logger LOGGER = LogManager.getLogger();
 
     private final JobId jobId;
@@ -46,6 +47,7 @@
 
     @Override
     public void runWork() {
+        LOGGER.debug("node {} finished job clean-up {}", nodeId, jobId);
         IJobManager jobManager = ccs.getJobManager();
         final JobRun run = jobManager.get(jobId);
         if (run == null) {
@@ -82,4 +84,9 @@
     public String toString() {
         return getName() + " jobId:" + jobId + ", nodeId:" + nodeId;
     }
+
+    @Override
+    public Level logLevel() {
+        return Level.TRACE;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterNodeWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterNodeWork.java
index ec21785..810fda2 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterNodeWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterNodeWork.java
@@ -31,6 +31,7 @@
 import org.apache.hyracks.control.common.controllers.NodeRegistration;
 import org.apache.hyracks.control.common.ipc.NodeControllerRemoteProxy;
 import org.apache.hyracks.control.common.work.SynchronizableWork;
+import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -83,4 +84,9 @@
             nc.sendRegistrationResult(params, e);
         }
     }
+
+    @Override
+    public Level logLevel() {
+        return Level.TRACE;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RemoveDeadNodesWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RemoveDeadNodesWork.java
index cbdf98a..9f740ef 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RemoveDeadNodesWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RemoveDeadNodesWork.java
@@ -50,9 +50,7 @@
             Collection<JobId> affectedJobIds = result.getRight();
             int size = affectedJobIds.size();
             if (size > 0) {
-                if (LOGGER.isInfoEnabled()) {
-                    LOGGER.info("Number of affected jobs: " + size);
-                }
+                LOGGER.info("number of affected jobs due to dead nodes removal {}", size);
                 IJobManager jobManager = ccs.getJobManager();
                 for (JobId jobId : affectedJobIds) {
                     JobRun run = jobManager.get(jobId);
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/TaskCompleteWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/TaskCompleteWork.java
index 0c53142..80dbd2e 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/TaskCompleteWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/TaskCompleteWork.java
@@ -29,8 +29,13 @@
 import org.apache.hyracks.control.common.job.profiling.om.JobProfile;
 import org.apache.hyracks.control.common.job.profiling.om.JobletProfile;
 import org.apache.hyracks.control.common.job.profiling.om.TaskProfile;
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
 
 public class TaskCompleteWork extends AbstractTaskLifecycleWork {
+
+    private static final Logger LOGGER = LogManager.getLogger();
     private final TaskProfile statistics;
 
     public TaskCompleteWork(ClusterControllerService ccs, JobId jobId, TaskAttemptId taId, String nodeId,
@@ -46,6 +51,7 @@
         if (run == null) {
             return;
         }
+        LOGGER.debug("node completed task {}:{}:{}", nodeId, jobId, taId);
         if (statistics != null) {
             JobProfile jobProfile = run.getJobProfile();
             Map<String, JobletProfile> jobletProfiles = jobProfile.getJobletProfiles();
@@ -63,4 +69,9 @@
     public String toString() {
         return getName() + ": [" + nodeId + "[" + jobId + ":" + taId + "]";
     }
+
+    @Override
+    public Level logLevel() {
+        return Level.TRACE;
+    }
 }
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 33d391f..48fd403 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,6 +22,7 @@
 
 import org.apache.hyracks.api.dataflow.TaskAttemptId;
 import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.util.ErrorMessageUtil;
 import org.apache.hyracks.control.cc.ClusterControllerService;
 import org.apache.hyracks.control.cc.job.IJobManager;
 import org.apache.hyracks.control.cc.job.JobRun;
@@ -50,6 +51,7 @@
 
     @Override
     public String toString() {
-        return getName() + ": [" + jobId + ":" + taId + ":" + nodeId + "]";
+        return getName() + ": [" + jobId + ":" + taId + ":" + nodeId + "] "
+                + ErrorMessageUtil.getCauseMessage(exceptions.get(0));
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/WaitForJobCompletionWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/WaitForJobCompletionWork.java
index ed3e574..63d5340 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/WaitForJobCompletionWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/WaitForJobCompletionWork.java
@@ -29,6 +29,7 @@
 import org.apache.hyracks.control.cc.job.JobRun;
 import org.apache.hyracks.control.common.work.IResultCallback;
 import org.apache.hyracks.control.common.work.SynchronizableWork;
+import org.apache.logging.log4j.Level;
 
 public class WaitForJobCompletionWork extends SynchronizableWork {
     private final ClusterControllerService ccs;
@@ -92,4 +93,9 @@
     public String toString() {
         return getName() + " jobId:" + jobId;
     }
+
+    @Override
+    public Level logLevel() {
+        return Level.TRACE;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/ApplicationMessageWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/ApplicationMessageWork.java
index d1360d8..f386a89 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/ApplicationMessageWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/ApplicationMessageWork.java
@@ -29,6 +29,7 @@
 import org.apache.logging.log4j.Logger;
 
 public class ApplicationMessageWork extends AbstractWork {
+
     private static final Logger LOGGER = LogManager.getLogger();
     private final byte[] message;
     private final DeploymentId deploymentId;
@@ -59,12 +60,12 @@
     }
 
     @Override
-    public String toString() {
-        return getName() + ": nodeId: " + nodeId;
+    public Level logLevel() {
+        return Level.TRACE;
     }
 
     @Override
-    public Level logLevel() {
-        return Level.TRACE;
+    public String toString() {
+        return getName() + ": nodeId: " + nodeId;
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/CleanupJobletWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/CleanupJobletWork.java
index 75edd38..2036d72 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/CleanupJobletWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/CleanupJobletWork.java
@@ -25,10 +25,12 @@
 import org.apache.hyracks.control.common.work.AbstractWork;
 import org.apache.hyracks.control.nc.Joblet;
 import org.apache.hyracks.control.nc.NodeControllerService;
+import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
 public class CleanupJobletWork extends AbstractWork {
+
     private static final Logger LOGGER = LogManager.getLogger();
 
     private final NodeControllerService ncs;
@@ -45,7 +47,7 @@
 
     @Override
     public void run() {
-        LOGGER.debug("cleaning up after job: {}", jobId);
+        LOGGER.debug("cleaning up {}, status:{}", jobId, status);
         ncs.removeJobParameterByteStore(jobId);
         ncs.getPartitionManager().jobCompleted(jobId, status);
         Map<JobId, Joblet> jobletMap = ncs.getJobletMap();
@@ -59,4 +61,9 @@
     public String toString() {
         return getName() + " jobId:" + jobId + ", status:" + status;
     }
+
+    @Override
+    public Level logLevel() {
+        return Level.TRACE;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/NotifyTaskCompleteWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/NotifyTaskCompleteWork.java
index 60860c5..52469dc 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/NotifyTaskCompleteWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/NotifyTaskCompleteWork.java
@@ -18,6 +18,8 @@
  */
 package org.apache.hyracks.control.nc.work;
 
+import org.apache.hyracks.api.dataflow.TaskAttemptId;
+import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.control.common.job.profiling.om.TaskProfile;
 import org.apache.hyracks.control.common.work.AbstractWork;
 import org.apache.hyracks.control.nc.NodeControllerService;
@@ -38,13 +40,16 @@
 
     @Override
     public void run() {
-        TaskProfile taskProfile = new TaskProfile(task.getTaskAttemptId(), task.getPartitionSendProfile(),
+        JobId jobId = task.getJoblet().getJobId();
+        TaskAttemptId taskAttemptId = task.getTaskAttemptId();
+        LOGGER.debug("notifying CC of task complete {}:{}", jobId, taskAttemptId);
+        TaskProfile taskProfile = new TaskProfile(taskAttemptId, task.getPartitionSendProfile(),
                 task.getStatsCollector(), task.getWarnings(), task.getWarningCollector().getTotalWarningsCount());
         try {
-            ncs.getClusterController(task.getJobletContext().getJobId().getCcId()).notifyTaskComplete(
-                    task.getJobletContext().getJobId(), task.getTaskAttemptId(), ncs.getId(), taskProfile);
+            ncs.getClusterController(task.getJobletContext().getJobId().getCcId())
+                    .notifyTaskComplete(task.getJobletContext().getJobId(), taskAttemptId, ncs.getId(), taskProfile);
         } catch (Exception e) {
-            LOGGER.log(Level.ERROR, "Failed notifying task complete for " + task.getTaskAttemptId(), e);
+            LOGGER.log(Level.ERROR, "Failed notifying task complete for {}", taskAttemptId, e);
         }
         task.getJoblet().removeTask(task);
     }
@@ -54,4 +59,9 @@
         return getName() + ": [" + ncs.getId() + "[" + task.getJoblet().getJobId() + ":" + task.getTaskAttemptId()
                 + "]";
     }
+
+    @Override
+    public Level logLevel() {
+        return Level.TRACE;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/StartTasksWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/StartTasksWork.java
index f277046..dd4a956 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/StartTasksWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/StartTasksWork.java
@@ -318,6 +318,6 @@
 
     @Override
     public String toString() {
-        return getName() + " jobId:" + jobId;
+        return getName() + " jobId:" + jobId + " tasks:" + taskDescriptors.size();
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/util/ComponentUtils.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/util/ComponentUtils.java
index 1ff9fa8..842ec61 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/util/ComponentUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/util/ComponentUtils.java
@@ -34,7 +34,6 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 import org.apache.hyracks.storage.common.buffercache.IPageWriteFailureCallback;
-import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -80,11 +79,7 @@
      */
     public static void get(ILSMIndex index, IValueReference key, ArrayBackedValueStorage value)
             throws HyracksDataException {
-        boolean loggable = LOGGER.isDebugEnabled();
         value.reset();
-        if (loggable) {
-            LOGGER.log(Level.DEBUG, "Getting " + key + " from index " + index);
-        }
         // Lock the opTracker to ensure index components don't change
         synchronized (index.getOperationTracker()) {
             ILSMMemoryComponent cmc = index.getCurrentMemoryComponent();
@@ -92,33 +87,17 @@
                 index.getCurrentMemoryComponent().getMetadata().get(key, value);
             }
             if (value.getLength() == 0) {
-                if (loggable) {
-                    LOGGER.log(Level.DEBUG, key + " was not found in mutable memory component of " + index);
-                }
-                // was not found in the in current mutable component, search in the other in memory components
+                // was not found in the in current mutable component, search in the other in-memory components
                 fromImmutableMemoryComponents(index, key, value);
                 if (value.getLength() == 0) {
-                    if (loggable) {
-                        LOGGER.log(Level.DEBUG, key + " was not found in all immmutable memory components of " + index);
-                    }
-                    // was not found in the in all in memory components, search in the disk components
+                    // was not found in all in-memory components, search in the disk components
                     fromDiskComponents(index, key, value);
-                    if (loggable) {
-                        if (value.getLength() == 0) {
-                            LOGGER.log(Level.DEBUG, key + " was not found in all disk components of " + index);
-                        } else {
-                            LOGGER.log(Level.DEBUG, key + " was found in disk components of " + index);
-                        }
-                    }
-                } else {
-                    if (loggable) {
-                        LOGGER.log(Level.DEBUG, key + " was found in the immutable memory components of " + index);
+                    if (value.getLength() == 0) {
+                        LOGGER.debug("{} was NOT found", key);
                     }
                 }
             } else {
-                if (loggable) {
-                    LOGGER.log(Level.DEBUG, key + " was found in mutable memory component of " + index);
-                }
+                LOGGER.debug("{} was found in mutable memory component {}", key, cmc);
             }
         }
     }
@@ -143,17 +122,11 @@
 
     private static void fromDiskComponents(ILSMIndex index, IValueReference key, ArrayBackedValueStorage value)
             throws HyracksDataException {
-        boolean loggable = LOGGER.isDebugEnabled();
-        if (loggable) {
-            LOGGER.log(Level.DEBUG, "Getting " + key + " from disk components of " + index);
-        }
         for (ILSMDiskComponent c : index.getDiskComponents()) {
-            if (loggable) {
-                LOGGER.log(Level.DEBUG, "Getting " + key + " from disk components " + c);
-            }
             c.getMetadata().get(key, value);
             if (value.getLength() != 0) {
                 // Found
+                LOGGER.debug("{} was found in disk component {}", key, c);
                 return;
             }
         }
@@ -161,21 +134,10 @@
 
     private static void fromImmutableMemoryComponents(ILSMIndex index, IValueReference key,
             ArrayBackedValueStorage value) throws HyracksDataException {
-        boolean loggable = LOGGER.isDebugEnabled();
-        if (loggable) {
-            LOGGER.log(Level.DEBUG, "Getting " + key + " from immutable memory components of " + index);
-        }
         List<ILSMMemoryComponent> memComponents = index.getMemoryComponents();
         int numOtherMemComponents = memComponents.size() - 1;
         int next = index.getCurrentMemoryComponentIndex();
-        if (loggable) {
-            LOGGER.log(Level.DEBUG, index + " has " + numOtherMemComponents + " immutable memory components");
-        }
         for (int i = 0; i < numOtherMemComponents; i++) {
-            if (loggable) {
-                LOGGER.log(Level.DEBUG,
-                        "trying to get " + key + " from immutable memory components number: " + (i + 1));
-            }
             next = next - 1;
             if (next < 0) {
                 next = memComponents.size() - 1;
@@ -185,6 +147,7 @@
                 c.getMetadata().get(key, value);
                 if (value.getLength() != 0) {
                     // Found
+                    LOGGER.debug("{} was found in immutable memory component {}", key, c);
                     return;
                 }
             }