diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala
index 283bb80f1c78898be9c8d9e19c70322fafae9b49..b6c30a007d88f53a28c8ca540a6c153f38eeeafa 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala
@@ -43,10 +43,10 @@ object CrossValidatorExample {
     val conf = new SparkConf().setAppName("CrossValidatorExample")
     val sc = new SparkContext(conf)
     val sqlContext = new SQLContext(sc)
-    import sqlContext._
+    import sqlContext.implicits._
 
     // Prepare training documents, which are labeled.
-    val training = sparkContext.parallelize(Seq(
+    val training = sc.parallelize(Seq(
       LabeledDocument(0L, "a b c d e spark", 1.0),
       LabeledDocument(1L, "b d", 0.0),
       LabeledDocument(2L, "spark f g h", 1.0),
@@ -92,7 +92,7 @@ object CrossValidatorExample {
     val cvModel = crossval.fit(training)
 
     // Prepare test documents, which are unlabeled.
-    val test = sparkContext.parallelize(Seq(
+    val test = sc.parallelize(Seq(
       Document(4L, "spark i j k"),
       Document(5L, "l m n"),
       Document(6L, "mapreduce spark"),
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/MovieLensALS.scala b/examples/src/main/scala/org/apache/spark/examples/ml/MovieLensALS.scala
index b7885829459a310b2be3b475de953fd05a13372f..836ea2e01201e24fad951bf19079714ad0af9b73 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/MovieLensALS.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/MovieLensALS.scala
@@ -109,7 +109,7 @@ object MovieLensALS {
     val conf = new SparkConf().setAppName(s"MovieLensALS with $params")
     val sc = new SparkContext(conf)
     val sqlContext = new SQLContext(sc)
-    import sqlContext._
+    import sqlContext.implicits._
 
     val ratings = sc.textFile(params.ratings).map(Rating.parseRating).cache()
 
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala
index 95cc9801eaeb9f3163af9975440ad8970a2748aa..4d1530cd1349f5e56d7f161e6fe710f2788afce0 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala
@@ -37,12 +37,12 @@ object SimpleParamsExample {
     val conf = new SparkConf().setAppName("SimpleParamsExample")
     val sc = new SparkContext(conf)
     val sqlContext = new SQLContext(sc)
-    import sqlContext._
+    import sqlContext.implicits._
 
     // Prepare training data.
     // We use LabeledPoint, which is a case class.  Spark SQL can convert RDDs of Java Beans
     // into DataFrames, where it uses the bean metadata to infer the schema.
-    val training = sparkContext.parallelize(Seq(
+    val training = sc.parallelize(Seq(
       LabeledPoint(1.0, Vectors.dense(0.0, 1.1, 0.1)),
       LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)),
       LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)),
@@ -81,7 +81,7 @@ object SimpleParamsExample {
     println("Model 2 was fit using parameters: " + model2.fittingParamMap)
 
     // Prepare test documents.
-    val test = sparkContext.parallelize(Seq(
+    val test = sc.parallelize(Seq(
       LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)),
       LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)),
       LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5))))
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala
index 065db62b0f5eda24d03e38e172f03fea43c61dd8..dbbe01dd5ce8ed473ea3addbfcfaa7879820c3ca 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala
@@ -44,10 +44,10 @@ object SimpleTextClassificationPipeline {
     val conf = new SparkConf().setAppName("SimpleTextClassificationPipeline")
     val sc = new SparkContext(conf)
     val sqlContext = new SQLContext(sc)
-    import sqlContext._
+    import sqlContext.implicits._
 
     // Prepare training documents, which are labeled.
-    val training = sparkContext.parallelize(Seq(
+    val training = sc.parallelize(Seq(
       LabeledDocument(0L, "a b c d e spark", 1.0),
       LabeledDocument(1L, "b d", 0.0),
       LabeledDocument(2L, "spark f g h", 1.0),
@@ -71,7 +71,7 @@ object SimpleTextClassificationPipeline {
     val model = pipeline.fit(training)
 
     // Prepare test documents, which are unlabeled.
-    val test = sparkContext.parallelize(Seq(
+    val test = sc.parallelize(Seq(
       Document(4L, "spark i j k"),
       Document(5L, "l m n"),
       Document(6L, "mapreduce spark"),
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DatasetExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DatasetExample.scala
index ab58375649d257e520b2f474fb950ed56d4c8dd1..89b6255991a38cf8d1052b9f48efe1aed2c1b293 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/DatasetExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DatasetExample.scala
@@ -71,7 +71,7 @@ object DatasetExample {
     val conf = new SparkConf().setAppName(s"DatasetExample with $params")
     val sc = new SparkContext(conf)
     val sqlContext = new SQLContext(sc)
-    import sqlContext._ // for implicit conversions
+    import sqlContext.implicits._  // for implicit conversions
 
     // Load input data
     val origData: RDD[LabeledPoint] = params.dataFormat match {
diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala
index 82a0b637b3cff6c1c6abde31f3807d5b2ffca074..1eac3c8d03e39cead1c5175d8e644028eb1e853b 100644
--- a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala
@@ -32,7 +32,7 @@ object RDDRelation {
     val sqlContext = new SQLContext(sc)
 
     // Importing the SQL context gives access to all the SQL functions and implicit conversions.
-    import sqlContext._
+    import sqlContext.implicits._
 
     val rdd = sc.parallelize((1 to 100).map(i => Record(i, s"val_$i")))
     // Any RDD containing case classes can be registered as a table.  The schema of the table is
@@ -41,15 +41,15 @@ object RDDRelation {
 
     // Once tables have been registered, you can run SQL queries over them.
     println("Result of SELECT *:")
-    sql("SELECT * FROM records").collect().foreach(println)
+    sqlContext.sql("SELECT * FROM records").collect().foreach(println)
 
     // Aggregation queries are also supported.
-    val count = sql("SELECT COUNT(*) FROM records").collect().head.getLong(0)
+    val count = sqlContext.sql("SELECT COUNT(*) FROM records").collect().head.getLong(0)
     println(s"COUNT(*): $count")
 
     // The results of SQL queries are themselves RDDs and support all normal RDD functions.  The
     // items in the RDD are of type Row, which allows you to access each column by ordinal.
-    val rddFromSql = sql("SELECT key, value FROM records WHERE key < 10")
+    val rddFromSql = sqlContext.sql("SELECT key, value FROM records WHERE key < 10")
 
     println("Result of RDD.map:")
     rddFromSql.map(row => s"Key: ${row(0)}, Value: ${row(1)}").collect().foreach(println)
@@ -68,7 +68,7 @@ object RDDRelation {
 
     // These files can also be registered as tables.
     parquetFile.registerTempTable("parquetFile")
-    sql("SELECT * FROM parquetFile").collect().foreach(println)
+    sqlContext.sql("SELECT * FROM parquetFile").collect().foreach(println)
 
     sc.stop()
   }
diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala
index 5725da1848114f69d4d90faef8e96194bba2d0d9..15754cdfcc35ed0dfa9b2f8488f38660da1abcb7 100644
--- a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala
@@ -43,7 +43,8 @@ object HiveFromSpark {
     // HiveContext. When not configured by the hive-site.xml, the context automatically
     // creates metastore_db and warehouse in the current directory.
     val hiveContext = new HiveContext(sc)
-    import hiveContext._
+    import hiveContext.implicits._
+    import hiveContext.sql
 
     sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)")
     sql(s"LOAD DATA LOCAL INPATH '${kv1File.getAbsolutePath}' INTO TABLE src")
diff --git a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala
index c7bec7a845a11f17e6113467db1ba365a0fc34c4..09456b289e2dc52cda8319ba2556af6cf56272e0 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala
@@ -122,7 +122,7 @@ class ALSModel private[ml] (
   def setPredictionCol(value: String): this.type = set(predictionCol, value)
 
   override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = {
-    import dataset.sqlContext.createDataFrame
+    import dataset.sqlContext.implicits._
     val map = this.paramMap ++ paramMap
     val users = userFactors.toDataFrame("id", "features")
     val items = itemFactors.toDataFrame("id", "features")
diff --git a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala
index acc447742bad0ebdb11981ad28355c955ecdf82a..cb7d57de35c348e4db862bdcd5c90ef41fa27203 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala
@@ -350,7 +350,7 @@ class ALSSuite extends FunSuite with MLlibTestSparkContext with Logging {
       numItemBlocks: Int = 3,
       targetRMSE: Double = 0.05): Unit = {
     val sqlContext = this.sqlContext
-    import sqlContext.createDataFrame
+    import sqlContext.implicits._
     val als = new ALS()
       .setRank(rank)
       .setRegParam(regParam)
diff --git a/repl/scala-2.10/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/scala-2.10/src/test/scala/org/apache/spark/repl/ReplSuite.scala
index e594ad868ea1c7ccede270baff1ab259dab5be15..201f2672d54742bfdf4b171b67d14bcec4a5a2f8 100644
--- a/repl/scala-2.10/src/test/scala/org/apache/spark/repl/ReplSuite.scala
+++ b/repl/scala-2.10/src/test/scala/org/apache/spark/repl/ReplSuite.scala
@@ -255,12 +255,12 @@ class ReplSuite extends FunSuite {
     assertDoesNotContain("Exception", output)
   }
 
-  test("SPARK-2576 importing SQLContext.createDataFrame.") {
+  test("SPARK-2576 importing SQLContext.implicits._") {
     // We need to use local-cluster to test this case.
     val output = runInterpreter("local-cluster[1,1,512]",
       """
         |val sqlContext = new org.apache.spark.sql.SQLContext(sc)
-        |import sqlContext.createDataFrame
+        |import sqlContext.implicits._
         |case class TestCaseClass(value: Int)
         |sc.parallelize(1 to 10).map(x => TestCaseClass(x)).toDataFrame.collect()
       """.stripMargin)
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 5ab5494f8042f1d5ac9798450fb783a8b21671d1..01620aa0acd4944aed0b8c96ae09eccb9a0e7634 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
@@ -166,6 +166,7 @@ class SQLContext(@transient val sparkContext: SparkContext)
   // scalastyle:off
   // Disable style checker so "implicits" object can start with lowercase i
   /**
+   * (Scala-specific)
    * Implicit methods available in Scala for converting common Scala objects into [[DataFrame]]s.
    */
   object implicits {
@@ -192,8 +193,7 @@ class SQLContext(@transient val sparkContext: SparkContext)
    *
    * @group userf
    */
-  // TODO: Remove implicit here.
-  implicit def createDataFrame[A <: Product : TypeTag](rdd: RDD[A]): DataFrame = {
+  def createDataFrame[A <: Product : TypeTag](rdd: RDD[A]): DataFrame = {
     SparkPlan.currentContext.set(self)
     val schema = ScalaReflection.schemaFor[A].dataType.asInstanceOf[StructType]
     val attributeSeq = schema.toAttributes
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala
index 9d6c529574da0e0b6c12b95184c92b8a89fbdbfe..8d3e094e3344dfc20d4d24cefeb70b906bb6bdaa 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala
@@ -88,6 +88,7 @@ trait ParquetTest {
   protected def withParquetFile[T <: Product: ClassTag: TypeTag]
       (data: Seq[T])
       (f: String => Unit): Unit = {
+    import sqlContext.implicits._
     withTempPath { file =>
       sparkContext.parallelize(data).saveAsParquetFile(file.getCanonicalPath)
       f(file.getCanonicalPath)
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala
index acb5677c4b77b6364052c2942e3e6e9a28839098..1318750a4a3b0e13d4ab15a750ceed379ecacb21 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala
@@ -34,6 +34,8 @@ case class BigData(s: String)
 class CachedTableSuite extends QueryTest {
   TestData // Load test tables.
 
+  import org.apache.spark.sql.test.TestSQLContext.implicits._
+
   def rddIdOf(tableName: String): Int = {
     val executedPlan = table(tableName).queryExecution.executedPlan
     executedPlan.collect {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
index e588555ad01d4fb78ba75a09d967d215f2debbfe..74c29459d2e47752d65b2d1cdf3b6f3f332571ac 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
@@ -21,7 +21,8 @@ import org.apache.spark.sql.Dsl._
 import org.apache.spark.sql.types._
 
 /* Implicits */
-import org.apache.spark.sql.test.TestSQLContext.{createDataFrame, logicalPlanToSparkQuery}
+import org.apache.spark.sql.test.TestSQLContext.logicalPlanToSparkQuery
+import org.apache.spark.sql.test.TestSQLContext.implicits._
 
 import scala.language.postfixOps
 
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
index 8f3d4265a286d8c1ced75c8ce3396b0daa687840..dc8ee41712fcdd73869b5770f4415cdcaa70ec20 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
@@ -35,6 +35,8 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll {
   // Make sure the tables are loaded.
   TestData
 
+  import org.apache.spark.sql.test.TestSQLContext.implicits._
+
   var origZone: TimeZone = _
   override protected def beforeAll() {
     origZone = TimeZone.getDefault
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala
index f26fcc0385b68a9010db7b85265667c6a48d088e..93782619826f0cd3d11bf62f9820c3657d4b62ae 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala
@@ -75,6 +75,9 @@ case class ComplexReflectData(
     dataField: Data)
 
 class ScalaReflectionRelationSuite extends FunSuite {
+
+  import org.apache.spark.sql.test.TestSQLContext.implicits._
+
   test("query case class RDD") {
     val data = ReflectData("a", 1, 1L, 1.toFloat, 1.toDouble, 1.toShort, 1.toByte, true,
                            new java.math.BigDecimal(1), new Date(12345), new Timestamp(12345), Seq(1,2,3))
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala
index dd781169ca57f7d9825cc204c717d14be994c6bd..0ed437edd05fdbbe2815857c3009f83b52167641 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala
@@ -22,9 +22,8 @@ import java.sql.Timestamp
 import org.apache.spark.sql.catalyst.plans.logical
 import org.apache.spark.sql.Dsl._
 import org.apache.spark.sql.test._
+import org.apache.spark.sql.test.TestSQLContext.implicits._
 
-/* Implicits */
-import org.apache.spark.sql.test.TestSQLContext._
 
 case class TestData(key: Int, value: String)
 
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala
index 117a51173481d0b9e274cc4dc582de1a183f83be..3c1657cd5fc3a4a0db719754af78d5bc834bf20b 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala
@@ -22,7 +22,8 @@ import scala.beans.{BeanInfo, BeanProperty}
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.Dsl._
 import org.apache.spark.sql.test.TestSQLContext
-import org.apache.spark.sql.test.TestSQLContext.{udf => _, _}
+import org.apache.spark.sql.test.TestSQLContext.{sparkContext, sql}
+import org.apache.spark.sql.test.TestSQLContext.implicits._
 import org.apache.spark.sql.types._
 
 
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala
index 3d33484ab0eb9c9db83e595f2eb49741b0d225de..86b1b5fda1c0f06d142ef654ba5c38de787e235a 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala
@@ -28,6 +28,8 @@ class InMemoryColumnarQuerySuite extends QueryTest {
   // Make sure the tables are loaded.
   TestData
 
+  import org.apache.spark.sql.test.TestSQLContext.implicits._
+
   test("simple columnar query") {
     val plan = executePlan(testData.logicalPlan).executedPlan
     val scan = InMemoryRelation(useCompression = true, 5, MEMORY_ONLY, plan, None)
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala
index fe9a69edbb92077ae3ff52fe1c798a65348969d2..55a9f735b350632673f897c9860169242a23d152 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala
@@ -26,6 +26,8 @@ class PartitionBatchPruningSuite extends FunSuite with BeforeAndAfterAll with Be
   val originalColumnBatchSize = conf.columnBatchSize
   val originalInMemoryPartitionPruning = conf.inMemoryPartitionPruning
 
+  import org.apache.spark.sql.test.TestSQLContext.implicits._
+
   override protected def beforeAll(): Unit = {
     // Make a table with 5 partitions, 2 batches per partition, 10 elements per batch
     setConf(SQLConf.COLUMN_BATCH_SIZE, "10")
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala
index d9ab16baf9a66b571e744c792dc27d265ec01713..0bc246c645602aa687a7dd0f4b68b5a0289535ad 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala
@@ -98,6 +98,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest {
   }
 
   test("fixed-length decimals") {
+    import org.apache.spark.sql.test.TestSQLContext.implicits._
+
     def makeDecimalRDD(decimal: DecimalType): DataFrame =
       sparkContext
         .parallelize(0 to 1000)
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala
index 4dd96bd5a1b77f0fb3baacab4cc35ca22a752552..869d01eb398c590016d656bc0f3e593a29fcd2ac 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala
@@ -30,6 +30,8 @@ import org.apache.spark.sql.hive.test.TestHive._
 case class TestData(key: Int, value: String)
 
 class InsertIntoHiveTableSuite extends QueryTest {
+  import org.apache.spark.sql.hive.test.TestHive.implicits._
+
   val testData = TestHive.sparkContext.parallelize(
     (1 to 100).map(i => TestData(i, i.toString)))
   testData.registerTempTable("testData")
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 85795acb658e2dc201b97ba34bb35976b4a2f08f..c23575fe968983629e2b7bf99bdc42bb609707b9 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
@@ -36,6 +36,9 @@ import org.apache.spark.sql.hive.test.TestHive._
  * Tests for persisting tables created though the data sources API into the metastore.
  */
 class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach {
+
+  import org.apache.spark.sql.hive.test.TestHive.implicits._
+
   override def afterEach(): Unit = {
     reset()
     if (ctasPath.exists()) Utils.deleteRecursively(ctasPath)
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
index a321452cef74ff1b4d331a70f6789bcecc454903..407d6058c33ed49414c997ddf8cf19e15b922360 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
@@ -43,6 +43,8 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter {
   private val originalTimeZone = TimeZone.getDefault
   private val originalLocale = Locale.getDefault
 
+  import org.apache.spark.sql.hive.test.TestHive.implicits._
+
   override def beforeAll() {
     TestHive.cacheTables = true
     // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*)
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala
index 422e843d2b0d25c5abbc808964af279ed0cc3229..178ece447758efc28ccfe21612571656008b0ee2 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala
@@ -18,7 +18,8 @@
 package org.apache.spark.sql.hive.execution
 
 import org.apache.spark.sql.hive.test.TestHive
-import org.apache.spark.sql.hive.test.TestHive._
+import org.apache.spark.sql.hive.test.TestHive.{sparkContext, sql}
+import org.apache.spark.sql.hive.test.TestHive.implicits._
 
 case class Nested(a: Int, B: Int)
 case class Data(a: Int, B: Int, n: Nested, nestedArray: Seq[Nested])
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala
index dd0df1a9f63206f83d3d942b02427a78188ad7f0..1e99003d3e9b578d04b21daeab498fe911fefb44 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala
@@ -47,7 +47,9 @@ case class ListStringCaseClass(l: Seq[String])
  * A test suite for Hive custom UDFs.
  */
 class HiveUdfSuite extends QueryTest {
-  import TestHive._
+
+  import TestHive.{udf, sql}
+  import TestHive.implicits._
 
   test("spark sql udf test that returns a struct") {
     udf.register("getStruct", (_: Int) => Fields(1, 2, 3, 4, 5))
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 268e5f6f01dff5488ebf05605cea3261349cecd8..22310ffadd25e206d0e705cceb4a985c7c8e186e 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
@@ -35,6 +35,9 @@ case class Nested3(f3: Int)
  * valid, but Hive currently cannot execute it.
  */
 class SQLQuerySuite extends QueryTest {
+
+  import org.apache.spark.sql.hive.test.TestHive.implicits._
+
   test("SPARK-4512 Fix attribute reference resolution error when using SORT BY") {
     checkAnswer(
       sql("SELECT * FROM (SELECT key + key AS a FROM src SORT BY value) t ORDER BY t.a"),
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala
index 79fd99d9f89ffc38e78430019e6008948563d679..30441bbbdf817333d78b0fcf4dc58576ac67ea94 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala
@@ -149,6 +149,8 @@ abstract class ParquetPartitioningTest extends QueryTest with BeforeAndAfterAll
   var partitionedTableDir: File = null
   var partitionedTableDirWithKey: File = null
 
+  import org.apache.spark.sql.hive.test.TestHive.implicits._
+
   override def beforeAll(): Unit = {
     partitionedTableDir = File.createTempFile("parquettests", "sparksql")
     partitionedTableDir.delete()