Skip to content

Commit 422646b

Browse files
committed
Remove scalalogging-slf4j dependency
1 parent d934801 commit 422646b

File tree

30 files changed

+83
-82
lines changed

30 files changed

+83
-82
lines changed

core/src/main/scala/org/apache/spark/Logging.scala

Lines changed: 7 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -39,13 +39,17 @@ trait Logging {
3939
// be serialized and used on another machine
4040
@transient private var log_ : Logger = null
4141

42+
// Method to get the logger name for this object
43+
protected def logName = {
44+
// Ignore trailing $'s in the class names for Scala objects
45+
this.getClass.getName.stripSuffix("$")
46+
}
47+
4248
// Method to get or create the logger for this object
4349
protected def log: Logger = {
4450
if (log_ == null) {
4551
initializeIfNecessary()
46-
var className = this.getClass.getName
47-
// Ignore trailing $'s in the class names for Scala objects
48-
log_ = LoggerFactory.getLogger(className.stripSuffix("$"))
52+
log_ = LoggerFactory.getLogger(logName)
4953
}
5054
log_
5155
}

sql/catalyst/pom.xml

Lines changed: 0 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -54,11 +54,6 @@
5454
<artifactId>spark-core_${scala.binary.version}</artifactId>
5555
<version>${project.version}</version>
5656
</dependency>
57-
<dependency>
58-
<groupId>com.typesafe</groupId>
59-
<artifactId>scalalogging-slf4j_${scala.binary.version}</artifactId>
60-
<version>1.0.1</version>
61-
</dependency>
6257
<dependency>
6358
<groupId>org.scalatest</groupId>
6459
<artifactId>scalatest_${scala.binary.version}</artifactId>

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -109,12 +109,12 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool
109109
object ResolveReferences extends Rule[LogicalPlan] {
110110
def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
111111
case q: LogicalPlan if q.childrenResolved =>
112-
logger.trace(s"Attempting to resolve ${q.simpleString}")
112+
logTrace(s"Attempting to resolve ${q.simpleString}")
113113
q transformExpressions {
114114
case u @ UnresolvedAttribute(name) =>
115115
// Leave unchanged if resolution fails. Hopefully will be resolved next round.
116116
val result = q.resolve(name).getOrElse(u)
117-
logger.debug(s"Resolving $u to $result")
117+
logDebug(s"Resolving $u to $result")
118118
result
119119
}
120120
}

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -75,7 +75,7 @@ trait HiveTypeCoercion {
7575
// Leave the same if the dataTypes match.
7676
case Some(newType) if a.dataType == newType.dataType => a
7777
case Some(newType) =>
78-
logger.debug(s"Promoting $a to $newType in ${q.simpleString}}")
78+
logDebug(s"Promoting $a to $newType in ${q.simpleString}}")
7979
newType
8080
}
8181
}
@@ -154,7 +154,7 @@ trait HiveTypeCoercion {
154154
(Alias(Cast(l, StringType), l.name)(), r)
155155

156156
case (l, r) if l.dataType != r.dataType =>
157-
logger.debug(s"Resolving mismatched union input ${l.dataType}, ${r.dataType}")
157+
logDebug(s"Resolving mismatched union input ${l.dataType}, ${r.dataType}")
158158
findTightestCommonType(l.dataType, r.dataType).map { widestType =>
159159
val newLeft =
160160
if (l.dataType == widestType) l else Alias(Cast(l, widestType), l.name)()
@@ -170,15 +170,15 @@ trait HiveTypeCoercion {
170170

171171
val newLeft =
172172
if (castedLeft.map(_.dataType) != left.output.map(_.dataType)) {
173-
logger.debug(s"Widening numeric types in union $castedLeft ${left.output}")
173+
logDebug(s"Widening numeric types in union $castedLeft ${left.output}")
174174
Project(castedLeft, left)
175175
} else {
176176
left
177177
}
178178

179179
val newRight =
180180
if (castedRight.map(_.dataType) != right.output.map(_.dataType)) {
181-
logger.debug(s"Widening numeric types in union $castedRight ${right.output}")
181+
logDebug(s"Widening numeric types in union $castedRight ${right.output}")
182182
Project(castedRight, right)
183183
} else {
184184
right

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,7 @@
1717

1818
package org.apache.spark.sql.catalyst.expressions
1919

20-
import org.apache.spark.sql.catalyst.Logging
20+
import org.apache.spark.Logging
2121
import org.apache.spark.sql.catalyst.errors.attachTree
2222
import org.apache.spark.sql.catalyst.types._
2323
import org.apache.spark.sql.catalyst.trees

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,7 @@
1717

1818
package org.apache.spark.sql.catalyst.expressions.codegen
1919

20-
import com.typesafe.scalalogging.slf4j.Logging
20+
import org.apache.spark.Logging
2121
import org.apache.spark.sql.catalyst.expressions._
2222
import org.apache.spark.sql.catalyst.types.{StringType, NumericType}
2323

@@ -92,7 +92,7 @@ object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[Row]] wit
9292
}
9393
new $orderingName()
9494
"""
95-
logger.debug(s"Generated Ordering: $code")
95+
logDebug(s"Generated Ordering: $code")
9696
toolBox.eval(code).asInstanceOf[Ordering[Row]]
9797
}
9898
}

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -25,5 +25,4 @@ package object catalyst {
2525
*/
2626
protected[catalyst] object ScalaReflectionLock
2727

28-
protected[catalyst] type Logging = com.typesafe.scalalogging.slf4j.Logging
2928
}

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,7 @@
1717

1818
package org.apache.spark.sql.catalyst.planning
1919

20-
import org.apache.spark.sql.catalyst.Logging
20+
import org.apache.spark.Logging
2121
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
2222
import org.apache.spark.sql.catalyst.trees.TreeNode
2323

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.planning
2020
import scala.annotation.tailrec
2121

2222
import org.apache.spark.sql.catalyst.expressions._
23-
import org.apache.spark.sql.catalyst.Logging
23+
import org.apache.spark.Logging
2424
import org.apache.spark.sql.catalyst.plans._
2525
import org.apache.spark.sql.catalyst.plans.logical._
2626

@@ -184,7 +184,7 @@ object ExtractEquiJoinKeys extends Logging with PredicateHelper {
184184

185185
def unapply(plan: LogicalPlan): Option[ReturnType] = plan match {
186186
case join @ Join(left, right, joinType, condition) =>
187-
logger.debug(s"Considering join on: $condition")
187+
logDebug(s"Considering join on: $condition")
188188
// Find equi-join predicates that can be evaluated before the join, and thus can be used
189189
// as join keys.
190190
val (joinPredicates, otherPredicates) =
@@ -202,7 +202,7 @@ object ExtractEquiJoinKeys extends Logging with PredicateHelper {
202202
val rightKeys = joinKeys.map(_._2)
203203

204204
if (joinKeys.nonEmpty) {
205-
logger.debug(s"leftKeys:${leftKeys} | rightKeys:${rightKeys}")
205+
logDebug(s"leftKeys:${leftKeys} | rightKeys:${rightKeys}")
206206
Some((joinType, leftKeys, rightKeys, otherPredicates.reduceOption(And), left, right))
207207
} else {
208208
None

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,7 @@
1717

1818
package org.apache.spark.sql.catalyst.rules
1919

20-
import org.apache.spark.sql.catalyst.Logging
20+
import org.apache.spark.Logging
2121
import org.apache.spark.sql.catalyst.trees.TreeNode
2222

2323
abstract class Rule[TreeType <: TreeNode[_]] extends Logging {

0 commit comments

Comments
 (0)