Skip to content

Commit

Permalink
add v1 read fallback API in DS v2
Browse files Browse the repository at this point in the history
  • Loading branch information
cloud-fan committed Oct 28, 2019
1 parent cbe6ead commit bb195f3
Show file tree
Hide file tree
Showing 6 changed files with 298 additions and 32 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -31,8 +31,8 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap
class BasicInMemoryTableCatalog extends TableCatalog {
import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._

protected val tables: util.Map[Identifier, InMemoryTable] =
new ConcurrentHashMap[Identifier, InMemoryTable]()
protected val tables: util.Map[Identifier, Table] =
new ConcurrentHashMap[Identifier, Table]()

private var _name: Option[String] = None

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,43 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.sql.connector.read

import org.apache.spark.annotation.{Experimental, Unstable}
import org.apache.spark.sql.sources.BaseRelation

/**
* A trait that should be implemented by V1 DataSources that would like to leverage the DataSource
* V2 read code paths.
*
* This interface is designed to provide Spark DataSources time to migrate to DataSource V2 and
* will be removed in a future Spark release.
*
* @since 3.0.0
*/
@Experimental
@Unstable
trait V1Scan extends Scan {

/**
* Creates an `BaseRelation` that can scan data from DataSource v1 to RDD[Row]. The returned
* relation must be a `TableScan` instance.
*
* @since 3.0.0
*/
def toV1Relation(): BaseRelation
}
Original file line number Diff line number Diff line change
Expand Up @@ -97,17 +97,14 @@ trait DataSourceScanExec extends LeafExecNode {

/** Physical plan node for scanning data from a relation. */
case class RowDataSourceScanExec(
fullOutput: Seq[Attribute],
requiredColumnsIndex: Seq[Int],
output: Seq[Attribute],
filters: Set[Filter],
handledFilters: Set[Filter],
rdd: RDD[InternalRow],
@transient relation: BaseRelation,
override val tableIdentifier: Option[TableIdentifier])
extends DataSourceScanExec with InputRDDCodegen {

def output: Seq[Attribute] = requiredColumnsIndex.map(fullOutput)

override lazy val metrics =
Map("numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"))

Expand Down Expand Up @@ -141,7 +138,8 @@ case class RowDataSourceScanExec(
// Don't care about `rdd` and `tableIdentifier` when canonicalizing.
override def doCanonicalize(): SparkPlan =
copy(
fullOutput.map(QueryPlan.normalizeExpressions(_, fullOutput)),
// Only the required column names matter when checking equality.
output.map(a => a.withExprId(ExprId(-1))),
rdd = null,
tableIdentifier = None)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -290,7 +290,6 @@ case class DataSourceStrategy(conf: SQLConf) extends Strategy with Logging with
case l @ LogicalRelation(baseRelation: TableScan, _, _, _) =>
RowDataSourceScanExec(
l.output,
l.output.indices,
Set.empty,
Set.empty,
toCatalystRDD(l, baseRelation.buildScan()),
Expand Down Expand Up @@ -362,8 +361,7 @@ case class DataSourceStrategy(conf: SQLConf) extends Strategy with Logging with
.map(relation.attributeMap)

val scan = RowDataSourceScanExec(
relation.output,
requestedColumns.map(relation.output.indexOf),
requestedColumns,
pushedFilters.toSet,
handledFilters,
scanBuilder(requestedColumns, candidatePredicates, pushedFilters),
Expand All @@ -384,8 +382,7 @@ case class DataSourceStrategy(conf: SQLConf) extends Strategy with Logging with
(projectSet ++ filterSet -- handledSet).map(relation.attributeMap).toSeq

val scan = RowDataSourceScanExec(
relation.output,
requestedColumns.map(relation.output.indexOf),
requestedColumns,
pushedFilters.toSet,
handledFilters,
scanBuilder(requestedColumns, candidatePredicates, pushedFilters),
Expand All @@ -403,14 +400,7 @@ case class DataSourceStrategy(conf: SQLConf) extends Strategy with Logging with
relation: LogicalRelation,
output: Seq[Attribute],
rdd: RDD[Row]): RDD[InternalRow] = {
if (relation.relation.needConversion) {
val converters = RowEncoder(StructType.fromAttributes(output))
rdd.mapPartitions { iterator =>
iterator.map(converters.toRow)
}
} else {
rdd.asInstanceOf[RDD[InternalRow]]
}
DataSourceStrategy.toCatalystRDD(relation.relation, output, rdd)
}

/**
Expand Down Expand Up @@ -618,4 +608,21 @@ object DataSourceStrategy {

(nonconvertiblePredicates ++ unhandledPredicates, pushedFilters, handledFilters)
}

/**
* Convert RDD of Row into RDD of InternalRow with objects in catalyst types
*/
private[sql] def toCatalystRDD(
relation: BaseRelation,
output: Seq[Attribute],
rdd: RDD[Row]): RDD[InternalRow] = {
if (relation.needConversion) {
val converters = RowEncoder(StructType.fromAttributes(output))
rdd.mapPartitions { iterator =>
iterator.map(converters.toRow)
}
} else {
rdd.asInstanceOf[RDD[InternalRow]]
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -25,12 +25,13 @@ import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, Attri
import org.apache.spark.sql.catalyst.planning.PhysicalOperation
import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, AppendData, CreateNamespace, CreateTableAsSelect, CreateV2Table, DeleteFromTable, DescribeTable, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, Repartition, ReplaceTable, ReplaceTableAsSelect, SetCatalogAndNamespace, ShowNamespaces, ShowTables}
import org.apache.spark.sql.connector.catalog.{StagingTableCatalog, TableCapability}
import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownFilters, SupportsPushDownRequiredColumns}
import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownFilters, SupportsPushDownRequiredColumns, V1Scan}
import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBatchStream}
import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SparkPlan}
import org.apache.spark.sql.execution.{FilterExec, ProjectExec, RowDataSourceScanExec, SparkPlan}
import org.apache.spark.sql.execution.datasources.DataSourceStrategy
import org.apache.spark.sql.execution.streaming.continuous.{ContinuousCoalesceExec, WriteToContinuousDataSource, WriteToContinuousDataSourceExec}
import org.apache.spark.sql.sources
import org.apache.spark.sql.sources.{Filter, TableScan}
import org.apache.spark.sql.util.CaseInsensitiveStringMap

object DataSourceV2Strategy extends Strategy with PredicateHelper {
Expand All @@ -42,7 +43,7 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper {
*/
private def pushFilters(
scanBuilder: ScanBuilder,
filters: Seq[Expression]): (Seq[Expression], Seq[Expression]) = {
filters: Seq[Expression]): (Seq[Filter], Seq[Expression]) = {
scanBuilder match {
case r: SupportsPushDownFilters =>
// A map from translated data source leaf node filters to original catalyst filter
Expand Down Expand Up @@ -70,11 +71,7 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper {
val postScanFilters = r.pushFilters(translatedFilters.toArray).map { filter =>
DataSourceStrategy.rebuildExpressionFromFilter(filter, translatedFilterToExpr)
}
// The filters which are marked as pushed to this data source
val pushedFilters = r.pushedFilters().map { filter =>
DataSourceStrategy.rebuildExpressionFromFilter(filter, translatedFilterToExpr)
}
(pushedFilters, untranslatableExprs ++ postScanFilters)
(r.pushedFilters(), untranslatableExprs ++ postScanFilters)

case _ => (Nil, filters)
}
Expand All @@ -83,7 +80,7 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper {
/**
* Applies column pruning to the data source, w.r.t. the references of the given expressions.
*
* @return the created `ScanConfig`(since column pruning is the last step of operator pushdown),
* @return the physical plan (since column pruning is the last step of operator pushdown),
* and new output attributes after column pruning.
*/
// TODO: nested column pruning.
Expand Down Expand Up @@ -136,12 +133,39 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper {
|Output: ${output.mkString(", ")}
""".stripMargin)

val batchExec = BatchScanExec(output, scan)
val (scanExec, needsUnsafeConversion) = scan match {
case v1Scan: V1Scan =>
val v1Relation = v1Scan.toV1Relation()
if (v1Relation.schema != output.toStructType) {
throw new IllegalArgumentException(
"The fallback v1 relation reports inconsistent schema:\n" +
"Schema of v2 scan: " + output.toStructType + "\n" +
"Schema of v1 relation: " + v1Relation.schema)
}
val rdd = v1Relation match {
case s: TableScan => s.buildScan()
case _ =>
throw new IllegalArgumentException(
"`V1Scan.toV1Relation` must return a `TableScan` instance.")
}
val unsafeRowRDD = DataSourceStrategy.toCatalystRDD(v1Relation, output, rdd)
val dsScan = RowDataSourceScanExec(
output,
pushedFilters.toSet,
pushedFilters.toSet,
unsafeRowRDD,
v1Relation,
tableIdentifier = None)
(dsScan, false)
case _ =>
val batchScan = BatchScanExec(output, scan)
(batchScan, !batchScan.supportsColumnar)
}

val filterCondition = postScanFilters.reduceLeftOption(And)
val withFilter = filterCondition.map(FilterExec(_, batchExec)).getOrElse(batchExec)
val withFilter = filterCondition.map(FilterExec(_, scanExec)).getOrElse(scanExec)

val withProjection = if (withFilter.output != project || !batchExec.supportsColumnar) {
val withProjection = if (withFilter.output != project || needsUnsafeConversion) {
ProjectExec(project, withFilter)
} else {
withFilter
Expand Down
Loading

0 comments on commit bb195f3

Please sign in to comment.