You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by pw...@apache.org on 2014/04/23 05:02:43 UTC

git commit: SPARK-1562 Fix visibility / annotation of Spark SQL APIs

Repository: spark
Updated Branches:
  refs/heads/master 662c860eb -> aa77f8a6a


SPARK-1562 Fix visibility / annotation of Spark SQL APIs

Author: Michael Armbrust <mi...@databricks.com>

Closes #489 from marmbrus/sqlDocFixes and squashes the following commits:

acee4f3 [Michael Armbrust] Fix visibility / annotation of Spark SQL APIs


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/aa77f8a6
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/aa77f8a6
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/aa77f8a6

Branch: refs/heads/master
Commit: aa77f8a6a604efe0d02bc8412b3f1ba3903b7a57
Parents: 662c860
Author: Michael Armbrust <mi...@databricks.com>
Authored: Tue Apr 22 20:02:33 2014 -0700
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Tue Apr 22 20:02:33 2014 -0700

----------------------------------------------------------------------
 project/SparkBuild.scala                        |  4 +--
 .../org/apache/spark/sql/SchemaRDDLike.scala    |  2 +-
 .../apache/spark/sql/execution/Aggregate.scala  |  3 ++
 .../apache/spark/sql/execution/Exchange.scala   |  7 +++-
 .../apache/spark/sql/execution/Generate.scala   |  3 ++
 .../apache/spark/sql/execution/SparkPlan.scala  | 13 ++++++--
 .../sql/execution/SparkSqlSerializer.scala      |  8 ++---
 .../spark/sql/execution/SparkStrategies.scala   |  2 +-
 .../spark/sql/execution/basicOperators.scala    | 34 +++++++++++++++++++-
 .../org/apache/spark/sql/execution/debug.scala  |  4 +--
 .../org/apache/spark/sql/execution/joins.scala  | 18 +++++++++++
 .../apache/spark/sql/execution/package.scala    |  1 +
 .../spark/sql/parquet/ParquetTestData.scala     |  2 +-
 .../org/apache/spark/SparkHadoopWriter.scala    |  9 +++---
 .../spark/sql/hive/HiveMetastoreCatalog.scala   |  8 +++--
 .../org/apache/spark/sql/hive/HiveQl.scala      | 14 ++++----
 .../apache/spark/sql/hive/HiveStrategies.scala  |  3 +-
 .../spark/sql/hive/ScriptTransformation.scala   |  6 +++-
 .../org/apache/spark/sql/hive/TestHive.scala    |  3 +-
 .../apache/spark/sql/hive/hiveOperators.scala   | 13 ++++++--
 .../org/apache/spark/sql/hive/hiveUdfs.scala    | 18 +++++------
 .../spark/sql/hive/CachedTableSuite.scala       |  1 +
 .../sql/hive/InsertIntoHiveTableSuite.scala     |  5 ++-
 .../spark/sql/hive/api/java/JavaHiveSuite.scala |  2 +-
 .../hive/execution/BigDataBenchmarkSuite.scala  |  2 +-
 .../hive/execution/ConcurrentHiveSuite.scala    |  2 +-
 .../sql/hive/execution/HiveComparisonTest.scala |  2 +-
 .../hive/execution/HiveCompatibilitySuite.scala |  2 +-
 .../sql/hive/execution/HiveQuerySuite.scala     |  2 +-
 .../hive/execution/HiveResolutionSuite.scala    |  4 +--
 .../spark/sql/hive/execution/PruningSuite.scala |  2 +-
 .../spark/sql/parquet/HiveParquetSuite.scala    |  3 +-
 32 files changed, 143 insertions(+), 59 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/aa77f8a6/project/SparkBuild.scala
