cloud-fan commented on a change in pull request #26231: [SPARK-29572][SQL] add 
v1 read fallback API in DS v2
URL: https://github.com/apache/spark/pull/26231#discussion_r367230956
 
 

 ##########
 File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
 ##########
 @@ -19,40 +19,72 @@ package org.apache.spark.sql.execution.datasources.v2
 
 import scala.collection.JavaConverters._
 
-import org.apache.spark.sql.{AnalysisException, Strategy}
+import org.apache.spark.sql.{AnalysisException, SparkSession, Strategy}
 import org.apache.spark.sql.catalyst.analysis.{ResolvedNamespace, 
ResolvedTable}
-import org.apache.spark.sql.catalyst.expressions.{And, PredicateHelper, 
SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.{And, Expression, 
NamedExpression, PredicateHelper, SubqueryExpression}
 import org.apache.spark.sql.catalyst.planning.PhysicalOperation
 import org.apache.spark.sql.catalyst.plans.logical.{AlterNamespaceSetLocation, 
AlterNamespaceSetOwner, AlterNamespaceSetProperties, AlterTable, AppendData, 
CommentOnNamespace, CommentOnTable, CreateNamespace, CreateTableAsSelect, 
CreateV2Table, DeleteFromTable, DescribeNamespace, DescribeTable, 
DropNamespace, DropTable, LogicalPlan, OverwriteByExpression, 
OverwritePartitionsDynamic, RefreshTable, RenameTable, Repartition, 
ReplaceTable, ReplaceTableAsSelect, SetCatalogAndNamespace, 
ShowCurrentNamespace, ShowNamespaces, ShowTableProperties, ShowTables}
-import org.apache.spark.sql.connector.catalog.{Identifier, 
StagingTableCatalog, SupportsNamespaces, TableCapability, TableCatalog, 
TableChange}
+import org.apache.spark.sql.connector.catalog.{StagingTableCatalog, 
SupportsNamespaces, TableCapability, TableCatalog, TableChange}
+import org.apache.spark.sql.connector.read.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, LeafExecNode, 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.{BaseRelation, TableScan}
 import org.apache.spark.sql.util.CaseInsensitiveStringMap
 
-object DataSourceV2Strategy extends Strategy with PredicateHelper {
+class DataSourceV2Strategy(session: SparkSession) extends Strategy with 
PredicateHelper {
 
   import DataSourceV2Implicits._
   import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
 
+  private def withProjectAndFilter(
+      project: Seq[NamedExpression],
+      filters: Seq[Expression],
+      scan: LeafExecNode,
+      needsUnsafeConversion: Boolean): SparkPlan = {
+    val filterCondition = filters.reduceLeftOption(And)
+    val withFilter = filterCondition.map(FilterExec(_, scan)).getOrElse(scan)
+
+    if (withFilter.output != project || needsUnsafeConversion) {
+      ProjectExec(project, withFilter)
+    } else {
+      withFilter
+    }
+  }
+
   override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
+    case PhysicalOperation(project, filters,
+        relation @ DataSourceV2ScanRelation(_, v1Scan: V1Scan, output)) =>
+      val pushedFilters = 
relation.getTagValue(V2ScanRelationPushDown.PUSHED_FILTERS_TAG)
+        .getOrElse(Seq.empty)
+      val v1Relation = v1Scan.toV1TableScan[BaseRelation with 
TableScan](session.sqlContext)
+      if (v1Relation.schema != v1Scan.readSchema()) {
+        throw new IllegalArgumentException(
+          "The fallback v1 relation reports inconsistent schema:\n" +
+            "Schema of v2 scan:     " + v1Scan.readSchema() + "\n" +
+            "Schema of v1 relation: " + v1Relation.schema)
+      }
+      val rdd = v1Relation.buildScan()
+      val unsafeRowRDD = DataSourceStrategy.toCatalystRDD(v1Relation, output, 
rdd)
+      val originalOutputNames = relation.table.schema().map(_.name)
+      val requiredColumnsIndex = 
output.map(_.name).map(originalOutputNames.indexOf)
+      val dsScan = RowDataSourceScanExec(
+        output,
+        requiredColumnsIndex,
+        pushedFilters.toSet,
 
 Review comment:
   Yes it should be the entire set of filters, but it's not a big deal. 
`RowDataSourceScanExec.filters` is only used in `toString`, to let people know 
which filters are pushed but not accepted by the source.
   
   Anyway we can retain the full filter set like the pushed filters, I'll fix 
it.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to