aboutsummaryrefslogtreecommitdiff
path: root/sql
diff options
context:
space:
mode:
authorJosh Rosen <joshrosen@databricks.com>2015-07-14 16:08:17 -0700
committerReynold Xin <rxin@databricks.com>2015-07-14 16:08:17 -0700
commit11e5c372862ec00e57460b37ccfee51c6d93c5f7 (patch)
treee12b1ab4a40a6b3f090669490c1563970bb7a4aa /sql
parent740b034f1ca885a386f5a9ef7e0c81c714b047ff (diff)
downloadspark-11e5c372862ec00e57460b37ccfee51c6d93c5f7.tar.gz
spark-11e5c372862ec00e57460b37ccfee51c6d93c5f7.tar.bz2
spark-11e5c372862ec00e57460b37ccfee51c6d93c5f7.zip
[SPARK-8962] Add Scalastyle rule to ban direct use of Class.forName; fix existing uses
This pull request adds a Scalastyle regex rule which fails the style check if `Class.forName` is used directly. `Class.forName` always loads classes from the default / system classloader, but in a majority of cases, we should be using Spark's own `Utils.classForName` instead, which tries to load classes from the current thread's context classloader and falls back to the classloader which loaded Spark when the context classloader is not defined. <!-- Reviewable:start --> [<img src="https://reviewable.io/review_button.png" height=40 alt="Review on Reviewable"/>](https://reviewable.io/reviews/apache/spark/7350) <!-- Reviewable:end --> Author: Josh Rosen <joshrosen@databricks.com> Closes #7350 from JoshRosen/ban-Class.forName and squashes the following commits: e3e96f7 [Josh Rosen] Merge remote-tracking branch 'origin/master' into ban-Class.forName c0b7885 [Josh Rosen] Hopefully fix the last two cases d707ba7 [Josh Rosen] Fix uses of Class.forName that I missed in my first cleanup pass 046470d [Josh Rosen] Merge remote-tracking branch 'origin/master' into ban-Class.forName 62882ee [Josh Rosen] Fix uses of Class.forName or add exclusion. d9abade [Josh Rosen] Add stylechecker rule to ban uses of Class.forName
Diffstat (limited to 'sql')
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala3
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala3
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala7
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala3
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala3
-rw-r--r--sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala2
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala4
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala9
-rw-r--r--sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala8
9 files changed, 21 insertions, 21 deletions
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala
index 57718228e4..da83a7f0ba 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala
@@ -27,6 +27,7 @@ import org.json4s.jackson.JsonMethods._
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.util.Utils
/**
@@ -146,7 +147,7 @@ object DataType {
("pyClass", _),
("sqlType", _),
("type", JString("udt"))) =>
- Class.forName(udtClass).newInstance().asInstanceOf[UserDefinedType[_]]
+ Utils.classForName(udtClass).newInstance().asInstanceOf[UserDefinedType[_]]
}
private def parseStructField(json: JValue): StructField = json match {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
index 477dea9164..46bd60daa1 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
@@ -554,8 +554,7 @@ class SQLContext(@transient val sparkContext: SparkContext)
val className = beanClass.getName
val rowRdd = rdd.mapPartitions { iter =>
// BeanInfo is not serializable so we must rediscover it remotely for each partition.
- val localBeanInfo = Introspector.getBeanInfo(
- Class.forName(className, true, Utils.getContextOrSparkClassLoader))
+ val localBeanInfo = Introspector.getBeanInfo(Utils.classForName(className))
val extractors =
localBeanInfo.getPropertyDescriptors.filterNot(_.getName == "class").map(_.getReadMethod)
val methodsToConverts = extractors.zip(attributeSeq).map { case (e, attr) =>
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala
index 704cf56f38..e0bea65a15 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala
@@ -33,6 +33,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap}
import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.{DataFrame, SQLContext}
+import org.apache.spark.util.Utils
/**
* Relation that consists of data stored in a Parquet columnar format.
@@ -108,7 +109,7 @@ private[sql] object ParquetRelation {
//
// Therefore we need to force the class to be loaded.
// This should really be resolved by Parquet.
- Class.forName(classOf[ParquetLog].getName)
+ Utils.classForName(classOf[ParquetLog].getName)
// Note: Logger.getLogger("parquet") has a default logger
// that appends to Console which needs to be cleared.
@@ -119,12 +120,12 @@ private[sql] object ParquetRelation {
// Disables a WARN log message in ParquetOutputCommitter. We first ensure that
// ParquetOutputCommitter is loaded and the static LOG field gets initialized.
// See https://issues.apache.org/jira/browse/SPARK-5968 for details
- Class.forName(classOf[ParquetOutputCommitter].getName)
+ Utils.classForName(classOf[ParquetOutputCommitter].getName)
JLogger.getLogger(classOf[ParquetOutputCommitter].getName).setLevel(Level.OFF)
// Similar as above, disables a unnecessary WARN log message in ParquetRecordReader.
// See https://issues.apache.org/jira/browse/PARQUET-220 for details
- Class.forName(classOf[ParquetRecordReader[_]].getName)
+ Utils.classForName(classOf[ParquetRecordReader[_]].getName)
JLogger.getLogger(classOf[ParquetRecordReader[_]].getName).setLevel(Level.OFF)
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala
index 566a52dc1b..0f82f13088 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala
@@ -26,6 +26,7 @@ import org.scalatest.BeforeAndAfter
import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
class JDBCSuite extends SparkFunSuite with BeforeAndAfter {
val url = "jdbc:h2:mem:testdb0"
@@ -46,7 +47,7 @@ class JDBCSuite extends SparkFunSuite with BeforeAndAfter {
import ctx.sql
before {
- Class.forName("org.h2.Driver")
+ Utils.classForName("org.h2.Driver")
// Extra properties that will be specified for our database. We need these to test
// usage of parameters from OPTIONS clause in queries.
val properties = new Properties()
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala
index d949ef4226..84b52ca2c7 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala
@@ -25,6 +25,7 @@ import org.scalatest.BeforeAndAfter
import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.{SaveMode, Row}
import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
class JDBCWriteSuite extends SparkFunSuite with BeforeAndAfter {
val url = "jdbc:h2:mem:testdb2"
@@ -41,7 +42,7 @@ class JDBCWriteSuite extends SparkFunSuite with BeforeAndAfter {
import ctx.sql
before {
- Class.forName("org.h2.Driver")
+ Utils.classForName("org.h2.Driver")
conn = DriverManager.getConnection(url)
conn.prepareStatement("create schema test").executeUpdate()
diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala
index 301aa5a641..39b31523e0 100644
--- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala
+++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala
@@ -417,7 +417,7 @@ object ServerMode extends Enumeration {
}
abstract class HiveThriftJdbcTest extends HiveThriftServer2Test {
- Class.forName(classOf[HiveDriver].getCanonicalName)
+ Utils.classForName(classOf[HiveDriver].getCanonicalName)
private def jdbcUri = if (mode == ServerMode.http) {
s"""jdbc:hive2://localhost:$serverPort/
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
index d65d29daac..dc35569085 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
@@ -78,9 +78,7 @@ class HadoopTableReader(
override def makeRDDForTable(hiveTable: HiveTable): RDD[InternalRow] =
makeRDDForTable(
hiveTable,
- Class.forName(
- relation.tableDesc.getSerdeClassName, true, Utils.getContextOrSparkClassLoader)
- .asInstanceOf[Class[Deserializer]],
+ Utils.classForName(relation.tableDesc.getSerdeClassName).asInstanceOf[Class[Deserializer]],
filterOpt = None)
/**
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala
index 1f280c6429..8adda54754 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala
@@ -21,9 +21,6 @@ import java.io.{File, PrintStream}
import java.util.{Map => JMap}
import javax.annotation.concurrent.GuardedBy
-import org.apache.spark.sql.catalyst.expressions.Expression
-import org.apache.spark.util.CircularBuffer
-
import scala.collection.JavaConversions._
import scala.language.reflectiveCalls
@@ -37,7 +34,9 @@ import org.apache.hadoop.hive.ql.session.SessionState
import org.apache.hadoop.hive.ql.{Driver, metadata}
import org.apache.spark.Logging
+import org.apache.spark.sql.catalyst.expressions.Expression
import org.apache.spark.sql.execution.QueryExecutionException
+import org.apache.spark.util.{CircularBuffer, Utils}
/**
@@ -249,10 +248,10 @@ private[hive] class ClientWrapper(
}
private def toInputFormat(name: String) =
- Class.forName(name).asInstanceOf[Class[_ <: org.apache.hadoop.mapred.InputFormat[_, _]]]
+ Utils.classForName(name).asInstanceOf[Class[_ <: org.apache.hadoop.mapred.InputFormat[_, _]]]
private def toOutputFormat(name: String) =
- Class.forName(name)
+ Utils.classForName(name)
.asInstanceOf[Class[_ <: org.apache.hadoop.hive.ql.io.HiveOutputFormat[_, _]]]
private def toQlTable(table: HiveTable): metadata.Table = {
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala
index 917900e5f4..bee2ecbedb 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala
@@ -120,8 +120,8 @@ object SparkSubmitClassLoaderTest extends Logging {
logInfo("Testing load classes at the driver side.")
// First, we load classes at driver side.
try {
- Class.forName(args(0), true, Thread.currentThread().getContextClassLoader)
- Class.forName(args(1), true, Thread.currentThread().getContextClassLoader)
+ Utils.classForName(args(0))
+ Utils.classForName(args(1))
} catch {
case t: Throwable =>
throw new Exception("Could not load user class from jar:\n", t)
@@ -131,8 +131,8 @@ object SparkSubmitClassLoaderTest extends Logging {
val result = df.mapPartitions { x =>
var exception: String = null
try {
- Class.forName(args(0), true, Thread.currentThread().getContextClassLoader)
- Class.forName(args(1), true, Thread.currentThread().getContextClassLoader)
+ Utils.classForName(args(0))
+ Utils.classForName(args(1))
} catch {
case t: Throwable =>
exception = t + "\n" + t.getStackTraceString