aboutsummaryrefslogtreecommitdiff
path: root/sql/catalyst
diff options
context:
space:
mode:
authorGuoQiang Li <witgo@qq.com>2014-08-01 23:55:11 -0700
committerPatrick Wendell <pwendell@gmail.com>2014-08-01 23:55:11 -0700
commitadc8303294e26efb4ed15e5f5ba1062f7988625d (patch)
tree09d3231b718713f384e51711b31d737c5edcf6c7 /sql/catalyst
parent4bc3bb29a4b6ab24b6b7e1f8df26414c41c80ace (diff)
downloadspark-adc8303294e26efb4ed15e5f5ba1062f7988625d.tar.gz
spark-adc8303294e26efb4ed15e5f5ba1062f7988625d.tar.bz2
spark-adc8303294e26efb4ed15e5f5ba1062f7988625d.zip
[SPARK-1470][SPARK-1842] Use the scala-logging wrapper instead of the directly sfl4j api
Author: GuoQiang Li <witgo@qq.com> Closes #1369 from witgo/SPARK-1470_new and squashes the following commits: 66a1641 [GuoQiang Li] IncompatibleResultTypeProblem 73a89ba [GuoQiang Li] Use the scala-logging wrapper instead of the directly sfl4j api.
Diffstat (limited to 'sql/catalyst')
-rw-r--r--sql/catalyst/pom.xml5
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala4
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala8
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala2
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala4
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala1
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala2
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala6
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala2
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala12
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala8
11 files changed, 25 insertions, 29 deletions
diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml
index 54fa96baa1..58d44e7923 100644
--- a/sql/catalyst/pom.xml
+++ b/sql/catalyst/pom.xml
@@ -55,11 +55,6 @@
<version>${project.version}</version>
</dependency>
<dependency>
- <groupId>com.typesafe</groupId>
- <artifactId>scalalogging-slf4j_${scala.binary.version}</artifactId>
- <version>1.0.1</version>
- </dependency>
- <dependency>
<groupId>org.scalatest</groupId>
<artifactId>scalatest_${scala.binary.version}</artifactId>
<scope>test</scope>
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index 74c0104e5b..2b36582215 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -109,12 +109,12 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool
object ResolveReferences extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
case q: LogicalPlan if q.childrenResolved =>
- logger.trace(s"Attempting to resolve ${q.simpleString}")
+ log.trace(s"Attempting to resolve ${q.simpleString}")
q transformExpressions {
case u @ UnresolvedAttribute(name) =>
// Leave unchanged if resolution fails. Hopefully will be resolved next round.
val result = q.resolve(name).getOrElse(u)
- logger.debug(s"Resolving $u to $result")
+ log.debug(s"Resolving $u to $result")
result
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala
index 47c7ad076a..eafbb70dc3 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala
@@ -75,7 +75,7 @@ trait HiveTypeCoercion {
// Leave the same if the dataTypes match.
case Some(newType) if a.dataType == newType.dataType => a
case Some(newType) =>
- logger.debug(s"Promoting $a to $newType in ${q.simpleString}}")
+ log.debug(s"Promoting $a to $newType in ${q.simpleString}}")
newType
}
}
@@ -154,7 +154,7 @@ trait HiveTypeCoercion {
(Alias(Cast(l, StringType), l.name)(), r)
case (l, r) if l.dataType != r.dataType =>
- logger.debug(s"Resolving mismatched union input ${l.dataType}, ${r.dataType}")
+ log.debug(s"Resolving mismatched union input ${l.dataType}, ${r.dataType}")
findTightestCommonType(l.dataType, r.dataType).map { widestType =>
val newLeft =
if (l.dataType == widestType) l else Alias(Cast(l, widestType), l.name)()
@@ -170,7 +170,7 @@ trait HiveTypeCoercion {
val newLeft =
if (castedLeft.map(_.dataType) != left.output.map(_.dataType)) {
- logger.debug(s"Widening numeric types in union $castedLeft ${left.output}")
+ log.debug(s"Widening numeric types in union $castedLeft ${left.output}")
Project(castedLeft, left)
} else {
left
@@ -178,7 +178,7 @@ trait HiveTypeCoercion {
val newRight =
if (castedRight.map(_.dataType) != right.output.map(_.dataType)) {
- logger.debug(s"Widening numeric types in union $castedRight ${right.output}")
+ log.debug(s"Widening numeric types in union $castedRight ${right.output}")
Project(castedRight, right)
} else {
right
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala
index f38f99569f..0913f15888 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala
@@ -17,7 +17,7 @@
package org.apache.spark.sql.catalyst.expressions
-import org.apache.spark.sql.catalyst.Logging
+import org.apache.spark.Logging
import org.apache.spark.sql.catalyst.errors.attachTree
import org.apache.spark.sql.catalyst.types._
import org.apache.spark.sql.catalyst.trees
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala
index 4211998f75..e2552d432c 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala
@@ -17,7 +17,7 @@
package org.apache.spark.sql.catalyst.expressions.codegen
-import com.typesafe.scalalogging.slf4j.Logging
+import org.apache.spark.Logging
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.types.{StringType, NumericType}
@@ -92,7 +92,7 @@ object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[Row]] wit
}
new $orderingName()
"""
- logger.debug(s"Generated Ordering: $code")
+ log.debug(s"Generated Ordering: $code")
toolBox.eval(code).asInstanceOf[Ordering[Row]]
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala
index ca9642954e..bdd07bbeb2 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala
@@ -25,5 +25,4 @@ package object catalyst {
*/
protected[catalyst] object ScalaReflectionLock
- protected[catalyst] type Logging = com.typesafe.scalalogging.slf4j.Logging
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala
index 781ba489b4..5839c9f7c4 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala
@@ -17,7 +17,7 @@
package org.apache.spark.sql.catalyst.planning
-import org.apache.spark.sql.catalyst.Logging
+import org.apache.spark.Logging
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.catalyst.trees.TreeNode
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala
index bc763a4e06..06c5ffe92a 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala
@@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.planning
import scala.annotation.tailrec
import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.Logging
+import org.apache.spark.Logging
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.logical._
@@ -184,7 +184,7 @@ object ExtractEquiJoinKeys extends Logging with PredicateHelper {
def unapply(plan: LogicalPlan): Option[ReturnType] = plan match {
case join @ Join(left, right, joinType, condition) =>
- logger.debug(s"Considering join on: $condition")
+ log.debug(s"Considering join on: $condition")
// Find equi-join predicates that can be evaluated before the join, and thus can be used
// as join keys.
val (joinPredicates, otherPredicates) =
@@ -202,7 +202,7 @@ object ExtractEquiJoinKeys extends Logging with PredicateHelper {
val rightKeys = joinKeys.map(_._2)
if (joinKeys.nonEmpty) {
- logger.debug(s"leftKeys:${leftKeys} | rightKeys:${rightKeys}")
+ log.debug(s"leftKeys:${leftKeys} | rightKeys:${rightKeys}")
Some((joinType, leftKeys, rightKeys, otherPredicates.reduceOption(And), left, right))
} else {
None
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala
index f8960b3fe7..03414b2301 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala
@@ -17,7 +17,7 @@
package org.apache.spark.sql.catalyst.rules
-import org.apache.spark.sql.catalyst.Logging
+import org.apache.spark.Logging
import org.apache.spark.sql.catalyst.trees.TreeNode
abstract class Rule[TreeType <: TreeNode[_]] extends Logging {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala
index 6aa407c836..20bf8eed7d 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala
@@ -17,7 +17,7 @@
package org.apache.spark.sql.catalyst.rules
-import org.apache.spark.sql.catalyst.Logging
+import org.apache.spark.Logging
import org.apache.spark.sql.catalyst.trees.TreeNode
import org.apache.spark.sql.catalyst.util.sideBySide
@@ -60,7 +60,7 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging {
case (plan, rule) =>
val result = rule(plan)
if (!result.fastEquals(plan)) {
- logger.trace(
+ log.trace(
s"""
|=== Applying Rule ${rule.ruleName} ===
|${sideBySide(plan.treeString, result.treeString).mkString("\n")}
@@ -73,26 +73,26 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging {
if (iteration > batch.strategy.maxIterations) {
// Only log if this is a rule that is supposed to run more than once.
if (iteration != 2) {
- logger.info(s"Max iterations (${iteration - 1}) reached for batch ${batch.name}")
+ log.info(s"Max iterations (${iteration - 1}) reached for batch ${batch.name}")
}
continue = false
}
if (curPlan.fastEquals(lastPlan)) {
- logger.trace(s"Fixed point reached for batch ${batch.name} after $iteration iterations.")
+ log.trace(s"Fixed point reached for batch ${batch.name} after $iteration iterations.")
continue = false
}
lastPlan = curPlan
}
if (!batchStartPlan.fastEquals(curPlan)) {
- logger.debug(
+ log.debug(
s"""
|=== Result of Batch ${batch.name} ===
|${sideBySide(plan.treeString, curPlan.treeString).mkString("\n")}
""".stripMargin)
} else {
- logger.trace(s"Batch ${batch.name} has no effect.")
+ log.trace(s"Batch ${batch.name} has no effect.")
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala
index 9a28d035a1..d725a92c06 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala
@@ -17,6 +17,8 @@
package org.apache.spark.sql.catalyst
+import org.apache.spark.Logging
+
/**
* A library for easily manipulating trees of operators. Operators that extend TreeNode are
* granted the following interface:
@@ -31,8 +33,8 @@ package org.apache.spark.sql.catalyst
* <li>debugging support - pretty printing, easy splicing of trees, etc.</li>
* </ul>
*/
-package object trees {
+package object trees extends Logging {
// Since we want tree nodes to be lightweight, we create one logger for all treenode instances.
- protected val logger =
- com.typesafe.scalalogging.slf4j.Logger(org.slf4j.LoggerFactory.getLogger("catalyst.trees"))
+ protected override def logName = "catalyst.trees"
+
}