[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;