[SPARK-16964][SQL] Remove private[sql] and private[spark] from sql.execution package [Backport]
## What changes were proposed in this pull request? This PR backports https://github.com/apache/spark/pull/14554 to branch-2.0. I have also changed the visibility of a few similar Hive classes. ## How was this patch tested? (Only a package visibility change) Author: Herman van Hovell <hvanhov...@databricks.com> Author: Reynold Xin <r...@databricks.com> Closes #14652 from hvanhovell/SPARK-16964. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/1c569711 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/1c569711 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/1c569711 Branch: refs/heads/branch-2.0 Commit: 1c56971167a0ebb3c422ccc7cc3d6904015fe2ec Parents: 237ae54 Author: Herman van Hovell <hvanhov...@databricks.com> Authored: Tue Aug 16 01:15:31 2016 -0700 Committer: Reynold Xin <r...@databricks.com> Committed: Tue Aug 16 01:15:31 2016 -0700 ---------------------------------------------------------------------- .../spark/sql/execution/CacheManager.scala | 22 +++++++++--------- .../spark/sql/execution/ExistingRDD.scala | 18 +++++++-------- .../apache/spark/sql/execution/ExpandExec.scala | 2 +- .../spark/sql/execution/FileRelation.scala | 2 +- .../spark/sql/execution/GenerateExec.scala | 2 +- .../sql/execution/LocalTableScanExec.scala | 4 ++-- .../spark/sql/execution/RowIterator.scala | 2 +- .../spark/sql/execution/SQLExecution.scala | 2 +- .../apache/spark/sql/execution/SortExec.scala | 6 ++--- .../apache/spark/sql/execution/SparkPlan.scala | 14 ++++++------ .../spark/sql/execution/SparkPlanInfo.scala | 2 +- .../spark/sql/execution/SparkStrategies.scala | 6 ++--- .../sql/execution/UnsafeRowSerializer.scala | 4 ++-- .../sql/execution/WholeStageCodegenExec.scala | 2 +- .../execution/aggregate/HashAggregateExec.scala | 2 +- .../execution/aggregate/SortAggregateExec.scala | 2 +- .../spark/sql/execution/aggregate/udaf.scala | 6 ++--- .../sql/execution/basicPhysicalOperators.scala | 6 ++--- .../execution/columnar/InMemoryRelation.scala | 8 +++---- .../columnar/InMemoryTableScanExec.scala | 4 ++-- .../spark/sql/execution/command/commands.scala | 4 ++-- .../datasources/DataSourceStrategy.scala | 8 +++---- .../datasources/FileSourceStrategy.scala | 2 +- .../InsertIntoDataSourceCommand.scala | 2 +- .../InsertIntoHadoopFsRelationCommand.scala | 2 +- .../datasources/PartitioningUtils.scala | 24 +++++++++++--------- .../execution/datasources/WriterContainer.scala | 8 +++---- .../sql/execution/datasources/bucket.scala | 2 +- .../execution/datasources/csv/CSVOptions.scala | 2 +- .../execution/datasources/csv/CSVParser.scala | 4 ++-- .../execution/datasources/csv/CSVRelation.scala | 4 ++-- .../datasources/fileSourceInterfaces.scala | 6 ++--- .../execution/datasources/jdbc/JDBCRDD.scala | 8 +++---- .../datasources/parquet/ParquetFileFormat.scala | 17 +++++++------- .../datasources/parquet/ParquetFilters.scala | 2 +- .../datasources/parquet/ParquetOptions.scala | 6 ++--- .../spark/sql/execution/datasources/rules.scala | 6 ++--- .../spark/sql/execution/debug/package.scala | 2 +- .../exchange/BroadcastExchangeExec.scala | 2 +- .../exchange/ExchangeCoordinator.scala | 4 ++-- .../execution/exchange/ShuffleExchange.scala | 9 ++++---- .../execution/joins/BroadcastHashJoinExec.scala | 2 +- .../joins/BroadcastNestedLoopJoinExec.scala | 2 +- .../execution/joins/CartesianProductExec.scala | 5 ++-- .../execution/joins/ShuffledHashJoinExec.scala | 2 +- .../sql/execution/joins/SortMergeJoinExec.scala | 2 +- .../spark/sql/execution/metric/SQLMetrics.scala | 10 ++++---- .../execution/python/ExtractPythonUDFs.scala | 4 ++-- .../sql/execution/r/MapPartitionsRWrapper.scala | 4 ++-- .../sql/execution/stat/FrequentItems.scala | 4 ++-- .../sql/execution/stat/StatFunctions.scala | 8 +++---- .../streaming/IncrementalExecution.scala | 2 +- .../execution/streaming/StreamExecution.scala | 19 ++++++++-------- .../execution/streaming/StreamProgress.scala | 2 +- .../execution/streaming/state/StateStore.scala | 2 +- .../streaming/state/StateStoreCoordinator.scala | 4 ++-- .../spark/sql/execution/ui/ExecutionPage.scala | 2 +- .../spark/sql/execution/ui/SQLListener.scala | 6 ++--- .../apache/spark/sql/execution/ui/SQLTab.scala | 4 ++-- .../spark/sql/execution/ui/SparkPlanGraph.scala | 6 ++--- .../apache/spark/sql/internal/SharedState.scala | 2 -- .../CreateHiveTableAsSelectCommand.scala | 1 - .../sql/hive/execution/HiveTableScanExec.scala | 2 +- .../hive/execution/ScriptTransformation.scala | 3 --- .../spark/sql/hive/orc/OrcFileFormat.scala | 2 +- 65 files changed, 166 insertions(+), 174 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala index de2503a..83b7c77 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala @@ -31,7 +31,7 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.storage.StorageLevel.MEMORY_AND_DISK /** Holds a cached logical plan and its data */ -private[sql] case class CachedData(plan: LogicalPlan, cachedRepresentation: InMemoryRelation) +case class CachedData(plan: LogicalPlan, cachedRepresentation: InMemoryRelation) /** * Provides support in a SQLContext for caching query results and automatically using these cached @@ -41,7 +41,7 @@ private[sql] case class CachedData(plan: LogicalPlan, cachedRepresentation: InMe * * Internal to Spark SQL. */ -private[sql] class CacheManager extends Logging { +class CacheManager extends Logging { @transient private val cachedData = new scala.collection.mutable.ArrayBuffer[CachedData] @@ -68,13 +68,13 @@ private[sql] class CacheManager extends Logging { } /** Clears all cached tables. */ - private[sql] def clearCache(): Unit = writeLock { + def clearCache(): Unit = writeLock { cachedData.foreach(_.cachedRepresentation.cachedColumnBuffers.unpersist()) cachedData.clear() } /** Checks if the cache is empty. */ - private[sql] def isEmpty: Boolean = readLock { + def isEmpty: Boolean = readLock { cachedData.isEmpty } @@ -83,7 +83,7 @@ private[sql] class CacheManager extends Logging { * Unlike `RDD.cache()`, the default storage level is set to be `MEMORY_AND_DISK` because * recomputing the in-memory columnar representation of the underlying table is expensive. */ - private[sql] def cacheQuery( + def cacheQuery( query: Dataset[_], tableName: Option[String] = None, storageLevel: StorageLevel = MEMORY_AND_DISK): Unit = writeLock { @@ -108,7 +108,7 @@ private[sql] class CacheManager extends Logging { * Tries to remove the data for the given [[Dataset]] from the cache. * No operation, if it's already uncached. */ - private[sql] def uncacheQuery(query: Dataset[_], blocking: Boolean = true): Boolean = writeLock { + def uncacheQuery(query: Dataset[_], blocking: Boolean = true): Boolean = writeLock { val planToCache = query.queryExecution.analyzed val dataIndex = cachedData.indexWhere(cd => planToCache.sameResult(cd.plan)) val found = dataIndex >= 0 @@ -120,17 +120,17 @@ private[sql] class CacheManager extends Logging { } /** Optionally returns cached data for the given [[Dataset]] */ - private[sql] def lookupCachedData(query: Dataset[_]): Option[CachedData] = readLock { + def lookupCachedData(query: Dataset[_]): Option[CachedData] = readLock { lookupCachedData(query.queryExecution.analyzed) } /** Optionally returns cached data for the given [[LogicalPlan]]. */ - private[sql] def lookupCachedData(plan: LogicalPlan): Option[CachedData] = readLock { + def lookupCachedData(plan: LogicalPlan): Option[CachedData] = readLock { cachedData.find(cd => plan.sameResult(cd.plan)) } /** Replaces segments of the given logical plan with cached versions where possible. */ - private[sql] def useCachedData(plan: LogicalPlan): LogicalPlan = { + def useCachedData(plan: LogicalPlan): LogicalPlan = { plan transformDown { case currentFragment => lookupCachedData(currentFragment) @@ -143,7 +143,7 @@ private[sql] class CacheManager extends Logging { * Invalidates the cache of any data that contains `plan`. Note that it is possible that this * function will over invalidate. */ - private[sql] def invalidateCache(plan: LogicalPlan): Unit = writeLock { + def invalidateCache(plan: LogicalPlan): Unit = writeLock { cachedData.foreach { case data if data.plan.collect { case p if p.sameResult(plan) => p }.nonEmpty => data.cachedRepresentation.recache() @@ -155,7 +155,7 @@ private[sql] class CacheManager extends Logging { * Invalidates the cache of any data that contains `resourcePath` in one or more * `HadoopFsRelation` node(s) as part of its logical plan. */ - private[sql] def invalidateCachedPath( + def invalidateCachedPath( sparkSession: SparkSession, resourcePath: String): Unit = writeLock { val (fs, qualifiedPath) = { val path = new Path(resourcePath) http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala index 09203e6..ba30bed 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala @@ -75,7 +75,7 @@ object RDDConversions { } /** Logical plan node for scanning data from an RDD. */ -private[sql] case class LogicalRDD( +case class LogicalRDD( output: Seq[Attribute], rdd: RDD[InternalRow])(session: SparkSession) extends LogicalPlan with MultiInstanceRelation { @@ -106,12 +106,12 @@ private[sql] case class LogicalRDD( } /** Physical plan node for scanning data from an RDD. */ -private[sql] case class RDDScanExec( +case class RDDScanExec( output: Seq[Attribute], rdd: RDD[InternalRow], override val nodeName: String) extends LeafExecNode { - private[sql] override lazy val metrics = Map( + override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) protected override def doExecute(): RDD[InternalRow] = { @@ -130,7 +130,7 @@ private[sql] case class RDDScanExec( } } -private[sql] trait DataSourceScanExec extends LeafExecNode { +trait DataSourceScanExec extends LeafExecNode { val rdd: RDD[InternalRow] val relation: BaseRelation val metastoreTableIdentifier: Option[TableIdentifier] @@ -147,7 +147,7 @@ private[sql] trait DataSourceScanExec extends LeafExecNode { } /** Physical plan node for scanning data from a relation. */ -private[sql] case class RowDataSourceScanExec( +case class RowDataSourceScanExec( output: Seq[Attribute], rdd: RDD[InternalRow], @transient relation: BaseRelation, @@ -156,7 +156,7 @@ private[sql] case class RowDataSourceScanExec( override val metastoreTableIdentifier: Option[TableIdentifier]) extends DataSourceScanExec with CodegenSupport { - private[sql] override lazy val metrics = + override lazy val metrics = Map("numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) val outputUnsafeRows = relation match { @@ -222,7 +222,7 @@ private[sql] case class RowDataSourceScanExec( } /** Physical plan node for scanning data from a batched relation. */ -private[sql] case class BatchedDataSourceScanExec( +case class BatchedDataSourceScanExec( output: Seq[Attribute], rdd: RDD[InternalRow], @transient relation: BaseRelation, @@ -231,7 +231,7 @@ private[sql] case class BatchedDataSourceScanExec( override val metastoreTableIdentifier: Option[TableIdentifier]) extends DataSourceScanExec with CodegenSupport { - private[sql] override lazy val metrics = + override lazy val metrics = Map("numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), "scanTime" -> SQLMetrics.createTimingMetric(sparkContext, "scan time")) @@ -337,7 +337,7 @@ private[sql] case class BatchedDataSourceScanExec( } } -private[sql] object DataSourceScanExec { +object DataSourceScanExec { // Metadata keys val INPUT_PATHS = "InputPaths" val PUSHED_FILTERS = "PushedFilters" http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/ExpandExec.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExpandExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExpandExec.scala index 4c046f7..d5603b3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExpandExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExpandExec.scala @@ -39,7 +39,7 @@ case class ExpandExec( child: SparkPlan) extends UnaryExecNode with CodegenSupport { - private[sql] override lazy val metrics = Map( + override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) // The GroupExpressions can output data with arbitrary partitioning, so set it http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/FileRelation.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/FileRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/FileRelation.scala index 7a2a9ee..a299fed 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/FileRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/FileRelation.scala @@ -22,7 +22,7 @@ package org.apache.spark.sql.execution * the list of paths that it returns will be returned to a user who calls `inputPaths` on any * DataFrame that queries this relation. */ -private[sql] trait FileRelation { +trait FileRelation { /** Returns the list of files that will be read when scanning this relation. */ def inputFiles: Array[String] } http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala index 8b62c55..39189a2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala @@ -55,7 +55,7 @@ case class GenerateExec( child: SparkPlan) extends UnaryExecNode { - private[sql] override lazy val metrics = Map( + override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) override def producedAttributes: AttributeSet = AttributeSet(output) http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala index df2f238..9f53a99 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala @@ -26,11 +26,11 @@ import org.apache.spark.sql.execution.metric.SQLMetrics /** * Physical plan node for scanning data from a local collection. */ -private[sql] case class LocalTableScanExec( +case class LocalTableScanExec( output: Seq[Attribute], rows: Seq[InternalRow]) extends LeafExecNode { - private[sql] override lazy val metrics = Map( + override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) private val unsafeRows: Array[InternalRow] = { http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/RowIterator.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/RowIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/RowIterator.scala index 7462dbc..717ff93 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/RowIterator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/RowIterator.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.InternalRow * iterator to consume the next row, whereas RowIterator combines these calls into a single * [[advanceNext()]] method. */ -private[sql] abstract class RowIterator { +abstract class RowIterator { /** * Advance this iterator by a single row. Returns `false` if this iterator has no more rows * and `true` otherwise. If this returns `true`, then the new row can be retrieved by calling http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala index 6cb1a44..ec07aab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.execution.ui.{SparkListenerSQLExecutionEnd, SparkListenerSQLExecutionStart} -private[sql] object SQLExecution { +object SQLExecution { val EXECUTION_ID_KEY = "spark.sql.execution.id" http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala index 66a16ac..cde3ed4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala @@ -22,11 +22,9 @@ import org.apache.spark.executor.TaskMetrics import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode, GenerateUnsafeProjection} +import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.plans.physical.{Distribution, OrderedDistribution, UnspecifiedDistribution} import org.apache.spark.sql.execution.metric.SQLMetrics -import org.apache.spark.sql.types._ -import org.apache.spark.util.collection.unsafe.sort.RadixSort; /** * Performs (external) sorting. @@ -52,7 +50,7 @@ case class SortExec( private val enableRadixSort = sqlContext.conf.enableRadixSort - override private[sql] lazy val metrics = Map( + override lazy val metrics = Map( "sortTime" -> SQLMetrics.createTimingMetric(sparkContext, "sort time"), "peakMemory" -> SQLMetrics.createSizeMetric(sparkContext, "peak memory"), "spillSize" -> SQLMetrics.createSizeMetric(sparkContext, "spill size")) http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/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 045ccc7..79cb409 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 @@ -72,24 +72,24 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ /** * Return all metadata that describes more details of this SparkPlan. */ - private[sql] def metadata: Map[String, String] = Map.empty + def metadata: Map[String, String] = Map.empty /** * Return all metrics containing metrics of this SparkPlan. */ - private[sql] def metrics: Map[String, SQLMetric] = Map.empty + def metrics: Map[String, SQLMetric] = Map.empty /** * Reset all the metrics. */ - private[sql] def resetMetrics(): Unit = { + def resetMetrics(): Unit = { metrics.valuesIterator.foreach(_.reset()) } /** * Return a LongSQLMetric according to the name. */ - private[sql] def longMetric(name: String): SQLMetric = metrics(name) + def longMetric(name: String): SQLMetric = metrics(name) // TODO: Move to `DistributedPlan` /** Specifies how data is partitioned across different nodes in the cluster. */ @@ -395,7 +395,7 @@ object SparkPlan { ThreadUtils.newDaemonCachedThreadPool("subquery", 16)) } -private[sql] trait LeafExecNode extends SparkPlan { +trait LeafExecNode extends SparkPlan { override def children: Seq[SparkPlan] = Nil override def producedAttributes: AttributeSet = outputSet } @@ -407,7 +407,7 @@ object UnaryExecNode { } } -private[sql] trait UnaryExecNode extends SparkPlan { +trait UnaryExecNode extends SparkPlan { def child: SparkPlan override def children: Seq[SparkPlan] = child :: Nil @@ -415,7 +415,7 @@ private[sql] trait UnaryExecNode extends SparkPlan { override def outputPartitioning: Partitioning = child.outputPartitioning } -private[sql] trait BinaryExecNode extends SparkPlan { +trait BinaryExecNode extends SparkPlan { def left: SparkPlan def right: SparkPlan http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala index f84070a..7aa9312 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala @@ -47,7 +47,7 @@ class SparkPlanInfo( } } -private[sql] object SparkPlanInfo { +private[execution] object SparkPlanInfo { def fromSparkPlan(plan: SparkPlan): SparkPlanInfo = { val children = plan match { http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/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 b619d4e..6d7c193 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 @@ -17,7 +17,6 @@ package org.apache.spark.sql.execution -import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD import org.apache.spark.sql.{AnalysisException, Strategy} import org.apache.spark.sql.catalyst.InternalRow @@ -43,13 +42,12 @@ import org.apache.spark.sql.streaming.StreamingQuery * writing libraries should instead consider using the stable APIs provided in * [[org.apache.spark.sql.sources]] */ -@DeveloperApi abstract class SparkStrategy extends GenericStrategy[SparkPlan] { override protected def planLater(plan: LogicalPlan): SparkPlan = PlanLater(plan) } -private[sql] case class PlanLater(plan: LogicalPlan) extends LeafExecNode { +case class PlanLater(plan: LogicalPlan) extends LeafExecNode { override def output: Seq[Attribute] = plan.output @@ -58,7 +56,7 @@ private[sql] case class PlanLater(plan: LogicalPlan) extends LeafExecNode { } } -private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { +abstract class SparkStrategies extends QueryPlanner[SparkPlan] { self: SparkPlanner => /** http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/UnsafeRowSerializer.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/UnsafeRowSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/UnsafeRowSerializer.scala index 4849234..8ab5533 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/UnsafeRowSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/UnsafeRowSerializer.scala @@ -40,12 +40,12 @@ import org.apache.spark.unsafe.Platform * * @param numFields the number of fields in the row being serialized. */ -private[sql] class UnsafeRowSerializer( +class UnsafeRowSerializer( numFields: Int, dataSize: SQLMetric = null) extends Serializer with Serializable { override def newInstance(): SerializerInstance = new UnsafeRowSerializerInstance(numFields, dataSize) - override private[spark] def supportsRelocationOfSerializedObjects: Boolean = true + override def supportsRelocationOfSerializedObjects: Boolean = true } private class UnsafeRowSerializerInstance( http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index ac4c3aa..fb57ed7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -295,7 +295,7 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co override def outputPartitioning: Partitioning = child.outputPartitioning override def outputOrdering: Seq[SortOrder] = child.outputOrdering - override private[sql] lazy val metrics = Map( + override lazy val metrics = Map( "pipelineTime" -> SQLMetrics.createTimingMetric(sparkContext, WholeStageCodegenExec.PIPELINE_DURATION_METRIC)) http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala index 54d7340..cfc47ab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala @@ -54,7 +54,7 @@ case class HashAggregateExec( child.output ++ aggregateBufferAttributes ++ aggregateAttributes ++ aggregateExpressions.flatMap(_.aggregateFunction.inputAggBufferAttributes) - override private[sql] lazy val metrics = Map( + override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), "peakMemory" -> SQLMetrics.createSizeMetric(sparkContext, "peak memory"), "spillSize" -> SQLMetrics.createSizeMetric(sparkContext, "spill size"), http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala index 05dbacf..7c41e5e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala @@ -49,7 +49,7 @@ case class SortAggregateExec( AttributeSet(resultExpressions.diff(groupingExpressions).map(_.toAttribute)) ++ AttributeSet(aggregateBufferAttributes) - override private[sql] lazy val metrics = Map( + override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) override def output: Seq[Attribute] = resultExpressions.map(_.toAttribute) http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala index b047bc0..586e145 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala @@ -204,7 +204,7 @@ sealed trait BufferSetterGetterUtils { /** * A Mutable [[Row]] representing a mutable aggregation buffer. */ -private[sql] class MutableAggregationBufferImpl ( +private[aggregate] class MutableAggregationBufferImpl( schema: StructType, toCatalystConverters: Array[Any => Any], toScalaConverters: Array[Any => Any], @@ -266,7 +266,7 @@ private[sql] class MutableAggregationBufferImpl ( /** * A [[Row]] representing an immutable aggregation buffer. */ -private[sql] class InputAggregationBuffer private[sql] ( +private[aggregate] class InputAggregationBuffer( schema: StructType, toCatalystConverters: Array[Any => Any], toScalaConverters: Array[Any => Any], @@ -319,7 +319,7 @@ private[sql] class InputAggregationBuffer private[sql] ( * The internal wrapper used to hook a [[UserDefinedAggregateFunction]] `udaf` in the * internal aggregation code path. */ -private[sql] case class ScalaUDAF( +case class ScalaUDAF( children: Seq[Expression], udaf: UserDefinedAggregateFunction, mutableAggBufferOffset: Int = 0, http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala index 185c79f..e6f7081 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala @@ -102,7 +102,7 @@ case class FilterExec(condition: Expression, child: SparkPlan) } } - private[sql] override lazy val metrics = Map( + override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) override def inputRDDs(): Seq[RDD[InternalRow]] = { @@ -228,7 +228,7 @@ case class SampleExec( child: SparkPlan) extends UnaryExecNode with CodegenSupport { override def output: Seq[Attribute] = child.output - private[sql] override lazy val metrics = Map( + override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) protected override def doExecute(): RDD[InternalRow] = { @@ -317,7 +317,7 @@ case class RangeExec(range: org.apache.spark.sql.catalyst.plans.logical.Range) override val output: Seq[Attribute] = range.output - private[sql] override lazy val metrics = Map( + override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) // output attributes should not affect the results http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala index 079e122..479934a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -34,7 +34,7 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.util.CollectionAccumulator -private[sql] object InMemoryRelation { +object InMemoryRelation { def apply( useCompression: Boolean, batchSize: Int, @@ -55,15 +55,15 @@ private[sql] object InMemoryRelation { private[columnar] case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow) -private[sql] case class InMemoryRelation( +case class InMemoryRelation( output: Seq[Attribute], useCompression: Boolean, batchSize: Int, storageLevel: StorageLevel, @transient child: SparkPlan, tableName: Option[String])( - @transient private[sql] var _cachedColumnBuffers: RDD[CachedBatch] = null, - private[sql] val batchStats: CollectionAccumulator[InternalRow] = + @transient var _cachedColumnBuffers: RDD[CachedBatch] = null, + val batchStats: CollectionAccumulator[InternalRow] = child.sqlContext.sparkContext.collectionAccumulator[InternalRow]) extends logical.LeafNode with MultiInstanceRelation { http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index 183e494..e63b313 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.types.UserDefinedType -private[sql] case class InMemoryTableScanExec( +case class InMemoryTableScanExec( attributes: Seq[Attribute], predicates: Seq[Expression], @transient relation: InMemoryRelation) @@ -36,7 +36,7 @@ private[sql] case class InMemoryTableScanExec( override protected def innerChildren: Seq[QueryPlan[_]] = Seq(relation) ++ super.innerChildren - private[sql] override lazy val metrics = Map( + override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) override def output: Seq[Attribute] = attributes http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala index 7eaad81..cce1489 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.types._ * A logical command that is executed for its side-effects. `RunnableCommand`s are * wrapped in `ExecutedCommand` during execution. */ -private[sql] trait RunnableCommand extends LogicalPlan with logical.Command { +trait RunnableCommand extends LogicalPlan with logical.Command { override def output: Seq[Attribute] = Seq.empty override def children: Seq[LogicalPlan] = Seq.empty def run(sparkSession: SparkSession): Seq[Row] @@ -45,7 +45,7 @@ private[sql] trait RunnableCommand extends LogicalPlan with logical.Command { * A physical operator that executes the run method of a `RunnableCommand` and * saves the result to prevent multiple executions. */ -private[sql] case class ExecutedCommandExec(cmd: RunnableCommand) extends SparkPlan { +case class ExecutedCommandExec(cmd: RunnableCommand) extends SparkPlan { /** * A concrete command should override this lazy field to wrap up any side effects caused by the * command or any other computation that should be evaluated exactly once. The value of this field http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index 27133f0..bd65d02 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -43,7 +43,7 @@ import org.apache.spark.unsafe.types.UTF8String * Replaces generic operations with specific variants that are designed to work with Spark * SQL Data Sources. */ -private[sql] case class DataSourceAnalysis(conf: CatalystConf) extends Rule[LogicalPlan] { +case class DataSourceAnalysis(conf: CatalystConf) extends Rule[LogicalPlan] { def resolver: Resolver = { if (conf.caseSensitiveAnalysis) { @@ -54,7 +54,7 @@ private[sql] case class DataSourceAnalysis(conf: CatalystConf) extends Rule[Logi } // The access modifier is used to expose this method to tests. - private[sql] def convertStaticPartitions( + def convertStaticPartitions( sourceAttributes: Seq[Attribute], providedPartitions: Map[String, Option[String]], targetAttributes: Seq[Attribute], @@ -202,7 +202,7 @@ private[sql] case class DataSourceAnalysis(conf: CatalystConf) extends Rule[Logi * Replaces [[SimpleCatalogRelation]] with data source table if its table property contains data * source information. */ -private[sql] class FindDataSourceTable(sparkSession: SparkSession) extends Rule[LogicalPlan] { +class FindDataSourceTable(sparkSession: SparkSession) extends Rule[LogicalPlan] { private def readDataSourceTable(sparkSession: SparkSession, table: CatalogTable): LogicalPlan = { val userSpecifiedSchema = DDLUtils.getSchemaFromTableProperties(table) @@ -242,7 +242,7 @@ private[sql] class FindDataSourceTable(sparkSession: SparkSession) extends Rule[ /** * A Strategy for planning scans over data sources defined using the sources API. */ -private[sql] object DataSourceStrategy extends Strategy with Logging { +object DataSourceStrategy extends Strategy with Logging { def apply(plan: LogicalPlan): Seq[execution.SparkPlan] = plan match { case PhysicalOperation(projects, filters, l @ LogicalRelation(t: CatalystScan, _, _)) => pruneFilterProjectRaw( http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala index 8af9562..74510f9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala @@ -54,7 +54,7 @@ import org.apache.spark.sql.execution.SparkPlan * is under the threshold with the addition of the next file, add it. If not, open a new bucket * and add it. Proceed to the next file. */ -private[sql] object FileSourceStrategy extends Strategy with Logging { +object FileSourceStrategy extends Strategy with Logging { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case PhysicalOperation(projects, filters, l @ LogicalRelation(files: HadoopFsRelation, _, table)) => http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoDataSourceCommand.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoDataSourceCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoDataSourceCommand.scala index 8549ae9..b2ff68a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoDataSourceCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoDataSourceCommand.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.sources.InsertableRelation /** * Inserts the results of `query` in to a relation that extends [[InsertableRelation]]. */ -private[sql] case class InsertIntoDataSourceCommand( +case class InsertIntoDataSourceCommand( logicalRelation: LogicalRelation, query: LogicalPlan, overwrite: Boolean) http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala index 1426dcf..d8b8fae 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala @@ -55,7 +55,7 @@ import org.apache.spark.sql.internal.SQLConf * 4. If all tasks are committed, commit the job, otherwise aborts the job; If any exception is * thrown during job commitment, also aborts the job. */ -private[sql] case class InsertIntoHadoopFsRelationCommand( +case class InsertIntoHadoopFsRelationCommand( outputPath: Path, partitionColumns: Seq[Attribute], bucketSpec: Option[BucketSpec], http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala index c356109..5044642 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala @@ -31,6 +31,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Cast, Literal} import org.apache.spark.sql.types._ +// TODO: We should tighten up visibility of the classes here once we clean up Hive coupling. object PartitionDirectory { def apply(values: InternalRow, path: String): PartitionDirectory = @@ -41,22 +42,23 @@ object PartitionDirectory { * Holds a directory in a partitioned collection of files as well as as the partition values * in the form of a Row. Before scanning, the files at `path` need to be enumerated. */ -private[sql] case class PartitionDirectory(values: InternalRow, path: Path) +case class PartitionDirectory(values: InternalRow, path: Path) -private[sql] case class PartitionSpec( +case class PartitionSpec( partitionColumns: StructType, partitions: Seq[PartitionDirectory]) -private[sql] object PartitionSpec { +object PartitionSpec { val emptySpec = PartitionSpec(StructType(Seq.empty[StructField]), Seq.empty[PartitionDirectory]) } -private[sql] object PartitioningUtils { +object PartitioningUtils { // This duplicates default value of Hive `ConfVars.DEFAULTPARTITIONNAME`, since sql/core doesn't // depend on Hive. - private[sql] val DEFAULT_PARTITION_NAME = "__HIVE_DEFAULT_PARTITION__" + val DEFAULT_PARTITION_NAME = "__HIVE_DEFAULT_PARTITION__" - private[sql] case class PartitionValues(columnNames: Seq[String], literals: Seq[Literal]) { + private[datasources] case class PartitionValues(columnNames: Seq[String], literals: Seq[Literal]) + { require(columnNames.size == literals.size) } @@ -83,7 +85,7 @@ private[sql] object PartitioningUtils { * path = "hdfs://<host>:<port>/path/to/partition/a=2/b=world/c=6.28"))) * }}} */ - private[sql] def parsePartitions( + private[datasources] def parsePartitions( paths: Seq[Path], defaultPartitionName: String, typeInference: Boolean, @@ -166,7 +168,7 @@ private[sql] object PartitioningUtils { * hdfs://<host>:<port>/path/to/partition * }}} */ - private[sql] def parsePartition( + private[datasources] def parsePartition( path: Path, defaultPartitionName: String, typeInference: Boolean, @@ -249,7 +251,7 @@ private[sql] object PartitioningUtils { * DoubleType -> StringType * }}} */ - private[sql] def resolvePartitions( + def resolvePartitions( pathsWithPartitionValues: Seq[(Path, PartitionValues)]): Seq[PartitionValues] = { if (pathsWithPartitionValues.isEmpty) { Seq.empty @@ -275,7 +277,7 @@ private[sql] object PartitioningUtils { } } - private[sql] def listConflictingPartitionColumns( + private[datasources] def listConflictingPartitionColumns( pathWithPartitionValues: Seq[(Path, PartitionValues)]): String = { val distinctPartColNames = pathWithPartitionValues.map(_._2.columnNames).distinct @@ -308,7 +310,7 @@ private[sql] object PartitioningUtils { * [[IntegerType]], [[LongType]], [[DoubleType]], [[DecimalType.SYSTEM_DEFAULT]], and * [[StringType]]. */ - private[sql] def inferPartitionColumnValue( + private[datasources] def inferPartitionColumnValue( raw: String, defaultPartitionName: String, typeInference: Boolean): Literal = { http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala index 9a0b46c..e25924b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala @@ -40,14 +40,14 @@ import org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter /** A container for all the details required when writing to a table. */ -case class WriteRelation( +private[datasources] case class WriteRelation( sparkSession: SparkSession, dataSchema: StructType, path: String, prepareJobForWrite: Job => OutputWriterFactory, bucketSpec: Option[BucketSpec]) -private[sql] abstract class BaseWriterContainer( +private[datasources] abstract class BaseWriterContainer( @transient val relation: WriteRelation, @transient private val job: Job, isAppend: Boolean) @@ -234,7 +234,7 @@ private[sql] abstract class BaseWriterContainer( /** * A writer that writes all of the rows in a partition to a single file. */ -private[sql] class DefaultWriterContainer( +private[datasources] class DefaultWriterContainer( relation: WriteRelation, job: Job, isAppend: Boolean) @@ -293,7 +293,7 @@ private[sql] class DefaultWriterContainer( * done by maintaining a HashMap of open files until `maxFiles` is reached. If this occurs, the * writer externally sorts the remaining rows and then writes out them out one file at a time. */ -private[sql] class DynamicPartitionWriterContainer( +private[datasources] class DynamicPartitionWriterContainer( relation: WriteRelation, job: Job, partitionColumns: Seq[Attribute], http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/bucket.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/bucket.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/bucket.scala index 6008d73..2bafe96 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/bucket.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/bucket.scala @@ -31,7 +31,7 @@ private[sql] case class BucketSpec( bucketColumnNames: Seq[String], sortColumnNames: Seq[String]) -private[sql] object BucketingUtils { +object BucketingUtils { // The file name of bucketed data should have 3 parts: // 1. some other information in the head of file name // 2. bucket id part, some numbers, starts with "_" http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala index 22fb816..10fe541 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala @@ -23,7 +23,7 @@ import java.text.SimpleDateFormat import org.apache.spark.internal.Logging import org.apache.spark.sql.execution.datasources.{CompressionCodecs, ParseModes} -private[sql] class CSVOptions(@transient private val parameters: Map[String, String]) +private[csv] class CSVOptions(@transient private val parameters: Map[String, String]) extends Logging with Serializable { private def getChar(paramName: String, default: Char): Char = { http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala index 7929ebb..0a99654 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala @@ -30,7 +30,7 @@ import org.apache.spark.internal.Logging * @param params Parameters object * @param headers headers for the columns */ -private[sql] abstract class CsvReader(params: CSVOptions, headers: Seq[String]) { +private[csv] abstract class CsvReader(params: CSVOptions, headers: Seq[String]) { protected lazy val parser: CsvParser = { val settings = new CsvParserSettings() @@ -60,7 +60,7 @@ private[sql] abstract class CsvReader(params: CSVOptions, headers: Seq[String]) * @param params Parameters object for configuration * @param headers headers for columns */ -private[sql] class LineCsvWriter(params: CSVOptions, headers: Seq[String]) extends Logging { +private[csv] class LineCsvWriter(params: CSVOptions, headers: Seq[String]) extends Logging { private val writerSettings = new CsvWriterSettings private val format = writerSettings.getFormat http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala index 083ac33..10d84f4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala @@ -159,7 +159,7 @@ object CSVRelation extends Logging { } } -private[sql] class CSVOutputWriterFactory(params: CSVOptions) extends OutputWriterFactory { +private[csv] class CSVOutputWriterFactory(params: CSVOptions) extends OutputWriterFactory { override def newInstance( path: String, bucketId: Option[Int], @@ -170,7 +170,7 @@ private[sql] class CSVOutputWriterFactory(params: CSVOptions) extends OutputWrit } } -private[sql] class CsvOutputWriter( +private[csv] class CsvOutputWriter( path: String, dataSchema: StructType, context: TaskAttemptContext, http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala index 438fccb..4351bed 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala @@ -76,7 +76,7 @@ abstract class OutputWriterFactory extends Serializable { * through the [[OutputWriterFactory]] implementation. * @since 2.0.0 */ - private[sql] def newWriter(path: String): OutputWriter = { + def newWriter(path: String): OutputWriter = { throw new UnsupportedOperationException("newInstance with just path not supported") } } @@ -249,7 +249,7 @@ trait FileFormat { * appends partition values to [[InternalRow]]s produced by the reader function [[buildReader]] * returns. */ - private[sql] def buildReaderWithPartitionValues( + def buildReaderWithPartitionValues( sparkSession: SparkSession, dataSchema: StructType, partitionSchema: StructType, @@ -356,7 +356,7 @@ trait FileCatalog { /** * Helper methods for gathering metadata from HDFS. */ -private[sql] object HadoopFsRelation extends Logging { +object HadoopFsRelation extends Logging { /** Checks if we should filter out this path name. */ def shouldFilterOut(pathName: String): Boolean = { http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala index 24e2c1a..f655155 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala @@ -38,11 +38,11 @@ import org.apache.spark.unsafe.types.UTF8String /** * Data corresponding to one partition of a JDBCRDD. */ -private[sql] case class JDBCPartition(whereClause: String, idx: Int) extends Partition { +case class JDBCPartition(whereClause: String, idx: Int) extends Partition { override def index: Int = idx } -private[sql] object JDBCRDD extends Logging { +object JDBCRDD extends Logging { /** * Maps a JDBC type to a Catalyst type. This function is called only when @@ -192,7 +192,7 @@ private[sql] object JDBCRDD extends Logging { * Turns a single Filter into a String representing a SQL expression. * Returns None for an unhandled filter. */ - private[jdbc] def compileFilter(f: Filter): Option[String] = { + def compileFilter(f: Filter): Option[String] = { Option(f match { case EqualTo(attr, value) => s"$attr = ${compileValue(value)}" case EqualNullSafe(attr, value) => @@ -275,7 +275,7 @@ private[sql] object JDBCRDD extends Logging { * driver code and the workers must be able to access the database; the driver * needs to fetch the schema while the workers need to fetch the data. */ -private[sql] class JDBCRDD( +private[jdbc] class JDBCRDD( sc: SparkContext, getConnection: () => Connection, schema: StructType, http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index 260071e..9498088 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -52,7 +52,7 @@ import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ import org.apache.spark.util.SerializableConfiguration -private[sql] class ParquetFileFormat +class ParquetFileFormat extends FileFormat with DataSourceRegister with Logging @@ -268,7 +268,7 @@ private[sql] class ParquetFileFormat true } - override private[sql] def buildReaderWithPartitionValues( + override def buildReaderWithPartitionValues( sparkSession: SparkSession, dataSchema: StructType, partitionSchema: StructType, @@ -429,7 +429,7 @@ private[sql] class ParquetFileFormat * writes the data to the path used to generate the output writer. Callers of this factory * has to ensure which files are to be considered as committed. */ -private[sql] class ParquetOutputWriterFactory( +private[parquet] class ParquetOutputWriterFactory( sqlConf: SQLConf, dataSchema: StructType, hadoopConf: Configuration, @@ -478,7 +478,7 @@ private[sql] class ParquetOutputWriterFactory( * Returns a [[OutputWriter]] that writes data to the give path without using * [[OutputCommitter]]. */ - override private[sql] def newWriter(path: String): OutputWriter = new OutputWriter { + override def newWriter(path: String): OutputWriter = new OutputWriter { // Create TaskAttemptContext that is used to pass on Configuration to the ParquetRecordWriter private val hadoopTaskAttempId = new TaskAttemptID(new TaskID(new JobID, TaskType.MAP, 0), 0) @@ -525,7 +525,7 @@ private[sql] class ParquetOutputWriterFactory( // NOTE: This class is instantiated and used on executor side only, no need to be serializable. -private[sql] class ParquetOutputWriter( +private[parquet] class ParquetOutputWriter( path: String, bucketId: Option[Int], context: TaskAttemptContext) @@ -563,12 +563,13 @@ private[sql] class ParquetOutputWriter( override def write(row: Row): Unit = throw new UnsupportedOperationException("call writeInternal") - override protected[sql] def writeInternal(row: InternalRow): Unit = recordWriter.write(null, row) + override def writeInternal(row: InternalRow): Unit = recordWriter.write(null, row) override def close(): Unit = recordWriter.close(context) } -private[sql] object ParquetFileFormat extends Logging { + +object ParquetFileFormat extends Logging { /** * If parquet's block size (row group size) setting is larger than the min split size, * we use parquet's block size setting as the min split size. Otherwise, we will create @@ -715,7 +716,7 @@ private[sql] object ParquetFileFormat extends Logging { * distinguish binary and string). This method generates a correct schema by merging Metastore * schema data types and Parquet schema field names. */ - private[sql] def mergeMetastoreParquetSchema( + def mergeMetastoreParquetSchema( metastoreSchema: StructType, parquetSchema: StructType): StructType = { def schemaConflictMessage: String = http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala index 70ae829..2edd275 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala @@ -28,7 +28,7 @@ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName import org.apache.spark.sql.sources import org.apache.spark.sql.types._ -private[sql] object ParquetFilters { +object ParquetFilters { case class SetInFilter[T <: Comparable[T]]( valueSet: Set[T]) extends UserDefinedPredicate[T] with Serializable { http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala index dd2e915..3eec582 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.internal.SQLConf /** * Options for the Parquet data source. */ -private[sql] class ParquetOptions( +private[parquet] class ParquetOptions( @transient private val parameters: Map[String, String], @transient private val sqlConf: SQLConf) extends Serializable { @@ -56,8 +56,8 @@ private[sql] class ParquetOptions( } -private[sql] object ParquetOptions { - private[sql] val MERGE_SCHEMA = "mergeSchema" +object ParquetOptions { + val MERGE_SCHEMA = "mergeSchema" // The parquet compression short names private val shortParquetCompressionCodecNames = Map( http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index 15b9d14..05908d9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.sources.{BaseRelation, InsertableRelation} /** * Try to replaces [[UnresolvedRelation]]s with [[ResolveDataSource]]. */ -private[sql] class ResolveDataSource(sparkSession: SparkSession) extends Rule[LogicalPlan] { +class ResolveDataSource(sparkSession: SparkSession) extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { case u: UnresolvedRelation if u.tableIdentifier.database.isDefined => try { @@ -67,7 +67,7 @@ private[sql] class ResolveDataSource(sparkSession: SparkSession) extends Rule[Lo * table. It also does data type casting and field renaming, to make sure that the columns to be * inserted have the correct data type and fields have the correct names. */ -private[sql] case class PreprocessTableInsertion(conf: SQLConf) extends Rule[LogicalPlan] { +case class PreprocessTableInsertion(conf: SQLConf) extends Rule[LogicalPlan] { private def preprocess( insert: InsertIntoTable, tblName: String, @@ -147,7 +147,7 @@ private[sql] case class PreprocessTableInsertion(conf: SQLConf) extends Rule[Log /** * A rule to do various checks before inserting into or writing to a data source table. */ -private[sql] case class PreWriteCheck(conf: SQLConf, catalog: SessionCatalog) +case class PreWriteCheck(conf: SQLConf, catalog: SessionCatalog) extends (LogicalPlan => Unit) { def failAnalysis(msg: String): Unit = { throw new AnalysisException(msg) } http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala index e89f792..082f97a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala @@ -104,7 +104,7 @@ package object debug { } } - private[sql] case class DebugExec(child: SparkPlan) extends UnaryExecNode with CodegenSupport { + case class DebugExec(child: SparkPlan) extends UnaryExecNode with CodegenSupport { def output: Seq[Attribute] = child.output class SetAccumulator[T] extends AccumulatorV2[T, HashSet[T]] { http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala index bd0841d..a809076 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala @@ -38,7 +38,7 @@ case class BroadcastExchangeExec( mode: BroadcastMode, child: SparkPlan) extends Exchange { - override private[sql] lazy val metrics = Map( + override lazy val metrics = Map( "dataSize" -> SQLMetrics.createMetric(sparkContext, "data size (bytes)"), "collectTime" -> SQLMetrics.createMetric(sparkContext, "time to collect (ms)"), "buildTime" -> SQLMetrics.createMetric(sparkContext, "time to build (ms)"), http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ExchangeCoordinator.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ExchangeCoordinator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ExchangeCoordinator.scala index 2ea6ee3..57da85f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ExchangeCoordinator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ExchangeCoordinator.scala @@ -79,7 +79,7 @@ import org.apache.spark.sql.execution.{ShuffledRowRDD, SparkPlan} * - post-shuffle partition 1: pre-shuffle partition 2 * - post-shuffle partition 2: pre-shuffle partition 3 and 4 */ -private[sql] class ExchangeCoordinator( +class ExchangeCoordinator( numExchanges: Int, advisoryTargetPostShuffleInputSize: Long, minNumPostShufflePartitions: Option[Int] = None) @@ -112,7 +112,7 @@ private[sql] class ExchangeCoordinator( * Estimates partition start indices for post-shuffle partitions based on * mapOutputStatistics provided by all pre-shuffle stages. */ - private[sql] def estimatePartitionStartIndices( + def estimatePartitionStartIndices( mapOutputStatistics: Array[MapOutputStatistics]): Array[Int] = { // If we have mapOutputStatistics.length < numExchange, it is because we do not submit // a stage when the number of partitions of this dependency is 0. http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchange.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchange.scala index afe0fbe..7a4a251 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchange.scala @@ -40,7 +40,7 @@ case class ShuffleExchange( child: SparkPlan, @transient coordinator: Option[ExchangeCoordinator]) extends Exchange { - override private[sql] lazy val metrics = Map( + override lazy val metrics = Map( "dataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size")) override def nodeName: String = { @@ -81,7 +81,8 @@ case class ShuffleExchange( * the partitioning scheme defined in `newPartitioning`. Those partitions of * the returned ShuffleDependency will be the input of shuffle. */ - private[sql] def prepareShuffleDependency(): ShuffleDependency[Int, InternalRow, InternalRow] = { + private[exchange] def prepareShuffleDependency() + : ShuffleDependency[Int, InternalRow, InternalRow] = { ShuffleExchange.prepareShuffleDependency( child.execute(), child.output, newPartitioning, serializer) } @@ -92,7 +93,7 @@ case class ShuffleExchange( * partition start indices array. If this optional array is defined, the returned * [[ShuffledRowRDD]] will fetch pre-shuffle partitions based on indices of this array. */ - private[sql] def preparePostShuffleRDD( + private[exchange] def preparePostShuffleRDD( shuffleDependency: ShuffleDependency[Int, InternalRow, InternalRow], specifiedPartitionStartIndices: Option[Array[Int]] = None): ShuffledRowRDD = { // If an array of partition start indices is provided, we need to use this array @@ -194,7 +195,7 @@ object ShuffleExchange { * the partitioning scheme defined in `newPartitioning`. Those partitions of * the returned ShuffleDependency will be the input of shuffle. */ - private[sql] def prepareShuffleDependency( + def prepareShuffleDependency( rdd: RDD[InternalRow], outputAttributes: Seq[Attribute], newPartitioning: Partitioning, http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala index 7c194ab..0f24baa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala @@ -45,7 +45,7 @@ case class BroadcastHashJoinExec( right: SparkPlan) extends BinaryExecNode with HashJoin with CodegenSupport { - override private[sql] lazy val metrics = Map( + override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) override def requiredChildDistribution: Seq[Distribution] = { http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala index 4d43765..6a9965f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala @@ -37,7 +37,7 @@ case class BroadcastNestedLoopJoinExec( condition: Option[Expression], withinBroadcastThreshold: Boolean = true) extends BinaryExecNode { - override private[sql] lazy val metrics = Map( + override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) /** BuildRight means the right relation <=> the broadcast relation. */ http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala index 3a0b6ef..c97fffe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala @@ -34,7 +34,6 @@ import org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter * will be much faster than building the right partition for every row in left RDD, it also * materialize the right RDD (in case of the right RDD is nondeterministic). */ -private[spark] class UnsafeCartesianRDD(left : RDD[UnsafeRow], right : RDD[UnsafeRow], numFieldsOfRight: Int) extends CartesianRDD[UnsafeRow, UnsafeRow](left.sparkContext, left, right) { @@ -78,7 +77,7 @@ class UnsafeCartesianRDD(left : RDD[UnsafeRow], right : RDD[UnsafeRow], numField for (x <- rdd1.iterator(partition.s1, context); y <- createIter()) yield (x, y) CompletionIterator[(UnsafeRow, UnsafeRow), Iterator[(UnsafeRow, UnsafeRow)]]( - resultIter, sorter.cleanupResources) + resultIter, sorter.cleanupResources()) } } @@ -89,7 +88,7 @@ case class CartesianProductExec( condition: Option[Expression]) extends BinaryExecNode { override def output: Seq[Attribute] = left.output ++ right.output - override private[sql] lazy val metrics = Map( + override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) protected override def doPrepare(): Unit = { http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala index 0036f9a..afb6e5e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala @@ -39,7 +39,7 @@ case class ShuffledHashJoinExec( right: SparkPlan) extends BinaryExecNode with HashJoin { - override private[sql] lazy val metrics = Map( + override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), "buildDataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size of build side"), "buildTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to build hash map")) http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala index fac6b8d..5c9c1e6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala @@ -40,7 +40,7 @@ case class SortMergeJoinExec( left: SparkPlan, right: SparkPlan) extends BinaryExecNode with CodegenSupport { - override private[sql] lazy val metrics = Map( + override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) override def output: Seq[Attribute] = { http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala index 9817a56..15afa0b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala @@ -55,17 +55,17 @@ class SQLMetric(val metricType: String, initValue: Long = 0L) extends Accumulato override def value: Long = _value // Provide special identifier as metadata so we can tell that this is a `SQLMetric` later - private[spark] override def toInfo(update: Option[Any], value: Option[Any]): AccumulableInfo = { + override def toInfo(update: Option[Any], value: Option[Any]): AccumulableInfo = { new AccumulableInfo( id, name, update, value, true, true, Some(AccumulatorContext.SQL_ACCUM_IDENTIFIER)) } } -private[sql] object SQLMetrics { - private[sql] val SUM_METRIC = "sum" - private[sql] val SIZE_METRIC = "size" - private[sql] val TIMING_METRIC = "timing" +object SQLMetrics { + private val SUM_METRIC = "sum" + private val SIZE_METRIC = "size" + private val TIMING_METRIC = "timing" def createMetric(sc: SparkContext, name: String): SQLMetric = { val acc = new SQLMetric(SUM_METRIC) http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala index 829bcae..16e4484 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.execution.SparkPlan * Extracts all the Python UDFs in logical aggregate, which depends on aggregate expression or * grouping key, evaluate them after aggregate. */ -private[spark] object ExtractPythonUDFFromAggregate extends Rule[LogicalPlan] { +object ExtractPythonUDFFromAggregate extends Rule[LogicalPlan] { /** * Returns whether the expression could only be evaluated within aggregate. @@ -90,7 +90,7 @@ private[spark] object ExtractPythonUDFFromAggregate extends Rule[LogicalPlan] { * This has the limitation that the input to the Python UDF is not allowed include attributes from * multiple child operators. */ -private[spark] object ExtractPythonUDFs extends Rule[SparkPlan] { +object ExtractPythonUDFs extends Rule[SparkPlan] { private def hasPythonUDF(e: Expression): Boolean = { e.find(_.isInstanceOf[PythonUDF]).isDefined http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala index 70539da..d2178e9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala @@ -21,12 +21,12 @@ import org.apache.spark.api.r._ import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.api.r.SQLUtils._ import org.apache.spark.sql.Row -import org.apache.spark.sql.types.{BinaryType, StructField, StructType} +import org.apache.spark.sql.types.StructType /** * A function wrapper that applies the given R function to each partition. */ -private[sql] case class MapPartitionsRWrapper( +case class MapPartitionsRWrapper( func: Array[Byte], packageNames: Array[Byte], broadcastVars: Array[Broadcast[Object]], http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala index b19344f..b9dbfcf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.{Column, DataFrame, Dataset, Row} import org.apache.spark.sql.catalyst.plans.logical.LocalRelation import org.apache.spark.sql.types._ -private[sql] object FrequentItems extends Logging { +object FrequentItems extends Logging { /** A helper class wrapping `MutableMap[Any, Long]` for simplicity. */ private class FreqItemCounter(size: Int) extends Serializable { @@ -79,7 +79,7 @@ private[sql] object FrequentItems extends Logging { * than 1e-4. * @return A Local DataFrame with the Array of frequent items for each column. */ - private[sql] def singlePassFreqItems( + def singlePassFreqItems( df: DataFrame, cols: Seq[String], support: Double): DataFrame = { http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala index ea58df7..50eecb4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String -private[sql] object StatFunctions extends Logging { +object StatFunctions extends Logging { import QuantileSummaries.Stats @@ -337,7 +337,7 @@ private[sql] object StatFunctions extends Logging { } /** Calculate the Pearson Correlation Coefficient for the given columns */ - private[sql] def pearsonCorrelation(df: DataFrame, cols: Seq[String]): Double = { + def pearsonCorrelation(df: DataFrame, cols: Seq[String]): Double = { val counts = collectStatisticalData(df, cols, "correlation") counts.Ck / math.sqrt(counts.MkX * counts.MkY) } @@ -407,13 +407,13 @@ private[sql] object StatFunctions extends Logging { * @param cols the column names * @return the covariance of the two columns. */ - private[sql] def calculateCov(df: DataFrame, cols: Seq[String]): Double = { + def calculateCov(df: DataFrame, cols: Seq[String]): Double = { val counts = collectStatisticalData(df, cols, "covariance") counts.cov } /** Generate a table of frequencies for the elements of two columns. */ - private[sql] def crossTabulate(df: DataFrame, col1: String, col2: String): DataFrame = { + def crossTabulate(df: DataFrame, col1: String, col2: String): DataFrame = { val tableName = s"${col1}_$col2" val counts = df.groupBy(col1, col2).agg(count("*")).take(1e6.toInt) if (counts.length == 1e6.toInt) { http://git-wip-us.apache.org/repos/asf/spark/blob/1c569711/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala index 7367c68..05294df 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.streaming.OutputMode * A variant of [[QueryExecution]] that allows the execution of the given [[LogicalPlan]] * plan incrementally. Possibly preserving state in between each execution. */ -class IncrementalExecution private[sql]( +class IncrementalExecution( sparkSession: SparkSession, logicalPlan: LogicalPlan, val outputMode: OutputMode, --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org