Repository: spark
Updated Branches:
  refs/heads/master 07f92ef1f -> 2b2c8c332


http://git-wip-us.apache.org/repos/asf/spark/blob/2b2c8c33/sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala
index c89a151..28ad7ae 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala
@@ -18,8 +18,8 @@
 package org.apache.spark.sql.test
 
 import org.apache.spark.{SparkConf, SparkContext}
-import org.apache.spark.sql.{SQLConf, SQLContext}
-
+import org.apache.spark.sql.SQLContext
+import org.apache.spark.sql.internal.SQLConf
 
 /**
  * A special [[SQLContext]] prepared for testing.
@@ -39,7 +39,7 @@ private[sql] class TestSQLContext(sc: SparkContext) extends 
SQLContext(sc) { sel
       super.clear()
 
       // Make sure we start with the default test configs even after clear
-      TestSQLContext.overrideConfs.map {
+      TestSQLContext.overrideConfs.foreach {
         case (key, value) => setConfString(key, value)
       }
     }

http://git-wip-us.apache.org/repos/asf/spark/blob/2b2c8c33/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala
 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala
index 66eaa3e..f32ba5f 100644
--- 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala
+++ 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala
@@ -32,10 +32,10 @@ import org.apache.hive.service.server.{HiveServer2, 
HiveServerServerOptionsProce
 import org.apache.spark.{Logging, SparkContext}
 import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd, 
SparkListenerJobStart}
-import org.apache.spark.sql.SQLConf
 import org.apache.spark.sql.hive.HiveContext
 import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._
 import org.apache.spark.sql.hive.thriftserver.ui.ThriftServerTab
+import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.util.{ShutdownHookManager, Utils}
 
 /**

http://git-wip-us.apache.org/repos/asf/spark/blob/2b2c8c33/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
index 8fef22c..458d4f2 100644
--- 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
+++ 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
@@ -33,9 +33,10 @@ import 
org.apache.hive.service.cli.operation.ExecuteStatementOperation
 import org.apache.hive.service.cli.session.HiveSession
 
 import org.apache.spark.Logging
-import org.apache.spark.sql.{DataFrame, Row => SparkRow, SQLConf}
+import org.apache.spark.sql.{DataFrame, Row => SparkRow}
 import org.apache.spark.sql.execution.SetCommand
 import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes}
+import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types._
 import org.apache.spark.util.{Utils => SparkUtils}
 

http://git-wip-us.apache.org/repos/asf/spark/blob/2b2c8c33/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala
----------------------------------------------------------------------
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 5f9952a..c05527b 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
@@ -202,7 +202,7 @@ class HiveThriftBinaryServerSuite extends 
HiveThriftJdbcTest {
   }
 
   test("test multiple session") {
-    import org.apache.spark.sql.SQLConf
+    import org.apache.spark.sql.internal.SQLConf
     var defaultV1: String = null
     var defaultV2: String = null
     var data: ArrayBuffer[Int] = null

http://git-wip-us.apache.org/repos/asf/spark/blob/2b2c8c33/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
 
b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
index 973ba6b..d15f883 100644
--- 
a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
+++ 
b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
@@ -23,8 +23,8 @@ import java.util.{Locale, TimeZone}
 import org.apache.spark.sql.catalyst.rules.RuleExecutor
 import org.scalatest.BeforeAndAfter
 
-import org.apache.spark.sql.SQLConf
 import org.apache.spark.sql.hive.test.TestHive
+import org.apache.spark.sql.internal.SQLConf
 
 /**
  * Runs the test cases that are included in the hive distribution.

http://git-wip-us.apache.org/repos/asf/spark/blob/2b2c8c33/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
index 88fda16..d511dd6 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
@@ -40,8 +40,6 @@ import org.apache.hadoop.util.VersionInfo
 import org.apache.spark.{Logging, SparkContext}
 import org.apache.spark.api.java.JavaSparkContext
 import org.apache.spark.sql._
-import org.apache.spark.sql.SQLConf.SQLConfEntry
-import org.apache.spark.sql.SQLConf.SQLConfEntry._
 import org.apache.spark.sql.catalyst.{InternalRow, ParserInterface}
 import org.apache.spark.sql.catalyst.analysis._
 import org.apache.spark.sql.catalyst.expressions.{Expression, LeafExpression}
@@ -52,6 +50,9 @@ import 
org.apache.spark.sql.execution.datasources.{DataSourceStrategy, PreInsert
 import org.apache.spark.sql.execution.ui.SQLListener
 import org.apache.spark.sql.hive.client._
 import org.apache.spark.sql.hive.execution.{DescribeHiveTableCommand, 
HiveNativeCommand}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.internal.SQLConf.SQLConfEntry
+import org.apache.spark.sql.internal.SQLConf.SQLConfEntry._
 import org.apache.spark.sql.types._
 import org.apache.spark.unsafe.types.UTF8String
 import org.apache.spark.util.Utils

http://git-wip-us.apache.org/repos/asf/spark/blob/2b2c8c33/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
index d316664..145b5f7 100644
--- 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
+++ 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
@@ -27,13 +27,13 @@ import org.apache.hadoop.hive.ql.{Context, ErrorMsg}
 import org.apache.hadoop.mapred.{FileOutputFormat, JobConf}
 
 import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.SQLConf
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.expressions.Attribute
 import org.apache.spark.sql.execution.{SparkPlan, UnaryNode}
 import org.apache.spark.sql.hive._
 import org.apache.spark.sql.hive.HiveShim.{ShimFileSinkDesc => FileSinkDesc}
 import org.apache.spark.SparkException
+import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.util.SerializableJobConf
 
 private[hive]

http://git-wip-us.apache.org/repos/asf/spark/blob/2b2c8c33/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala
index 246108e..9d0622b 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala
@@ -30,7 +30,7 @@ import org.apache.hadoop.hive.ql.processors._
 import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe
 
 import org.apache.spark.{SparkConf, SparkContext}
-import org.apache.spark.sql.{SQLConf, SQLContext}
+import org.apache.spark.sql.SQLContext
 import org.apache.spark.sql.catalyst.analysis._
 import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder
 import org.apache.spark.sql.catalyst.expressions.ExpressionInfo
@@ -39,6 +39,7 @@ import org.apache.spark.sql.execution.CacheTableCommand
 import org.apache.spark.sql.hive._
 import org.apache.spark.sql.hive.client.HiveClientImpl
 import org.apache.spark.sql.hive.execution.HiveNativeCommand
+import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.util.{ShutdownHookManager, Utils}
 
 // SPARK-3729: Test key required to check for initialization errors with 
config.

http://git-wip-us.apache.org/repos/asf/spark/blob/2b2c8c33/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala
index f8764d4..90d65d9 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala
@@ -20,9 +20,10 @@ package org.apache.spark.sql.hive
 import java.io.File
 
 import org.apache.spark.SparkFunSuite
-import org.apache.spark.sql.{QueryTest, Row, SaveMode, SQLConf}
+import org.apache.spark.sql.{QueryTest, Row, SaveMode}
 import org.apache.spark.sql.catalyst.catalog.CatalogTableType
 import org.apache.spark.sql.hive.test.TestHiveSingleton
+import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.test.{ExamplePointUDT, SQLTestUtils}
 import org.apache.spark.sql.types.{DecimalType, StringType, StructType}
 

http://git-wip-us.apache.org/repos/asf/spark/blob/2b2c8c33/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
index cd85e23..cb23959 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
@@ -29,6 +29,7 @@ import 
org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable
 import org.apache.spark.sql.execution.datasources.LogicalRelation
 import org.apache.spark.sql.execution.datasources.parquet.ParquetRelation
 import org.apache.spark.sql.hive.test.TestHiveSingleton
+import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.test.SQLTestUtils
 import org.apache.spark.sql.types._
 import org.apache.spark.util.Utils

http://git-wip-us.apache.org/repos/asf/spark/blob/2b2c8c33/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala
index 4a73153..a9823ae 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala
@@ -21,9 +21,10 @@ import java.sql.Timestamp
 
 import org.apache.hadoop.hive.conf.HiveConf
 
-import org.apache.spark.sql.{Row, SQLConf}
+import org.apache.spark.sql.Row
 import 
org.apache.spark.sql.execution.datasources.parquet.ParquetCompatibilityTest
 import org.apache.spark.sql.hive.test.TestHiveSingleton
+import org.apache.spark.sql.internal.SQLConf
 
 class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with 
TestHiveSingleton {
   /**

http://git-wip-us.apache.org/repos/asf/spark/blob/2b2c8c33/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala
index f49ee69..78569c5 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala
@@ -21,6 +21,7 @@ import com.google.common.io.Files
 
 import org.apache.spark.sql._
 import org.apache.spark.sql.hive.test.TestHiveSingleton
+import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.test.SQLTestUtils
 import org.apache.spark.util.Utils
 

http://git-wip-us.apache.org/repos/asf/spark/blob/2b2c8c33/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala
index 91bedf9..3952e71 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala
@@ -19,12 +19,13 @@ package org.apache.spark.sql.hive
 
 import scala.reflect.ClassTag
 
-import org.apache.spark.sql.{QueryTest, Row, SQLConf}
+import org.apache.spark.sql.{QueryTest, Row}
 import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.catalyst.parser.SimpleParserConf
 import org.apache.spark.sql.execution.joins._
 import org.apache.spark.sql.hive.execution._
 import org.apache.spark.sql.hive.test.TestHiveSingleton
+import org.apache.spark.sql.internal.SQLConf
 
 class StatisticsSuite extends QueryTest with TestHiveSingleton {
   import hiveContext.sql

http://git-wip-us.apache.org/repos/asf/spark/blob/2b2c8c33/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala
index caf1db9..45634a4 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala
@@ -26,6 +26,7 @@ import 
org.apache.spark.sql.expressions.{MutableAggregationBuffer, UserDefinedAg
 import org.apache.spark.sql.functions._
 import org.apache.spark.sql.hive.aggregate.{MyDoubleAvg, MyDoubleSum}
 import org.apache.spark.sql.hive.test.TestHiveSingleton
+import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.test.SQLTestUtils
 import org.apache.spark.sql.types._
 

http://git-wip-us.apache.org/repos/asf/spark/blob/2b2c8c33/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
index 6624494..ff1719e 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
@@ -29,6 +29,7 @@ import 
org.apache.spark.sql.execution.datasources.parquet.ParquetRelation
 import org.apache.spark.sql.functions._
 import org.apache.spark.sql.hive.{HiveContext, MetastoreRelation}
 import org.apache.spark.sql.hive.test.TestHiveSingleton
+import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.test.SQLTestUtils
 import org.apache.spark.sql.types._
 import org.apache.spark.unsafe.types.CalendarInterval

http://git-wip-us.apache.org/repos/asf/spark/blob/2b2c8c33/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcHadoopFsRelationSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcHadoopFsRelationSuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcHadoopFsRelationSuite.scala
index e8a6112..528f40b 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcHadoopFsRelationSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcHadoopFsRelationSuite.scala
@@ -20,7 +20,8 @@ package org.apache.spark.sql.hive.orc
 import org.apache.hadoop.fs.Path
 
 import org.apache.spark.deploy.SparkHadoopUtil
-import org.apache.spark.sql.{Row, SQLConf}
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.sources.HadoopFsRelationTest
 import org.apache.spark.sql.types._
 

http://git-wip-us.apache.org/repos/asf/spark/blob/2b2c8c33/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala
index 2156806..b11d1d9 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala
@@ -27,6 +27,7 @@ import org.apache.spark.sql._
 import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.hive.test.TestHive._
 import org.apache.spark.sql.hive.test.TestHive.implicits._
+import org.apache.spark.sql.internal.SQLConf
 
 case class AllDataTypesWithNonPrimitiveType(
     stringField: String,

http://git-wip-us.apache.org/repos/asf/spark/blob/2b2c8c33/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala
index a6ca7d0..68d5c7d 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala
@@ -25,6 +25,7 @@ import 
org.apache.spark.sql.execution.datasources.{InsertIntoDataSource, InsertI
 import org.apache.spark.sql.execution.datasources.parquet.ParquetRelation
 import org.apache.spark.sql.hive.execution.HiveTableScan
 import org.apache.spark.sql.hive.test.TestHiveSingleton
+import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.test.SQLTestUtils
 import org.apache.spark.sql.types._
 import org.apache.spark.util.Utils
@@ -755,6 +756,7 @@ abstract class ParquetPartitioningTest extends QueryTest 
with SQLTestUtils with
 
   /**
    * Drop named tables if they exist
+ *
    * @param tableNames tables to drop
    */
   def dropTables(tableNames: String*): Unit = {

http://git-wip-us.apache.org/repos/asf/spark/blob/2b2c8c33/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala 
b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala
index 62f6b99..9a52276 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala
@@ -28,6 +28,7 @@ import org.apache.spark.sql.execution.exchange.ShuffleExchange
 import org.apache.spark.sql.execution.joins.SortMergeJoin
 import org.apache.spark.sql.functions._
 import org.apache.spark.sql.hive.test.TestHiveSingleton
+import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.test.SQLTestUtils
 import org.apache.spark.util.Utils
 import org.apache.spark.util.collection.BitSet

http://git-wip-us.apache.org/repos/asf/spark/blob/2b2c8c33/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala 
b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala
index a32f8fb..c37b21b 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala
@@ -20,12 +20,13 @@ package org.apache.spark.sql.sources
 import java.io.File
 import java.net.URI
 
-import org.apache.spark.sql.{AnalysisException, QueryTest, SQLConf}
+import org.apache.spark.sql.{AnalysisException, QueryTest}
 import org.apache.spark.sql.catalyst.expressions.UnsafeProjection
 import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
 import org.apache.spark.sql.execution.datasources.BucketingUtils
 import org.apache.spark.sql.functions._
 import org.apache.spark.sql.hive.test.TestHiveSingleton
+import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.test.SQLTestUtils
 
 class BucketedWriteSuite extends QueryTest with SQLTestUtils with 
TestHiveSingleton {

http://git-wip-us.apache.org/repos/asf/spark/blob/2b2c8c33/sql/hive/src/test/scala/org/apache/spark/sql/sources/ParquetHadoopFsRelationSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/sources/ParquetHadoopFsRelationSuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/sources/ParquetHadoopFsRelationSuite.scala
index ba2a483..5ce58e8 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/sources/ParquetHadoopFsRelationSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/sources/ParquetHadoopFsRelationSuite.scala
@@ -24,6 +24,7 @@ import org.apache.hadoop.fs.Path
 
 import org.apache.spark.deploy.SparkHadoopUtil
 import org.apache.spark.sql._
+import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types._
 
 

http://git-wip-us.apache.org/repos/asf/spark/blob/2b2c8c33/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala
index 1a4b3ec..2a921a0 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala
@@ -30,6 +30,7 @@ import org.apache.spark.deploy.SparkHadoopUtil
 import org.apache.spark.sql._
 import org.apache.spark.sql.execution.datasources.LogicalRelation
 import org.apache.spark.sql.hive.test.TestHiveSingleton
+import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.test.SQLTestUtils
 import org.apache.spark.sql.types._
 


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org

Reply via email to