diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala
index 9cc7b2ac79205f89f20ea84c74fe33df68d728f4..f542f5cf40506bf1693f8f0fbb9e7fd6bc3d299f 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala
@@ -382,7 +382,7 @@ object CatalystTypeConverters {
    * Typical use case would be converting a collection of rows that have the same schema. You will
    * call this function once to get a converter, and apply it to every row.
    */
-  private[sql] def createToCatalystConverter(dataType: DataType): Any => Any = {
+  def createToCatalystConverter(dataType: DataType): Any => Any = {
     if (isPrimitive(dataType)) {
       // Although the `else` branch here is capable of handling inbound conversion of primitives,
       // we add some special-case handling for those types here. The motivation for this relates to
@@ -409,7 +409,7 @@ object CatalystTypeConverters {
    * Typical use case would be converting a collection of rows that have the same schema. You will
    * call this function once to get a converter, and apply it to every row.
    */
-  private[sql] def createToScalaConverter(dataType: DataType): Any => Any = {
+  def createToScalaConverter(dataType: DataType): Any => Any = {
     if (isPrimitive(dataType)) {
       identity
     } else {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala
index 76f87f64ba5caf5e4ad67074ed526821c53c2978..7923cfce8210094714d8995ebc166fcc4dc682a8 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala
@@ -708,7 +708,7 @@ object ScalaReflection extends ScalaReflection {
   /**
    * Whether the fields of the given type is defined entirely by its constructor parameters.
    */
-  private[sql] def definedByConstructorParams(tpe: Type): Boolean = {
+  def definedByConstructorParams(tpe: Type): Boolean = {
     tpe <:< localTypeOf[Product] || tpe <:< localTypeOf[DefinedByConstructorParams]
   }
 
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index 2efa997ff22d23b656fe4770ecd3e7d46986be31..660f523698e7f40a9ece02df4dfde2b7ced0eba0 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -246,7 +246,7 @@ class Analyzer(
       }.isDefined
     }
 
-    private[sql] def hasGroupingFunction(e: Expression): Boolean = {
+    private[analysis] def hasGroupingFunction(e: Expression): Boolean = {
       e.collectFirst {
         case g: Grouping => g
         case g: GroupingID => g
@@ -1412,7 +1412,7 @@ class Analyzer(
      * Construct the output attributes for a [[Generator]], given a list of names.  If the list of
      * names is empty names are assigned from field names in generator.
      */
-    private[sql] def makeGeneratorOutput(
+    private[analysis] def makeGeneratorOutput(
         generator: Generator,
         names: Seq[String]): Seq[Attribute] = {
       val elementAttrs = generator.elementSchema.toAttributes
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala
index 9a040f8644fb59ab3b4c2808dd7a776153bc7259..8503b8dcf81a75777e932475c374dd2dc6466aa5 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala
@@ -63,7 +63,7 @@ object TypeCoercion {
 
   // See https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Types.
   // The conversion for integral and floating point types have a linear widening hierarchy:
-  private[sql] val numericPrecedence =
+  val numericPrecedence =
     IndexedSeq(
       ByteType,
       ShortType,
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
index 1856dc4d642d0f8aa52ee66e254cee6f96ea170b..e36241a4367b95130282b94f4f8abe0f9a74b801 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
@@ -750,7 +750,7 @@ class SessionCatalog(
    *
    * This performs reflection to decide what type of [[Expression]] to return in the builder.
    */
-  private[sql] def makeFunctionBuilder(name: String, functionClassName: String): FunctionBuilder = {
+  def makeFunctionBuilder(name: String, functionClassName: String): FunctionBuilder = {
     // TODO: at least support UDAFs here
     throw new UnsupportedOperationException("Use sqlContext.udf.register(...) instead.")
   }
@@ -794,7 +794,7 @@ class SessionCatalog(
   /**
    * Look up the [[ExpressionInfo]] associated with the specified function, assuming it exists.
    */
-  private[spark] def lookupFunctionInfo(name: FunctionIdentifier): ExpressionInfo = synchronized {
+  def lookupFunctionInfo(name: FunctionIdentifier): ExpressionInfo = synchronized {
     // TODO: just make function registry take in FunctionIdentifier instead of duplicating this
     val database = name.database.orElse(Some(currentDb)).map(formatDatabaseName)
     val qualifiedName = name.copy(database = database)
@@ -906,7 +906,7 @@ class SessionCatalog(
    *
    * This is mainly used for tests.
    */
-  private[sql] def reset(): Unit = synchronized {
+  def reset(): Unit = synchronized {
     setCurrentDatabase(DEFAULT_DATABASE)
     listDatabases().filter(_ != DEFAULT_DATABASE).foreach { db =>
       dropDatabase(db, ignoreIfNotExists = false, cascade = true)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/package.scala
index 03708fb7afd448349e00555a010dd5ddaef061c5..59f7969e561441bbf91ce18ed780bdd425fce185 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/package.scala
@@ -26,7 +26,7 @@ package object encoders {
    * references from a specific schema.)  This requirement allows us to preserve whether a given
    * object type is being bound by name or by ordinal when doing resolution.
    */
-  private[sql] def encoderFor[A : Encoder]: ExpressionEncoder[A] = implicitly[Encoder[A]] match {
+  def encoderFor[A : Encoder]: ExpressionEncoder[A] = implicitly[Encoder[A]] match {
     case e: ExpressionEncoder[A] =>
       e.assertUnresolved()
       e
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
index 1f37b68846ae4a3e161617dbae86d2cef04ae718..7abbbe257d830b13296dcea07c5981a6b837e86c 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
@@ -526,7 +526,7 @@ abstract class BinaryOperator extends BinaryExpression with ExpectsInputTypes {
 }
 
 
-private[sql] object BinaryOperator {
+object BinaryOperator {
   def unapply(e: BinaryOperator): Option[(Expression, Expression)] = Some((e.left, e.right))
 }
 
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/MonotonicallyIncreasingID.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/MonotonicallyIncreasingID.scala
index 75c6bb2d84dfb689422871d6133100bb1383fbab..5b4922e0cf2b7581bbf1b164297836d4498047ae 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/MonotonicallyIncreasingID.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/MonotonicallyIncreasingID.scala
@@ -40,7 +40,7 @@ import org.apache.spark.sql.types.{DataType, LongType}
       represent the record number within each partition. The assumption is that the data frame has
       less than 1 billion partitions, and each partition has less than 8 billion records.""",
   extended = "> SELECT _FUNC_();\n 0")
-private[sql] case class MonotonicallyIncreasingID() extends LeafExpression with Nondeterministic {
+case class MonotonicallyIncreasingID() extends LeafExpression with Nondeterministic {
 
   /**
    * Record ID within each partition. By being transient, count's value is reset to 0 every time
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SparkPartitionID.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SparkPartitionID.scala
index 71af59a7a8529469397c296bc31b323f8b793e68..1f675d5b07270f6f8aa0f9edd229772340927617 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SparkPartitionID.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SparkPartitionID.scala
@@ -28,7 +28,7 @@ import org.apache.spark.sql.types.{DataType, IntegerType}
 @ExpressionDescription(
   usage = "_FUNC_() - Returns the current partition id of the Spark task",
   extended = "> SELECT _FUNC_();\n 0")
-private[sql] case class SparkPartitionID() extends LeafExpression with Nondeterministic {
+case class SparkPartitionID() extends LeafExpression with Nondeterministic {
 
   override def nullable: Boolean = false
 
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala
index 504cea52797deb2ae9ee6fa93d210d3c56519fd7..7a39e568fa2895f24347d90dadb8475c9dc2713b 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala
@@ -24,14 +24,14 @@ import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
 import org.apache.spark.sql.types._
 
 /** The mode of an [[AggregateFunction]]. */
-private[sql] sealed trait AggregateMode
+sealed trait AggregateMode
 
 /**
  * An [[AggregateFunction]] with [[Partial]] mode is used for partial aggregation.
  * This function updates the given aggregation buffer with the original input of this
  * function. When it has processed all input rows, the aggregation buffer is returned.
  */
-private[sql] case object Partial extends AggregateMode
+case object Partial extends AggregateMode
 
 /**
  * An [[AggregateFunction]] with [[PartialMerge]] mode is used to merge aggregation buffers
@@ -39,7 +39,7 @@ private[sql] case object Partial extends AggregateMode
  * This function updates the given aggregation buffer by merging multiple aggregation buffers.
  * When it has processed all input rows, the aggregation buffer is returned.
  */
-private[sql] case object PartialMerge extends AggregateMode
+case object PartialMerge extends AggregateMode
 
 /**
  * An [[AggregateFunction]] with [[Final]] mode is used to merge aggregation buffers
@@ -47,7 +47,7 @@ private[sql] case object PartialMerge extends AggregateMode
  * This function updates the given aggregation buffer by merging multiple aggregation buffers.
  * When it has processed all input rows, the final result of this function is returned.
  */
-private[sql] case object Final extends AggregateMode
+case object Final extends AggregateMode
 
 /**
  * An [[AggregateFunction]] with [[Complete]] mode is used to evaluate this function directly
@@ -55,13 +55,13 @@ private[sql] case object Final extends AggregateMode
  * This function updates the given aggregation buffer with the original input of this
  * function. When it has processed all input rows, the final result of this function is returned.
  */
-private[sql] case object Complete extends AggregateMode
+case object Complete extends AggregateMode
 
 /**
  * A place holder expressions used in code-gen, it does not change the corresponding value
  * in the row.
  */
-private[sql] case object NoOp extends Expression with Unevaluable {
+case object NoOp extends Expression with Unevaluable {
   override def nullable: Boolean = true
   override def dataType: DataType = NullType
   override def children: Seq[Expression] = Nil
@@ -84,7 +84,7 @@ object AggregateExpression {
  * A container for an [[AggregateFunction]] with its [[AggregateMode]] and a field
  * (`isDistinct`) indicating if DISTINCT keyword is specified for this function.
  */
-private[sql] case class AggregateExpression(
+case class AggregateExpression(
     aggregateFunction: AggregateFunction,
     mode: AggregateMode,
     isDistinct: Boolean,
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
index 91ffac0ba2a6006ce6294fa91b0ba67825b09cfc..7ff8795d4f05ea3984d128e42486f90866fff2ad 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
@@ -125,7 +125,7 @@ abstract class BinaryArithmetic extends BinaryOperator {
   }
 }
 
-private[sql] object BinaryArithmetic {
+object BinaryArithmetic {
   def unapply(e: BinaryArithmetic): Option[(Expression, Expression)] = Some((e.left, e.right))
 }
 
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala
index 0ca715f42472ac7a8b227fabe498b2854d1e2354..09e22aaf3e3d8637e3fd0c4e6940377f0feb0a60 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala
@@ -84,8 +84,8 @@ case class CreateArray(children: Seq[Expression]) extends Expression {
 @ExpressionDescription(
   usage = "_FUNC_(key0, value0, key1, value1...) - Creates a map with the given key/value pairs.")
 case class CreateMap(children: Seq[Expression]) extends Expression {
-  private[sql] lazy val keys = children.indices.filter(_ % 2 == 0).map(children)
-  private[sql] lazy val values = children.indices.filter(_ % 2 != 0).map(children)
+  lazy val keys = children.indices.filter(_ % 2 == 0).map(children)
+  lazy val values = children.indices.filter(_ % 2 != 0).map(children)
 
   override def foldable: Boolean = children.forall(_.foldable)
 
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala
index 3b4468f55ca73811d62f5648b02615673251e069..abb5594bfa7f89f1477cdde5a3fc9174378bd0ad 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala
@@ -106,7 +106,7 @@ trait ExtractValue extends Expression
 case class GetStructField(child: Expression, ordinal: Int, name: Option[String] = None)
   extends UnaryExpression with ExtractValue {
 
-  private[sql] lazy val childSchema = child.dataType.asInstanceOf[StructType]
+  lazy val childSchema = child.dataType.asInstanceOf[StructType]
 
   override def dataType: DataType = childSchema(ordinal).dataType
   override def nullable: Boolean = child.nullable || childSchema(ordinal).nullable
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala
index d2c94ec1df4d5c4db2b0e0848406e6b1ba8b9893..369207587d8601966a6cf56a55bd3a5aec1c729d 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala
@@ -554,7 +554,7 @@ object XxHash64Function extends InterpretedHashFunction {
 @ExpressionDescription(
   usage = "_FUNC_() - Returns the current database.",
   extended = "> SELECT _FUNC_()")
-private[sql] case class CurrentDatabase() extends LeafExpression with Unevaluable {
+case class CurrentDatabase() extends LeafExpression with Unevaluable {
   override def dataType: DataType = StringType
   override def foldable: Boolean = true
   override def nullable: Boolean = false
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
index 734bacf727e361470202c7ffe44538c56662b45c..799858a6865e565919280ed8df60a6869d860259 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
@@ -394,13 +394,13 @@ abstract class BinaryComparison extends BinaryOperator with Predicate {
 }
 
 
-private[sql] object BinaryComparison {
+object BinaryComparison {
   def unapply(e: BinaryComparison): Option[(Expression, Expression)] = Some((e.left, e.right))
 }
 
 
 /** An extractor that matches both standard 3VL equality and null-safe equality. */
-private[sql] object Equality {
+object Equality {
   def unapply(e: BinaryComparison): Option[(Expression, Expression)] = e match {
     case EqualTo(l, r) => Some((l, r))
     case EqualNullSafe(l, r) => Some((l, r))
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala
index e036982e70f9983c64555be7a1ae65837be61ce4..73dceb35ac50e3fa2d02d06370cdf34ad33b04d1 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala
@@ -218,7 +218,7 @@ class GenericRowWithSchema(values: Array[Any], override val schema: StructType)
  * Note that, while the array is not copied, and thus could technically be mutated after creation,
  * this is not allowed.
  */
-class GenericInternalRow(private[sql] val values: Array[Any]) extends BaseGenericInternalRow {
+class GenericInternalRow(val values: Array[Any]) extends BaseGenericInternalRow {
   /** No-arg constructor for serialization. */
   protected def this() = this(null)
 
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
index b31f5aa11c229d8ad7732755d26cd5adcaa79dc7..eb612c4c12c754ba35a42e12dbbcd65b7a7e8583 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
@@ -127,7 +127,7 @@ abstract class SetOperation(left: LogicalPlan, right: LogicalPlan) extends Binar
   }
 }
 
-private[sql] object SetOperation {
+object SetOperation {
   def unapply(p: SetOperation): Option[(LogicalPlan, LogicalPlan)] = Some((p.left, p.right))
 }
 
@@ -365,7 +365,7 @@ case class InsertIntoTable(
   override def children: Seq[LogicalPlan] = child :: Nil
   override def output: Seq[Attribute] = Seq.empty
 
-  private[spark] lazy val expectedColumns = {
+  lazy val expectedColumns = {
     if (table.output.isEmpty) {
       None
     } else {
@@ -509,7 +509,7 @@ case class Window(
   def windowOutputSet: AttributeSet = AttributeSet(windowExpressions.map(_.toAttribute))
 }
 
-private[sql] object Expand {
+object Expand {
   /**
    * Extract attribute set according to the grouping id.
    *
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/AbstractScalaRowIterator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/AbstractScalaRowIterator.scala
index 6d35f140cf23f7b719906972736106ad4f3c7045..0c7205b3c66515af88b44c60dc770d51ed35941b 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/AbstractScalaRowIterator.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/AbstractScalaRowIterator.scala
@@ -23,7 +23,7 @@ package org.apache.spark.sql.catalyst.util
  * `Row` in order to work around a spurious IntelliJ compiler error. This cannot be an abstract
  * class because that leads to compilation errors under Scala 2.11.
  */
-private[spark] class AbstractScalaRowIterator[T] extends Iterator[T] {
+class AbstractScalaRowIterator[T] extends Iterator[T] {
   override def hasNext: Boolean = throw new NotImplementedError
 
   override def next(): T = throw new NotImplementedError