----------------------------------------------------------------------
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 1ca4e0d..8523585 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -367,7 +367,7 @@ object SparkBuild extends Build {
     publish := {},
 
     unidocProjectFilter in (ScalaUnidoc, unidoc) :=
-      inAnyProject -- inProjects(repl, examples, tools, yarn, yarnAlpha),
+      inAnyProject -- inProjects(repl, examples, tools, catalyst, yarn, yarnAlpha),
     unidocProjectFilter in (JavaUnidoc, unidoc) :=
       inAnyProject -- inProjects(repl, examples, bagel, graphx, catalyst, tools, yarn, yarnAlpha),
 
@@ -457,7 +457,7 @@ object SparkBuild extends Build {
   def catalystSettings = sharedSettings ++ Seq(
     name := "catalyst",
     // The mechanics of rewriting expression ids to compare trees in some test cases makes
-    // assumptions about the the expression ids being contiguious.  Running tests in parallel breaks
+    // assumptions about the the expression ids being contiguous.  Running tests in parallel breaks
     // this non-deterministically.  TODO: FIX THIS.
     parallelExecution in Test := false,
     libraryDependencies ++= Seq(

http://git-wip-us.apache.org/repos/asf/spark/blob/aa77f8a6/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala
index a390ab6..3a895e1 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala
@@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.plans.logical._
 /**
  * Contains functions that are shared between all SchemaRDD types (i.e., Scala, Java)
  */
-trait SchemaRDDLike {
+private[sql] trait SchemaRDDLike {
   @transient val sqlContext: SQLContext
   @transient protected[spark] val logicalPlan: LogicalPlan
 

http://git-wip-us.apache.org/repos/asf/spark/blob/aa77f8a6/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala
index 3a4f071..36b3b95 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala
@@ -19,12 +19,14 @@ package org.apache.spark.sql.execution
 
 import java.util.HashMap
 
+import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.SparkContext
 import org.apache.spark.sql.catalyst.errors._
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.physical._
 
 /**
+ * :: DeveloperApi ::
  * Groups input data by `groupingExpressions` and computes the `aggregateExpressions` for each
  * group.
  *
@@ -34,6 +36,7 @@ import org.apache.spark.sql.catalyst.plans.physical._
  * @param aggregateExpressions expressions that are computed for each group.
  * @param child the input data source.
  */
+@DeveloperApi
 case class Aggregate(
     partial: Boolean,
     groupingExpressions: Seq[Expression],

http://git-wip-us.apache.org/repos/asf/spark/blob/aa77f8a6/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala
index 070557e..3b4acb7 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala
@@ -17,6 +17,7 @@
 
 package org.apache.spark.sql.execution
 
+import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.{HashPartitioner, RangePartitioner, SparkConf}
 import org.apache.spark.rdd.ShuffledRDD
 import org.apache.spark.sql.Row
@@ -26,6 +27,10 @@ import org.apache.spark.sql.catalyst.plans.physical._
 import org.apache.spark.sql.catalyst.rules.Rule
 import org.apache.spark.util.MutablePair
 
+/**
+ * :: DeveloperApi ::
+ */
+@DeveloperApi
 case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends UnaryNode {
 
   override def outputPartitioning = newPartitioning
@@ -81,7 +86,7 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una
  * [[catalyst.plans.physical.Distribution Distribution]] requirements for each operator by inserting
  * [[Exchange]] Operators where required.
  */
-object AddExchange extends Rule[SparkPlan] {
+private[sql] object AddExchange extends Rule[SparkPlan] {
   // TODO: Determine the number of partitions.
   val numPartitions = 150
 

http://git-wip-us.apache.org/repos/asf/spark/blob/aa77f8a6/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala
index cff4887..da1e08b 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala
@@ -17,9 +17,11 @@
 
 package org.apache.spark.sql.execution
 
+import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.sql.catalyst.expressions.{Generator, JoinedRow, Literal, Projection}
 
 /**
+ * :: DeveloperApi ::
  * Applies a [[catalyst.expressions.Generator Generator]] to a stream of input rows, combining the
  * output of each into a new stream of rows.  This operation is similar to a `flatMap` in functional
  * programming with one important additional feature, which allows the input rows to be joined with
@@ -29,6 +31,7 @@ import org.apache.spark.sql.catalyst.expressions.{Generator, JoinedRow, Literal,
  * @param outer when true, each input row will be output at least once, even if the output of the
  *              given `generator` is empty. `outer` has no effect when `join` is false.
  */
+@DeveloperApi
 case class Generate(
     generator: Generator,
     join: Boolean,

http://git-wip-us.apache.org/repos/asf/spark/blob/aa77f8a6/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala
index 5d89697..50124dd 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala
@@ -17,6 +17,7 @@
 
 package org.apache.spark.sql.execution
 
+import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.{Logging, Row}
 import org.apache.spark.sql.catalyst.trees
@@ -26,6 +27,10 @@ import org.apache.spark.sql.catalyst.plans.{QueryPlan, logical}
 import org.apache.spark.sql.catalyst.plans.physical._
 import org.apache.spark.sql.columnar.InMemoryColumnarTableScan
 
+/**
+ * :: DeveloperApi ::
+ */
+@DeveloperApi
 abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging {
   self: Product =>
 
@@ -51,6 +56,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging {
 }
 
 /**
+ * :: DeveloperApi ::
  * Allows already planned SparkQueries to be linked into logical query plans.
  *
  * Note that in general it is not valid to use this class to link multiple copies of the same
@@ -59,6 +65,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging {
  * replace the output attributes with new copies of themselves without breaking any attribute
  * linking.
  */
+@DeveloperApi
 case class SparkLogicalPlan(alreadyPlanned: SparkPlan)
   extends logical.LogicalPlan with MultiInstanceRelation {
 
@@ -77,15 +84,15 @@ case class SparkLogicalPlan(alreadyPlanned: SparkPlan)
   }
 }
 
-trait LeafNode extends SparkPlan with trees.LeafNode[SparkPlan] {
+private[sql] trait LeafNode extends SparkPlan with trees.LeafNode[SparkPlan] {
   self: Product =>
 }
 
-trait UnaryNode extends SparkPlan with trees.UnaryNode[SparkPlan] {
+private[sql] trait UnaryNode extends SparkPlan with trees.UnaryNode[SparkPlan] {
   self: Product =>
   override def outputPartitioning: Partitioning = child.outputPartitioning
 }
 
-trait BinaryNode extends SparkPlan with trees.BinaryNode[SparkPlan] {
+private[sql] trait BinaryNode extends SparkPlan with trees.BinaryNode[SparkPlan] {
   self: Product =>
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/aa77f8a6/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala
index c30ae5b..5067c14 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala
@@ -27,7 +27,7 @@ import org.apache.spark.serializer.KryoSerializer
 import org.apache.spark.util.MutablePair
 import org.apache.spark.util.Utils
 
-class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(conf) {
+private[sql] class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(conf) {
   override def newKryo(): Kryo = {
     val kryo = new Kryo()
     kryo.setRegistrationRequired(false)
@@ -50,7 +50,7 @@ class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(conf) {
   }
 }
 
-object SparkSqlSerializer {
+private[sql] object SparkSqlSerializer {
   // TODO (lian) Using KryoSerializer here is workaround, needs further investigation
   // Using SparkSqlSerializer here makes BasicQuerySuite to fail because of Kryo serialization
   // related error.
@@ -68,7 +68,7 @@ object SparkSqlSerializer {
   }
 }
 
-class BigDecimalSerializer extends Serializer[BigDecimal] {
+private[sql] class BigDecimalSerializer extends Serializer[BigDecimal] {
   def write(kryo: Kryo, output: Output, bd: math.BigDecimal) {
     // TODO: There are probably more efficient representations than strings...
     output.writeString(bd.toString())
@@ -83,7 +83,7 @@ class BigDecimalSerializer extends Serializer[BigDecimal] {
  * Maps do not have a no arg constructor and so cannot be serialized by default. So, we serialize
  * them as `Array[(k,v)]`.
  */
-class MapSerializer extends Serializer[Map[_,_]] {
+private[sql] class MapSerializer extends Serializer[Map[_,_]] {
   def write(kryo: Kryo, output: Output, map: Map[_,_]) {
     kryo.writeObject(output, map.flatMap(e => Seq(e._1, e._2)).toArray)
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/aa77f8a6/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
index fe8bd5a..500fde1 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
@@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.catalyst.plans.physical._
 import org.apache.spark.sql.parquet._
 
-abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
+private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
   self: SQLContext#SparkPlanner =>
 
   object HashJoin extends Strategy {

http://git-wip-us.apache.org/repos/asf/spark/blob/aa77f8a6/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala
index eedcc7d..e4cf202 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala
@@ -19,6 +19,7 @@ package org.apache.spark.sql.execution
 
 import scala.reflect.runtime.universe.TypeTag
 
+import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.{HashPartitioner, SparkConf, SparkContext}
 import org.apache.spark.rdd.{RDD, ShuffledRDD}
 import org.apache.spark.sql.catalyst.ScalaReflection
@@ -27,6 +28,10 @@ import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.physical.{OrderedDistribution, UnspecifiedDistribution}
 import org.apache.spark.util.MutablePair
 
+/**
+ * :: DeveloperApi ::
+ */
+@DeveloperApi
 case class Project(projectList: Seq[NamedExpression], child: SparkPlan) extends UnaryNode {
   override def output = projectList.map(_.toAttribute)
 
@@ -36,6 +41,10 @@ case class Project(projectList: Seq[NamedExpression], child: SparkPlan) extends
   }
 }
 
+/**
+ * :: DeveloperApi ::
+ */
+@DeveloperApi
 case class Filter(condition: Expression, child: SparkPlan) extends UnaryNode {
   override def output = child.output
 
@@ -44,6 +53,10 @@ case class Filter(condition: Expression, child: SparkPlan) extends UnaryNode {
   }
 }
 
+/**
+ * :: DeveloperApi ::
+ */
+@DeveloperApi
 case class Sample(fraction: Double, withReplacement: Boolean, seed: Int, child: SparkPlan)
     extends UnaryNode {
 
@@ -53,6 +66,10 @@ case class Sample(fraction: Double, withReplacement: Boolean, seed: Int, child:
   override def execute() = child.execute().sample(withReplacement, fraction, seed)
 }
 
+/**
+ * :: DeveloperApi ::
+ */
+@DeveloperApi
 case class Union(children: Seq[SparkPlan])(@transient sc: SparkContext) extends SparkPlan {
   // TODO: attributes output by union should be distinct for nullability purposes
   override def output = children.head.output
@@ -62,12 +79,14 @@ case class Union(children: Seq[SparkPlan])(@transient sc: SparkContext) extends
 }
 
 /**
+ * :: DeveloperApi ::
  * Take the first limit elements. Note that the implementation is different depending on whether
  * this is a terminal operator or not. If it is terminal and is invoked using executeCollect,
  * this operator uses Spark's take method on the Spark driver. If it is not terminal or is
  * invoked using execute, we first take the limit on each partition, and then repartition all the
  * data to a single partition to compute the global limit.
  */
+@DeveloperApi
 case class Limit(limit: Int, child: SparkPlan)(@transient sc: SparkContext) extends UnaryNode {
   // TODO: Implement a partition local limit, and use a strategy to generate the proper limit plan:
   // partition local limit -> exchange into one partition -> partition local limit again
@@ -91,10 +110,12 @@ case class Limit(limit: Int, child: SparkPlan)(@transient sc: SparkContext) exte
 }
 
 /**
+ * :: DeveloperApi ::
  * Take the first limit elements as defined by the sortOrder. This is logically equivalent to
  * having a [[Limit]] operator after a [[Sort]] operator. This could have been named TopK, but
  * Spark's top operator does the opposite in ordering so we name it TakeOrdered to avoid confusion.
  */
+@DeveloperApi
 case class TakeOrdered(limit: Int, sortOrder: Seq[SortOrder], child: SparkPlan)
                       (@transient sc: SparkContext) extends UnaryNode {
   override def otherCopyArgs = sc :: Nil
@@ -111,7 +132,10 @@ case class TakeOrdered(limit: Int, sortOrder: Seq[SortOrder], child: SparkPlan)
   override def execute() = sc.makeRDD(executeCollect(), 1)
 }
 
-
+/**
+ * :: DeveloperApi ::
+ */
+@DeveloperApi
 case class Sort(
     sortOrder: Seq[SortOrder],
     global: Boolean,
@@ -134,6 +158,10 @@ case class Sort(
   override def output = child.output
 }
 
+/**
+ * :: DeveloperApi ::
+ */
+@DeveloperApi
 object ExistingRdd {
   def convertToCatalyst(a: Any): Any = a match {
     case s: Seq[Any] => s.map(convertToCatalyst)
@@ -167,6 +195,10 @@ object ExistingRdd {
   }
 }
 
+/**
+ * :: DeveloperApi ::
+ */
+@DeveloperApi
 case class ExistingRdd(output: Seq[Attribute], rdd: RDD[Row]) extends LeafNode {
   override def execute() = rdd
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/aa77f8a6/sql/core/src/main/scala/org/apache/spark/sql/execution/debug.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug.scala
index 40982f1..a0d2910 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug.scala
@@ -17,7 +17,7 @@
 
 package org.apache.spark.sql.execution
 
-object DebugQuery {
+private[sql] object DebugQuery {
   def apply(plan: SparkPlan): SparkPlan = {
     val visited = new collection.mutable.HashSet[Long]()
     plan transform {
@@ -28,7 +28,7 @@ object DebugQuery {
   }
 }
 
-case class DebugNode(child: SparkPlan) extends UnaryNode {
+private[sql] case class DebugNode(child: SparkPlan) extends UnaryNode {
   def references = Set.empty
   def output = child.output
   def execute() = {

http://git-wip-us.apache.org/repos/asf/spark/blob/aa77f8a6/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala
index c89dae9..31cc269 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala
@@ -21,14 +21,24 @@ import scala.collection.mutable.{ArrayBuffer, BitSet}
 
 import org.apache.spark.SparkContext
 
+import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans._
 import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, Partitioning}
 
+@DeveloperApi
 sealed abstract class BuildSide
+
+@DeveloperApi
 case object BuildLeft extends BuildSide
+
+@DeveloperApi
 case object BuildRight extends BuildSide
 
+/**
+ * :: DeveloperApi ::
+ */
+@DeveloperApi
 case class HashJoin(
     leftKeys: Seq[Expression],
     rightKeys: Seq[Expression],
@@ -130,6 +140,10 @@ case class HashJoin(
   }
 }
 
+/**
+ * :: DeveloperApi ::
+ */
+@DeveloperApi
 case class CartesianProduct(left: SparkPlan, right: SparkPlan) extends BinaryNode {
   def output = left.output ++ right.output
 
@@ -138,6 +152,10 @@ case class CartesianProduct(left: SparkPlan, right: SparkPlan) extends BinaryNod
   }
 }
 
+/**
+ * :: DeveloperApi ::
+ */
+@DeveloperApi
 case class BroadcastNestedLoopJoin(
     streamed: SparkPlan, broadcast: SparkPlan, joinType: JoinType, condition: Option[Expression])
     (@transient sc: SparkContext)

http://git-wip-us.apache.org/repos/asf/spark/blob/aa77f8a6/sql/core/src/main/scala/org/apache/spark/sql/execution/package.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/package.scala
index e4a2dec..66237f8 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/package.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/package.scala
@@ -18,6 +18,7 @@
 package org.apache.spark.sql
 
 /**
+ * :: DeveloperApi ::
  * An execution engine for relational query plans that runs on top Spark and returns RDDs.
  *
  * Note that the operators in this package are created automatically by a query planner using a

http://git-wip-us.apache.org/repos/asf/spark/blob/aa77f8a6/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala
index 728e3dd..f37976f 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala
@@ -28,7 +28,7 @@ import parquet.schema.{MessageType, MessageTypeParser}
 import org.apache.spark.sql.catalyst.expressions.GenericRow
 import org.apache.spark.util.Utils
 
-object ParquetTestData {
+private[sql] object ParquetTestData {
 
   val testSchema =
     """message myrecord {

http://git-wip-us.apache.org/repos/asf/spark/blob/aa77f8a6/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala b/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala
index 7219c03..ab7862f 100644
--- a/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.spark
+package org.apache.spark.sql.hive
 
 import java.io.IOException
 import java.text.NumberFormat
@@ -28,12 +28,13 @@ import org.apache.hadoop.hive.ql.plan.FileSinkDesc
 import org.apache.hadoop.mapred._
 import org.apache.hadoop.io.Writable
 
+import org.apache.spark.{Logging, SerializableWritable, SparkHadoopWriter}
+
 /**
  * Internal helper class that saves an RDD using a Hive OutputFormat.
  * It is based on [[SparkHadoopWriter]].
  */
-protected[spark]
-class SparkHiveHadoopWriter(
+private[hive] class SparkHiveHadoopWriter(
     @transient jobConf: JobConf,
     fileSinkConf: FileSinkDesc)
   extends Logging
@@ -179,7 +180,7 @@ class SparkHiveHadoopWriter(
   }
 }
 
-object SparkHiveHadoopWriter {
+private[hive] object SparkHiveHadoopWriter {
   def createPathFromString(path: String, conf: JobConf): Path = {
     if (path == null) {
       throw new IllegalArgumentException("Output path is null")

http://git-wip-us.apache.org/repos/asf/spark/blob/aa77f8a6/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
index ca75cec..6c90788 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
@@ -34,12 +34,13 @@ import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.rules._
 import org.apache.spark.sql.catalyst.types._
 import org.apache.spark.sql.execution.SparkLogicalPlan
+import org.apache.spark.sql.hive.execution.{HiveTableScan, InsertIntoHiveTable}
 import org.apache.spark.sql.columnar.InMemoryColumnarTableScan
 
 /* Implicit conversions */
 import scala.collection.JavaConversions._
 
-class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Logging {
+private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Logging {
   import HiveMetastoreTypes._
 
   val client = Hive.get(hive.hiveconf)
@@ -171,7 +172,7 @@ class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Logging {
   override def unregisterAllTables() = {}
 }
 
-object HiveMetastoreTypes extends RegexParsers {
+private[hive] object HiveMetastoreTypes extends RegexParsers {
   protected lazy val primitiveType: Parser[DataType] =
     "string" ^^^ StringType |
     "float" ^^^ FloatType |
@@ -229,7 +230,8 @@ object HiveMetastoreTypes extends RegexParsers {
   }
 }
 
-case class MetastoreRelation(databaseName: String, tableName: String, alias: Option[String])
+private[hive] case class MetastoreRelation
+    (databaseName: String, tableName: String, alias: Option[String])
     (val table: TTable, val partitions: Seq[TPartition])
   extends BaseRelation {
   // TODO: Can we use org.apache.hadoop.hive.ql.metadata.Table as the type of table and

http://git-wip-us.apache.org/repos/asf/spark/blob/aa77f8a6/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
index 4dac25b..1777e96 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
@@ -36,20 +36,20 @@ import scala.collection.JavaConversions._
  * back for Hive to execute natively.  Will be replaced with a native command that contains the
  * cmd string.
  */
-case object NativePlaceholder extends Command
+private[hive] case object NativePlaceholder extends Command
 
-case class DfsCommand(cmd: String) extends Command
+private[hive] case class DfsCommand(cmd: String) extends Command
 
-case class ShellCommand(cmd: String) extends Command
+private[hive] case class ShellCommand(cmd: String) extends Command
 
-case class SourceCommand(filePath: String) extends Command
+private[hive] case class SourceCommand(filePath: String) extends Command
 
-case class AddJar(jarPath: String) extends Command
+private[hive] case class AddJar(jarPath: String) extends Command
 
-case class AddFile(filePath: String) extends Command
+private[hive] case class AddFile(filePath: String) extends Command
 
 /** Provides a mapping from HiveQL statements to catalyst logical plans and expression trees. */
-object HiveQl {
+private[hive] object HiveQl {
   protected val nativeCommands = Seq(
     "TOK_DESCFUNCTION",
     "TOK_DESCTABLE",

http://git-wip-us.apache.org/repos/asf/spark/blob/aa77f8a6/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala
index ac817b2..d9a6e0e 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala
@@ -23,9 +23,10 @@ import org.apache.spark.sql.catalyst.planning._
 import org.apache.spark.sql.catalyst.plans._
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.execution._
+import org.apache.spark.sql.hive.execution._
 import org.apache.spark.sql.columnar.InMemoryColumnarTableScan
 
-trait HiveStrategies {
+private[hive] trait HiveStrategies {
   // Possibly being too clever with types here... or not clever enough.
   self: SQLContext#SparkPlanner =>
 

http://git-wip-us.apache.org/repos/asf/spark/blob/aa77f8a6/sql/hive/src/main/scala/org/apache/spark/sql/hive/ScriptTransformation.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/ScriptTransformation.scala
index 2610100..610fa9c 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/ScriptTransformation.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/ScriptTransformation.scala
@@ -15,23 +15,27 @@
  * limitations under the License.
  */
 
-package org.apache.spark.sql.hive
+package org.apache.spark.sql.hive.execution
 
 import java.io.{BufferedReader, InputStreamReader}
 
+import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.execution._
+import org.apache.spark.sql.hive.HiveContext
 
 /* Implicit conversions */
 import scala.collection.JavaConversions._
 
 /**
+ * :: DeveloperApi ::
  * Transforms the input by forking and running the specified script.
  *
  * @param input the set of expression that should be passed to the script.
  * @param script the command that should be executed.
  * @param output the attributes that are produced by the script.
  */
+@DeveloperApi
 case class ScriptTransformation(
     input: Seq[Expression],
     script: String,

http://git-wip-us.apache.org/repos/asf/spark/blob/aa77f8a6/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala
index b1a26fd..74110ee 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.spark.sql.hive
+package org.apache.spark.sql.hive.test
 
 import java.io.File
 import java.util.{Set => JavaSet}
@@ -34,6 +34,7 @@ import org.apache.spark.{SparkConf, SparkContext}
 import org.apache.spark.sql.catalyst.analysis._
 import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, NativeCommand}
 import org.apache.spark.sql.catalyst.util._
+import org.apache.spark.sql.hive._
 
 /* Implicit conversions */
 import scala.collection.JavaConversions._

http://git-wip-us.apache.org/repos/asf/spark/blob/aa77f8a6/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala
index 821fb22..96faebc 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.spark.sql.hive
+package org.apache.spark.sql.hive.execution
 
 import org.apache.hadoop.hive.common.`type`.{HiveDecimal, HiveVarchar}
 import org.apache.hadoop.hive.metastore.MetaStoreUtils
@@ -30,23 +30,26 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaHiveVarcharOb
 import org.apache.hadoop.io.Writable
 import org.apache.hadoop.mapred._
 
+import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.types.{BooleanType, DataType}
 import org.apache.spark.sql.execution._
-import org.apache.spark.{SparkHiveHadoopWriter, TaskContext, SparkException}
+import org.apache.spark.sql.hive._
+import org.apache.spark.{TaskContext, SparkException}
 
 /* Implicits */
 import scala.collection.JavaConversions._
 
 /**
+ * :: DeveloperApi ::
  * The Hive table scan operator.  Column and partition pruning are both handled.
  *
- * @constructor
  * @param attributes Attributes to be fetched from the Hive table.
  * @param relation The Hive table be be scanned.
  * @param partitionPruningPred An optional partition pruning predicate for partitioned table.
  */
+@DeveloperApi
 case class HiveTableScan(
     attributes: Seq[Attribute],
     relation: MetastoreRelation,
@@ -160,6 +163,10 @@ case class HiveTableScan(
   def output = attributes
 }
 
+/**
+ * :: DeveloperApi ::
+ */
+@DeveloperApi
 case class InsertIntoHiveTable(
     table: MetastoreRelation,
     partition: Map[String, Option[String]],

http://git-wip-us.apache.org/repos/asf/spark/blob/aa77f8a6/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala
index 55a4363..a09270e 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala
@@ -37,7 +37,7 @@ import org.apache.spark.sql.catalyst.types._
 /* Implicit conversions */
 import scala.collection.JavaConversions._
 
-object HiveFunctionRegistry
+private[hive] object HiveFunctionRegistry
   extends analysis.FunctionRegistry with HiveFunctionFactory with HiveInspectors {
 
   def lookupFunction(name: String, children: Seq[Expression]): Expression = {
@@ -99,7 +99,7 @@ object HiveFunctionRegistry
   }
 }
 
-trait HiveFunctionFactory {
+private[hive] trait HiveFunctionFactory {
   def getFunctionInfo(name: String) = FunctionRegistry.getFunctionInfo(name)
   def getFunctionClass(name: String) = getFunctionInfo(name).getFunctionClass
   def createFunction[UDFType](name: String) =
@@ -130,7 +130,7 @@ trait HiveFunctionFactory {
   }
 }
 
-abstract class HiveUdf extends Expression with Logging with HiveFunctionFactory {
+private[hive] abstract class HiveUdf extends Expression with Logging with HiveFunctionFactory {
   self: Product =>
 
   type UDFType
@@ -148,7 +148,7 @@ abstract class HiveUdf extends Expression with Logging with HiveFunctionFactory
   override def toString = s"$nodeName#${functionInfo.getDisplayName}(${children.mkString(",")})"
 }
 
-case class HiveSimpleUdf(name: String, children: Seq[Expression]) extends HiveUdf {
+private[hive] case class HiveSimpleUdf(name: String, children: Seq[Expression]) extends HiveUdf {
   import org.apache.spark.sql.hive.HiveFunctionRegistry._
   type UDFType = UDF
 
@@ -201,7 +201,7 @@ case class HiveSimpleUdf(name: String, children: Seq[Expression]) extends HiveUd
   }
 }
 
-case class HiveGenericUdf(name: String, children: Seq[Expression])
+private[hive] case class HiveGenericUdf(name: String, children: Seq[Expression])
   extends HiveUdf with HiveInspectors {
 
   import org.apache.hadoop.hive.ql.udf.generic.GenericUDF._
@@ -228,7 +228,7 @@ case class HiveGenericUdf(name: String, children: Seq[Expression])
   }
 }
 
-trait HiveInspectors {
+private[hive] trait HiveInspectors {
 
   def unwrapData(data: Any, oi: ObjectInspector): Any = oi match {
     case pi: PrimitiveObjectInspector => pi.getPrimitiveJavaObject(data)
@@ -329,7 +329,7 @@ trait HiveInspectors {
   }
 }
 
-case class HiveGenericUdaf(
+private[hive] case class HiveGenericUdaf(
     name: String,
     children: Seq[Expression]) extends AggregateExpression
   with HiveInspectors
@@ -371,7 +371,7 @@ case class HiveGenericUdaf(
  * Operators that require maintaining state in between input rows should instead be implemented as
  * user defined aggregations, which have clean semantics even in a partitioned execution.
  */
-case class HiveGenericUdtf(
+private[hive] case class HiveGenericUdtf(
     name: String,
     aliasNames: Seq[String],
     children: Seq[Expression])
@@ -438,7 +438,7 @@ case class HiveGenericUdtf(
   override def toString = s"$nodeName#$name(${children.mkString(",")})"
 }
 
-case class HiveUdafFunction(
+private[hive] case class HiveUdafFunction(
     functionName: String,
     exprs: Seq[Expression],
     base: AggregateExpression)

http://git-wip-us.apache.org/repos/asf/spark/blob/aa77f8a6/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala
index 79ec1f1..f9a162e 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala
@@ -20,6 +20,7 @@ package org.apache.spark.sql.hive
 import org.apache.spark.sql.execution.SparkLogicalPlan
 import org.apache.spark.sql.columnar.InMemoryColumnarTableScan
 import org.apache.spark.sql.hive.execution.HiveComparisonTest
+import org.apache.spark.sql.hive.test.TestHive
 
 class CachedTableSuite extends HiveComparisonTest {
   TestHive.loadTestTable("src")

http://git-wip-us.apache.org/repos/asf/spark/blob/aa77f8a6/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala
----------------------------------------------------------------------
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 ad29e06..833f350 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
@@ -17,12 +17,11 @@
 
 package org.apache.spark.sql.hive
 
-import java.io.File
-
 import org.apache.spark.sql.QueryTest
+import org.apache.spark.sql.hive.test.TestHive
 
 /* Implicits */
-import org.apache.spark.sql.hive.TestHive._
+import org.apache.spark.sql.hive.test.TestHive._
 
 case class TestData(key: Int, value: String)
 

http://git-wip-us.apache.org/repos/asf/spark/blob/aa77f8a6/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveSuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveSuite.scala
index 8137f99..9c5d7c8 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveSuite.scala
@@ -21,7 +21,7 @@ import org.scalatest.FunSuite
 
 import org.apache.spark.api.java.JavaSparkContext
 import org.apache.spark.sql.test.TestSQLContext
-import org.apache.spark.sql.hive.TestHive
+import org.apache.spark.sql.hive.test.TestHive
 
 // Implicits
 import scala.collection.JavaConversions._

http://git-wip-us.apache.org/repos/asf/spark/blob/aa77f8a6/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala
index 8488f23..9b9a823 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.hive.execution
 
 import java.io.File
 
-import org.apache.spark.sql.hive.TestHive._
+import org.apache.spark.sql.hive.test.TestHive._
 
 /**
  * A set of test cases based on the big-data-benchmark.

http://git-wip-us.apache.org/repos/asf/spark/blob/aa77f8a6/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala
index ac87f2c..23ece7e 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala
@@ -18,7 +18,7 @@
 package org.apache.spark.sql.hive.execution
 
 import org.apache.spark.{SparkConf, SparkContext}
-import org.apache.spark.sql.hive.TestHiveContext
+import org.apache.spark.sql.hive.test.TestHiveContext
 import org.scalatest.{BeforeAndAfterAll, FunSuite}
 
 class ConcurrentHiveSuite extends FunSuite with BeforeAndAfterAll {

http://git-wip-us.apache.org/repos/asf/spark/blob/aa77f8a6/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
index 6c91f40..ea17e6e 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
@@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{ExplainCommand, NativeComman
 import org.apache.spark.sql.catalyst.util._
 import org.apache.spark.sql.execution.Sort
 import org.scalatest.{BeforeAndAfterAll, FunSuite, GivenWhenThen}
-import org.apache.spark.sql.hive.TestHive
+import org.apache.spark.sql.hive.test.TestHive
 
 /**
  * Allows the creations of tests that execute the same query against both hive

http://git-wip-us.apache.org/repos/asf/spark/blob/aa77f8a6/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
index c3cfa3d..dfe88b9 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.hive.execution
 
 import org.scalatest.BeforeAndAfter
 
-import org.apache.spark.sql.hive.TestHive
+import org.apache.spark.sql.hive.test.TestHive
 
 /**
  * Runs the test cases that are included in the hive distribution.

http://git-wip-us.apache.org/repos/asf/spark/blob/aa77f8a6/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
----------------------------------------------------------------------
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 a09667a..d224d2e 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
@@ -17,7 +17,7 @@
 
 package org.apache.spark.sql.hive.execution
 
-import org.apache.spark.sql.hive.TestHive._
+import org.apache.spark.sql.hive.test.TestHive._
 
 /**
  * A set of test cases expressed in Hive QL that are not covered by the tests included in the hive distribution.

http://git-wip-us.apache.org/repos/asf/spark/blob/aa77f8a6/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala
----------------------------------------------------------------------
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 8883e5b..67594b5 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
@@ -17,8 +17,8 @@
 
 package org.apache.spark.sql.hive.execution
 
-import org.apache.spark.sql.hive.TestHive
-import org.apache.spark.sql.hive.TestHive._
+import org.apache.spark.sql.hive.test.TestHive
+import org.apache.spark.sql.hive.test.TestHive._
 
 case class Data(a: Int, B: Int, n: Nested)
 case class Nested(a: Int, B: Int)

http://git-wip-us.apache.org/repos/asf/spark/blob/aa77f8a6/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala
index d9ccb93..25eca39 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala
@@ -17,7 +17,7 @@
 
 package org.apache.spark.sql.hive.execution
 
-import org.apache.spark.sql.hive.{HiveTableScan, TestHive}
+import org.apache.spark.sql.hive.test.TestHive
 
 /* Implicit conversions */
 import scala.collection.JavaConversions._

http://git-wip-us.apache.org/repos/asf/spark/blob/aa77f8a6/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala
index 843c681..91ad59d 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala
@@ -22,11 +22,10 @@ import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite}
 import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Row}
 import org.apache.spark.sql.catalyst.types.{DataType, StringType, IntegerType}
 import org.apache.spark.sql.{parquet, SchemaRDD}
-import org.apache.spark.sql.hive.TestHive
 import org.apache.spark.util.Utils
 
 // Implicits
-import org.apache.spark.sql.hive.TestHive._
+import org.apache.spark.sql.hive.test.TestHive._
 
 class HiveParquetSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAfterEach {