[GitHub] [spark] brkyvz commented on a change in pull request #26231: [SPARK-29572][SQL] add v1 read fallback API in DS v2

2020-01-15 Thread GitBox
brkyvz 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_r366990811
 
 

 ##
 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:
   this is incorrect, right? There were other filters that weren't handled. As 
I understand, this should be the entire set of filters


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-mai

[GitHub] [spark] brkyvz commented on a change in pull request #26231: [SPARK-29572][SQL] add v1 read fallback API in DS v2

2020-01-15 Thread GitBox
brkyvz 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_r366994522
 
 

 ##
 File path: 
sql/core/src/test/scala/org/apache/spark/sql/connector/V1ReadFallbackSuite.scala
 ##
 @@ -0,0 +1,199 @@
+/*
+ * 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
+
+import java.util
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.{DataFrame, QueryTest, Row, SparkSession, 
SQLContext}
+import org.apache.spark.sql.connector.catalog.{Identifier, SupportsRead, 
Table, TableCapability, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, 
SupportsPushDownFilters, SupportsPushDownRequiredColumns, V1Scan}
+import org.apache.spark.sql.execution.RowDataSourceScanExec
+import org.apache.spark.sql.sources.{BaseRelation, Filter, GreaterThan, 
TableScan}
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+abstract class V1ReadFallbackSuite extends QueryTest with SharedSparkSession {
+  protected def baseTableScan(): DataFrame
+
+  test("full scan") {
+val df = baseTableScan()
+val v1Scan = df.queryExecution.executedPlan.collect {
+  case s: RowDataSourceScanExec => s
+}
+assert(v1Scan.length == 1)
+checkAnswer(df, Seq(Row(1, 10), Row(2, 20), Row(3, 30)))
+  }
+
+  test("column pruning") {
+val df = baseTableScan().select("i")
+val v1Scan = df.queryExecution.executedPlan.collect {
+  case s: RowDataSourceScanExec => s
+}
+assert(v1Scan.length == 1)
+assert(v1Scan.head.output.map(_.name) == Seq("i"))
+checkAnswer(df, Seq(Row(1), Row(2), Row(3)))
+  }
+
+  test("filter push down") {
+val df = baseTableScan().filter("i > 1 and j < 30")
+val v1Scan = df.queryExecution.executedPlan.collect {
+  case s: RowDataSourceScanExec => s
+}
+assert(v1Scan.length == 1)
+// `j < 30` can't be pushed.
+assert(v1Scan.head.handledFilters.size == 1)
+checkAnswer(df, Seq(Row(2, 20)))
+  }
+
+  test("filter push down + column pruning") {
+val df = baseTableScan().filter("i > 1").select("i")
+val v1Scan = df.queryExecution.executedPlan.collect {
+  case s: RowDataSourceScanExec => s
+}
+assert(v1Scan.length == 1)
+assert(v1Scan.head.output.map(_.name) == Seq("i"))
+assert(v1Scan.head.handledFilters.size == 1)
+checkAnswer(df, Seq(Row(2), Row(3)))
+  }
+}
+
+class V1ReadFallbackWithDataFrameReaderSuite extends V1ReadFallbackSuite {
+  override protected def baseTableScan(): DataFrame = {
+spark.read.format(classOf[V1ReadFallbackTableProvider].getName).load()
+  }
+}
+
+class V1ReadFallbackWithCatalogSuite extends V1ReadFallbackSuite {
+  override def beforeAll(): Unit = {
+super.beforeAll()
+spark.conf.set("spark.sql.catalog.read_fallback", 
classOf[V1ReadFallbackCatalog].getName)
+sql("CREATE TABLE read_fallback.tbl(i int, j int) USING foo")
+  }
+
+  override def afterAll(): Unit = {
+spark.conf.unset("spark.sql.catalog.read_fallback")
+super.afterAll()
+  }
+
+  override protected def baseTableScan(): DataFrame = {
+spark.table("read_fallback.tbl")
+  }
+}
+
+class V1ReadFallbackCatalog extends BasicInMemoryTableCatalog {
+  override def createTable(
+  ident: Identifier,
+  schema: StructType,
+  partitions: Array[Transform],
+  properties: util.Map[String, String]): Table = {
+// To simplify the test implementation, only support fixed schema.
+if (schema != V1ReadFallbackCatalog.schema || partitions.nonEmpty) {
+  throw new UnsupportedOperationException
+}
+val table = new TableWithV1ReadFallback(ident.toString)
+tables.put(ident, table)
+table
+  }
+}
+
+object V1ReadFallbackCatalog {
+  val schema = new StructType().add("i", "int").add("j", "int")
+}
+
+class V1ReadFallbackTableProvider extends TableProvider {
+  override def getTable(options: CaseInsensitiveStringMap): Table = {
+new TableWithV1ReadFa

[GitHub] [spark] brkyvz commented on a change in pull request #26231: [SPARK-29572][SQL] add v1 read fallback API in DS v2

2020-01-15 Thread GitBox
brkyvz 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_r366989910
 
 

 ##
 File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala
 ##
 @@ -51,6 +54,7 @@ object V2ScanRelationPushDown extends Rule[LogicalPlan] {
  """.stripMargin)
 
   val scanRelation = DataSourceV2ScanRelation(relation.table, scan, output)
+  scanRelation.setTagValue(PUSHED_FILTERS_TAG, pushedFilters)
 
 Review comment:
   Can the `pushedFilters` be just a parameter of `DataSourceV2ScanRelation`?


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



[GitHub] [spark] brkyvz commented on a change in pull request #26231: [SPARK-29572][SQL] add v1 read fallback API in DS v2

2020-01-09 Thread GitBox
brkyvz 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_r345517260
 
 

 ##
 File path: 
sql/core/src/main/scala/org/apache/spark/sql/connector/read/V1Scan.scala
 ##
 @@ -0,0 +1,44 @@
+/*
+ * 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.SQLContext
+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.
 
 Review comment:
   Why does it need to be a `TableScan`? Can't it be a HadoopFsRelation? Can't 
it be a PrunedFilteredScan?


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



[GitHub] [spark] brkyvz commented on a change in pull request #26231: [SPARK-29572][SQL] add v1 read fallback API in DS v2

2020-01-09 Thread GitBox
brkyvz 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_r365042772
 
 

 ##
 File path: 
sql/core/src/main/java/org/apache/spark/sql/connector/read/V1Scan.java
 ##
 @@ -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.Unstable;
+import org.apache.spark.sql.SQLContext;
+import org.apache.spark.sql.sources.BaseRelation;
+import org.apache.spark.sql.sources.TableScan;
+
+/**
+ * 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
+ */
+@Unstable
+public interface V1Scan extends Scan {
+
+  /**
+   * Creates an `BaseRelation` with `TableScan` that can scan data from 
DataSource v1 to RDD[Row].
 
 Review comment:
   nit: Create a `BaseRelation`


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



[GitHub] [spark] brkyvz commented on a change in pull request #26231: [SPARK-29572][SQL] add v1 read fallback API in DS v2

2020-01-09 Thread GitBox
brkyvz 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_r365042944
 
 

 ##
 File path: 
sql/core/src/main/java/org/apache/spark/sql/connector/read/V1Scan.java
 ##
 @@ -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.Unstable;
+import org.apache.spark.sql.SQLContext;
+import org.apache.spark.sql.sources.BaseRelation;
+import org.apache.spark.sql.sources.TableScan;
+
+/**
+ * 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
+ */
+@Unstable
+public interface V1Scan extends Scan {
+
+  /**
+   * Creates an `BaseRelation` with `TableScan` that can scan data from 
DataSource v1 to RDD[Row].
+   *
+   * @since 3.0.0
+   */
+   T toV1TableScan(SQLContext context);
 
 Review comment:
   it kind of seems weird to me that we're introducing new APIs that use 
deprecated APIs


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



[GitHub] [spark] brkyvz commented on a change in pull request #26231: [SPARK-29572][SQL] add v1 read fallback API in DS v2

2020-01-09 Thread GitBox
brkyvz 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_r365045539
 
 

 ##
 File path: 
sql/core/src/main/java/org/apache/spark/sql/connector/read/V1Scan.java
 ##
 @@ -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.Unstable;
+import org.apache.spark.sql.SQLContext;
+import org.apache.spark.sql.sources.BaseRelation;
+import org.apache.spark.sql.sources.TableScan;
+
+/**
+ * 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
+ */
+@Unstable
+public interface V1Scan extends Scan {
 
 Review comment:
   Can we not push down filters and schema pruning down to this scan? We 
support these in the V1 APIs. Then you can avoid the pushed Filters tag


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



[GitHub] [spark] brkyvz commented on a change in pull request #26231: [SPARK-29572][SQL] add v1 read fallback API in DS v2

2020-01-09 Thread GitBox
brkyvz 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_r365045814
 
 

 ##
 File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
 ##
 @@ -19,38 +19,75 @@ package org.apache.spark.sql.execution.datasources.v2
 
 import scala.collection.JavaConverters._
 
-import org.apache.spark.sql.{AnalysisException, Strategy}
-import org.apache.spark.sql.catalyst.expressions.{And, PredicateHelper, 
SubqueryExpression}
+import org.apache.spark.sql.{AnalysisException, SparkSession, Strategy}
+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.{AlterNamespaceSetProperties, 
AlterTable, AppendData, 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.{StagingTableCatalog, 
TableCapability}
+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.TableScan
 import org.apache.spark.sql.util.CaseInsensitiveStringMap
 
-object DataSourceV2Strategy extends Strategy with PredicateHelper {
+class DataSourceV2Strategy(session: SparkSession) extends Strategy with 
PredicateHelper {
 
   import DataSourceV2Implicits._
 
+  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(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 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 originalOutputNames = relation.table.schema().map(_.name)
+  val requiredColumnsIndex = 
output.map(_.name).map(originalOutputNames.indexOf)
+  val dsScan = RowDataSourceScanExec(
 
 Review comment:
   how about an alternate constructor?


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