[NO ISSUE][OTH] Fix Job + Algebricks Logging Level

- user model changes: no
- storage format changes: no
- interface changes: no

Details:
- Move job execution + Algebricks trace logs to trace
  log level.

Change-Id: I3a94fb251523ff7056fa4ba7db503c8f49bfbfbb
Reviewed-on: https://asterix-gerrit.ics.uci.edu/2853
Sonar-Qube: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Contrib: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Michael Blow <mblow@apache.org>
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
index 06f7cc2..a2c500a 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
@@ -246,8 +246,8 @@
                 IAObject o = (IAObject) serde.deserialize(dis);
                 return new Pair<>(true, new ConstantExpression(new AsterixConstantValue(o)));
             } catch (HyracksDataException | AlgebricksException e) {
-                if (AlgebricksConfig.ALGEBRICKS_LOGGER.isDebugEnabled()) {
-                    AlgebricksConfig.ALGEBRICKS_LOGGER.debug("Exception caught at constant folding: " + e, e);
+                if (AlgebricksConfig.ALGEBRICKS_LOGGER.isTraceEnabled()) {
+                    AlgebricksConfig.ALGEBRICKS_LOGGER.trace("Exception caught at constant folding: " + e, e);
                 }
                 return new Pair<>(false, null);
             }
diff --git a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java
index 2d6123e..a25e2a3 100644
--- a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java
+++ b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java
@@ -93,7 +93,7 @@
                     @Override
                     public JobSpecification createJob(Object appContext,
                             IJobletEventListenerFactory jobEventListenerFactory) throws AlgebricksException {
-                        AlgebricksConfig.ALGEBRICKS_LOGGER.debug("Starting Job Generation.\n");
+                        AlgebricksConfig.ALGEBRICKS_LOGGER.trace("Starting Job Generation.\n");
                         JobGenContext context = new JobGenContext(null, metadata, appContext,
                                 serializerDeserializerProvider, hashFunctionFactoryProvider, hashFunctionFamilyProvider,
                                 comparatorFactoryProvider, typeTraitProvider, binaryBooleanInspectorFactory,
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
index 2d5780d..d0d121f 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
@@ -313,12 +313,10 @@
                 newGbyList.add(p);
             }
         }
-        if (changed) {
-            if (AlgebricksConfig.ALGEBRICKS_LOGGER.isDebugEnabled()) {
-                AlgebricksConfig.ALGEBRICKS_LOGGER
-                        .debug(">>>> Group-by list changed from " + GroupByOperator.veListToString(gByList) + " to "
-                                + GroupByOperator.veListToString(newGbyList) + ".\n");
-            }
+        if (changed && AlgebricksConfig.ALGEBRICKS_LOGGER.isTraceEnabled()) {
+            AlgebricksConfig.ALGEBRICKS_LOGGER
+                    .trace(">>>> Group-by list changed from " + GroupByOperator.veListToString(gByList) + " to "
+                            + GroupByOperator.veListToString(newGbyList) + ".\n");
         }
         gByList.clear();
         gByList.addAll(newGbyList);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
index ae93386..6dfe254 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
@@ -80,7 +80,7 @@
             computeLogicalPropertiesRec(ref.getValue(), visitor, context);
         }
         op.accept(visitor, context);
-        if (AlgebricksConfig.DEBUG && AlgebricksConfig.ALGEBRICKS_LOGGER.isTraceEnabled()) {
+        if (AlgebricksConfig.ALGEBRICKS_LOGGER.isTraceEnabled()) {
             AlgebricksConfig.ALGEBRICKS_LOGGER.trace(
                     "Logical properties visitor for " + op + ": " + context.getLogicalPropertiesVector(op) + "\n");
         }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
index c63e8a1..8779777 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
@@ -22,7 +22,7 @@
 import org.apache.logging.log4j.Logger;
 
 public class AlgebricksConfig {
-    public static final boolean DEBUG = true;
+
     public static final String ALGEBRICKS_LOGGER_NAME = "org.apache.hyracks.algebricks";
     public static final Logger ALGEBRICKS_LOGGER = LogManager.getLogger(ALGEBRICKS_LOGGER_NAME);
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/AbstractRuleController.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/AbstractRuleController.java
index cad62c4..cf4375f 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/AbstractRuleController.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/AbstractRuleController.java
@@ -66,7 +66,7 @@
     }
 
     private String getPlanString(Mutable<ILogicalOperator> opRef) throws AlgebricksException {
-        if (AlgebricksConfig.ALGEBRICKS_LOGGER.isDebugEnabled() && context != null) {
+        if (AlgebricksConfig.ALGEBRICKS_LOGGER.isTraceEnabled() && context != null) {
             LogicalOperatorPrettyPrintVisitor pvisitor = context.getPrettyPrintVisitor();
             pvisitor.reset(new AlgebricksAppendable());
             PlanPrettyPrinter.printOperator((AbstractLogicalOperator) opRef.getValue(), pvisitor, 0);
@@ -77,10 +77,10 @@
 
     private void printRuleApplication(IAlgebraicRewriteRule rule, String beforePlan, String afterPlan)
             throws AlgebricksException {
-        if (AlgebricksConfig.ALGEBRICKS_LOGGER.isDebugEnabled()) {
-            AlgebricksConfig.ALGEBRICKS_LOGGER.debug(">>>> Rule " + rule.getClass() + " fired.\n");
-            AlgebricksConfig.ALGEBRICKS_LOGGER.debug(">>>> Before plan\n" + beforePlan + "\n");
-            AlgebricksConfig.ALGEBRICKS_LOGGER.debug(">>>> After plan\n" + afterPlan + "\n");
+        if (AlgebricksConfig.ALGEBRICKS_LOGGER.isTraceEnabled()) {
+            AlgebricksConfig.ALGEBRICKS_LOGGER.trace(">>>> Rule " + rule.getClass() + " fired.\n");
+            AlgebricksConfig.ALGEBRICKS_LOGGER.trace(">>>> Before plan\n" + beforePlan + "\n");
+            AlgebricksConfig.ALGEBRICKS_LOGGER.trace(">>>> After plan\n" + afterPlan + "\n");
         }
     }
 
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/HeuristicOptimizer.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/HeuristicOptimizer.java
index 9119d6c..8eb9b90 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/HeuristicOptimizer.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/HeuristicOptimizer.java
@@ -74,15 +74,15 @@
         if (plan == null) {
             return;
         }
-        if (AlgebricksConfig.DEBUG) {
-            AlgebricksConfig.ALGEBRICKS_LOGGER.debug("Starting logical optimizations.\n");
+        if (AlgebricksConfig.ALGEBRICKS_LOGGER.isTraceEnabled()) {
+            AlgebricksConfig.ALGEBRICKS_LOGGER.trace("Starting logical optimizations.\n");
         }
 
-        logPlanAt("Logical Plan", Level.DEBUG);
+        logPlanAt("Logical Plan", Level.TRACE);
         runOptimizationSets(plan, logicalRewrites);
         computeSchemaBottomUpForPlan(plan);
         runPhysicalOptimizations(plan, physicalRewrites);
-        logPlanAt("Optimized Plan", Level.DEBUG);
+        logPlanAt("Optimized Plan", Level.TRACE);
     }
 
     private void logPlanAt(String name, Level lvl) throws AlgebricksException {
@@ -126,8 +126,8 @@
     private void runPhysicalOptimizations(ILogicalPlan plan,
             List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> physicalRewrites)
             throws AlgebricksException {
-        if (AlgebricksConfig.DEBUG) {
-            AlgebricksConfig.ALGEBRICKS_LOGGER.debug("Starting physical optimizations.\n");
+        if (AlgebricksConfig.ALGEBRICKS_LOGGER.isTraceEnabled()) {
+            AlgebricksConfig.ALGEBRICKS_LOGGER.trace("Starting physical optimizations.\n");
         }
         // PhysicalOptimizationsUtil.computeFDsAndEquivalenceClasses(plan);
         runOptimizationSets(plan, physicalRewrites);
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
index 9ca4bdb..6f7f86a 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
@@ -124,8 +124,8 @@
         // somewhere else, too.
 
         physicalOptimizationConfig = context.getPhysicalOptimizationConfig();
-        if (AlgebricksConfig.ALGEBRICKS_LOGGER.isDebugEnabled()) {
-            AlgebricksConfig.ALGEBRICKS_LOGGER.debug(">>>> Optimizing operator " + op.getPhysicalOperator() + ".\n");
+        if (AlgebricksConfig.ALGEBRICKS_LOGGER.isTraceEnabled()) {
+            AlgebricksConfig.ALGEBRICKS_LOGGER.trace(">>>> Optimizing operator " + op.getPhysicalOperator() + ".\n");
         }
 
         PhysicalOptimizationsUtil.computeFDsAndEquivalenceClasses(op, context);
@@ -320,7 +320,7 @@
         }
 
         if (opIsRedundantSort) {
-            if (AlgebricksConfig.DEBUG && loggerTraceEnabled) {
+            if (loggerTraceEnabled) {
                 AlgebricksConfig.ALGEBRICKS_LOGGER
                         .trace(">>>> Removing redundant SORT operator " + op.getPhysicalOperator() + "\n");
                 printOp(op);
@@ -477,7 +477,7 @@
 
     private void addLocalEnforcers(AbstractLogicalOperator op, int i, List<ILocalStructuralProperty> localProperties,
             boolean nestedPlan, IOptimizationContext context) throws AlgebricksException {
-        if (AlgebricksConfig.DEBUG && AlgebricksConfig.ALGEBRICKS_LOGGER.isTraceEnabled()) {
+        if (AlgebricksConfig.ALGEBRICKS_LOGGER.isTraceEnabled()) {
             AlgebricksConfig.ALGEBRICKS_LOGGER
                     .trace(">>>> Adding local enforcers for local props = " + localProperties + "\n");
         }
@@ -549,7 +549,7 @@
         }
         oo.getInputs().add(topOp);
         context.computeAndSetTypeEnvironmentForOperator(oo);
-        if (AlgebricksConfig.DEBUG && AlgebricksConfig.ALGEBRICKS_LOGGER.isTraceEnabled()) {
+        if (AlgebricksConfig.ALGEBRICKS_LOGGER.isTraceEnabled()) {
             AlgebricksConfig.ALGEBRICKS_LOGGER.trace(">>>> Added sort enforcer " + oo.getPhysicalOperator() + ".\n");
         }
         return new MutableObject<ILogicalOperator>(oo);
@@ -632,9 +632,9 @@
             exchg.setExecutionMode(AbstractLogicalOperator.ExecutionMode.PARTITIONED);
             OperatorPropertiesUtil.computeSchemaAndPropertiesRecIfNull(exchg, context);
             context.computeAndSetTypeEnvironmentForOperator(exchg);
-            if (AlgebricksConfig.DEBUG && AlgebricksConfig.ALGEBRICKS_LOGGER.isDebugEnabled()) {
+            if (AlgebricksConfig.ALGEBRICKS_LOGGER.isTraceEnabled()) {
                 AlgebricksConfig.ALGEBRICKS_LOGGER
-                        .debug(">>>> Added partitioning enforcer " + exchg.getPhysicalOperator() + ".\n");
+                        .trace(">>>> Added partitioning enforcer " + exchg.getPhysicalOperator() + ".\n");
                 printOp((AbstractLogicalOperator) op);
             }
         }
@@ -652,8 +652,8 @@
     private void printOp(AbstractLogicalOperator op) throws AlgebricksException {
         LogicalOperatorPrettyPrintVisitor pvisitor = new LogicalOperatorPrettyPrintVisitor();
         PlanPrettyPrinter.printOperator(op, pvisitor, 0);
-        if (AlgebricksConfig.ALGEBRICKS_LOGGER.isDebugEnabled()) {
-            AlgebricksConfig.ALGEBRICKS_LOGGER.debug(pvisitor.get().toString());
+        if (AlgebricksConfig.ALGEBRICKS_LOGGER.isTraceEnabled()) {
+            AlgebricksConfig.ALGEBRICKS_LOGGER.trace(pvisitor.get().toString());
         }
     }
 
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceGroupByForSubplanRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceGroupByForSubplanRule.java
index a988075..c1e613b 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceGroupByForSubplanRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceGroupByForSubplanRule.java
@@ -202,8 +202,8 @@
             //retain the intersection
             pkVars.retainAll(producedVars);
         }
-        if (AlgebricksConfig.ALGEBRICKS_LOGGER.isDebugEnabled()) {
-            AlgebricksConfig.ALGEBRICKS_LOGGER.debug("Found FD for introducing group-by: " + pkVars);
+        if (AlgebricksConfig.ALGEBRICKS_LOGGER.isTraceEnabled()) {
+            AlgebricksConfig.ALGEBRICKS_LOGGER.trace("Found FD for introducing group-by: " + pkVars);
         }
 
         Mutable<ILogicalOperator> rightRef = join.getInputs().get(1);
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/util/JoinUtils.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/util/JoinUtils.java
index 98e48fd..6306338 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/util/JoinUtils.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/util/JoinUtils.java
@@ -114,18 +114,18 @@
         ILogicalOperator opBuild = op.getInputs().get(1).getValue();
         LogicalPropertiesVisitor.computeLogicalPropertiesDFS(opBuild, context);
         ILogicalPropertiesVector v = context.getLogicalPropertiesVector(opBuild);
-        boolean loggerDebugEnabled = AlgebricksConfig.ALGEBRICKS_LOGGER.isDebugEnabled();
-        if (loggerDebugEnabled) {
+        boolean loggerTraceEnabled = AlgebricksConfig.ALGEBRICKS_LOGGER.isTraceEnabled();
+        if (loggerTraceEnabled) {
             AlgebricksConfig.ALGEBRICKS_LOGGER
-                    .debug("// HybridHashJoin inner branch -- Logical properties for " + opBuild + ": " + v + "\n");
+                    .trace("// HybridHashJoin inner branch -- Logical properties for " + opBuild + ": " + v + "\n");
         }
         if (v != null) {
             int size2 = v.getMaxOutputFrames();
             HybridHashJoinPOperator hhj = (HybridHashJoinPOperator) op.getPhysicalOperator();
             if (size2 > 0 && size2 * hhj.getFudgeFactor() <= hhj.getMemSizeInFrames()) {
-                if (loggerDebugEnabled) {
+                if (loggerTraceEnabled) {
                     AlgebricksConfig.ALGEBRICKS_LOGGER
-                            .debug("// HybridHashJoin inner branch " + opBuild + " fits in memory\n");
+                            .trace("// HybridHashJoin inner branch " + opBuild + " fits in memory\n");
                 }
                 // maintains the local properties on the probe side
                 op.setPhysicalOperator(
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 06af2d8..dee36e9 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
@@ -197,8 +197,8 @@
         Set<TaskCluster> taskClusterRoots = new HashSet<>();
         findRunnableTaskClusterRoots(taskClusterRoots,
                 jobRun.getActivityClusterGraph().getActivityClusterMap().values());
-        if (LOGGER.isInfoEnabled()) {
-            LOGGER.log(Level.INFO,
+        if (LOGGER.isTraceEnabled()) {
+            LOGGER.log(Level.TRACE,
                     "Runnable TC roots: " + taskClusterRoots + ", inProgressTaskClusters: " + inProgressTaskClusters);
         }
         if (taskClusterRoots.isEmpty() && inProgressTaskClusters.isEmpty()) {
@@ -227,19 +227,19 @@
                 queue.add(new RankedRunnableTaskCluster(priority, tc));
             }
         }
-        if (LOGGER.isDebugEnabled()) {
-            LOGGER.debug("Ranked TCs: " + queue);
+        if (LOGGER.isTraceEnabled()) {
+            LOGGER.trace("Ranked TCs: " + queue);
         }
 
         Map<String, List<TaskAttemptDescriptor>> taskAttemptMap = new HashMap<>();
         for (RankedRunnableTaskCluster rrtc : queue) {
             TaskCluster tc = rrtc.getTaskCluster();
-            if (LOGGER.isDebugEnabled()) {
-                LOGGER.debug("Found runnable TC: " + tc);
+            if (LOGGER.isTraceEnabled()) {
+                LOGGER.trace("Found runnable TC: " + tc);
                 List<TaskClusterAttempt> attempts = tc.getAttempts();
-                LOGGER.debug("Attempts so far:" + attempts.size());
+                LOGGER.trace("Attempts so far:" + attempts.size());
                 for (TaskClusterAttempt tcAttempt : attempts) {
-                    LOGGER.debug("Status: " + tcAttempt.getStatus());
+                    LOGGER.trace("Status: " + tcAttempt.getStatus());
                 }
             }
             assignTaskLocations(tc, taskAttemptMap);
@@ -259,16 +259,16 @@
      * Runnability(Non-schedulable TaskCluster) = {NOT_RUNNABLE, _}
      */
     private Runnability assignRunnabilityRank(TaskCluster goal, Map<TaskCluster, Runnability> runnabilityMap) {
-        if (LOGGER.isDebugEnabled()) {
-            LOGGER.debug("Computing runnability: " + goal);
+        if (LOGGER.isTraceEnabled()) {
+            LOGGER.trace("Computing runnability: " + goal);
         }
         if (runnabilityMap.containsKey(goal)) {
             return runnabilityMap.get(goal);
         }
         TaskClusterAttempt lastAttempt = findLastTaskClusterAttempt(goal);
         if (lastAttempt != null) {
-            if (LOGGER.isDebugEnabled()) {
-                LOGGER.debug("Last Attempt Status: " + lastAttempt.getStatus());
+            if (LOGGER.isTraceEnabled()) {
+                LOGGER.trace("Last Attempt Status: " + lastAttempt.getStatus());
             }
             if (lastAttempt.getStatus() == TaskClusterAttempt.TaskClusterStatus.COMPLETED) {
                 Runnability runnability = new Runnability(Runnability.Tag.COMPLETED, Integer.MIN_VALUE);
@@ -285,15 +285,15 @@
         PartitionMatchMaker pmm = jobRun.getPartitionMatchMaker();
         Runnability aggregateRunnability = new Runnability(Runnability.Tag.RUNNABLE, 0);
         for (PartitionId pid : goal.getRequiredPartitions()) {
-            if (LOGGER.isDebugEnabled()) {
-                LOGGER.debug("Inspecting required partition: " + pid);
+            if (LOGGER.isTraceEnabled()) {
+                LOGGER.trace("Inspecting required partition: " + pid);
             }
             Runnability runnability;
             ConnectorDescriptorId cdId = pid.getConnectorDescriptorId();
             IConnectorPolicy cPolicy = connectorPolicyMap.get(cdId);
             PartitionState maxState = pmm.getMaximumAvailableState(pid);
-            if (LOGGER.isDebugEnabled()) {
-                LOGGER.debug("Policy: " + cPolicy + " maxState: " + maxState);
+            if (LOGGER.isTraceEnabled()) {
+                LOGGER.trace("Policy: " + cPolicy + " maxState: " + maxState);
             }
             if (PartitionState.COMMITTED.equals(maxState)) {
                 runnability = new Runnability(Runnability.Tag.RUNNABLE, 0);
@@ -329,8 +329,8 @@
                 // already not runnable -- cannot get better. bail.
                 break;
             }
-            if (LOGGER.isDebugEnabled()) {
-                LOGGER.debug("aggregateRunnability: " + aggregateRunnability);
+            if (LOGGER.isTraceEnabled()) {
+                LOGGER.trace("aggregateRunnability: " + aggregateRunnability);
             }
         }
         runnabilityMap.put(goal, aggregateRunnability);
@@ -512,8 +512,8 @@
                 if (node != null) {
                     node.getActiveJobIds().add(jobRun.getJobId());
                     boolean changed = jobRun.getParticipatingNodeIds().add(nodeId);
-                    if (LOGGER.isDebugEnabled()) {
-                        LOGGER.debug("Starting: " + taskDescriptors + " at " + entry.getKey());
+                    if (LOGGER.isTraceEnabled()) {
+                        LOGGER.trace("Starting: " + taskDescriptors + " at " + entry.getKey());
                     }
                     byte[] jagBytes = changed ? acgBytes : null;
                     node.getNodeController().startTasks(deploymentId, jobId, jagBytes, taskDescriptors,
@@ -539,14 +539,14 @@
 
     private void abortTaskCluster(TaskClusterAttempt tcAttempt,
             TaskClusterAttempt.TaskClusterStatus failedOrAbortedStatus) {
-        LOGGER.debug("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()) {
             TaskAttemptId taId = ta.getTaskAttemptId();
             TaskAttempt.TaskStatus status = ta.getStatus();
             abortTaskIds.add(taId);
-            LOGGER.debug("Checking " + taId + ": " + ta.getStatus());
+            LOGGER.trace(() -> "Checking " + taId + ": " + ta.getStatus());
             if (status == TaskAttempt.TaskStatus.RUNNING || status == TaskAttempt.TaskStatus.COMPLETED) {
                 ta.setStatus(TaskAttempt.TaskStatus.ABORTED, null);
                 ta.setEndTime(System.currentTimeMillis());
@@ -561,13 +561,13 @@
             }
         }
         final JobId jobId = jobRun.getJobId();
-        LOGGER.info("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.isInfoEnabled()) {
-                    LOGGER.info("Aborting: " + abortTaskAttempts + " at " + key);
+                if (LOGGER.isTraceEnabled()) {
+                    LOGGER.trace("Aborting: " + abortTaskAttempts + " at " + key);
                 }
                 try {
                     node.getNodeController().abortTasks(jobId, abortTaskAttempts);
@@ -587,7 +587,7 @@
     }
 
     private void abortDoomedTaskClusters() throws HyracksException {
-        LOGGER.log(Level.INFO, "aborting doomed task clusters");
+        LOGGER.trace("aborting doomed task clusters");
         Set<TaskCluster> doomedTaskClusters = new HashSet<>();
         for (TaskCluster tc : inProgressTaskClusters) {
             // Start search at TCs that produce no outputs (sinks)
@@ -596,7 +596,7 @@
             }
         }
 
-        LOGGER.log(Level.INFO, "number of doomed task clusters found = " + doomedTaskClusters.size());
+        LOGGER.trace(() -> "number of doomed task clusters found = " + doomedTaskClusters.size());
         for (TaskCluster tc : doomedTaskClusters) {
             TaskClusterAttempt tca = findLastTaskClusterAttempt(tc);
             if (tca != null) {
@@ -683,28 +683,28 @@
      */
     public void notifyTaskFailure(TaskAttempt ta, List<Exception> exceptions) {
         try {
-            LOGGER.log(Level.INFO, "Received failure notification for TaskAttempt " + ta.getTaskAttemptId());
+            LOGGER.debug("Received failure notification for TaskAttempt " + ta.getTaskAttemptId());
             TaskAttemptId taId = ta.getTaskAttemptId();
             TaskCluster tc = ta.getTask().getTaskCluster();
             TaskClusterAttempt lastAttempt = findLastTaskClusterAttempt(tc);
             if (lastAttempt != null && taId.getAttempt() == lastAttempt.getAttempt()) {
-                LOGGER.log(Level.INFO, "Marking TaskAttempt " + ta.getTaskAttemptId() + " as failed");
+                LOGGER.trace(() -> "Marking TaskAttempt " + ta.getTaskAttemptId() + " as failed");
                 ta.setStatus(TaskAttempt.TaskStatus.FAILED, exceptions);
                 abortTaskCluster(lastAttempt, TaskClusterAttempt.TaskClusterStatus.FAILED);
                 abortDoomedTaskClusters();
                 int maxReattempts = jobRun.getActivityClusterGraph().getMaxReattempts();
-                LOGGER.log(Level.INFO, "Marking TaskAttempt " + ta.getTaskAttemptId()
+                LOGGER.trace(() -> "Marking TaskAttempt " + ta.getTaskAttemptId()
                         + " as failed and the number of max re-attempts = " + maxReattempts);
                 if (lastAttempt.getAttempt() >= maxReattempts || isCancelled()) {
-                    LOGGER.log(Level.INFO, "Aborting the job of " + ta.getTaskAttemptId());
+                    LOGGER.debug(() -> "Aborting the job of " + ta.getTaskAttemptId());
                     abortJob(exceptions, NoOpCallback.INSTANCE);
                     return;
                 }
-                LOGGER.log(Level.INFO, "We will try to start runnable activity clusters of " + ta.getTaskAttemptId());
+                LOGGER.debug(() -> "We will try to start runnable activity clusters of " + ta.getTaskAttemptId());
                 startRunnableActivityClusters();
             } else {
-                LOGGER.warn(
-                        "Ignoring task failure notification: " + taId + " -- Current last attempt = " + lastAttempt);
+                LOGGER.warn(() -> "Ignoring task failure notification: " + taId + " -- Current last attempt = "
+                        + lastAttempt);
             }
         } catch (Exception e) {
             abortJob(Collections.singletonList(e), NoOpCallback.INSTANCE);
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/work/AbstractWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/work/AbstractWork.java
index b7f3332..9ec3214 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/work/AbstractWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/work/AbstractWork.java
@@ -23,7 +23,7 @@
 public abstract class AbstractWork implements Runnable {
 
     public Level logLevel() {
-        return Level.INFO;
+        return Level.DEBUG;
     }
 
     public String getName() {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/work/WorkQueue.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/work/WorkQueue.java
index fb812e1..2e20aa1 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/work/WorkQueue.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/work/WorkQueue.java
@@ -83,8 +83,8 @@
         if (DEBUG) {
             LOGGER.log(Level.DEBUG, "Enqueue (" + hashCode() + "): " + enqueueCount.incrementAndGet());
         }
-        if (LOGGER.isDebugEnabled()) {
-            LOGGER.debug("Scheduling: " + event);
+        if (LOGGER.isTraceEnabled()) {
+            LOGGER.trace("Scheduling: " + event);
         }
         queue.offer(event);
     }
@@ -128,19 +128,21 @@
                 } catch (Exception e) {
                     LOGGER.log(Level.WARN, "Exception while executing " + r, e);
                 } finally {
-                    auditWaitsAndBlocks(r, before);
+                    if (LOGGER.isTraceEnabled()) {
+                        traceWaitsAndBlocks(r, before);
+                    }
                 }
             }
         }
 
-        protected void auditWaitsAndBlocks(AbstractWork r, ThreadInfo before) {
+        protected void traceWaitsAndBlocks(AbstractWork r, ThreadInfo before) {
             ThreadInfo after = threadMXBean.getThreadInfo(thread.getId());
             final long waitedDelta = after.getWaitedCount() - before.getWaitedCount();
             final long blockedDelta = after.getBlockedCount() - before.getBlockedCount();
             if (waitedDelta > 0 || blockedDelta > 0) {
-                LOGGER.warn("Work " + r + " waited " + waitedDelta + " times (~"
-                        + (after.getWaitedTime() - before.getWaitedTime()) + "ms), blocked " + blockedDelta
-                        + " times (~" + (after.getBlockedTime() - before.getBlockedTime()) + "ms)");
+                LOGGER.trace("Work {} waited {} times (~{}ms), blocked {} times (~{}ms)", r, waitedDelta,
+                        after.getWaitedTime() - before.getWaitedTime(), blockedDelta,
+                        after.getBlockedTime() - before.getBlockedTime());
             }
         }
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Joblet.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Joblet.java
index 55bc192..1d2b77a 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Joblet.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Joblet.java
@@ -230,7 +230,7 @@
     public void close() {
         long stillAllocated = memoryAllocation.get();
         if (stillAllocated > 0) {
-            LOGGER.info(() -> "Freeing leaked " + stillAllocated + " bytes");
+            LOGGER.trace(() -> "Freeing leaked " + stillAllocated + " bytes");
             serviceCtx.getMemoryManager().deallocate(stillAllocated);
         }
         nodeController.getExecutor().execute(() -> deallocatableRegistry.close());
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 660621e..09a4c18 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
@@ -147,7 +147,7 @@
                 ActivityId aid = tid.getActivityId();
                 ActivityCluster ac = acg.getActivityMap().get(aid);
                 IActivity han = ac.getActivityMap().get(aid);
-                LOGGER.info("Initializing {} -> {} for {}", taId, han, jobId);
+                LOGGER.trace("Initializing {} -> {} for {}", taId, han, jobId);
                 final int partition = tid.getPartition();
                 List<IConnectorDescriptor> inputs = ac.getActivityInputMap().get(aid);
                 task = null;
@@ -159,7 +159,7 @@
                     for (int i = 0; i < inputs.size(); ++i) {
                         IConnectorDescriptor conn = inputs.get(i);
                         IConnectorPolicy cPolicy = connectorPoliciesMap.get(conn.getConnectorId());
-                        LOGGER.info("input: {}: {}", i, conn.getConnectorId());
+                        LOGGER.trace("input: {}: {}", i, conn.getConnectorId());
                         RecordDescriptor recordDesc = ac.getConnectorRecordDescriptorMap().get(conn.getConnectorId());
                         IPartitionCollector collector =
                                 createPartitionCollector(td, partition, task, i, conn, recordDesc, cPolicy);
@@ -176,7 +176,7 @@
 
                         IPartitionWriterFactory pwFactory =
                                 createPartitionWriterFactory(task, cPolicy, jobId, conn, partition, taId, flags);
-                        LOGGER.info("input: {}: {}", i, conn.getConnectorId());
+                        LOGGER.trace("input: {}: {}", i, conn.getConnectorId());
                         IFrameWriter writer = conn.createPartitioner(task, recordDesc, pwFactory, partition,
                                 td.getPartitionCount(), td.getOutputPartitionCounts()[i]);
                         writer = enforce ? EnforceFrameWriter.enforce(writer) : writer;