You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by rdblue <gi...@git.apache.org> on 2018/01/24 19:00:20 UTC

[GitHub] spark pull request #20387: SPARK-22386: DataSourceV2: Use immutable logical ...

GitHub user rdblue opened a pull request:

    https://github.com/apache/spark/pull/20387

    SPARK-22386: DataSourceV2: Use immutable logical plans.

    ## What changes were proposed in this pull request?
    
    DataSourceV2 should use immutable catalyst trees instead of wrapping a mutable DataSourceV2Reader. This commit updates DataSourceV2Relation and consolidates much of the DataSourceV2 API requirements for the read path in it. Instead of wrapping a reader that changes, the relation lazily produces a reader from its configuration.
    
    This commit also updates the predicate and projection push-down. Instead of the implementation from SPARK-22197, this reuses the rule matching from the Hive and DataSource read paths (using `PhysicalOperation`) and copies most of the implementation of `SparkPlanner.pruneFilterProject`, with updates for DataSourceV2. By reusing the implementation from other read paths, this should have fewer regressions from other read paths and is less code to maintain.
    
    The new push-down rules also support the following edge cases:
    
    * The output of DataSourceV2Relation should be what is returned by the reader, in case the reader can only partially satisfy the requested schema projection
    * The requested projection passed to the DataSourceV2Reader should include filter columns
    * The push-down rule may be run more than once if filters are not pushed through projections
    
    ## How was this patch tested?
    
    Existing push-down and read tests.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/rdblue/spark SPARK-22386-push-down-immutable-trees

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/20387.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #20387
    
----
commit d3233e1a8b1d4d153146b1a536dee34246920b0d
Author: Ryan Blue <bl...@...>
Date:   2018-01-17T21:58:12Z

    SPAKR-22386: DataSourceV2: Use immutable logical plans.

----


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r167343401
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,17 +17,131 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
     import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    -import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, ExprId}
    +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema, WriteSupport}
    +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics}
    +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    output: Seq[AttributeReference],
    -    reader: DataSourceReader)
    -  extends LeafNode with MultiInstanceRelation with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    projection: Option[Seq[AttributeReference]] = None,
    +    filters: Option[Seq[Expression]] = None,
    +    userSchema: Option[StructType] = None) extends LeafNode with MultiInstanceRelation {
    +
    +  override def simpleString: String = {
    +    s"DataSourceV2Relation(source=$sourceName, " +
    +      s"schema=[${output.map(a => s"$a ${a.dataType.simpleString}").mkString(", ")}], " +
    +      s"filters=[${pushedFilters.mkString(", ")}], options=$options)"
    +  }
    +
    +  override lazy val schema: StructType = reader.readSchema()
    +
    +  override lazy val output: Seq[AttributeReference] = {
    +    projection match {
    +      case Some(attrs) =>
    +        // use the projection attributes to avoid assigning new ids. fields that are not projected
    +        // will be assigned new ids, which is okay because they are not projected.
    +        val attrMap = attrs.map(a => a.name -> a).toMap
    +        schema.map(f => attrMap.getOrElse(f.name,
    +          AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()))
    +      case _ =>
    +        schema.toAttributes
    +    }
    +  }
    +
    +  private lazy val v2Options: DataSourceOptions = {
    +    // ensure path and table options are set correctly
    +    val updatedOptions = new mutable.HashMap[String, String]
    +    updatedOptions ++= options
    +
    +    new DataSourceOptions(options.asJava)
    +  }
    +
    +  private val sourceName: String = {
    +    source match {
    +      case registered: DataSourceRegister =>
    +        registered.shortName()
    +      case _ =>
    +        source.getClass.getSimpleName
    +    }
    +  }
    +
    +  lazy val (
    +      reader: DataSourceReader,
    +      unsupportedFilters: Seq[Expression],
    +      pushedFilters: Seq[Expression]) = {
    +    val newReader = userSchema match {
    +      case Some(s) =>
    +        asReadSupportWithSchema.createReader(s, v2Options)
    +      case _ =>
    +        asReadSupport.createReader(v2Options)
    +    }
    +
    +    projection.foreach { attrs =>
    +      DataSourceV2Relation.pushRequiredColumns(newReader, attrs.toStructType)
    +    }
    +
    +    val (remainingFilters, pushedFilters) = filters match {
    +      case Some(filterSeq) =>
    +        DataSourceV2Relation.pushFilters(newReader, filterSeq)
    +      case _ =>
    +        (Nil, Nil)
    +    }
    +
    +    (newReader, remainingFilters, pushedFilters)
    +  }
    +
    +  private lazy val asReadSupport: ReadSupport = {
    --- End diff --
    
    It's a little hard to map the new logic to what we have in https://github.com/apache/spark/pull/20387/files#diff-f70bda59304588cc3abfa3a9840653f4L201 .
    
    Can we keep the previous code style? i.e. `val reader = (ds, userSpecifiedSchema) match ...`


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/541/
    Test PASSed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r166168357
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,17 +17,151 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.TableIdentifier
     import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    -import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
    +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema, WriteSupport}
    +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics}
    +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    fullOutput: Seq[AttributeReference],
    -    reader: DataSourceReader)
    -  extends LeafNode with MultiInstanceRelation with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    path: Option[String] = None,
    +    table: Option[TableIdentifier] = None,
    --- End diff --
    
    Let me confirm your expectation for the data source v2 framework. First we define some standard information, like shema, path, table. We provide dedicated methods in `DataFrameReader` and `DataFrameWriter` for end users to specify them, e.g. `DataFrameReader.schame/path/table`. Then these information are passed to the read/write logical plan directly, i.e. `DataSourceV2Relation` and `WriteToDataSourceV2`. At the end the read/write logical plan construct the `DataSourceOption`, put these standard information to the option with standard option keys. Do I understand it correctly?


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r167288341
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,17 +17,130 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
     import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    -import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
    +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema, WriteSupport}
    +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics}
    +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    output: Seq[AttributeReference],
    -    reader: DataSourceReader)
    -  extends LeafNode with MultiInstanceRelation with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    projection: Option[Seq[AttributeReference]] = None,
    +    filters: Option[Seq[Expression]] = None,
    +    userSchema: Option[StructType] = None) extends LeafNode with MultiInstanceRelation {
    +
    +  override def simpleString: String = {
    +    s"DataSourceV2Relation(source=$sourceName, " +
    +      s"schema=[${output.map(a => s"$a ${a.dataType.simpleString}").mkString(", ")}], " +
    +      s"filters=[${pushedFilters.mkString(", ")}], options=$options)"
    +  }
    +
    +  override lazy val schema: StructType = reader.readSchema()
    +
    +  override lazy val output: Seq[AttributeReference] = {
    +    projection match {
    +      case Some(attrs) =>
    +        // use the projection attributes to avoid assigning new ids. fields that are not projected
    +        // will be assigned new ids, which is okay because they are not projected.
    +        val attrMap = attrs.map(a => a.name -> a).toMap
    +        schema.map(f => attrMap.getOrElse(f.name,
    +          AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()))
    +      case _ =>
    +        schema.toAttributes
    +    }
    +  }
    +
    +  private lazy val v2Options: DataSourceOptions = {
    +    // ensure path and table options are set correctly
    +    val updatedOptions = new mutable.HashMap[String, String]
    +    updatedOptions ++= options
    +
    +    new DataSourceOptions(options.asJava)
    +  }
    +
    +  private val sourceName: String = {
    +    source match {
    +      case registered: DataSourceRegister =>
    +        registered.shortName()
    +      case _ =>
    +        source.getClass.getSimpleName
    +    }
    +  }
    +
    +  lazy val (
    +      reader: DataSourceReader,
    +      unsupportedFilters: Seq[Expression],
    +      pushedFilters: Seq[Expression]) = {
    +    val newReader = userSchema match {
    +      case Some(s) =>
    +        asReadSupportWithSchema.createReader(s, v2Options)
    +      case _ =>
    +        asReadSupport.createReader(v2Options)
    +    }
    +
    +    projection.foreach { attrs =>
    +      DataSourceV2Relation.pushRequiredColumns(newReader, attrs.toStructType)
    +    }
    +
    +    val (remainingFilters, pushedFilters) = filters match {
    +      case Some(filterSeq) =>
    +        DataSourceV2Relation.pushFilters(newReader, filterSeq)
    +      case _ =>
    +        (Nil, Nil)
    +    }
    +
    +    (newReader, remainingFilters, pushedFilters)
    +  }
     
    -  override def canEqual(other: Any): Boolean = other.isInstanceOf[DataSourceV2Relation]
    +  def writer(dfSchema: StructType, mode: SaveMode): Option[DataSourceWriter] = {
    --- End diff --
    
    There's a key difference here that you're missing. The streaming API was rushed and committed without adequate review input. The comment indicated that the commit was probably unfinished and, if so, the remaining changes could be put into a completely different commit.
    
    In contrast, this contained a few additional things that make `DataSourceV2Relation` complete and more useful for the next PRs. Thinking about how this relation should be configured from all the code paths where it will be used and preparing for those cases just isn't the same kind of problem.
    
    I do appreciate you taking my recommendation to heart, but if you think this is "the same mistake," then I don't think you've quite understood what I'm saying.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r167385722
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,17 +17,80 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.sql.AnalysisException
     import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    -import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
    +import org.apache.spark.sql.catalyst.plans.QueryPlan
    +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema}
    +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics}
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    output: Seq[AttributeReference],
    -    reader: DataSourceReader)
    -  extends LeafNode with MultiInstanceRelation with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    projection: Seq[AttributeReference],
    +    filters: Option[Seq[Expression]] = None,
    +    userSchema: Option[StructType] = None) extends LeafNode with MultiInstanceRelation {
    --- End diff --
    
    nit: may be more clear to call it `userSpecifiedSchema`


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    @cloud-fan, is there anything else that needs to be updated, or is this ready to be merged?


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    I think there will be a lot of discussion about data source v2 table support. For now how about we remove the table/path stuff and get this PR in ASAP?


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r168933227
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownOperatorsToDataSource.scala ---
    @@ -17,130 +17,55 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    -import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeMap, AttributeSet, Expression, NamedExpression, PredicateHelper}
    -import org.apache.spark.sql.catalyst.optimizer.RemoveRedundantProject
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, AttributeSet}
    +import org.apache.spark.sql.catalyst.planning.PhysicalOperation
     import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project}
     import org.apache.spark.sql.catalyst.rules.Rule
    -import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    -import org.apache.spark.sql.sources
    -import org.apache.spark.sql.sources.v2.reader._
     
    -/**
    - * Pushes down various operators to the underlying data source for better performance. Operators are
    - * being pushed down with a specific order. As an example, given a LIMIT has a FILTER child, you
    - * can't push down LIMIT if FILTER is not completely pushed down. When both are pushed down, the
    - * data source should execute FILTER before LIMIT. And required columns are calculated at the end,
    - * because when more operators are pushed down, we may need less columns at Spark side.
    - */
    -object PushDownOperatorsToDataSource extends Rule[LogicalPlan] with PredicateHelper {
    -  override def apply(plan: LogicalPlan): LogicalPlan = {
    -    // Note that, we need to collect the target operator along with PROJECT node, as PROJECT may
    -    // appear in many places for column pruning.
    -    // TODO: Ideally column pruning should be implemented via a plan property that is propagated
    -    // top-down, then we can simplify the logic here and only collect target operators.
    -    val filterPushed = plan transformUp {
    -      case FilterAndProject(fields, condition, r @ DataSourceV2Relation(_, reader)) =>
    -        val (candidates, nonDeterministic) =
    -          splitConjunctivePredicates(condition).partition(_.deterministic)
    -
    -        val stayUpFilters: Seq[Expression] = reader match {
    -          case r: SupportsPushDownCatalystFilters =>
    -            r.pushCatalystFilters(candidates.toArray)
    -
    -          case r: SupportsPushDownFilters =>
    -            // A map from original Catalyst expressions to corresponding translated data source
    -            // filters. If a predicate is not in this map, it means it cannot be pushed down.
    -            val translatedMap: Map[Expression, sources.Filter] = candidates.flatMap { p =>
    -              DataSourceStrategy.translateFilter(p).map(f => p -> f)
    -            }.toMap
    -
    -            // Catalyst predicate expressions that cannot be converted to data source filters.
    -            val nonConvertiblePredicates = candidates.filterNot(translatedMap.contains)
    -
    -            // Data source filters that cannot be pushed down. An unhandled filter means
    -            // the data source cannot guarantee the rows returned can pass the filter.
    -            // As a result we must return it so Spark can plan an extra filter operator.
    -            val unhandledFilters = r.pushFilters(translatedMap.values.toArray).toSet
    -            val unhandledPredicates = translatedMap.filter { case (_, f) =>
    -              unhandledFilters.contains(f)
    -            }.keys
    -
    -            nonConvertiblePredicates ++ unhandledPredicates
    -
    -          case _ => candidates
    -        }
    -
    -        val filterCondition = (stayUpFilters ++ nonDeterministic).reduceLeftOption(And)
    -        val withFilter = filterCondition.map(Filter(_, r)).getOrElse(r)
    -        if (withFilter.output == fields) {
    -          withFilter
    -        } else {
    -          Project(fields, withFilter)
    -        }
    -    }
    -
    -    // TODO: add more push down rules.
    -
    -    val columnPruned = pushDownRequiredColumns(filterPushed, filterPushed.outputSet)
    -    // After column pruning, we may have redundant PROJECT nodes in the query plan, remove them.
    -    RemoveRedundantProject(columnPruned)
    -  }
    -
    -  // TODO: nested fields pruning
    -  private def pushDownRequiredColumns(
    -      plan: LogicalPlan, requiredByParent: AttributeSet): LogicalPlan = {
    -    plan match {
    -      case p @ Project(projectList, child) =>
    -        val required = projectList.flatMap(_.references)
    -        p.copy(child = pushDownRequiredColumns(child, AttributeSet(required)))
    -
    -      case f @ Filter(condition, child) =>
    -        val required = requiredByParent ++ condition.references
    -        f.copy(child = pushDownRequiredColumns(child, required))
    -
    -      case relation: DataSourceV2Relation => relation.reader match {
    -        case reader: SupportsPushDownRequiredColumns =>
    -          // TODO: Enable the below assert after we make `DataSourceV2Relation` immutable. Fow now
    -          // it's possible that the mutable reader being updated by someone else, and we need to
    -          // always call `reader.pruneColumns` here to correct it.
    -          // assert(relation.output.toStructType == reader.readSchema(),
    -          //  "Schema of data source reader does not match the relation plan.")
    -
    -          val requiredColumns = relation.output.filter(requiredByParent.contains)
    -          reader.pruneColumns(requiredColumns.toStructType)
    -
    -          val nameToAttr = relation.output.map(_.name).zip(relation.output).toMap
    -          val newOutput = reader.readSchema().map(_.name).map(nameToAttr)
    -          relation.copy(output = newOutput)
    -
    -        case _ => relation
    +object PushDownOperatorsToDataSource extends Rule[LogicalPlan] {
    +  override def apply(
    +      plan: LogicalPlan): LogicalPlan = plan transformUp {
    +    // PhysicalOperation guarantees that filters are deterministic; no need to check
    +    case PhysicalOperation(project, newFilters, relation : DataSourceV2Relation) =>
    +      // merge the filters
    +      val filters = relation.filters match {
    +        case Some(existing) =>
    +          existing ++ newFilters
    +        case _ =>
    +          newFilters
           }
     
    -      // TODO: there may be more operators that can be used to calculate the required columns. We
    -      // can add more and more in the future.
    -      case _ => plan.mapChildren(c => pushDownRequiredColumns(c, c.outputSet))
    -    }
    -  }
    -
    -  /**
    -   * Finds a Filter node(with an optional Project child) above data source relation.
    -   */
    -  object FilterAndProject {
    -    // returns the project list, the filter condition and the data source relation.
    -    def unapply(plan: LogicalPlan)
    -        : Option[(Seq[NamedExpression], Expression, DataSourceV2Relation)] = plan match {
    +      val projectAttrs = project.map(_.toAttribute)
    +      val projectSet = AttributeSet(project.flatMap(_.references))
    +      val filterSet = AttributeSet(filters.flatMap(_.references))
    +
    +      val projection = if (filterSet.subsetOf(projectSet) &&
    +          AttributeSet(projectAttrs) == projectSet) {
    +        // When the required projection contains all of the filter columns and column pruning alone
    +        // can produce the required projection, push the required projection.
    +        // A final projection may still be needed if the data source produces a different column
    +        // order or if it cannot prune all of the nested columns.
    +        projectAttrs
    +      } else {
    +        // When there are filter columns not already in the required projection or when the required
    +        // projection is more complicated than column pruning, base column pruning on the set of
    +        // all columns needed by both.
    +        (projectSet ++ filterSet).toSeq
    +      }
     
    -      case Filter(condition, r: DataSourceV2Relation) => Some((r.output, condition, r))
    +      val newRelation = relation.copy(
    +        projection = projection.asInstanceOf[Seq[AttributeReference]],
    --- End diff --
    
    There is no guarantee that the source will produce the requested schema, so it is not correct to assume that `projection` and `output` will be the same thing. The `output` must always be based on what the configured reader will return, so there is no way around this unless we want to change the data source API.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    **[Test build #87268 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87268/testReport)** for PR 20387 at commit [`87a36be`](https://github.com/apache/spark/commit/87a36be5b8dc2002a45bf71ffdc94c816f6d7355).


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    **[Test build #87001 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87001/testReport)** for PR 20387 at commit [`63f11a9`](https://github.com/apache/spark/commit/63f11a92ffefaf937dd0266e3e59d619d33ab873).
     * This patch **fails to build**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    > [The push-down rule may be run more than once if filters are not pushed through projections] looks weird, do you have a query to reproduce this issue?
    
    One of the DataSourceV2 tests hit this. I thought it was a good thing to push a single node down at a time and not depend on order.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Thanks for the update! Enjoy your vacation, and thanks for letting me know.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r167639047
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,17 +17,80 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.sql.AnalysisException
     import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    -import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
    +import org.apache.spark.sql.catalyst.plans.QueryPlan
    +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema}
    +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics}
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    output: Seq[AttributeReference],
    -    reader: DataSourceReader)
    -  extends LeafNode with MultiInstanceRelation with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    projection: Seq[AttributeReference],
    +    filters: Option[Seq[Expression]] = None,
    +    userSchema: Option[StructType] = None) extends LeafNode with MultiInstanceRelation {
    --- End diff --
    
    I think this is clear. In what way do you think this will be misinterpreted?


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r166998163
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,17 +17,130 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
     import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    -import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
    +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema, WriteSupport}
    +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics}
    +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    output: Seq[AttributeReference],
    -    reader: DataSourceReader)
    -  extends LeafNode with MultiInstanceRelation with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    projection: Option[Seq[AttributeReference]] = None,
    +    filters: Option[Seq[Expression]] = None,
    +    userSchema: Option[StructType] = None) extends LeafNode with MultiInstanceRelation {
    +
    +  override def simpleString: String = {
    +    s"DataSourceV2Relation(source=$sourceName, " +
    +      s"schema=[${output.map(a => s"$a ${a.dataType.simpleString}").mkString(", ")}], " +
    +      s"filters=[${pushedFilters.mkString(", ")}], options=$options)"
    +  }
    +
    +  override lazy val schema: StructType = reader.readSchema()
    +
    +  override lazy val output: Seq[AttributeReference] = {
    --- End diff --
    
    I'll have a look. I didn't realize you'd committed that one already.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r167385858
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -37,22 +100,129 @@ case class DataSourceV2Relation(
       }
     
       override def newInstance(): DataSourceV2Relation = {
    -    copy(output = output.map(_.newInstance()))
    +    // projection is used to maintain id assignment.
    +    // if projection is not set, use output so the copy is not equal to the original
    +    copy(projection = projection.map(_.newInstance()))
       }
     }
     
     /**
      * A specialization of DataSourceV2Relation with the streaming bit set to true. Otherwise identical
      * to the non-streaming relation.
      */
    -class StreamingDataSourceV2Relation(
    +case class StreamingDataSourceV2Relation(
         output: Seq[AttributeReference],
    -    reader: DataSourceReader) extends DataSourceV2Relation(output, reader) {
    +    reader: DataSourceReader)
    +    extends LeafNode with DataSourceReaderHolder with MultiInstanceRelation {
       override def isStreaming: Boolean = true
    +
    +  override def canEqual(other: Any): Boolean = other.isInstanceOf[StreamingDataSourceV2Relation]
    +
    +  override def newInstance(): LogicalPlan = copy(output = output.map(_.newInstance()))
     }
     
     object DataSourceV2Relation {
    -  def apply(reader: DataSourceReader): DataSourceV2Relation = {
    -    new DataSourceV2Relation(reader.readSchema().toAttributes, reader)
    +  private implicit class SourceHelpers(source: DataSourceV2) {
    +    def asReadSupport: ReadSupport = {
    +      source match {
    +        case support: ReadSupport =>
    +          support
    +        case _: ReadSupportWithSchema =>
    +          // this method is only called if there is no user-supplied schema. if there is no
    +          // user-supplied schema and ReadSupport was not implemented, throw a helpful exception.
    +          throw new AnalysisException(s"Data source requires a user-supplied schema: $name")
    +        case _ =>
    +          throw new AnalysisException(s"Data source is not readable: $name")
    +      }
    +    }
    +
    +    def asReadSupportWithSchema: ReadSupportWithSchema = {
    +      source match {
    +        case support: ReadSupportWithSchema =>
    +          support
    +        case _: ReadSupport =>
    --- End diff --
    
    another concern is: this check should be done ASAP so that we can fail earlier.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    > I'm ok to make it immutable if there is an significant benefit.
    
    Mutable nodes violate a basic assumption of catalyst, that trees are immutable. Here's a good quote from the SIGMOD paper (by @rxin, @yhuai, and @marmbrus et al.):
    
    > In our experience, functional transformations on immutable trees make the whole optimizer very easy to reason about and debug. They also enable parallelization in the optimizer, although we do not yet exploit this.
    
    Mixing mutable nodes into supposedly immutable trees is a bad idea. Other nodes in the tree assume that children do not change.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r167147910
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,17 +17,130 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
     import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    -import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
    +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema, WriteSupport}
    +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics}
    +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    output: Seq[AttributeReference],
    -    reader: DataSourceReader)
    -  extends LeafNode with MultiInstanceRelation with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    projection: Option[Seq[AttributeReference]] = None,
    +    filters: Option[Seq[Expression]] = None,
    +    userSchema: Option[StructType] = None) extends LeafNode with MultiInstanceRelation {
    +
    +  override def simpleString: String = {
    +    s"DataSourceV2Relation(source=$sourceName, " +
    +      s"schema=[${output.map(a => s"$a ${a.dataType.simpleString}").mkString(", ")}], " +
    +      s"filters=[${pushedFilters.mkString(", ")}], options=$options)"
    +  }
    +
    +  override lazy val schema: StructType = reader.readSchema()
    +
    +  override lazy val output: Seq[AttributeReference] = {
    --- End diff --
    
    I pulled your code and played with it. So your PR does fix the bug, but in a hacky way. Let's me explain what happened.
    1. `QueryPlan.canonicalized` is called, every expression in `DataSourceV2Relation` is canonicalized, including `DataSourceV2Relation.projection`. This means, the attributes in `projection` are all renamed to "none".
    2. `DataSourceV2Relation.output` is called, which triggers the creation of the reader, and applies filter push down and column pruning. Note that because all attributes are renamed to "none", we are actually pushing invalid filters and columns to data sources.
    3. line up `reader.schema` and `projection`, to get the actual output. Because all names are "none", it works.
    
    However step 2 is pretty dangerous, Spark doesn't define the behavior of pushing invalid filters and columns, especially what `reader.schema` should return after invalid columns are pushed down.
    
    I prefer my original fix, which put `output` in `DataSourceV2Relation`'s constructor parameters, and update it when doing column pruning in `PushDownOperatorsToDataSource`.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/86602/
    Test PASSed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    **[Test build #87170 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87170/testReport)** for PR 20387 at commit [`181946d`](https://github.com/apache/spark/commit/181946d1f1c5889661544830a77bd23c4b4f685a).


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/951/
    Test PASSed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    > For safety, I wanna keep it unchanged, and start something new for data source v2 only.
    
    I disagree.
    
    * **#20476 addresses a bug caused by the new implementation that is not a problem if we reuse the current push-down code.** Using an entirely new implementation to push filters and projection is going to introduce bugs, and that problem demonstrates that it is a real risk.
    * **Using unreliable push-down code is going to make it more difficult for anyone to use the v2 API.**
    * **This approach throws away work that has accumulated over the past few years that give us confidence in the current push-down code.** The other code paths have push-down tests that will help us catch bugs in the new push-down logic. If we limit the scope of this change to v2, we will not be able to reuse those tests and will have to write entirely new ones that cover all cases.
    
    Lastly, I think it is clear that we need a design for a new push-down mechanism. **Adding this to DataSourceV2 as feature creep is not a good way to redesign it.** I'd like to see a design document that addresses some of the open questions.
    
    I'd also prefer that this new implementation be removed from the v2 code path for 2.3.0. @marmbrus, what do you think?


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    @felixcheung, yes, we do already have a `table` option. That creates an `UnresolvedRelation` with the parsed table name as a `TableIdentifier`, which is not currently compatible with `DataSourceV2` because there is no standard way to pass the identifier's db and table name.
    
    Part of the intent here is to add support in `DataSourceV2Relation` for cases where we have a `TableIdentifier`, so that we can add a resolver rule that replaces `UnresolvedRelation` with `DataSourceV2Relation`. This is what we do in our Spark branch.
    
    @cloud-fan, what is your objection to support like this?


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    The reader is to be created and configured by the relation, then the relation needs to be able to set the table, path, and other properties. This adds necessary data to the relation that is no longer be passed directly to the reader from `DataFrameReader`.
    
    From the other thread on this, I think we agree that minimizing the number of places that work with `DataSourceOptions` and the specific option strings is a good idea. So it makes sense to define the relation using `TableIdentifier`. Other paths that create `DataSourceV2Relation` need the table name to be passed like this.
    
    I guess we *could* revert the change and add it in a separate commit, but I don't see a reason for the extra work. It would be impractical to backport a later `TableIdentifier` change without this immutability change. Similarly, why would someone want to move to an immutable plan, but leave some left-over logic for configuration in `DataFrameReader`?
    
    I don't see why we wouldn't want to have these options in the immutable relation node from the start. Do you have a case in mind that I'm missing?


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/199/
    Test PASSed.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r166691646
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,17 +17,139 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
     import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    -import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
    +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema, WriteSupport}
    +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics}
    +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    fullOutput: Seq[AttributeReference],
    -    reader: DataSourceReader)
    -  extends LeafNode with MultiInstanceRelation with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    path: Option[String] = None,
    --- End diff --
    
    You've missed the idea that I was trying to make with that point.
    
    The idea isn't to make "surgical" commits that hardly make any changes and must exactly match the JIRA description. If we did that, we would end up making only tiny commits, spending far too much time filling out JIRA issues, and buried under an unmanageable review backlog.
    
    The point is to think about whether the content of a commit would confuse someone managing a branch or cause avoidable conflicts when reverting or picking commits.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    > This is a new API...
    
    Are you saying you wanna add a new method in `DataFreameReader` that is different than `load`? In Scala, parameter name is part of the method signature, so for `def load(path: String)`, we can't change its semantic, the parameter is a path. It's fine if a data source impelementation teach its users that path will interpreted as database/tables by it, but this should not be a contract in Spark.
    
    I do agree that Spark should set a standard for specifying database and table, as it's very common. We can even argue that path is not a general concept for data sources, but we still provide special APIs for path.
    
    My proposal: How about we add a new methods `table` in `DataFrameReader`? The usage would look like: `spark.read.format("iceberg").table("db.table").load()`, what do you think? We should not specify `database`, as if we may have catalog federation and table name may have 3 parts `catalog.db.table`. Let's keep it general and let the data source to interprete it.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    **[Test build #87432 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87432/testReport)** for PR 20387 at commit [`b8e3623`](https://github.com/apache/spark/commit/b8e3623837047949b39141e46eb96f30de8aa21e).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    @cloud-fan, I've rebased and made the requested changes.
    
    #20603 reverts the last commit that adds back support for user-supplied schemas that are identical to the source schema.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    > I'd suggest that we just propogate the paths parameter to options, and data source implementations are free to interprete the path option to whatever they want, e.g. table and database names.
    
    What about code paths that expect table names? In our branch, we've added support for converting Hive relations (which have a `TableIdentifier`, not a path) and using `insertInto`. Table names are paths are the two main ways to identify tables and I think both should be supported.
    
    This is a new API, so it doesn't matter that `load` and `save` currently use paths. We can easily update that support for tables. If we don't, then there will be no common way to refer to tables: some implementations will use `table`, some will pass `db` separately, and some might use `database`. Standardizing this and adding support in Spark will produce more consistent behavior across data sources.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r167385829
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -37,22 +100,129 @@ case class DataSourceV2Relation(
       }
     
       override def newInstance(): DataSourceV2Relation = {
    -    copy(output = output.map(_.newInstance()))
    +    // projection is used to maintain id assignment.
    +    // if projection is not set, use output so the copy is not equal to the original
    +    copy(projection = projection.map(_.newInstance()))
       }
     }
     
     /**
      * A specialization of DataSourceV2Relation with the streaming bit set to true. Otherwise identical
      * to the non-streaming relation.
      */
    -class StreamingDataSourceV2Relation(
    +case class StreamingDataSourceV2Relation(
         output: Seq[AttributeReference],
    -    reader: DataSourceReader) extends DataSourceV2Relation(output, reader) {
    +    reader: DataSourceReader)
    +    extends LeafNode with DataSourceReaderHolder with MultiInstanceRelation {
       override def isStreaming: Boolean = true
    +
    +  override def canEqual(other: Any): Boolean = other.isInstanceOf[StreamingDataSourceV2Relation]
    +
    +  override def newInstance(): LogicalPlan = copy(output = output.map(_.newInstance()))
     }
     
     object DataSourceV2Relation {
    -  def apply(reader: DataSourceReader): DataSourceV2Relation = {
    -    new DataSourceV2Relation(reader.readSchema().toAttributes, reader)
    +  private implicit class SourceHelpers(source: DataSourceV2) {
    +    def asReadSupport: ReadSupport = {
    +      source match {
    +        case support: ReadSupport =>
    +          support
    +        case _: ReadSupportWithSchema =>
    +          // this method is only called if there is no user-supplied schema. if there is no
    +          // user-supplied schema and ReadSupport was not implemented, throw a helpful exception.
    +          throw new AnalysisException(s"Data source requires a user-supplied schema: $name")
    +        case _ =>
    +          throw new AnalysisException(s"Data source is not readable: $name")
    +      }
    +    }
    +
    +    def asReadSupportWithSchema: ReadSupportWithSchema = {
    +      source match {
    +        case support: ReadSupportWithSchema =>
    +          support
    +        case _: ReadSupport =>
    --- End diff --
    
    This is different from before: see https://github.com/apache/spark/pull/20387/files#diff-f70bda59304588cc3abfa3a9840653f4L214
    
    Even if `userSpecifiedSchema` is not none, it's still allowed to have `ReadSupport`, as long as its reader's schema is same as the user specified schema.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r166827720
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,17 +17,130 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
     import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    -import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
    +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema, WriteSupport}
    +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics}
    +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    output: Seq[AttributeReference],
    -    reader: DataSourceReader)
    -  extends LeafNode with MultiInstanceRelation with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    projection: Option[Seq[AttributeReference]] = None,
    +    filters: Option[Seq[Expression]] = None,
    +    userSchema: Option[StructType] = None) extends LeafNode with MultiInstanceRelation {
    +
    +  override def simpleString: String = {
    +    s"DataSourceV2Relation(source=$sourceName, " +
    +      s"schema=[${output.map(a => s"$a ${a.dataType.simpleString}").mkString(", ")}], " +
    +      s"filters=[${pushedFilters.mkString(", ")}], options=$options)"
    +  }
    +
    +  override lazy val schema: StructType = reader.readSchema()
    +
    +  override lazy val output: Seq[AttributeReference] = {
    +    projection match {
    +      case Some(attrs) =>
    +        // use the projection attributes to avoid assigning new ids. fields that are not projected
    +        // will be assigned new ids, which is okay because they are not projected.
    +        val attrMap = attrs.map(a => a.name -> a).toMap
    +        schema.map(f => attrMap.getOrElse(f.name,
    +          AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()))
    +      case _ =>
    +        schema.toAttributes
    +    }
    +  }
    +
    +  private lazy val v2Options: DataSourceOptions = {
    +    // ensure path and table options are set correctly
    +    val updatedOptions = new mutable.HashMap[String, String]
    +    updatedOptions ++= options
    +
    +    new DataSourceOptions(options.asJava)
    --- End diff --
    
    This makes us create a new `DataSourceOptions` everytime `DataSourceV2Relation` gets transformed and copied. Can we just create `DataSourceOptions` at `DataFrameReader` and pass it to `DataSourceV2Relation`?


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/86601/
    Test FAILed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    I dig into the commit history and recalled why I made these decisions:
    
    * having an mutable `DataSourceV2Relation`. This is mostly to avoid to keep adding more constructor parameters to `DataSourceV2Relation`, make the code easy to maintain. I'm ok to make it immutable if there is an significant benefit.
    * not using `PhysicalOperation`. This is because we will add more push down optimizations(e.g. limit, aggregate, join), and we have a specify push down order for them. It's hard to improve `PhysicalOperation` to support more operators and specific push down orders, so I created the new one. Eventually all data sources will be implemented as data source v2, so `PhysicalOperation` will go away.
    
    
    > The output of DataSourceV2Relation should be what is returned by the reader, in case the reader can only partially satisfy the requested schema projection
    
    Good catch! Since `DataSourceV2Reader` is mutable, the output can't be fixed, as it may change when we apply data source optimizations. Using `lazy val output ...` can fix this.
    
    
    > The requested projection passed to the DataSourceV2Reader should include filter columns
    
    I did this intentionally. If a column is only refered by pushed filters, Spark doesn't need this column. Even if we require this column from the data source, we just read it out and wait it to be pruned by the next operator.
    
    
    > The push-down rule may be run more than once if filters are not pushed through projections
    
    This looks weird, do you have a query to reproduce this issue?
    
    
    > This updates DataFrameReader to parse locations that do not look like paths as table names and pass the result as "database" and "table" keys in v2 options.
    
    Personally I'd suggest to use `spark.read.format("iceberg").option("table", "db.table").load()`, as `load` is defined as `def load(paths: String*)`, but I think your usage looks better. The communition protocol between Spark and data source is options, I'd suggest that we just propogate the `paths` parameter to options, and data source implementations are free to interprete the path option to whatever they want, e.g. table and database names.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Us...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r163908263
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,15 +17,149 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
     import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceV2, DataSourceV2Options, ReadSupport, ReadSupportWithSchema, WriteSupport}
     import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.sources.v2.writer.DataSourceV2Writer
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    fullOutput: Seq[AttributeReference],
    -    reader: DataSourceV2Reader) extends LeafNode with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    path: Option[String] = None,
    +    table: Option[TableIdentifier] = None,
    --- End diff --
    
    We could keep these in options, but because they are the main two ways to identify tables, they should be easier to work with. I'd even suggest adding them to the DataSourceV2 read and write APIs.
    
    Another benefit of adding these is that it is easier to use DataSourceV2Relation elsewhere. In our Spark build, I've added a rule to convert Hive relations to DataSourceV2Relation based on a table property. That's cleaner because we can pass the TableIdentifier instead of adding options to the map.


---

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


[GitHub] spark issue #20387: SPARK-22386: DataSourceV2: Use immutable logical plans.

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    **[Test build #86598 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86598/testReport)** for PR 20387 at commit [`d3233e1`](https://github.com/apache/spark/commit/d3233e1a8b1d4d153146b1a536dee34246920b0d).


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    **[Test build #86600 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86600/testReport)** for PR 20387 at commit [`9c4dcb5`](https://github.com/apache/spark/commit/9c4dcb5b693e729e89ddd7daa54b19c8f8eb3571).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `case class StreamingDataSourceV2Relation(`


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    @cloud-fan, please have a look at these changes. This will require follow-up for the Streaming side. I have yet to review the streaming interfaces for `DataSourceV2`, so I haven't made any changes there.
    
    In our Spark build, I've also moved the write path to  use DataSourceV2Relation, which I intend to do in a follow-up to this issue.
    
    @rxin FYI.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    I'm on vacation and will be back next week, will have a more thorough review at that time, sorry for the inconvenience!


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    **[Test build #87000 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87000/testReport)** for PR 20387 at commit [`9bb0141`](https://github.com/apache/spark/commit/9bb01416d68e9e2b7ed34745ba0a4b92721d98dd).
     * This patch **fails to build**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    **[Test build #87532 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87532/testReport)** for PR 20387 at commit [`1a603db`](https://github.com/apache/spark/commit/1a603dbe5528b447bff371d2e00abdbdee664a75).


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    I'm OK to replace the new push down implementation with `PhysicalOperation`, but please do that in an individual PR. If we do find the new implementation is necessary, it's easier for us to bring it back, if it was removed in an individual commit.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    For doing pushdown at logical or physical phase, I don't have a strong preference. I think at logical phase we should try our best to push down data-size-reduction operators(like filter, aggregate, limit, etc.) close to the bottom of the plan, as it should always be good. We should apply pushdown to data sources at physical phase, as it's not always good and we need to consider the cost. Currently it's done in logical phase because of the `computeStats` problem. Eventually we should compute the statistics and apply pushdown to data sources in physical phase.
    
    About how to apply pushdown to data sources, I think `PhysicalOperation` is in the right direction and the new pushdown rule also follows it. Generally the logical phase is responsible for pushing down data-size-reduction operators close to the data source relation, and in the physical phase we collect supported operators(currently it's only project and filter) above the data source relation and apply the pushdown once, so this doesn't need to be incremental.
    
    We definitely need to document the contract for ordering and interactions between different types of pushdowns. For now we don't need to worry about it as we only support column pruning and filter push down, and these 2 are orthogonal, it doesn't matter if data source run project first or filter first. Here are some initial thoughts on how to define the contract.
    
    Let's say Data Source V2 framework supports pushing down required columns(column pruning), filter, limit, aggregate. Semantically filter, limit and aggregate are not exchangeable, we should respect their order in the query. If we have all these operators in a query, how to tell the data source about the order of these operators?
    
    My proposal is, since `DataSourceReader` is mutable(not the plan!), we can ask the data source to remember which operators have been pushed down, via the order of when these `pushXXX` methods are called. And data source implementations should respect the order of pushdown when applying them internally.
    
    About `PhysicalOperation`, it's pretty simple and we probably need to change it a lot when adding more operator pushdown. Another concern is, `PhysicalOperation` is used in a lot of places, not only data source pushdown. For safety, I wanna keep it unchanged, and start something new for data source v2 only.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Us...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r163734181
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,15 +17,149 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
     import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceV2, DataSourceV2Options, ReadSupport, ReadSupportWithSchema, WriteSupport}
     import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.sources.v2.writer.DataSourceV2Writer
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    fullOutput: Seq[AttributeReference],
    -    reader: DataSourceV2Reader) extends LeafNode with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    path: Option[String] = None,
    +    table: Option[TableIdentifier] = None,
    --- End diff --
    
    why do we need these 2 parameters? Can't we get them from options when needed?


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    @cloud-fan: Rebased and removed path.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    overall I think it's a good idea to make the plan immutable.


---

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


[GitHub] spark issue #20387: SPARK-22386: DataSourceV2: Use immutable logical plans.

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r167344956
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownOperatorsToDataSource.scala ---
    @@ -17,130 +17,55 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    -import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeMap, AttributeSet, Expression, NamedExpression, PredicateHelper}
    -import org.apache.spark.sql.catalyst.optimizer.RemoveRedundantProject
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, AttributeSet}
    +import org.apache.spark.sql.catalyst.planning.PhysicalOperation
     import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project}
     import org.apache.spark.sql.catalyst.rules.Rule
    -import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    -import org.apache.spark.sql.sources
    -import org.apache.spark.sql.sources.v2.reader._
     
    -/**
    - * Pushes down various operators to the underlying data source for better performance. Operators are
    - * being pushed down with a specific order. As an example, given a LIMIT has a FILTER child, you
    - * can't push down LIMIT if FILTER is not completely pushed down. When both are pushed down, the
    - * data source should execute FILTER before LIMIT. And required columns are calculated at the end,
    - * because when more operators are pushed down, we may need less columns at Spark side.
    - */
    -object PushDownOperatorsToDataSource extends Rule[LogicalPlan] with PredicateHelper {
    -  override def apply(plan: LogicalPlan): LogicalPlan = {
    -    // Note that, we need to collect the target operator along with PROJECT node, as PROJECT may
    -    // appear in many places for column pruning.
    -    // TODO: Ideally column pruning should be implemented via a plan property that is propagated
    -    // top-down, then we can simplify the logic here and only collect target operators.
    -    val filterPushed = plan transformUp {
    -      case FilterAndProject(fields, condition, r @ DataSourceV2Relation(_, reader)) =>
    -        val (candidates, nonDeterministic) =
    -          splitConjunctivePredicates(condition).partition(_.deterministic)
    -
    -        val stayUpFilters: Seq[Expression] = reader match {
    -          case r: SupportsPushDownCatalystFilters =>
    -            r.pushCatalystFilters(candidates.toArray)
    -
    -          case r: SupportsPushDownFilters =>
    -            // A map from original Catalyst expressions to corresponding translated data source
    -            // filters. If a predicate is not in this map, it means it cannot be pushed down.
    -            val translatedMap: Map[Expression, sources.Filter] = candidates.flatMap { p =>
    -              DataSourceStrategy.translateFilter(p).map(f => p -> f)
    -            }.toMap
    -
    -            // Catalyst predicate expressions that cannot be converted to data source filters.
    -            val nonConvertiblePredicates = candidates.filterNot(translatedMap.contains)
    -
    -            // Data source filters that cannot be pushed down. An unhandled filter means
    -            // the data source cannot guarantee the rows returned can pass the filter.
    -            // As a result we must return it so Spark can plan an extra filter operator.
    -            val unhandledFilters = r.pushFilters(translatedMap.values.toArray).toSet
    -            val unhandledPredicates = translatedMap.filter { case (_, f) =>
    -              unhandledFilters.contains(f)
    -            }.keys
    -
    -            nonConvertiblePredicates ++ unhandledPredicates
    -
    -          case _ => candidates
    -        }
    -
    -        val filterCondition = (stayUpFilters ++ nonDeterministic).reduceLeftOption(And)
    -        val withFilter = filterCondition.map(Filter(_, r)).getOrElse(r)
    -        if (withFilter.output == fields) {
    -          withFilter
    -        } else {
    -          Project(fields, withFilter)
    -        }
    -    }
    -
    -    // TODO: add more push down rules.
    -
    -    val columnPruned = pushDownRequiredColumns(filterPushed, filterPushed.outputSet)
    -    // After column pruning, we may have redundant PROJECT nodes in the query plan, remove them.
    -    RemoveRedundantProject(columnPruned)
    -  }
    -
    -  // TODO: nested fields pruning
    -  private def pushDownRequiredColumns(
    -      plan: LogicalPlan, requiredByParent: AttributeSet): LogicalPlan = {
    -    plan match {
    -      case p @ Project(projectList, child) =>
    -        val required = projectList.flatMap(_.references)
    -        p.copy(child = pushDownRequiredColumns(child, AttributeSet(required)))
    -
    -      case f @ Filter(condition, child) =>
    -        val required = requiredByParent ++ condition.references
    -        f.copy(child = pushDownRequiredColumns(child, required))
    -
    -      case relation: DataSourceV2Relation => relation.reader match {
    -        case reader: SupportsPushDownRequiredColumns =>
    -          // TODO: Enable the below assert after we make `DataSourceV2Relation` immutable. Fow now
    -          // it's possible that the mutable reader being updated by someone else, and we need to
    -          // always call `reader.pruneColumns` here to correct it.
    -          // assert(relation.output.toStructType == reader.readSchema(),
    -          //  "Schema of data source reader does not match the relation plan.")
    -
    -          val requiredColumns = relation.output.filter(requiredByParent.contains)
    -          reader.pruneColumns(requiredColumns.toStructType)
    -
    -          val nameToAttr = relation.output.map(_.name).zip(relation.output).toMap
    -          val newOutput = reader.readSchema().map(_.name).map(nameToAttr)
    -          relation.copy(output = newOutput)
    -
    -        case _ => relation
    +object PushDownOperatorsToDataSource extends Rule[LogicalPlan] {
    +  override def apply(
    +      plan: LogicalPlan): LogicalPlan = plan transformUp {
    +    // PhysicalOperation guarantees that filters are deterministic; no need to check
    +    case PhysicalOperation(project, newFilters, relation : DataSourceV2Relation) =>
    +      // merge the filters
    +      val filters = relation.filters match {
    +        case Some(existing) =>
    +          existing ++ newFilters
    +        case _ =>
    +          newFilters
           }
     
    -      // TODO: there may be more operators that can be used to calculate the required columns. We
    -      // can add more and more in the future.
    -      case _ => plan.mapChildren(c => pushDownRequiredColumns(c, c.outputSet))
    -    }
    -  }
    -
    -  /**
    -   * Finds a Filter node(with an optional Project child) above data source relation.
    -   */
    -  object FilterAndProject {
    -    // returns the project list, the filter condition and the data source relation.
    -    def unapply(plan: LogicalPlan)
    -        : Option[(Seq[NamedExpression], Expression, DataSourceV2Relation)] = plan match {
    +      val projectAttrs = project.map(_.toAttribute)
    +      val projectSet = AttributeSet(project.flatMap(_.references))
    +      val filterSet = AttributeSet(filters.flatMap(_.references))
    +
    +      val projection = if (filterSet.subsetOf(projectSet) &&
    +          AttributeSet(projectAttrs) == projectSet) {
    +        // When the required projection contains all of the filter columns and column pruning alone
    +        // can produce the required projection, push the required projection.
    +        // A final projection may still be needed if the data source produces a different column
    +        // order or if it cannot prune all of the nested columns.
    +        projectAttrs
    +      } else {
    +        // When there are filter columns not already in the required projection or when the required
    +        // projection is more complicated than column pruning, base column pruning on the set of
    +        // all columns needed by both.
    +        (projectSet ++ filterSet).toSeq
    +      }
     
    -      case Filter(condition, r: DataSourceV2Relation) => Some((r.output, condition, r))
    +      val newRelation = relation.copy(
    +        projection = Some(projection.asInstanceOf[Seq[AttributeReference]]),
    +        filters = Some(filters))
    --- End diff --
    
    This is the issue I mentioned before: which one shall we pushdown first? `projection` or `filters`? It's OK for now because the order of column pruning and filter push down doesn't matter. But this will be a problem when adding limit pushdown.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/87426/
    Test FAILed.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r167369570
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,17 +17,131 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
     import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    -import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, ExprId}
    +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema, WriteSupport}
    +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics}
    +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    output: Seq[AttributeReference],
    -    reader: DataSourceReader)
    -  extends LeafNode with MultiInstanceRelation with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    projection: Option[Seq[AttributeReference]] = None,
    +    filters: Option[Seq[Expression]] = None,
    +    userSchema: Option[StructType] = None) extends LeafNode with MultiInstanceRelation {
    +
    +  override def simpleString: String = {
    +    s"DataSourceV2Relation(source=$sourceName, " +
    +      s"schema=[${output.map(a => s"$a ${a.dataType.simpleString}").mkString(", ")}], " +
    +      s"filters=[${pushedFilters.mkString(", ")}], options=$options)"
    +  }
    +
    +  override lazy val schema: StructType = reader.readSchema()
    +
    +  override lazy val output: Seq[AttributeReference] = {
    +    projection match {
    +      case Some(attrs) =>
    +        // use the projection attributes to avoid assigning new ids. fields that are not projected
    +        // will be assigned new ids, which is okay because they are not projected.
    +        val attrMap = attrs.map(a => a.name -> a).toMap
    +        schema.map(f => attrMap.getOrElse(f.name,
    +          AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()))
    +      case _ =>
    +        schema.toAttributes
    +    }
    +  }
    +
    +  private lazy val v2Options: DataSourceOptions = {
    +    // ensure path and table options are set correctly
    +    val updatedOptions = new mutable.HashMap[String, String]
    +    updatedOptions ++= options
    +
    +    new DataSourceOptions(options.asJava)
    +  }
    +
    +  private val sourceName: String = {
    +    source match {
    +      case registered: DataSourceRegister =>
    +        registered.shortName()
    +      case _ =>
    +        source.getClass.getSimpleName
    +    }
    +  }
    +
    +  lazy val (
    +      reader: DataSourceReader,
    +      unsupportedFilters: Seq[Expression],
    +      pushedFilters: Seq[Expression]) = {
    +    val newReader = userSchema match {
    +      case Some(s) =>
    +        asReadSupportWithSchema.createReader(s, v2Options)
    +      case _ =>
    +        asReadSupport.createReader(v2Options)
    +    }
    +
    +    projection.foreach { attrs =>
    +      DataSourceV2Relation.pushRequiredColumns(newReader, attrs.toStructType)
    +    }
    +
    +    val (remainingFilters, pushedFilters) = filters match {
    +      case Some(filterSeq) =>
    +        DataSourceV2Relation.pushFilters(newReader, filterSeq)
    +      case _ =>
    +        (Nil, Nil)
    +    }
    +
    +    (newReader, remainingFilters, pushedFilters)
    +  }
    +
    +  private lazy val asReadSupport: ReadSupport = {
    --- End diff --
    
    I think the implementation is pretty clear:
    
    ```java
    if (ds.isInstanceOf[ReadSupport] || ds.isInstanceOf[ReadSupportWithSchema]) {
      ...
    } else {
      loadV1Source(paths: _*)
    }
    ```


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/87123/
    Test PASSed.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r167005107
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,17 +17,130 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
     import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    -import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
    +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema, WriteSupport}
    +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics}
    +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    output: Seq[AttributeReference],
    -    reader: DataSourceReader)
    -  extends LeafNode with MultiInstanceRelation with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    projection: Option[Seq[AttributeReference]] = None,
    +    filters: Option[Seq[Expression]] = None,
    +    userSchema: Option[StructType] = None) extends LeafNode with MultiInstanceRelation {
    +
    +  override def simpleString: String = {
    +    s"DataSourceV2Relation(source=$sourceName, " +
    +      s"schema=[${output.map(a => s"$a ${a.dataType.simpleString}").mkString(", ")}], " +
    +      s"filters=[${pushedFilters.mkString(", ")}], options=$options)"
    +  }
    +
    +  override lazy val schema: StructType = reader.readSchema()
    +
    +  override lazy val output: Seq[AttributeReference] = {
    +    projection match {
    +      case Some(attrs) =>
    +        // use the projection attributes to avoid assigning new ids. fields that are not projected
    +        // will be assigned new ids, which is okay because they are not projected.
    +        val attrMap = attrs.map(a => a.name -> a).toMap
    +        schema.map(f => attrMap.getOrElse(f.name,
    +          AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()))
    +      case _ =>
    +        schema.toAttributes
    +    }
    +  }
    +
    +  private lazy val v2Options: DataSourceOptions = {
    +    // ensure path and table options are set correctly
    +    val updatedOptions = new mutable.HashMap[String, String]
    +    updatedOptions ++= options
    +
    +    new DataSourceOptions(options.asJava)
    +  }
    +
    +  private val sourceName: String = {
    +    source match {
    +      case registered: DataSourceRegister =>
    +        registered.shortName()
    +      case _ =>
    +        source.getClass.getSimpleName
    +    }
    +  }
    +
    +  lazy val (
    +      reader: DataSourceReader,
    +      unsupportedFilters: Seq[Expression],
    +      pushedFilters: Seq[Expression]) = {
    +    val newReader = userSchema match {
    +      case Some(s) =>
    +        asReadSupportWithSchema.createReader(s, v2Options)
    +      case _ =>
    +        asReadSupport.createReader(v2Options)
    +    }
    +
    +    projection.foreach { attrs =>
    +      DataSourceV2Relation.pushRequiredColumns(newReader, attrs.toStructType)
    +    }
    +
    +    val (remainingFilters, pushedFilters) = filters match {
    +      case Some(filterSeq) =>
    +        DataSourceV2Relation.pushFilters(newReader, filterSeq)
    +      case _ =>
    +        (Nil, Nil)
    +    }
    +
    +    (newReader, remainingFilters, pushedFilters)
    +  }
     
    -  override def canEqual(other: Any): Boolean = other.isInstanceOf[DataSourceV2Relation]
    +  def writer(dfSchema: StructType, mode: SaveMode): Option[DataSourceWriter] = {
    --- End diff --
    
    If you want this removed to get this commit in, please say so.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/876/
    Test PASSed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    **[Test build #87000 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87000/testReport)** for PR 20387 at commit [`9bb0141`](https://github.com/apache/spark/commit/9bb01416d68e9e2b7ed34745ba0a4b92721d98dd).


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r166392277
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,17 +17,151 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.TableIdentifier
     import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    -import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
    +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema, WriteSupport}
    +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics}
    +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    fullOutput: Seq[AttributeReference],
    -    reader: DataSourceReader)
    -  extends LeafNode with MultiInstanceRelation with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    path: Option[String] = None,
    +    table: Option[TableIdentifier] = None,
    --- End diff --
    
    I agree that `DataFrameReader` should not be the only place that creates `DataSourceV2Relation`, so handling `DataSourceOptions` in `DataFrameReader` is a bad idea.
    
    But handling `DataSourceOptions` in `DataSourceV2Relation` is not the only option. Like data source v1(see `DataSource`), we can have a central place to deal with the data source stuff, which can also minimize the number of places that need to handle `DataSourceOptions`.
    
    The actual benefits I see are:
    1. It's easier to write catalyst rules to match a `DataSourceV2Relation` with a specific table identifier.
    2. It's easier to reuse `DataSourceV2Relation` if we have data source v3 later, because we don't have any v2 specific stuff in the constructor.
    
    For 1, I think we can solve it by defining a `DataSourceV2Relation.unapply` to match the table identifier. For 2, I think it may not worth to consider it at this point.
    
    There might be more benefits I was missing, please correct me if I was wrong, thanks!


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r167374648
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,17 +17,131 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
     import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    -import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, ExprId}
    +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema, WriteSupport}
    +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics}
    +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    output: Seq[AttributeReference],
    -    reader: DataSourceReader)
    -  extends LeafNode with MultiInstanceRelation with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    projection: Option[Seq[AttributeReference]] = None,
    +    filters: Option[Seq[Expression]] = None,
    +    userSchema: Option[StructType] = None) extends LeafNode with MultiInstanceRelation {
    +
    +  override def simpleString: String = {
    +    s"DataSourceV2Relation(source=$sourceName, " +
    +      s"schema=[${output.map(a => s"$a ${a.dataType.simpleString}").mkString(", ")}], " +
    +      s"filters=[${pushedFilters.mkString(", ")}], options=$options)"
    +  }
    +
    +  override lazy val schema: StructType = reader.readSchema()
    +
    +  override lazy val output: Seq[AttributeReference] = {
    +    projection match {
    +      case Some(attrs) =>
    +        // use the projection attributes to avoid assigning new ids. fields that are not projected
    +        // will be assigned new ids, which is okay because they are not projected.
    +        val attrMap = attrs.map(a => a.name -> a).toMap
    +        schema.map(f => attrMap.getOrElse(f.name,
    +          AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()))
    +      case _ =>
    +        schema.toAttributes
    +    }
    +  }
    +
    +  private lazy val v2Options: DataSourceOptions = {
    +    // ensure path and table options are set correctly
    --- End diff --
    
    Removing it.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r167374952
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,17 +17,131 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
     import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    -import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, ExprId}
    +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema, WriteSupport}
    +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics}
    +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    output: Seq[AttributeReference],
    -    reader: DataSourceReader)
    -  extends LeafNode with MultiInstanceRelation with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    projection: Option[Seq[AttributeReference]] = None,
    --- End diff --
    
    I mentioned this below, but I should point it out on this thread, too: it is not correct to pass output to this relation. There's no guarantee that output will match the requested projection exactly, so in addition to the problem of leaking v2 details in the planner, this would make it easy to build a relation that doesn't correctly report its output.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r166827975
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,17 +17,130 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
     import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    -import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
    +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema, WriteSupport}
    +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics}
    +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    output: Seq[AttributeReference],
    -    reader: DataSourceReader)
    -  extends LeafNode with MultiInstanceRelation with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    projection: Option[Seq[AttributeReference]] = None,
    +    filters: Option[Seq[Expression]] = None,
    +    userSchema: Option[StructType] = None) extends LeafNode with MultiInstanceRelation {
    +
    +  override def simpleString: String = {
    +    s"DataSourceV2Relation(source=$sourceName, " +
    +      s"schema=[${output.map(a => s"$a ${a.dataType.simpleString}").mkString(", ")}], " +
    +      s"filters=[${pushedFilters.mkString(", ")}], options=$options)"
    +  }
    +
    +  override lazy val schema: StructType = reader.readSchema()
    +
    +  override lazy val output: Seq[AttributeReference] = {
    --- End diff --
    
    This reverts https://github.com/apache/spark/pull/20485 , can we still pass the test?


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r167344060
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,17 +17,131 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
     import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    -import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, ExprId}
    +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema, WriteSupport}
    +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics}
    +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    output: Seq[AttributeReference],
    -    reader: DataSourceReader)
    -  extends LeafNode with MultiInstanceRelation with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    projection: Option[Seq[AttributeReference]] = None,
    +    filters: Option[Seq[Expression]] = None,
    +    userSchema: Option[StructType] = None) extends LeafNode with MultiInstanceRelation {
    +
    +  override def simpleString: String = {
    +    s"DataSourceV2Relation(source=$sourceName, " +
    +      s"schema=[${output.map(a => s"$a ${a.dataType.simpleString}").mkString(", ")}], " +
    +      s"filters=[${pushedFilters.mkString(", ")}], options=$options)"
    +  }
    +
    +  override lazy val schema: StructType = reader.readSchema()
    +
    +  override lazy val output: Seq[AttributeReference] = {
    +    projection match {
    +      case Some(attrs) =>
    +        // use the projection attributes to avoid assigning new ids. fields that are not projected
    +        // will be assigned new ids, which is okay because they are not projected.
    +        val attrMap = attrs.map(a => a.name -> a).toMap
    +        schema.map(f => attrMap.getOrElse(f.name,
    +          AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()))
    +      case _ =>
    +        schema.toAttributes
    --- End diff --
    
    Sorry I just recalled, there is a rule(undocumented) that leaf node should make `output` as a constructor parameter. The reason is that, if the plan gets transformed and copied, the `output` can not change, otherwise parent node's expressions may refer to non-exist attributes.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    **[Test build #87002 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87002/testReport)** for PR 20387 at commit [`a7f0b90`](https://github.com/apache/spark/commit/a7f0b90b6ccb85c0801934ce7841831fe37b8739).


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    @cloud-fan, to your point about push-down order, I'm not saying that order doesn't matter at all, I'm saying that the push-down can run more than once and it should push the closest operators. That way, if you have a situation where operators can't be reordered but they can all be pushed, they all get pushed through multiple runs of the rule, each one further refining the relation.
    
    If we do it this way, then we don't need to traverse the logical plan to find out what to push down. We continue pushing projections until the plan stops changing. This is how the rest of the optimizer works, so I think it is a better approach from a design standpoint.
    
    My implementation also reuses more existing code that we have higher confidence in, which is a good thing. We can add things like limit pushdown later, by adding it properly to the existing code. I don't see a compelling reason to toss out the existing implementation, especially without the same level of testing.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    **[Test build #87434 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87434/testReport)** for PR 20387 at commit [`3b55609`](https://github.com/apache/spark/commit/3b55609b605fb461f6c2616d1da95a2d4b27ff4b).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    @cloud-fan, @dongjoon-hyun, @gatorsmile, I've rebased this on master and removed the support for SPARK-23204 that parses table identifiers. If you need other changes to get this in, let me know. As far as I'm aware, this isn't targeting 2.3.0 so it makes sense to keep the `PhysicalOperation` push-down rules.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    > `spark.read.format("iceberg").table("db.table").load()`
    
    I'm fine with this if you think it is confusing to parse the path as a table name in load. I think it is reasonable.
    
    I'd still like to keep the `Option[TableIdentifier]` parameter on the relation, so that we can support `table` or `insertInto` on the write path.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    **[Test build #87268 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87268/testReport)** for PR 20387 at commit [`87a36be`](https://github.com/apache/spark/commit/87a36be5b8dc2002a45bf71ffdc94c816f6d7355).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r167754111
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -37,22 +100,129 @@ case class DataSourceV2Relation(
       }
     
       override def newInstance(): DataSourceV2Relation = {
    -    copy(output = output.map(_.newInstance()))
    +    // projection is used to maintain id assignment.
    +    // if projection is not set, use output so the copy is not equal to the original
    +    copy(projection = projection.map(_.newInstance()))
       }
     }
     
     /**
      * A specialization of DataSourceV2Relation with the streaming bit set to true. Otherwise identical
      * to the non-streaming relation.
      */
    -class StreamingDataSourceV2Relation(
    +case class StreamingDataSourceV2Relation(
         output: Seq[AttributeReference],
    -    reader: DataSourceReader) extends DataSourceV2Relation(output, reader) {
    +    reader: DataSourceReader)
    +    extends LeafNode with DataSourceReaderHolder with MultiInstanceRelation {
       override def isStreaming: Boolean = true
    +
    +  override def canEqual(other: Any): Boolean = other.isInstanceOf[StreamingDataSourceV2Relation]
    +
    +  override def newInstance(): LogicalPlan = copy(output = output.map(_.newInstance()))
     }
     
     object DataSourceV2Relation {
    -  def apply(reader: DataSourceReader): DataSourceV2Relation = {
    -    new DataSourceV2Relation(reader.readSchema().toAttributes, reader)
    +  private implicit class SourceHelpers(source: DataSourceV2) {
    +    def asReadSupport: ReadSupport = {
    +      source match {
    +        case support: ReadSupport =>
    +          support
    +        case _: ReadSupportWithSchema =>
    +          // this method is only called if there is no user-supplied schema. if there is no
    +          // user-supplied schema and ReadSupport was not implemented, throw a helpful exception.
    +          throw new AnalysisException(s"Data source requires a user-supplied schema: $name")
    +        case _ =>
    +          throw new AnalysisException(s"Data source is not readable: $name")
    +      }
    +    }
    +
    +    def asReadSupportWithSchema: ReadSupportWithSchema = {
    +      source match {
    +        case support: ReadSupportWithSchema =>
    +          support
    +        case _: ReadSupport =>
    --- End diff --
    
    There was a historical reason we do this: https://github.com/apache/spark/pull/15046
    
    I agree it's more clear to not allow this since data source v2 is brand new. But this change worths a JIRA ticket and an individual PR, do you mind to create one? Or I can do that for you.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    @dongjoon-hyun, @gatorsmile, could you guys weigh in on some this discussion? I'd like to get additional perspectives on the changes I'm proposing.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    @cloud-fan, this is a single commit on purpose because predicate push-down makes plan changes. I think it's best to do these at once to avoid unnecessary work. That's why I started looking more closely at push-down in the first place: updating the other push-down code for immutable plans was a mess.
    
    I also think it is unlikely that we will need to revert the push-down changes here. If we end up redesigning push-down, then it is unlikely that the easiest starting point is to roll back this fix.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r167141001
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,17 +17,130 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
     import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    -import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
    +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema, WriteSupport}
    +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics}
    +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    output: Seq[AttributeReference],
    -    reader: DataSourceReader)
    -  extends LeafNode with MultiInstanceRelation with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    projection: Option[Seq[AttributeReference]] = None,
    +    filters: Option[Seq[Expression]] = None,
    +    userSchema: Option[StructType] = None) extends LeafNode with MultiInstanceRelation {
    +
    +  override def simpleString: String = {
    +    s"DataSourceV2Relation(source=$sourceName, " +
    +      s"schema=[${output.map(a => s"$a ${a.dataType.simpleString}").mkString(", ")}], " +
    +      s"filters=[${pushedFilters.mkString(", ")}], options=$options)"
    +  }
    +
    +  override lazy val schema: StructType = reader.readSchema()
    +
    +  override lazy val output: Seq[AttributeReference] = {
    +    projection match {
    +      case Some(attrs) =>
    +        // use the projection attributes to avoid assigning new ids. fields that are not projected
    +        // will be assigned new ids, which is okay because they are not projected.
    +        val attrMap = attrs.map(a => a.name -> a).toMap
    +        schema.map(f => attrMap.getOrElse(f.name,
    +          AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()))
    +      case _ =>
    +        schema.toAttributes
    +    }
    +  }
    +
    +  private lazy val v2Options: DataSourceOptions = {
    +    // ensure path and table options are set correctly
    +    val updatedOptions = new mutable.HashMap[String, String]
    +    updatedOptions ++= options
    +
    +    new DataSourceOptions(options.asJava)
    +  }
    +
    +  private val sourceName: String = {
    +    source match {
    +      case registered: DataSourceRegister =>
    +        registered.shortName()
    +      case _ =>
    +        source.getClass.getSimpleName
    +    }
    +  }
    +
    +  lazy val (
    +      reader: DataSourceReader,
    +      unsupportedFilters: Seq[Expression],
    +      pushedFilters: Seq[Expression]) = {
    +    val newReader = userSchema match {
    +      case Some(s) =>
    +        asReadSupportWithSchema.createReader(s, v2Options)
    +      case _ =>
    +        asReadSupport.createReader(v2Options)
    +    }
    +
    +    projection.foreach { attrs =>
    +      DataSourceV2Relation.pushRequiredColumns(newReader, attrs.toStructType)
    +    }
    +
    +    val (remainingFilters, pushedFilters) = filters match {
    +      case Some(filterSeq) =>
    +        DataSourceV2Relation.pushFilters(newReader, filterSeq)
    +      case _ =>
    +        (Nil, Nil)
    +    }
    +
    +    (newReader, remainingFilters, pushedFilters)
    +  }
     
    -  override def canEqual(other: Any): Boolean = other.isInstanceOf[DataSourceV2Relation]
    +  def writer(dfSchema: StructType, mode: SaveMode): Option[DataSourceWriter] = {
    --- End diff --
    
    I think we should avoid adding unused code that is needed in the future. The streaming data source v2 was a bad example and you already pointed it out. Hope we don't make the same mistake in the future.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/87532/
    Test PASSed.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Us...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r163909751
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,15 +17,149 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
     import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceV2, DataSourceV2Options, ReadSupport, ReadSupportWithSchema, WriteSupport}
     import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.sources.v2.writer.DataSourceV2Writer
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    fullOutput: Seq[AttributeReference],
    -    reader: DataSourceV2Reader) extends LeafNode with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    path: Option[String] = None,
    +    table: Option[TableIdentifier] = None,
    +    projection: Option[Seq[AttributeReference]] = None,
    +    filters: Option[Seq[Expression]] = None,
    --- End diff --
    
    I'm not sure I understand what you mean. When something is pushed, it creates a new immutable relation, so I think it has to be added to the relation. But I'm not sure that many things will be pushed besides the projection and filters. What are you thinking that we would need to add? Fragments of logical plan?
    
    Assuming we add the ability to push parts of the logical plan, then this would need to have a reference to the part that was pushed down. I'm not sure that would be this relation class, a subclass, or something else, but I would be fine adding a third push-down option here. The number of things to push down isn't very large, is it?


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r167004846
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,17 +17,130 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
     import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    -import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
    +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema, WriteSupport}
    +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics}
    +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    output: Seq[AttributeReference],
    -    reader: DataSourceReader)
    -  extends LeafNode with MultiInstanceRelation with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    projection: Option[Seq[AttributeReference]] = None,
    +    filters: Option[Seq[Expression]] = None,
    +    userSchema: Option[StructType] = None) extends LeafNode with MultiInstanceRelation {
    +
    +  override def simpleString: String = {
    +    s"DataSourceV2Relation(source=$sourceName, " +
    +      s"schema=[${output.map(a => s"$a ${a.dataType.simpleString}").mkString(", ")}], " +
    +      s"filters=[${pushedFilters.mkString(", ")}], options=$options)"
    +  }
    +
    +  override lazy val schema: StructType = reader.readSchema()
    +
    +  override lazy val output: Seq[AttributeReference] = {
    --- End diff --
    
    That commit was not reverted when I rebased. The test is still present and passing: https://github.com/apache/spark/blob/181946d1f1c5889661544830a77bd23c4b4f685a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala#L320-L336


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r166394747
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,17 +17,151 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.TableIdentifier
     import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    -import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
    +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema, WriteSupport}
    +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics}
    +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    fullOutput: Seq[AttributeReference],
    -    reader: DataSourceReader)
    -  extends LeafNode with MultiInstanceRelation with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    path: Option[String] = None,
    +    table: Option[TableIdentifier] = None,
    --- End diff --
    
    I'm not saying that `DataSourceOptions` have to be handled in the relation. Just that the relation should use the same classes to pass data, like `TableIdentifier`, that are used by the rest of the planner. I agree with those benefits of doing this.
    
    Is there anything that needs to change in this PR? We can move where the options are created in a follow-up, but let me know if you think this would prevent this from getting merged.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/637/
    Test PASSed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    @cloud-fan, can you have a look at this? I've made the requested changes and tests are passing.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    > We've added a resolution rule from UnresolvedRelation to DataSourceV2Relation that uses our implementation. UnresolvedRelation needs to pass its TableIdentifier to the v2 relation, which is why I added this.
    
    I've been thinking about this a little more. This is actually an existing problem for file-based data sources. The solution is, when converting an unresolved relation to data source relation, we add some new options to the existing data source options before passing the options to data source relation. See `FindDataSourceTable.readDataSourceTable` about how we handle the path option.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    **[Test build #87001 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87001/testReport)** for PR 20387 at commit [`63f11a9`](https://github.com/apache/spark/commit/63f11a92ffefaf937dd0266e3e59d619d33ab873).


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r166081367
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,17 +17,151 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.TableIdentifier
     import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    -import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
    +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema, WriteSupport}
    +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics}
    +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    fullOutput: Seq[AttributeReference],
    -    reader: DataSourceReader)
    -  extends LeafNode with MultiInstanceRelation with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    path: Option[String] = None,
    +    table: Option[TableIdentifier] = None,
    --- End diff --
    
    I should clarify this as well: I like the idea of standardizing how implementations access the table name from `DataSourceOptions`. However, I don't think that is sufficient to remove the table and path here in the definition of `DataSourceV2Relation` for the reasons above.
    
    I think we *should* follow this commit with a plan for how implementations access these options. For now, it is good to put the creation of those options in a single place.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    > Why pushdown is happening in logical optimization and not during query planning. My first instinct would be to have the optimizer get operators as close to the leaves as possible and then fuse (or push down) as we convert to physical plan. I'm probably missing something.
    
    I think there are two reasons, but I'm not fully convinced by either one:
    
    * [`computeStats`](https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala#L232) is defined on logical plans, so the result of filter push-down needs to be a logical plan if we want to be able to use accurate stats for a scan. I'm interested here to ensure that we correctly produce broadcast relations based on the actual scan stats, not the table-level stats. Maybe there's another way to do this?
    * One of the tests for DSv2 ends up invoking the push-down rule twice, which made me think about whether or not that should be valid. I think it probably should be. For example, what if a plan has nodes that can all be pushed, but they aren't in the right order? Or what if a projection wasn't pushed through a filter because of a rule problem, but it can still be pushed down? Incremental fusing during optimization might be an extensible way to handle odd cases, or it may be useless. I'm not quite sure yet.
    
    It would be great to hear your perspective on these.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r166386661
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,17 +17,151 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.TableIdentifier
     import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    -import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
    +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema, WriteSupport}
    +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics}
    +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    fullOutput: Seq[AttributeReference],
    -    reader: DataSourceReader)
    -  extends LeafNode with MultiInstanceRelation with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    path: Option[String] = None,
    +    table: Option[TableIdentifier] = None,
    --- End diff --
    
    I think that `TableIdentifier` and a string-to-string `Map` should be passed to `DataSourceV2Relation` and that either the relation or `DataSourceOptions` should be responsible for creating `DataSourceOptions` with well-defined properties to to pass the table information to implementations.
    
    This minimizes the number of places that need to handle `DataSourceOptions` (which is specific to v2) and uses `TableIdentifier` and `Map` to match the rest of the planner nodes. For example, other read paths that can create `DataSourceV2Relation`, like resolution rules, use `TableIdentifier`.
    
    I'm not currently advocating a position on how to configure `DataFrameReader` or `DataFrameWriter` or how to handle schemas.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r167343364
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,17 +17,131 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
     import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    -import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, ExprId}
    +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema, WriteSupport}
    +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics}
    +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    output: Seq[AttributeReference],
    -    reader: DataSourceReader)
    -  extends LeafNode with MultiInstanceRelation with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    projection: Option[Seq[AttributeReference]] = None,
    --- End diff --
    
    This is a bad idea because it means that any place that creates this relation needs to know to create a reader and get its schema to pass in. That's confusing because some code will pass a user schema and some code will pass a projection.
    
    Keeping this logic inside of the relation ensures that it is done correctly and without leaking details from the v2 API into the rest of the planner.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/87432/
    Test FAILed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/538/
    Test PASSed.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r166165005
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,17 +17,151 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.TableIdentifier
     import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    -import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
    +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema, WriteSupport}
    +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics}
    +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    fullOutput: Seq[AttributeReference],
    -    reader: DataSourceReader)
    -  extends LeafNode with MultiInstanceRelation with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    path: Option[String] = None,
    +    table: Option[TableIdentifier] = None,
    +    projection: Option[Seq[AttributeReference]] = None,
    +    filters: Option[Seq[Expression]] = None,
    +    userSchema: Option[StructType] = None) extends LeafNode with MultiInstanceRelation {
    +
    +  override def simpleString: String = {
    +    "DataSourceV2Relation(" +
    +      s"source=$sourceName${path.orElse(table).map(loc => s"($loc)").getOrElse("")}, " +
    +      s"schema=[${output.map(a => s"$a ${a.dataType.simpleString}").mkString(", ")}], " +
    +      s"filters=[${pushedFilters.mkString(", ")}] options=$options)"
    +  }
    +
    +  override lazy val schema: StructType = reader.readSchema()
    +
    +  override lazy val output: Seq[AttributeReference] = {
    +    projection match {
    +      case Some(attrs) =>
    +        // use the projection attributes to avoid assigning new ids. fields that are not projected
    +        // will be assigned new ids, which is okay because they are not projected.
    +        val attrMap = attrs.map(a => a.name -> a).toMap
    +        schema.map(f => attrMap.getOrElse(f.name,
    +          AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()))
    +      case _ =>
    +        schema.toAttributes
    +    }
    +  }
    +
    +  private lazy val v2Options: DataSourceOptions = {
    +    // ensure path and table options are set correctly
    +    val updatedOptions = new mutable.HashMap[String, String]
    +    updatedOptions ++= options
    +
    +    path match {
    +      case Some(p) =>
    +        updatedOptions.put("path", p)
    +      case None =>
    +        updatedOptions.remove("path")
    +    }
    +
    +    table.map { ident =>
    +      updatedOptions.put("table", ident.table)
    --- End diff --
    
    I think we agree here. I want to avoid doing this outside of either `DataSourceOptions` or `DataSourceV2Relation`. If we can create `DataSourceOptions` from a `Option[TableIdentifier]` and add the `getTable` accessor, then that works for me. My main motivation is to avoid having this piece of code copied throughout the SQL planner.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r166163929
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,17 +17,151 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.TableIdentifier
     import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    -import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
    +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema, WriteSupport}
    +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics}
    +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    fullOutput: Seq[AttributeReference],
    -    reader: DataSourceReader)
    -  extends LeafNode with MultiInstanceRelation with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    path: Option[String] = None,
    +    table: Option[TableIdentifier] = None,
    +    projection: Option[Seq[AttributeReference]] = None,
    +    filters: Option[Seq[Expression]] = None,
    +    userSchema: Option[StructType] = None) extends LeafNode with MultiInstanceRelation {
    +
    +  override def simpleString: String = {
    +    "DataSourceV2Relation(" +
    +      s"source=$sourceName${path.orElse(table).map(loc => s"($loc)").getOrElse("")}, " +
    +      s"schema=[${output.map(a => s"$a ${a.dataType.simpleString}").mkString(", ")}], " +
    +      s"filters=[${pushedFilters.mkString(", ")}] options=$options)"
    +  }
    +
    +  override lazy val schema: StructType = reader.readSchema()
    +
    +  override lazy val output: Seq[AttributeReference] = {
    +    projection match {
    +      case Some(attrs) =>
    +        // use the projection attributes to avoid assigning new ids. fields that are not projected
    +        // will be assigned new ids, which is okay because they are not projected.
    +        val attrMap = attrs.map(a => a.name -> a).toMap
    +        schema.map(f => attrMap.getOrElse(f.name,
    +          AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()))
    +      case _ =>
    +        schema.toAttributes
    +    }
    +  }
    +
    +  private lazy val v2Options: DataSourceOptions = {
    +    // ensure path and table options are set correctly
    +    val updatedOptions = new mutable.HashMap[String, String]
    +    updatedOptions ++= options
    +
    +    path match {
    +      case Some(p) =>
    +        updatedOptions.put("path", p)
    +      case None =>
    +        updatedOptions.remove("path")
    +    }
    +
    +    table.map { ident =>
    +      updatedOptions.put("table", ident.table)
    --- End diff --
    
    this is something I wanna avoid. At the end we still need to define some standard option keys, like `path` and `table`. It would be good if we can provide some methods for end users and data source developers to easily write/read these standard options. I agree this can be done in a different PR.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r168910531
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownOperatorsToDataSource.scala ---
    @@ -17,130 +17,55 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    -import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeMap, AttributeSet, Expression, NamedExpression, PredicateHelper}
    -import org.apache.spark.sql.catalyst.optimizer.RemoveRedundantProject
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, AttributeSet}
    +import org.apache.spark.sql.catalyst.planning.PhysicalOperation
     import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project}
     import org.apache.spark.sql.catalyst.rules.Rule
    -import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    -import org.apache.spark.sql.sources
    -import org.apache.spark.sql.sources.v2.reader._
     
    -/**
    - * Pushes down various operators to the underlying data source for better performance. Operators are
    - * being pushed down with a specific order. As an example, given a LIMIT has a FILTER child, you
    - * can't push down LIMIT if FILTER is not completely pushed down. When both are pushed down, the
    - * data source should execute FILTER before LIMIT. And required columns are calculated at the end,
    - * because when more operators are pushed down, we may need less columns at Spark side.
    - */
    -object PushDownOperatorsToDataSource extends Rule[LogicalPlan] with PredicateHelper {
    -  override def apply(plan: LogicalPlan): LogicalPlan = {
    -    // Note that, we need to collect the target operator along with PROJECT node, as PROJECT may
    -    // appear in many places for column pruning.
    -    // TODO: Ideally column pruning should be implemented via a plan property that is propagated
    -    // top-down, then we can simplify the logic here and only collect target operators.
    -    val filterPushed = plan transformUp {
    -      case FilterAndProject(fields, condition, r @ DataSourceV2Relation(_, reader)) =>
    -        val (candidates, nonDeterministic) =
    -          splitConjunctivePredicates(condition).partition(_.deterministic)
    -
    -        val stayUpFilters: Seq[Expression] = reader match {
    -          case r: SupportsPushDownCatalystFilters =>
    -            r.pushCatalystFilters(candidates.toArray)
    -
    -          case r: SupportsPushDownFilters =>
    -            // A map from original Catalyst expressions to corresponding translated data source
    -            // filters. If a predicate is not in this map, it means it cannot be pushed down.
    -            val translatedMap: Map[Expression, sources.Filter] = candidates.flatMap { p =>
    -              DataSourceStrategy.translateFilter(p).map(f => p -> f)
    -            }.toMap
    -
    -            // Catalyst predicate expressions that cannot be converted to data source filters.
    -            val nonConvertiblePredicates = candidates.filterNot(translatedMap.contains)
    -
    -            // Data source filters that cannot be pushed down. An unhandled filter means
    -            // the data source cannot guarantee the rows returned can pass the filter.
    -            // As a result we must return it so Spark can plan an extra filter operator.
    -            val unhandledFilters = r.pushFilters(translatedMap.values.toArray).toSet
    -            val unhandledPredicates = translatedMap.filter { case (_, f) =>
    -              unhandledFilters.contains(f)
    -            }.keys
    -
    -            nonConvertiblePredicates ++ unhandledPredicates
    -
    -          case _ => candidates
    -        }
    -
    -        val filterCondition = (stayUpFilters ++ nonDeterministic).reduceLeftOption(And)
    -        val withFilter = filterCondition.map(Filter(_, r)).getOrElse(r)
    -        if (withFilter.output == fields) {
    -          withFilter
    -        } else {
    -          Project(fields, withFilter)
    -        }
    -    }
    -
    -    // TODO: add more push down rules.
    -
    -    val columnPruned = pushDownRequiredColumns(filterPushed, filterPushed.outputSet)
    -    // After column pruning, we may have redundant PROJECT nodes in the query plan, remove them.
    -    RemoveRedundantProject(columnPruned)
    -  }
    -
    -  // TODO: nested fields pruning
    -  private def pushDownRequiredColumns(
    -      plan: LogicalPlan, requiredByParent: AttributeSet): LogicalPlan = {
    -    plan match {
    -      case p @ Project(projectList, child) =>
    -        val required = projectList.flatMap(_.references)
    -        p.copy(child = pushDownRequiredColumns(child, AttributeSet(required)))
    -
    -      case f @ Filter(condition, child) =>
    -        val required = requiredByParent ++ condition.references
    -        f.copy(child = pushDownRequiredColumns(child, required))
    -
    -      case relation: DataSourceV2Relation => relation.reader match {
    -        case reader: SupportsPushDownRequiredColumns =>
    -          // TODO: Enable the below assert after we make `DataSourceV2Relation` immutable. Fow now
    -          // it's possible that the mutable reader being updated by someone else, and we need to
    -          // always call `reader.pruneColumns` here to correct it.
    -          // assert(relation.output.toStructType == reader.readSchema(),
    -          //  "Schema of data source reader does not match the relation plan.")
    -
    -          val requiredColumns = relation.output.filter(requiredByParent.contains)
    -          reader.pruneColumns(requiredColumns.toStructType)
    -
    -          val nameToAttr = relation.output.map(_.name).zip(relation.output).toMap
    -          val newOutput = reader.readSchema().map(_.name).map(nameToAttr)
    -          relation.copy(output = newOutput)
    -
    -        case _ => relation
    +object PushDownOperatorsToDataSource extends Rule[LogicalPlan] {
    +  override def apply(
    +      plan: LogicalPlan): LogicalPlan = plan transformUp {
    +    // PhysicalOperation guarantees that filters are deterministic; no need to check
    +    case PhysicalOperation(project, newFilters, relation : DataSourceV2Relation) =>
    +      // merge the filters
    +      val filters = relation.filters match {
    +        case Some(existing) =>
    +          existing ++ newFilters
    +        case _ =>
    +          newFilters
           }
     
    -      // TODO: there may be more operators that can be used to calculate the required columns. We
    -      // can add more and more in the future.
    -      case _ => plan.mapChildren(c => pushDownRequiredColumns(c, c.outputSet))
    -    }
    -  }
    -
    -  /**
    -   * Finds a Filter node(with an optional Project child) above data source relation.
    -   */
    -  object FilterAndProject {
    -    // returns the project list, the filter condition and the data source relation.
    -    def unapply(plan: LogicalPlan)
    -        : Option[(Seq[NamedExpression], Expression, DataSourceV2Relation)] = plan match {
    +      val projectAttrs = project.map(_.toAttribute)
    +      val projectSet = AttributeSet(project.flatMap(_.references))
    +      val filterSet = AttributeSet(filters.flatMap(_.references))
    +
    +      val projection = if (filterSet.subsetOf(projectSet) &&
    +          AttributeSet(projectAttrs) == projectSet) {
    +        // When the required projection contains all of the filter columns and column pruning alone
    +        // can produce the required projection, push the required projection.
    +        // A final projection may still be needed if the data source produces a different column
    +        // order or if it cannot prune all of the nested columns.
    +        projectAttrs
    +      } else {
    +        // When there are filter columns not already in the required projection or when the required
    +        // projection is more complicated than column pruning, base column pruning on the set of
    +        // all columns needed by both.
    +        (projectSet ++ filterSet).toSeq
    +      }
     
    -      case Filter(condition, r: DataSourceV2Relation) => Some((r.output, condition, r))
    +      val newRelation = relation.copy(
    +        projection = projection.asInstanceOf[Seq[AttributeReference]],
    --- End diff --
    
    shall we just update `output` here? Since now we create a temp reader to get the schema before constructing `DataSourceV2Relation`, I don't see much value to have a `projection` separated from `output`.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r166827284
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,17 +17,130 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
     import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    -import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
    +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema, WriteSupport}
    +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics}
    +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    output: Seq[AttributeReference],
    -    reader: DataSourceReader)
    -  extends LeafNode with MultiInstanceRelation with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    projection: Option[Seq[AttributeReference]] = None,
    +    filters: Option[Seq[Expression]] = None,
    +    userSchema: Option[StructType] = None) extends LeafNode with MultiInstanceRelation {
    +
    +  override def simpleString: String = {
    +    s"DataSourceV2Relation(source=$sourceName, " +
    +      s"schema=[${output.map(a => s"$a ${a.dataType.simpleString}").mkString(", ")}], " +
    +      s"filters=[${pushedFilters.mkString(", ")}], options=$options)"
    +  }
    +
    +  override lazy val schema: StructType = reader.readSchema()
    +
    +  override lazy val output: Seq[AttributeReference] = {
    +    projection match {
    +      case Some(attrs) =>
    +        // use the projection attributes to avoid assigning new ids. fields that are not projected
    +        // will be assigned new ids, which is okay because they are not projected.
    +        val attrMap = attrs.map(a => a.name -> a).toMap
    +        schema.map(f => attrMap.getOrElse(f.name,
    +          AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()))
    +      case _ =>
    +        schema.toAttributes
    +    }
    +  }
    +
    +  private lazy val v2Options: DataSourceOptions = {
    +    // ensure path and table options are set correctly
    +    val updatedOptions = new mutable.HashMap[String, String]
    +    updatedOptions ++= options
    +
    +    new DataSourceOptions(options.asJava)
    +  }
    +
    +  private val sourceName: String = {
    +    source match {
    +      case registered: DataSourceRegister =>
    +        registered.shortName()
    +      case _ =>
    +        source.getClass.getSimpleName
    +    }
    +  }
    +
    +  lazy val (
    +      reader: DataSourceReader,
    +      unsupportedFilters: Seq[Expression],
    +      pushedFilters: Seq[Expression]) = {
    +    val newReader = userSchema match {
    +      case Some(s) =>
    +        asReadSupportWithSchema.createReader(s, v2Options)
    +      case _ =>
    +        asReadSupport.createReader(v2Options)
    +    }
    +
    +    projection.foreach { attrs =>
    +      DataSourceV2Relation.pushRequiredColumns(newReader, attrs.toStructType)
    +    }
    +
    +    val (remainingFilters, pushedFilters) = filters match {
    +      case Some(filterSeq) =>
    +        DataSourceV2Relation.pushFilters(newReader, filterSeq)
    +      case _ =>
    +        (Nil, Nil)
    +    }
    +
    +    (newReader, remainingFilters, pushedFilters)
    +  }
     
    -  override def canEqual(other: Any): Boolean = other.isInstanceOf[DataSourceV2Relation]
    +  def writer(dfSchema: StructType, mode: SaveMode): Option[DataSourceWriter] = {
    --- End diff --
    
    This method is not used anywhere.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Us...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r163913161
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,15 +17,149 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
     import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceV2, DataSourceV2Options, ReadSupport, ReadSupportWithSchema, WriteSupport}
     import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.sources.v2.writer.DataSourceV2Writer
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    fullOutput: Seq[AttributeReference],
    -    reader: DataSourceV2Reader) extends LeafNode with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    path: Option[String] = None,
    +    table: Option[TableIdentifier] = None,
    --- End diff --
    
    I guess another way to say this is that it's better to set reliable path, database, and table parameters after passing the explicitly, than to require all the places where DataSourceV2Relations are created do the same thing. Better to standardize passing these options in `v2Options`, and it would be even better to pass these directly to the readers and writers.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r165981421
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,17 +17,151 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.TableIdentifier
     import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    -import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
    +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema, WriteSupport}
    +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics}
    +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    fullOutput: Seq[AttributeReference],
    -    reader: DataSourceReader)
    -  extends LeafNode with MultiInstanceRelation with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    path: Option[String] = None,
    +    table: Option[TableIdentifier] = None,
    --- End diff --
    
    have you considered about https://github.com/apache/spark/pull/20387#issuecomment-362148217 ?
    
    I feel it's better to define these common options in `DataSourceOptions`, so that data source implementations can also get these common options easily.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    **[Test build #86602 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86602/testReport)** for PR 20387 at commit [`83203a6`](https://github.com/apache/spark/commit/83203a6e117f180b1839c815e4c3b3ef539f6b2b).


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/87434/
    Test PASSed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    **[Test build #86602 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86602/testReport)** for PR 20387 at commit [`83203a6`](https://github.com/apache/spark/commit/83203a6e117f180b1839c815e4c3b3ef539f6b2b).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #20387: SPARK-22386: DataSourceV2: Use immutable logical plans.

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/87002/
    Test FAILed.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r167346043
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownOperatorsToDataSource.scala ---
    @@ -17,130 +17,55 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    -import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeMap, AttributeSet, Expression, NamedExpression, PredicateHelper}
    -import org.apache.spark.sql.catalyst.optimizer.RemoveRedundantProject
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, AttributeSet}
    +import org.apache.spark.sql.catalyst.planning.PhysicalOperation
     import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project}
     import org.apache.spark.sql.catalyst.rules.Rule
    -import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    -import org.apache.spark.sql.sources
    -import org.apache.spark.sql.sources.v2.reader._
     
    -/**
    - * Pushes down various operators to the underlying data source for better performance. Operators are
    - * being pushed down with a specific order. As an example, given a LIMIT has a FILTER child, you
    - * can't push down LIMIT if FILTER is not completely pushed down. When both are pushed down, the
    - * data source should execute FILTER before LIMIT. And required columns are calculated at the end,
    - * because when more operators are pushed down, we may need less columns at Spark side.
    - */
    -object PushDownOperatorsToDataSource extends Rule[LogicalPlan] with PredicateHelper {
    -  override def apply(plan: LogicalPlan): LogicalPlan = {
    -    // Note that, we need to collect the target operator along with PROJECT node, as PROJECT may
    -    // appear in many places for column pruning.
    -    // TODO: Ideally column pruning should be implemented via a plan property that is propagated
    -    // top-down, then we can simplify the logic here and only collect target operators.
    -    val filterPushed = plan transformUp {
    -      case FilterAndProject(fields, condition, r @ DataSourceV2Relation(_, reader)) =>
    -        val (candidates, nonDeterministic) =
    -          splitConjunctivePredicates(condition).partition(_.deterministic)
    -
    -        val stayUpFilters: Seq[Expression] = reader match {
    -          case r: SupportsPushDownCatalystFilters =>
    -            r.pushCatalystFilters(candidates.toArray)
    -
    -          case r: SupportsPushDownFilters =>
    -            // A map from original Catalyst expressions to corresponding translated data source
    -            // filters. If a predicate is not in this map, it means it cannot be pushed down.
    -            val translatedMap: Map[Expression, sources.Filter] = candidates.flatMap { p =>
    -              DataSourceStrategy.translateFilter(p).map(f => p -> f)
    -            }.toMap
    -
    -            // Catalyst predicate expressions that cannot be converted to data source filters.
    -            val nonConvertiblePredicates = candidates.filterNot(translatedMap.contains)
    -
    -            // Data source filters that cannot be pushed down. An unhandled filter means
    -            // the data source cannot guarantee the rows returned can pass the filter.
    -            // As a result we must return it so Spark can plan an extra filter operator.
    -            val unhandledFilters = r.pushFilters(translatedMap.values.toArray).toSet
    -            val unhandledPredicates = translatedMap.filter { case (_, f) =>
    -              unhandledFilters.contains(f)
    -            }.keys
    -
    -            nonConvertiblePredicates ++ unhandledPredicates
    -
    -          case _ => candidates
    -        }
    -
    -        val filterCondition = (stayUpFilters ++ nonDeterministic).reduceLeftOption(And)
    -        val withFilter = filterCondition.map(Filter(_, r)).getOrElse(r)
    -        if (withFilter.output == fields) {
    -          withFilter
    -        } else {
    -          Project(fields, withFilter)
    -        }
    -    }
    -
    -    // TODO: add more push down rules.
    -
    -    val columnPruned = pushDownRequiredColumns(filterPushed, filterPushed.outputSet)
    -    // After column pruning, we may have redundant PROJECT nodes in the query plan, remove them.
    -    RemoveRedundantProject(columnPruned)
    -  }
    -
    -  // TODO: nested fields pruning
    -  private def pushDownRequiredColumns(
    -      plan: LogicalPlan, requiredByParent: AttributeSet): LogicalPlan = {
    -    plan match {
    -      case p @ Project(projectList, child) =>
    -        val required = projectList.flatMap(_.references)
    -        p.copy(child = pushDownRequiredColumns(child, AttributeSet(required)))
    -
    -      case f @ Filter(condition, child) =>
    -        val required = requiredByParent ++ condition.references
    -        f.copy(child = pushDownRequiredColumns(child, required))
    -
    -      case relation: DataSourceV2Relation => relation.reader match {
    -        case reader: SupportsPushDownRequiredColumns =>
    -          // TODO: Enable the below assert after we make `DataSourceV2Relation` immutable. Fow now
    -          // it's possible that the mutable reader being updated by someone else, and we need to
    -          // always call `reader.pruneColumns` here to correct it.
    -          // assert(relation.output.toStructType == reader.readSchema(),
    -          //  "Schema of data source reader does not match the relation plan.")
    -
    -          val requiredColumns = relation.output.filter(requiredByParent.contains)
    -          reader.pruneColumns(requiredColumns.toStructType)
    -
    -          val nameToAttr = relation.output.map(_.name).zip(relation.output).toMap
    -          val newOutput = reader.readSchema().map(_.name).map(nameToAttr)
    -          relation.copy(output = newOutput)
    -
    -        case _ => relation
    +object PushDownOperatorsToDataSource extends Rule[LogicalPlan] {
    +  override def apply(
    +      plan: LogicalPlan): LogicalPlan = plan transformUp {
    +    // PhysicalOperation guarantees that filters are deterministic; no need to check
    +    case PhysicalOperation(project, newFilters, relation : DataSourceV2Relation) =>
    +      // merge the filters
    +      val filters = relation.filters match {
    +        case Some(existing) =>
    +          existing ++ newFilters
    +        case _ =>
    +          newFilters
           }
     
    -      // TODO: there may be more operators that can be used to calculate the required columns. We
    -      // can add more and more in the future.
    -      case _ => plan.mapChildren(c => pushDownRequiredColumns(c, c.outputSet))
    -    }
    -  }
    -
    -  /**
    -   * Finds a Filter node(with an optional Project child) above data source relation.
    -   */
    -  object FilterAndProject {
    -    // returns the project list, the filter condition and the data source relation.
    -    def unapply(plan: LogicalPlan)
    -        : Option[(Seq[NamedExpression], Expression, DataSourceV2Relation)] = plan match {
    +      val projectAttrs = project.map(_.toAttribute)
    +      val projectSet = AttributeSet(project.flatMap(_.references))
    +      val filterSet = AttributeSet(filters.flatMap(_.references))
    +
    +      val projection = if (filterSet.subsetOf(projectSet) &&
    +          AttributeSet(projectAttrs) == projectSet) {
    +        // When the required projection contains all of the filter columns and column pruning alone
    +        // can produce the required projection, push the required projection.
    +        // A final projection may still be needed if the data source produces a different column
    +        // order or if it cannot prune all of the nested columns.
    +        projectAttrs
    +      } else {
    +        // When there are filter columns not already in the required projection or when the required
    +        // projection is more complicated than column pruning, base column pruning on the set of
    +        // all columns needed by both.
    +        (projectSet ++ filterSet).toSeq
    +      }
     
    -      case Filter(condition, r: DataSourceV2Relation) => Some((r.output, condition, r))
    +      val newRelation = relation.copy(
    +        projection = Some(projection.asInstanceOf[Seq[AttributeReference]]),
    +        filters = Some(filters))
    --- End diff --
    
    I agree, but I thought we decided that redesigning push-down is going to be done in a follow-up SPIP?


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r167320953
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,17 +17,130 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
     import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    -import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
    +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema, WriteSupport}
    +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics}
    +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    output: Seq[AttributeReference],
    -    reader: DataSourceReader)
    -  extends LeafNode with MultiInstanceRelation with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    projection: Option[Seq[AttributeReference]] = None,
    +    filters: Option[Seq[Expression]] = None,
    +    userSchema: Option[StructType] = None) extends LeafNode with MultiInstanceRelation {
    +
    +  override def simpleString: String = {
    +    s"DataSourceV2Relation(source=$sourceName, " +
    +      s"schema=[${output.map(a => s"$a ${a.dataType.simpleString}").mkString(", ")}], " +
    +      s"filters=[${pushedFilters.mkString(", ")}], options=$options)"
    +  }
    +
    +  override lazy val schema: StructType = reader.readSchema()
    +
    +  override lazy val output: Seq[AttributeReference] = {
    --- End diff --
    
    I agree that it's a bad idea to run push-down here. I fixed this by implementing `doCanonicalize` and returning a node that overrides the `output` val. I think that is cleaner than pulling logic outside of the relation. There's no need for every place that creates a relation to need to get the output of a reader.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r168910415
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -37,22 +100,147 @@ case class DataSourceV2Relation(
       }
     
       override def newInstance(): DataSourceV2Relation = {
    -    copy(output = output.map(_.newInstance()))
    +    // projection is used to maintain id assignment.
    +    // if projection is not set, use output so the copy is not equal to the original
    +    copy(projection = projection.map(_.newInstance()))
       }
     }
     
     /**
      * A specialization of DataSourceV2Relation with the streaming bit set to true. Otherwise identical
      * to the non-streaming relation.
      */
    -class StreamingDataSourceV2Relation(
    +case class StreamingDataSourceV2Relation(
         output: Seq[AttributeReference],
    -    reader: DataSourceReader) extends DataSourceV2Relation(output, reader) {
    +    reader: DataSourceReader)
    +    extends LeafNode with DataSourceReaderHolder with MultiInstanceRelation {
       override def isStreaming: Boolean = true
    +
    +  override def canEqual(other: Any): Boolean = other.isInstanceOf[StreamingDataSourceV2Relation]
    +
    +  override def newInstance(): LogicalPlan = copy(output = output.map(_.newInstance()))
    +
    +  override def computeStats(): Statistics = reader match {
    +    case r: SupportsReportStatistics =>
    +      Statistics(sizeInBytes = r.getStatistics.sizeInBytes().orElse(conf.defaultSizeInBytes))
    +    case _ =>
    +      Statistics(sizeInBytes = conf.defaultSizeInBytes)
    +  }
     }
     
     object DataSourceV2Relation {
    -  def apply(reader: DataSourceReader): DataSourceV2Relation = {
    -    new DataSourceV2Relation(reader.readSchema().toAttributes, reader)
    +  private implicit class SourceHelpers(source: DataSourceV2) {
    +    def asReadSupport: ReadSupport = {
    +      source match {
    +        case support: ReadSupport =>
    +          support
    +        case _: ReadSupportWithSchema =>
    +          // this method is only called if there is no user-supplied schema. if there is no
    +          // user-supplied schema and ReadSupport was not implemented, throw a helpful exception.
    +          throw new AnalysisException(s"Data source requires a user-supplied schema: $name")
    +        case _ =>
    +          throw new AnalysisException(s"Data source is not readable: $name")
    +      }
    +    }
    +
    +    def asReadSupportWithSchema: ReadSupportWithSchema = {
    +      source match {
    +        case support: ReadSupportWithSchema =>
    +          support
    +        case _: ReadSupport =>
    +          throw new AnalysisException(
    +            s"Data source does not support user-supplied schema: $name")
    --- End diff --
    
    here we should compare the schema before throwing an exception.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    **[Test build #86600 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86600/testReport)** for PR 20387 at commit [`9c4dcb5`](https://github.com/apache/spark/commit/9c4dcb5b693e729e89ddd7daa54b19c8f8eb3571).


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r167374579
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,17 +17,131 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
     import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    -import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, ExprId}
    +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema, WriteSupport}
    +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics}
    +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    output: Seq[AttributeReference],
    -    reader: DataSourceReader)
    -  extends LeafNode with MultiInstanceRelation with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    projection: Option[Seq[AttributeReference]] = None,
    +    filters: Option[Seq[Expression]] = None,
    +    userSchema: Option[StructType] = None) extends LeafNode with MultiInstanceRelation {
    +
    +  override def simpleString: String = {
    +    s"DataSourceV2Relation(source=$sourceName, " +
    +      s"schema=[${output.map(a => s"$a ${a.dataType.simpleString}").mkString(", ")}], " +
    +      s"filters=[${pushedFilters.mkString(", ")}], options=$options)"
    +  }
    +
    +  override lazy val schema: StructType = reader.readSchema()
    +
    +  override lazy val output: Seq[AttributeReference] = {
    +    projection match {
    +      case Some(attrs) =>
    +        // use the projection attributes to avoid assigning new ids. fields that are not projected
    +        // will be assigned new ids, which is okay because they are not projected.
    +        val attrMap = attrs.map(a => a.name -> a).toMap
    +        schema.map(f => attrMap.getOrElse(f.name,
    +          AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()))
    +      case _ =>
    +        schema.toAttributes
    --- End diff --
    
    I don't think it is correct to make `output` a constructor parameter here. The v2 read API allows implementations to return rows with a different schema than the one requested, so you don't know whether the projection and the output will actually match until you push the projection and look at the schema the reader returns.
    
    If `output` were a constructor parameter, then it would be easy accidentally create instances where the `output` doesn't match the actual rows returned by the source. That's why the current code uses `projection` to pass the requested columns, and always sets `output` correctly.
    
    To make the guarantee that the column ids don't change, we don't strictly need `output` to be a constructor param. In fact, right now the only time this matters is when the projection isn't set. Otherwise, the ids are taken from the projection. I've considered a couple of options, like caching the conversion from schema to attributes, but I think the easiest option is to make sure that `projection` is always set.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Us...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r164169060
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,15 +17,149 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
     import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceV2, DataSourceV2Options, ReadSupport, ReadSupportWithSchema, WriteSupport}
     import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.sources.v2.writer.DataSourceV2Writer
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    fullOutput: Seq[AttributeReference],
    -    reader: DataSourceV2Reader) extends LeafNode with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    path: Option[String] = None,
    +    table: Option[TableIdentifier] = None,
    --- End diff --
    
    That's why these are options. Passing either path or table name is the most common case, which we should have good support for. If tables are identified in other ways, that's supported.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    > I tried and can't figure out how to do it with PhysicalOperation, that's why I build something new for data source v2 pushdown.
    
    The problem is that we should get DSv2 working independently of a redesign of the push-down rules. Throwing an untested push-down rule into changes for DSv2 makes the new API less reliable, and hurts people that want to try it out and start using it. There is no benefit to doing this for 2.3.0.
    
    I also think a redesign of push-down should be properly designed, thought out, and tested. I'm all for fixing this if you can make the case that we need to, but we shouldn't needlessly mix together major changes.
    
    @cloud-fan, There's more discussion about this on #20476 that I encourage you to read.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Hi @rdblue , I think we all agree that the plan should be immutable, but other parts are still under discussion. Can you send a new PR that focus on making the plan immutable? so that we can merge that one first, and continue to discuss other parts in this PR.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    **[Test build #86601 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86601/testReport)** for PR 20387 at commit [`ac58844`](https://github.com/apache/spark/commit/ac58844118d543030fadfeda0a64b52ad659cf31).
     * This patch **fails to build**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r166526321
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,17 +17,139 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
     import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    -import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
    +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema, WriteSupport}
    +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics}
    +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    fullOutput: Seq[AttributeReference],
    -    reader: DataSourceReader)
    -  extends LeafNode with MultiInstanceRelation with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    path: Option[String] = None,
    --- End diff --
    
    Sorry I might not make it clear. What I proposed is to only include changes that are necessary for this PR. To make the plan immutable, what we need is:
    1. keep required columns and pushed filters in the plan. So here we add 2 parameters: `projection` and `filters`.
    2. keep things that are needed to create a `DataSourceReader`, i.e. `DataSourceV2`, `DataSourceOption` and `userSpecifiedSchema`.
    
    For 1 we are good. For 2, I think the most intuitive and surgical way is to create the `DataSourceOption` in `DataFrameReader`(the current behavior) and keep the `DataSourceOption` in `DataSourceV2Relation`.
    
    There may be an argument about putting these common information in the `simpleString`, I'm +1 on doing this, but this should be done after https://issues.apache.org/jira/browse/SPARK-23341 when the community have an agreement on an initial list of common information(may be more than just table and path).
    
    I really like your proposal about "don’t mix partial, unrelated changes into a commit", and I hope all the open source community can follow it and make high quality PRs. Thanks!


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/543/
    Test PASSed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    @cloud-fan, I'll update this PR and we can talk about passing configuration on the dev list.
    
    And as a reminder, please close #20445.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    **[Test build #87532 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87532/testReport)** for PR 20387 at commit [`1a603db`](https://github.com/apache/spark/commit/1a603dbe5528b447bff371d2e00abdbdee664a75).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r169239000
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,17 +17,80 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.sql.AnalysisException
     import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    -import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
    +import org.apache.spark.sql.catalyst.plans.QueryPlan
    +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema}
    +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics}
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    output: Seq[AttributeReference],
    -    reader: DataSourceReader)
    -  extends LeafNode with MultiInstanceRelation with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    projection: Seq[AttributeReference],
    +    filters: Option[Seq[Expression]] = None,
    +    userSpecifiedSchema: Option[StructType] = None) extends LeafNode with MultiInstanceRelation {
    +
    +  import DataSourceV2Relation._
    +
    +  override def simpleString: String = {
    +    s"DataSourceV2Relation(source=${source.name}, " +
    +      s"schema=[${output.map(a => s"$a ${a.dataType.simpleString}").mkString(", ")}], " +
    +      s"filters=[${pushedFilters.mkString(", ")}], options=$options)"
    +  }
    +
    +  override lazy val schema: StructType = reader.readSchema()
    +
    +  override lazy val output: Seq[AttributeReference] = {
    +    // use the projection attributes to avoid assigning new ids. fields that are not projected
    +    // will be assigned new ids, which is okay because they are not projected.
    +    val attrMap = projection.map(a => a.name -> a).toMap
    +    schema.map(f => attrMap.getOrElse(f.name,
    +      AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()))
    +  }
    +
    +  private lazy val v2Options: DataSourceOptions = makeV2Options(options)
    +
    +  lazy val (
    +      reader: DataSourceReader,
    +      unsupportedFilters: Seq[Expression],
    +      pushedFilters: Seq[Expression]) = {
    +    val newReader = userSpecifiedSchema match {
    +      case Some(s) =>
    +        source.asReadSupportWithSchema.createReader(s, v2Options)
    +      case _ =>
    +        source.asReadSupport.createReader(v2Options)
    +    }
    +
    +    DataSourceV2Relation.pushRequiredColumns(newReader, projection.toStructType)
     
    -  override def canEqual(other: Any): Boolean = other.isInstanceOf[DataSourceV2Relation]
    +    val (remainingFilters, pushedFilters) = filters match {
    +      case Some(filterSeq) =>
    +        DataSourceV2Relation.pushFilters(newReader, filterSeq)
    +      case _ =>
    +        (Nil, Nil)
    +    }
    +
    +    (newReader, remainingFilters, pushedFilters)
    +  }
    +
    +  override def doCanonicalize(): LogicalPlan = {
    +    val c = super.doCanonicalize().asInstanceOf[DataSourceV2Relation]
    +
    +    // override output with canonicalized output to avoid attempting to configure a reader
    +    val canonicalOutput: Seq[AttributeReference] = this.output
    +        .map(a => QueryPlan.normalizeExprId(a, projection))
    +
    +    new DataSourceV2Relation(c.source, c.options, c.projection) {
    --- End diff --
    
    This is hacky but I don't have a better idea now, let's revisit it later.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    > The reader is to be created and configured by the relation, then the relation needs to be able to set the table, path, and other properties. This adds necessary data to the relation that is no longer be passed directly to the reader from DataFrameReader.
    
    This is an interesting thing that I missed before. Can you give a use case about this? If we have a requirement for it, I totally I agree we should put them in `DataSourceV2Relation`.
    
    > I don't see why we wouldn't want to have these options in the immutable relation node from the start. 
    
    My main concern is, this PR itself doesn't show a strong use case for putting these information into `DataSourceV2Relation`. If we do need it in the future, we can add it when we really need it. `DataSourceV2Relation` is not a public API so I think we don't need to rush.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    +1 for @cloud-fan 's suggestion.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r167347513
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,17 +17,131 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
     import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    -import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, ExprId}
    +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema, WriteSupport}
    +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics}
    +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    output: Seq[AttributeReference],
    -    reader: DataSourceReader)
    -  extends LeafNode with MultiInstanceRelation with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    projection: Option[Seq[AttributeReference]] = None,
    +    filters: Option[Seq[Expression]] = None,
    +    userSchema: Option[StructType] = None) extends LeafNode with MultiInstanceRelation {
    +
    +  override def simpleString: String = {
    +    s"DataSourceV2Relation(source=$sourceName, " +
    +      s"schema=[${output.map(a => s"$a ${a.dataType.simpleString}").mkString(", ")}], " +
    +      s"filters=[${pushedFilters.mkString(", ")}], options=$options)"
    +  }
    +
    +  override lazy val schema: StructType = reader.readSchema()
    +
    +  override lazy val output: Seq[AttributeReference] = {
    +    projection match {
    +      case Some(attrs) =>
    +        // use the projection attributes to avoid assigning new ids. fields that are not projected
    +        // will be assigned new ids, which is okay because they are not projected.
    +        val attrMap = attrs.map(a => a.name -> a).toMap
    +        schema.map(f => attrMap.getOrElse(f.name,
    +          AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()))
    +      case _ =>
    +        schema.toAttributes
    +    }
    +  }
    +
    +  private lazy val v2Options: DataSourceOptions = {
    +    // ensure path and table options are set correctly
    +    val updatedOptions = new mutable.HashMap[String, String]
    +    updatedOptions ++= options
    +
    +    new DataSourceOptions(options.asJava)
    +  }
    +
    +  private val sourceName: String = {
    +    source match {
    +      case registered: DataSourceRegister =>
    +        registered.shortName()
    +      case _ =>
    +        source.getClass.getSimpleName
    +    }
    +  }
    +
    +  lazy val (
    +      reader: DataSourceReader,
    +      unsupportedFilters: Seq[Expression],
    +      pushedFilters: Seq[Expression]) = {
    +    val newReader = userSchema match {
    +      case Some(s) =>
    +        asReadSupportWithSchema.createReader(s, v2Options)
    +      case _ =>
    +        asReadSupport.createReader(v2Options)
    +    }
    +
    +    projection.foreach { attrs =>
    +      DataSourceV2Relation.pushRequiredColumns(newReader, attrs.toStructType)
    +    }
    +
    +    val (remainingFilters, pushedFilters) = filters match {
    +      case Some(filterSeq) =>
    +        DataSourceV2Relation.pushFilters(newReader, filterSeq)
    +      case _ =>
    +        (Nil, Nil)
    +    }
    +
    +    (newReader, remainingFilters, pushedFilters)
    +  }
    +
    +  private lazy val asReadSupport: ReadSupport = {
    --- End diff --
    
    e.g. I can't find where is the old fallback logic (https://github.com/apache/spark/pull/20387/files#diff-f70bda59304588cc3abfa3a9840653f4L198) in the new code. It takes more time for reviewers to make sure the new code is corrected.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r167295506
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,17 +17,130 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
     import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    -import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
    +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema, WriteSupport}
    +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics}
    +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    output: Seq[AttributeReference],
    -    reader: DataSourceReader)
    -  extends LeafNode with MultiInstanceRelation with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    projection: Option[Seq[AttributeReference]] = None,
    +    filters: Option[Seq[Expression]] = None,
    +    userSchema: Option[StructType] = None) extends LeafNode with MultiInstanceRelation {
    +
    +  override def simpleString: String = {
    +    s"DataSourceV2Relation(source=$sourceName, " +
    +      s"schema=[${output.map(a => s"$a ${a.dataType.simpleString}").mkString(", ")}], " +
    +      s"filters=[${pushedFilters.mkString(", ")}], options=$options)"
    +  }
    +
    +  override lazy val schema: StructType = reader.readSchema()
    +
    +  override lazy val output: Seq[AttributeReference] = {
    +    projection match {
    +      case Some(attrs) =>
    +        // use the projection attributes to avoid assigning new ids. fields that are not projected
    +        // will be assigned new ids, which is okay because they are not projected.
    +        val attrMap = attrs.map(a => a.name -> a).toMap
    +        schema.map(f => attrMap.getOrElse(f.name,
    +          AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()))
    +      case _ =>
    +        schema.toAttributes
    +    }
    +  }
    +
    +  private lazy val v2Options: DataSourceOptions = {
    +    // ensure path and table options are set correctly
    +    val updatedOptions = new mutable.HashMap[String, String]
    +    updatedOptions ++= options
    +
    +    new DataSourceOptions(options.asJava)
    --- End diff --
    
    I think you're missing that it's a bad idea to pass `DataSourceOptions` to the relation in the first place. As I've said, this class is specific to v2 readers and writers, and doesn't need to be leaked into logical plans.
    
    To make this a bit more clear: If we pass the the options to the relation, then every time a new code path instantiates the relation, someone has to know where that central place to create options is. You're trading the problem for one only slightly better.
    
    What if every relation required its own specific classes to pass the same information? That would be a mess. Rather than know what custom object to instantiate and where the helper for it is, it makes more sense to pass data to the relation. Then we can also use the info in match expressions.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    **[Test build #87004 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87004/testReport)** for PR 20387 at commit [`ab945a1`](https://github.com/apache/spark/commit/ab945a19efe666c41deae9c044002f3455220c1d).


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/87000/
    Test FAILed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Thanks for all your help getting this committed, @cloud-fan!


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r166165255
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,17 +17,151 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.TableIdentifier
     import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    -import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
    +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema, WriteSupport}
    +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics}
    +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    fullOutput: Seq[AttributeReference],
    -    reader: DataSourceReader)
    -  extends LeafNode with MultiInstanceRelation with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    path: Option[String] = None,
    +    table: Option[TableIdentifier] = None,
    +    projection: Option[Seq[AttributeReference]] = None,
    +    filters: Option[Seq[Expression]] = None,
    +    userSchema: Option[StructType] = None) extends LeafNode with MultiInstanceRelation {
    +
    +  override def simpleString: String = {
    +    "DataSourceV2Relation(" +
    +      s"source=$sourceName${path.orElse(table).map(loc => s"($loc)").getOrElse("")}, " +
    +      s"schema=[${output.map(a => s"$a ${a.dataType.simpleString}").mkString(", ")}], " +
    +      s"filters=[${pushedFilters.mkString(", ")}] options=$options)"
    +  }
    +
    +  override lazy val schema: StructType = reader.readSchema()
    +
    +  override lazy val output: Seq[AttributeReference] = {
    +    projection match {
    +      case Some(attrs) =>
    +        // use the projection attributes to avoid assigning new ids. fields that are not projected
    +        // will be assigned new ids, which is okay because they are not projected.
    +        val attrMap = attrs.map(a => a.name -> a).toMap
    +        schema.map(f => attrMap.getOrElse(f.name,
    +          AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()))
    +      case _ =>
    +        schema.toAttributes
    +    }
    +  }
    +
    +  private lazy val v2Options: DataSourceOptions = {
    +    // ensure path and table options are set correctly
    +    val updatedOptions = new mutable.HashMap[String, String]
    +    updatedOptions ++= options
    +
    +    path match {
    +      case Some(p) =>
    +        updatedOptions.put("path", p)
    +      case None =>
    +        updatedOptions.remove("path")
    +    }
    +
    +    table.map { ident =>
    +      updatedOptions.put("table", ident.table)
    --- End diff --
    
    Opened [SPARK-23341](https://issues.apache.org/jira/browse/SPARK-23341) for this.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/87007/
    Test PASSed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/87004/
    Test PASSed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    **[Test build #87007 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87007/testReport)** for PR 20387 at commit [`f1d9872`](https://github.com/apache/spark/commit/f1d9872a2699cdbd5c87b02e702dc8103335131d).


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/87001/
    Test FAILed.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r167345858
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,17 +17,131 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
     import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    -import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, ExprId}
    +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema, WriteSupport}
    +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics}
    +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    output: Seq[AttributeReference],
    -    reader: DataSourceReader)
    -  extends LeafNode with MultiInstanceRelation with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    projection: Option[Seq[AttributeReference]] = None,
    +    filters: Option[Seq[Expression]] = None,
    +    userSchema: Option[StructType] = None) extends LeafNode with MultiInstanceRelation {
    +
    +  override def simpleString: String = {
    +    s"DataSourceV2Relation(source=$sourceName, " +
    +      s"schema=[${output.map(a => s"$a ${a.dataType.simpleString}").mkString(", ")}], " +
    +      s"filters=[${pushedFilters.mkString(", ")}], options=$options)"
    +  }
    +
    +  override lazy val schema: StructType = reader.readSchema()
    +
    +  override lazy val output: Seq[AttributeReference] = {
    +    projection match {
    +      case Some(attrs) =>
    +        // use the projection attributes to avoid assigning new ids. fields that are not projected
    +        // will be assigned new ids, which is okay because they are not projected.
    +        val attrMap = attrs.map(a => a.name -> a).toMap
    +        schema.map(f => attrMap.getOrElse(f.name,
    +          AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()))
    +      case _ =>
    +        schema.toAttributes
    --- End diff --
    
    This is currently handled by setting the projection to always get the same ids, but it may not cover all copy cases. I'll look into how we can reliably move output to the constructor params, but this needs to avoid forcing callers to configure a reader.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    > See FindDataSourceTable.readDataSourceTable about how we handle the path option.
    
    Sorry, what do you want to change?


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r167342554
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,17 +17,131 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
     import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    -import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, ExprId}
    +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema, WriteSupport}
    +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics}
    +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    output: Seq[AttributeReference],
    -    reader: DataSourceReader)
    -  extends LeafNode with MultiInstanceRelation with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    projection: Option[Seq[AttributeReference]] = None,
    --- End diff --
    
    Since we need to keep the "pushed columns" anyway, can we just use the `output` field? The only difference between `output` and `projection` is the default value, this can be solved by creating a temp reader and get its schema in `DataFrameReader`. We already did this in `DataStreamReader.load`.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/spark/pull/20387


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/880/
    Test PASSed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by marmbrus <gi...@git.apache.org>.
Github user marmbrus commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Regarding, `computeStats`, the logical plan seems like it might not be the right place.  As we move towards more CBO it seems like we are going to need to pick physical operators before we can really reason about the cost of a subplan. With the caveat that I haven't though hard about this, I'd be supportive of moving these kinds of metrics to physical plan. +1 that we need to be able to consider pushdown when producing stats either way.
    
    On the second point, I don't think I understand DataSourceV2 enough yet to know the answer, but you ask a lot of questions that I think need to be defined as part of the API (if we haven't already).  What is the contract for ordering and interactions between different types of pushdown? Is it valid to pushdown in pieces or will we only call the method once? (sorry if this is written down and I've just missed it).
    
    My gut feeling is that we don't really want to fuse incrementally.  Its seems hard to reason about correctness and interactions between different things that have been pushed.  As I hinted at before, I think its most natural to split the concerns of pushdown within a query plan and fusing of operators. But maybe this is limited in someway I don't realize.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    > It's hard to improve PhysicalOperation to support more operators and specific push down orders, so I created the new one
    
    I'm concerned about the new one. The projection support seems really brittle because it calls out specific logical nodes and scans the entire plan. If we are doing push-down wrong on the current v1 and Hive code paths, then I'd like to see a proposal for fixing that without these drawbacks.
    
    I like that this PR pushes projections and filters just like the other paths. We should start there and add additional push-down as necessary.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Will do.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/537/
    Test PASSed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by felixcheung <gi...@git.apache.org>.
Github user felixcheung commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    don't we already have table in DataFrameReader? http://spark.apache.org/docs/latest/api/python/pyspark.sql.html?highlight=dataframereader#pyspark.sql.DataFrameReader.table
    http://spark.apache.org/docs/latest/api/scala/index.html#org.apache.spark.sql.DataFrameReader@table(tableName:String):org.apache.spark.sql.DataFrame


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    **[Test build #87123 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87123/testReport)** for PR 20387 at commit [`7ef90cb`](https://github.com/apache/spark/commit/7ef90cb7e20b903b3569470ae0e3c26a03cb6a2a).


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    > Can you give a use case about this?
    
    We've added a resolution rule from `UnresolvedRelation` to `DataSourceV2Relation` that uses our implementation. `UnresolvedRelation` needs to pass its `TableIdentifier` to the v2 relation, which is why I added this.
    
    We could separate this out, but I think it makes sense to get the options for an immutable plan node right from the start. And I think we agree that `TableIdentifier` will be passed this way. I just don't see any benefit to separating this out. I agree that there isn't a requirement in this PR and I can remove it if necessary.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    **[Test build #87432 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87432/testReport)** for PR 20387 at commit [`b8e3623`](https://github.com/apache/spark/commit/b8e3623837047949b39141e46eb96f30de8aa21e).


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/760/
    Test PASSed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    **[Test build #87274 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87274/testReport)** for PR 20387 at commit [`ce5f40d`](https://github.com/apache/spark/commit/ce5f40d6a512874e2dd45bab9256f77ff74e628b).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by marmbrus <gi...@git.apache.org>.
Github user marmbrus commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r165751660
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,15 +17,149 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
     import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceV2, DataSourceV2Options, ReadSupport, ReadSupportWithSchema, WriteSupport}
     import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.sources.v2.writer.DataSourceV2Writer
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    fullOutput: Seq[AttributeReference],
    -    reader: DataSourceV2Reader) extends LeafNode with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    path: Option[String] = None,
    +    table: Option[TableIdentifier] = None,
    +    projection: Option[Seq[AttributeReference]] = None,
    +    filters: Option[Seq[Expression]] = None,
    --- End diff --
    
    I like this pattern.  I think it is important that the arguments to a query plan node are comprehensive so that it is easy to understand what is going on in the output of `explain()`.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    This PR does 3 things:
    1. make `DataSourceV2Relation` immutable. This extends the constructor of `DataSourceV2Relation` to include pushed filters and pruned columns.
    2. carry some standard information(table, path, etc.) into `DataSourceV2Relation`. This extends the constructor of `DataSourceV2Relation` to include table identifier, path string etc.
    3. replace the new operator pushdown rule with `PhysicalOperation`.
    
    It will be great if we only focus on 1, but I'm also OK if we do 1 and 3 together. I don't think we should include 2 here, as it's unclear what's the benefit. These stardard information are only used to create `DataSourceOptions` inside `DataSourceV2Relation`, which can also be done in `DataFrameReader`. I suggest we don't change this part and just keep the `DataSourceOption` in the constructor of `DataSourceV2Relation`.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/539/
    Test PASSed.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r167346297
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,17 +17,131 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
     import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    -import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, ExprId}
    +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema, WriteSupport}
    +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics}
    +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    output: Seq[AttributeReference],
    -    reader: DataSourceReader)
    -  extends LeafNode with MultiInstanceRelation with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    projection: Option[Seq[AttributeReference]] = None,
    +    filters: Option[Seq[Expression]] = None,
    +    userSchema: Option[StructType] = None) extends LeafNode with MultiInstanceRelation {
    +
    +  override def simpleString: String = {
    +    s"DataSourceV2Relation(source=$sourceName, " +
    +      s"schema=[${output.map(a => s"$a ${a.dataType.simpleString}").mkString(", ")}], " +
    +      s"filters=[${pushedFilters.mkString(", ")}], options=$options)"
    +  }
    +
    +  override lazy val schema: StructType = reader.readSchema()
    +
    +  override lazy val output: Seq[AttributeReference] = {
    +    projection match {
    +      case Some(attrs) =>
    +        // use the projection attributes to avoid assigning new ids. fields that are not projected
    +        // will be assigned new ids, which is okay because they are not projected.
    +        val attrMap = attrs.map(a => a.name -> a).toMap
    +        schema.map(f => attrMap.getOrElse(f.name,
    +          AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()))
    +      case _ =>
    +        schema.toAttributes
    +    }
    +  }
    +
    +  private lazy val v2Options: DataSourceOptions = {
    +    // ensure path and table options are set correctly
    +    val updatedOptions = new mutable.HashMap[String, String]
    +    updatedOptions ++= options
    +
    +    new DataSourceOptions(options.asJava)
    +  }
    +
    +  private val sourceName: String = {
    +    source match {
    +      case registered: DataSourceRegister =>
    +        registered.shortName()
    +      case _ =>
    +        source.getClass.getSimpleName
    +    }
    +  }
    +
    +  lazy val (
    +      reader: DataSourceReader,
    +      unsupportedFilters: Seq[Expression],
    +      pushedFilters: Seq[Expression]) = {
    +    val newReader = userSchema match {
    +      case Some(s) =>
    +        asReadSupportWithSchema.createReader(s, v2Options)
    +      case _ =>
    +        asReadSupport.createReader(v2Options)
    +    }
    +
    +    projection.foreach { attrs =>
    +      DataSourceV2Relation.pushRequiredColumns(newReader, attrs.toStructType)
    +    }
    +
    +    val (remainingFilters, pushedFilters) = filters match {
    +      case Some(filterSeq) =>
    +        DataSourceV2Relation.pushFilters(newReader, filterSeq)
    +      case _ =>
    +        (Nil, Nil)
    +    }
    +
    +    (newReader, remainingFilters, pushedFilters)
    +  }
    +
    +  private lazy val asReadSupport: ReadSupport = {
    --- End diff --
    
    Is there a problem? I think this is a clear way to to handle the cases without mixing concerns.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/86600/
    Test FAILed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    **[Test build #87007 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87007/testReport)** for PR 20387 at commit [`f1d9872`](https://github.com/apache/spark/commit/f1d9872a2699cdbd5c87b02e702dc8103335131d).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #20387: SPARK-22386: DataSourceV2: Use immutable logical plans.

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    **[Test build #86598 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86598/testReport)** for PR 20387 at commit [`d3233e1`](https://github.com/apache/spark/commit/d3233e1a8b1d4d153146b1a536dee34246920b0d).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `case class StreamingDataSourceV2Relation(`


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    **[Test build #86601 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86601/testReport)** for PR 20387 at commit [`ac58844`](https://github.com/apache/spark/commit/ac58844118d543030fadfeda0a64b52ad659cf31).


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/755/
    Test PASSed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/875/
    Test PASSed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #20387: SPARK-22386: DataSourceV2: Use immutable logical plans.

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/86598/
    Test FAILed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    **[Test build #87426 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87426/testReport)** for PR 20387 at commit [`57e05c2`](https://github.com/apache/spark/commit/57e05c2babbcaec3ed3aa69765e1145539879c97).


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    > Let's keep it general and let the data source to interprete it.
    
    I think this is the wrong approach. The reason why we are using a special `DataSourceOptions` object is to ensure that data sources consistently ignore case when reading **their own options**. Consistency across data sources matters and we should be pushing for more consistency, not less.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/201/
    Test PASSed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Okay, I rebased again after SPARK-23303 was reverted.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r166997697
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,17 +17,130 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
     import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    -import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
    +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema, WriteSupport}
    +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics}
    +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    output: Seq[AttributeReference],
    -    reader: DataSourceReader)
    -  extends LeafNode with MultiInstanceRelation with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    projection: Option[Seq[AttributeReference]] = None,
    +    filters: Option[Seq[Expression]] = None,
    +    userSchema: Option[StructType] = None) extends LeafNode with MultiInstanceRelation {
    +
    +  override def simpleString: String = {
    +    s"DataSourceV2Relation(source=$sourceName, " +
    +      s"schema=[${output.map(a => s"$a ${a.dataType.simpleString}").mkString(", ")}], " +
    +      s"filters=[${pushedFilters.mkString(", ")}], options=$options)"
    +  }
    +
    +  override lazy val schema: StructType = reader.readSchema()
    +
    +  override lazy val output: Seq[AttributeReference] = {
    +    projection match {
    +      case Some(attrs) =>
    +        // use the projection attributes to avoid assigning new ids. fields that are not projected
    +        // will be assigned new ids, which is okay because they are not projected.
    +        val attrMap = attrs.map(a => a.name -> a).toMap
    +        schema.map(f => attrMap.getOrElse(f.name,
    +          AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()))
    +      case _ =>
    +        schema.toAttributes
    +    }
    +  }
    +
    +  private lazy val v2Options: DataSourceOptions = {
    +    // ensure path and table options are set correctly
    +    val updatedOptions = new mutable.HashMap[String, String]
    +    updatedOptions ++= options
    +
    +    new DataSourceOptions(options.asJava)
    +  }
    +
    +  private val sourceName: String = {
    +    source match {
    +      case registered: DataSourceRegister =>
    +        registered.shortName()
    +      case _ =>
    +        source.getClass.getSimpleName
    +    }
    +  }
    +
    +  lazy val (
    +      reader: DataSourceReader,
    +      unsupportedFilters: Seq[Expression],
    +      pushedFilters: Seq[Expression]) = {
    +    val newReader = userSchema match {
    +      case Some(s) =>
    +        asReadSupportWithSchema.createReader(s, v2Options)
    +      case _ =>
    +        asReadSupport.createReader(v2Options)
    +    }
    +
    +    projection.foreach { attrs =>
    +      DataSourceV2Relation.pushRequiredColumns(newReader, attrs.toStructType)
    +    }
    +
    +    val (remainingFilters, pushedFilters) = filters match {
    +      case Some(filterSeq) =>
    +        DataSourceV2Relation.pushFilters(newReader, filterSeq)
    +      case _ =>
    +        (Nil, Nil)
    +    }
    +
    +    (newReader, remainingFilters, pushedFilters)
    +  }
     
    -  override def canEqual(other: Any): Boolean = other.isInstanceOf[DataSourceV2Relation]
    +  def writer(dfSchema: StructType, mode: SaveMode): Option[DataSourceWriter] = {
    --- End diff --
    
    No, it isn't. But a relation should be able to return a writer. This is going to be needed as we improve the logical plans used by v2.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/200/
    Test PASSed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r167346995
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownOperatorsToDataSource.scala ---
    @@ -17,130 +17,55 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    -import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeMap, AttributeSet, Expression, NamedExpression, PredicateHelper}
    -import org.apache.spark.sql.catalyst.optimizer.RemoveRedundantProject
    +import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, AttributeSet}
    +import org.apache.spark.sql.catalyst.planning.PhysicalOperation
     import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project}
     import org.apache.spark.sql.catalyst.rules.Rule
    -import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    -import org.apache.spark.sql.sources
    -import org.apache.spark.sql.sources.v2.reader._
     
    -/**
    - * Pushes down various operators to the underlying data source for better performance. Operators are
    - * being pushed down with a specific order. As an example, given a LIMIT has a FILTER child, you
    - * can't push down LIMIT if FILTER is not completely pushed down. When both are pushed down, the
    - * data source should execute FILTER before LIMIT. And required columns are calculated at the end,
    - * because when more operators are pushed down, we may need less columns at Spark side.
    - */
    -object PushDownOperatorsToDataSource extends Rule[LogicalPlan] with PredicateHelper {
    -  override def apply(plan: LogicalPlan): LogicalPlan = {
    -    // Note that, we need to collect the target operator along with PROJECT node, as PROJECT may
    -    // appear in many places for column pruning.
    -    // TODO: Ideally column pruning should be implemented via a plan property that is propagated
    -    // top-down, then we can simplify the logic here and only collect target operators.
    -    val filterPushed = plan transformUp {
    -      case FilterAndProject(fields, condition, r @ DataSourceV2Relation(_, reader)) =>
    -        val (candidates, nonDeterministic) =
    -          splitConjunctivePredicates(condition).partition(_.deterministic)
    -
    -        val stayUpFilters: Seq[Expression] = reader match {
    -          case r: SupportsPushDownCatalystFilters =>
    -            r.pushCatalystFilters(candidates.toArray)
    -
    -          case r: SupportsPushDownFilters =>
    -            // A map from original Catalyst expressions to corresponding translated data source
    -            // filters. If a predicate is not in this map, it means it cannot be pushed down.
    -            val translatedMap: Map[Expression, sources.Filter] = candidates.flatMap { p =>
    -              DataSourceStrategy.translateFilter(p).map(f => p -> f)
    -            }.toMap
    -
    -            // Catalyst predicate expressions that cannot be converted to data source filters.
    -            val nonConvertiblePredicates = candidates.filterNot(translatedMap.contains)
    -
    -            // Data source filters that cannot be pushed down. An unhandled filter means
    -            // the data source cannot guarantee the rows returned can pass the filter.
    -            // As a result we must return it so Spark can plan an extra filter operator.
    -            val unhandledFilters = r.pushFilters(translatedMap.values.toArray).toSet
    -            val unhandledPredicates = translatedMap.filter { case (_, f) =>
    -              unhandledFilters.contains(f)
    -            }.keys
    -
    -            nonConvertiblePredicates ++ unhandledPredicates
    -
    -          case _ => candidates
    -        }
    -
    -        val filterCondition = (stayUpFilters ++ nonDeterministic).reduceLeftOption(And)
    -        val withFilter = filterCondition.map(Filter(_, r)).getOrElse(r)
    -        if (withFilter.output == fields) {
    -          withFilter
    -        } else {
    -          Project(fields, withFilter)
    -        }
    -    }
    -
    -    // TODO: add more push down rules.
    -
    -    val columnPruned = pushDownRequiredColumns(filterPushed, filterPushed.outputSet)
    -    // After column pruning, we may have redundant PROJECT nodes in the query plan, remove them.
    -    RemoveRedundantProject(columnPruned)
    -  }
    -
    -  // TODO: nested fields pruning
    -  private def pushDownRequiredColumns(
    -      plan: LogicalPlan, requiredByParent: AttributeSet): LogicalPlan = {
    -    plan match {
    -      case p @ Project(projectList, child) =>
    -        val required = projectList.flatMap(_.references)
    -        p.copy(child = pushDownRequiredColumns(child, AttributeSet(required)))
    -
    -      case f @ Filter(condition, child) =>
    -        val required = requiredByParent ++ condition.references
    -        f.copy(child = pushDownRequiredColumns(child, required))
    -
    -      case relation: DataSourceV2Relation => relation.reader match {
    -        case reader: SupportsPushDownRequiredColumns =>
    -          // TODO: Enable the below assert after we make `DataSourceV2Relation` immutable. Fow now
    -          // it's possible that the mutable reader being updated by someone else, and we need to
    -          // always call `reader.pruneColumns` here to correct it.
    -          // assert(relation.output.toStructType == reader.readSchema(),
    -          //  "Schema of data source reader does not match the relation plan.")
    -
    -          val requiredColumns = relation.output.filter(requiredByParent.contains)
    -          reader.pruneColumns(requiredColumns.toStructType)
    -
    -          val nameToAttr = relation.output.map(_.name).zip(relation.output).toMap
    -          val newOutput = reader.readSchema().map(_.name).map(nameToAttr)
    -          relation.copy(output = newOutput)
    -
    -        case _ => relation
    +object PushDownOperatorsToDataSource extends Rule[LogicalPlan] {
    +  override def apply(
    +      plan: LogicalPlan): LogicalPlan = plan transformUp {
    +    // PhysicalOperation guarantees that filters are deterministic; no need to check
    +    case PhysicalOperation(project, newFilters, relation : DataSourceV2Relation) =>
    +      // merge the filters
    +      val filters = relation.filters match {
    +        case Some(existing) =>
    +          existing ++ newFilters
    +        case _ =>
    +          newFilters
           }
     
    -      // TODO: there may be more operators that can be used to calculate the required columns. We
    -      // can add more and more in the future.
    -      case _ => plan.mapChildren(c => pushDownRequiredColumns(c, c.outputSet))
    -    }
    -  }
    -
    -  /**
    -   * Finds a Filter node(with an optional Project child) above data source relation.
    -   */
    -  object FilterAndProject {
    -    // returns the project list, the filter condition and the data source relation.
    -    def unapply(plan: LogicalPlan)
    -        : Option[(Seq[NamedExpression], Expression, DataSourceV2Relation)] = plan match {
    +      val projectAttrs = project.map(_.toAttribute)
    +      val projectSet = AttributeSet(project.flatMap(_.references))
    +      val filterSet = AttributeSet(filters.flatMap(_.references))
    +
    +      val projection = if (filterSet.subsetOf(projectSet) &&
    +          AttributeSet(projectAttrs) == projectSet) {
    +        // When the required projection contains all of the filter columns and column pruning alone
    +        // can produce the required projection, push the required projection.
    +        // A final projection may still be needed if the data source produces a different column
    +        // order or if it cannot prune all of the nested columns.
    +        projectAttrs
    +      } else {
    +        // When there are filter columns not already in the required projection or when the required
    +        // projection is more complicated than column pruning, base column pruning on the set of
    +        // all columns needed by both.
    +        (projectSet ++ filterSet).toSeq
    +      }
     
    -      case Filter(condition, r: DataSourceV2Relation) => Some((r.output, condition, r))
    +      val newRelation = relation.copy(
    +        projection = Some(projection.asInstanceOf[Seq[AttributeReference]]),
    +        filters = Some(filters))
    --- End diff --
    
    yea this is just FYI.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/87170/
    Test PASSed.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r167001183
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,17 +17,130 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
     import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    -import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
    +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema, WriteSupport}
    +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics}
    +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    output: Seq[AttributeReference],
    -    reader: DataSourceReader)
    -  extends LeafNode with MultiInstanceRelation with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    projection: Option[Seq[AttributeReference]] = None,
    +    filters: Option[Seq[Expression]] = None,
    +    userSchema: Option[StructType] = None) extends LeafNode with MultiInstanceRelation {
    +
    +  override def simpleString: String = {
    +    s"DataSourceV2Relation(source=$sourceName, " +
    +      s"schema=[${output.map(a => s"$a ${a.dataType.simpleString}").mkString(", ")}], " +
    +      s"filters=[${pushedFilters.mkString(", ")}], options=$options)"
    +  }
    +
    +  override lazy val schema: StructType = reader.readSchema()
    +
    +  override lazy val output: Seq[AttributeReference] = {
    +    projection match {
    +      case Some(attrs) =>
    +        // use the projection attributes to avoid assigning new ids. fields that are not projected
    +        // will be assigned new ids, which is okay because they are not projected.
    +        val attrMap = attrs.map(a => a.name -> a).toMap
    +        schema.map(f => attrMap.getOrElse(f.name,
    +          AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()))
    +      case _ =>
    +        schema.toAttributes
    +    }
    +  }
    +
    +  private lazy val v2Options: DataSourceOptions = {
    +    // ensure path and table options are set correctly
    +    val updatedOptions = new mutable.HashMap[String, String]
    +    updatedOptions ++= options
    +
    +    new DataSourceOptions(options.asJava)
    --- End diff --
    
    Also, keep in mind that this is a lazy val. It is only referenced when creating a reader or writer


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    **[Test build #87434 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87434/testReport)** for PR 20387 at commit [`3b55609`](https://github.com/apache/spark/commit/3b55609b605fb461f6c2616d1da95a2d4b27ff4b).


---

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


[GitHub] spark issue #20387: SPARK-22386: DataSourceV2: Use immutable logical plans.

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/197/
    Test PASSed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    > I thought it was a good thing to push a single node down at a time and not depend on order.
    
    The order must be taken care. For example, we can't push down a limit through Filter, unless the entire filter is pushed into the data source. Generally, if we pushed down multiple operators into a data source, we should clearly define what the order is to apply these operators in the data source.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r167753210
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,17 +17,80 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.sql.AnalysisException
     import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    -import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
    +import org.apache.spark.sql.catalyst.plans.QueryPlan
    +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema}
    +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics}
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    output: Seq[AttributeReference],
    -    reader: DataSourceReader)
    -  extends LeafNode with MultiInstanceRelation with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    projection: Seq[AttributeReference],
    +    filters: Option[Seq[Expression]] = None,
    +    userSchema: Option[StructType] = None) extends LeafNode with MultiInstanceRelation {
    --- End diff --
    
    because we call it `userSpecifiedSchema` in `DataFrameReader` and `DataSource`,  I think it's more clear to make the name consistent.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r166030958
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,17 +17,151 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.TableIdentifier
     import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    -import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
    +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema, WriteSupport}
    +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics}
    +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    fullOutput: Seq[AttributeReference],
    -    reader: DataSourceReader)
    -  extends LeafNode with MultiInstanceRelation with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    path: Option[String] = None,
    +    table: Option[TableIdentifier] = None,
    --- End diff --
    
    @cloud-fan, sorry if it was not clear: Yes, I have considered it and I think it is a bad idea. I sent a note to the dev list about this issue, as well if you want more context. There are two main reasons:
    
    1. Your proposal creates more places that are responsible for creating a `DataSourceOptions` with the right property names. All of the places where we have a `TableIdentifier` and want to convert to a `DataSourceV2Relation` need to copy the same logic and worry about using the same properties.
        What you propose is hard to maintain and error prone: what happens if we decide not to pass the database if it is `None` in a `TableIdentifier`? We would have to validate every place that creates a v2 relation. On the other hand, if we pass `TableIdentifier` here, we have one code path that converts. It is also easier for us to pass `TableIdentifier` to the data sources if we choose to update the API.
    2. There is no reason to use `DataSourceOptions` outside of v2 at this point. This PR doesn't expose the v2-specific options class to other places in the codebase. Instead, it uses a map for generic options and classes that can be used in pattern matching where possible. And again, this has fewer places that create v2 internal classes, which is easier for maintenance.
    
    If you want to add those methods to the options class so that implementations can easily access path and table name, then we can do that in a follow-up PR.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    **[Test build #87427 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87427/testReport)** for PR 20387 at commit [`adcb25a`](https://github.com/apache/spark/commit/adcb25a06240dc413f58b2d1240405b0a5485578).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    > We can add things like limit pushdown later, by adding it properly to the existing code.
    
    I tried and can't figure out how to do it with `PhysicalOperation`, that's why I build something new for data source v2 pushdown. I'm OK to reuse it if you can convince me `PhysicalOperation` is extendable, e.g. support limit push down.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    **[Test build #87002 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87002/testReport)** for PR 20387 at commit [`a7f0b90`](https://github.com/apache/spark/commit/a7f0b90b6ccb85c0801934ce7841831fe37b8739).
     * This patch **fails to build**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r167342730
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,17 +17,131 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
     import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    -import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, ExprId}
    +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema, WriteSupport}
    +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics}
    +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    output: Seq[AttributeReference],
    -    reader: DataSourceReader)
    -  extends LeafNode with MultiInstanceRelation with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    projection: Option[Seq[AttributeReference]] = None,
    +    filters: Option[Seq[Expression]] = None,
    +    userSchema: Option[StructType] = None) extends LeafNode with MultiInstanceRelation {
    +
    +  override def simpleString: String = {
    +    s"DataSourceV2Relation(source=$sourceName, " +
    +      s"schema=[${output.map(a => s"$a ${a.dataType.simpleString}").mkString(", ")}], " +
    +      s"filters=[${pushedFilters.mkString(", ")}], options=$options)"
    +  }
    +
    +  override lazy val schema: StructType = reader.readSchema()
    +
    +  override lazy val output: Seq[AttributeReference] = {
    +    projection match {
    +      case Some(attrs) =>
    +        // use the projection attributes to avoid assigning new ids. fields that are not projected
    +        // will be assigned new ids, which is okay because they are not projected.
    +        val attrMap = attrs.map(a => a.name -> a).toMap
    +        schema.map(f => attrMap.getOrElse(f.name,
    +          AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()))
    +      case _ =>
    +        schema.toAttributes
    +    }
    +  }
    +
    +  private lazy val v2Options: DataSourceOptions = {
    +    // ensure path and table options are set correctly
    --- End diff --
    
    the comment is out-dated.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    **[Test build #87427 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87427/testReport)** for PR 20387 at commit [`adcb25a`](https://github.com/apache/spark/commit/adcb25a06240dc413f58b2d1240405b0a5485578).


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Currently `DataSourceOptions` is the major way for Spark and users to pass information to the data source. It's very flexible and only defines one rule: the option key lookup should be case-insensitive.
    
    I agree with your point that more consistency is better. It's annoying if every data source needs to define their own option keys for table and database, and tell users about it. It's good if Spark can define some rules about what option keys should be used for some common information.
    
    My proposal:
    ```
    class DataSourceOptions {
      ...
      
      def getPath(): String = get("path")
    
      def getTimeZone(): String = get("timeZone")
    
      def getTableName(): String = get("table")
    }
    ```
    We can keep adding these options since this won't break binary compatibility.
    
    And then we just need to document it and tell both users and data source developers about how to specify and retrieve these common options.
    
    Then I think we don't need to add `table` and `database` parameters to `DataSourceV2Relation`, because we can easily do `relation.options.getTable`.
    
    BTW this doesn't change the API so I think it's fine to do it after 2.3.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Us...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r164633577
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,15 +17,149 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
     import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceV2, DataSourceV2Options, ReadSupport, ReadSupportWithSchema, WriteSupport}
     import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.sources.v2.writer.DataSourceV2Writer
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    fullOutput: Seq[AttributeReference],
    -    reader: DataSourceV2Reader) extends LeafNode with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    path: Option[String] = None,
    +    table: Option[TableIdentifier] = None,
    +    projection: Option[Seq[AttributeReference]] = None,
    +    filters: Option[Seq[Expression]] = None,
    +    userSchema: Option[StructType] = None) extends LeafNode {
    +
    +  override def simpleString: String = {
    +    "DataSourceV2Relation(" +
    +      s"source=$sourceName, schema=${schema.simpleString}, " +
    +      s"filters=$pushedFilters options=$options)"
    +  }
    +
    +  override lazy val schema: StructType = reader.readSchema()
    +
    +  override lazy val output: Seq[AttributeReference] = {
    +    projection match {
    +      case Some(attrs) =>
    +        // use the projection attributes to avoid assigning new ids. fields that are not projected
    +        // will be assigned new ids, which is okay because they are not projected.
    +        val attrMap = attrs.map(a => a.name -> a).toMap
    +        schema.map(f => attrMap.getOrElse(f.name,
    +          AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()))
    +      case _ =>
    +        schema.toAttributes
    +    }
    +  }
    +
    +  private lazy val v2Options: DataSourceV2Options = {
    +    // ensure path and table options are set correctly
    +    val updatedOptions = new mutable.HashMap[String, String]
    +    updatedOptions ++= options
     
    -  override def canEqual(other: Any): Boolean = other.isInstanceOf[DataSourceV2Relation]
    +    path match {
    +      case Some(p) =>
    +        updatedOptions.put("path", p)
    +      case None =>
    +        updatedOptions.remove("path")
    +    }
    +
    +    table.map { ident =>
    +      updatedOptions.put("table", ident.table)
    +      ident.database match {
    +        case Some(db) =>
    +          updatedOptions.put("database", db)
    +        case None =>
    +          updatedOptions.remove("database")
    +      }
    +    }
    +
    +    new DataSourceV2Options(options.asJava)
    +  }
    +
    +  private val sourceName: String = {
    +    source match {
    +      case registered: DataSourceRegister =>
    +        registered.shortName()
    +      case _ =>
    +        source.getClass.getSimpleName
    +    }
    +  }
    +
    +  lazy val (
    +      reader: DataSourceV2Reader,
    +      unsupportedFilters: Seq[Expression],
    +      pushedFilters: Seq[Expression]) = {
    +    val newReader = userSchema match {
    +      case Some(s) =>
    +        asReadSupportWithSchema.createReader(s, v2Options)
    --- End diff --
    
    I like this idea. Although `DataSourceReader` is mutable, we can create a new one every time when we wanna apply the operator pushdowns.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Use immut...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    **[Test build #87426 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87426/testReport)** for PR 20387 at commit [`57e05c2`](https://github.com/apache/spark/commit/57e05c2babbcaec3ed3aa69765e1145539879c97).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    thanks, merging to master!


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    **[Test build #87170 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87170/testReport)** for PR 20387 at commit [`181946d`](https://github.com/apache/spark/commit/181946d1f1c5889661544830a77bd23c4b4f685a).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r167142433
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,17 +17,130 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
     import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    -import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
    +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema, WriteSupport}
    +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics}
    +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    output: Seq[AttributeReference],
    -    reader: DataSourceReader)
    -  extends LeafNode with MultiInstanceRelation with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    projection: Option[Seq[AttributeReference]] = None,
    +    filters: Option[Seq[Expression]] = None,
    +    userSchema: Option[StructType] = None) extends LeafNode with MultiInstanceRelation {
    +
    +  override def simpleString: String = {
    +    s"DataSourceV2Relation(source=$sourceName, " +
    +      s"schema=[${output.map(a => s"$a ${a.dataType.simpleString}").mkString(", ")}], " +
    +      s"filters=[${pushedFilters.mkString(", ")}], options=$options)"
    +  }
    +
    +  override lazy val schema: StructType = reader.readSchema()
    +
    +  override lazy val output: Seq[AttributeReference] = {
    +    projection match {
    +      case Some(attrs) =>
    +        // use the projection attributes to avoid assigning new ids. fields that are not projected
    +        // will be assigned new ids, which is okay because they are not projected.
    +        val attrMap = attrs.map(a => a.name -> a).toMap
    +        schema.map(f => attrMap.getOrElse(f.name,
    +          AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()))
    +      case _ =>
    +        schema.toAttributes
    +    }
    +  }
    +
    +  private lazy val v2Options: DataSourceOptions = {
    +    // ensure path and table options are set correctly
    +    val updatedOptions = new mutable.HashMap[String, String]
    +    updatedOptions ++= options
    +
    +    new DataSourceOptions(options.asJava)
    --- End diff --
    
    We all agree that duplicating the logic of creating `DataSourceOptions` in many places is a bad idea. Currently there are 2 proposals:
    
    1. Have a central place to take care the data source v2 resolution logic, including option creating. This is the approach of data source v1, i.e. the class `DataSource`.
    2. Similar to proposal 1, but make `DataSourceV2Relation` the central place.
    
    For now we don't know which one is better, it depends on how data source v2 evolves in the future. At this point of time, I think we should pick the simplest approach, which is passing the `DataSourceOptions` to `DataSourceV2Relation`. Then we just need a one-line change in `DataFrameReader`, and don't need to add `v2Options` here.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r167638689
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -37,22 +100,129 @@ case class DataSourceV2Relation(
       }
     
       override def newInstance(): DataSourceV2Relation = {
    -    copy(output = output.map(_.newInstance()))
    +    // projection is used to maintain id assignment.
    +    // if projection is not set, use output so the copy is not equal to the original
    +    copy(projection = projection.map(_.newInstance()))
       }
     }
     
     /**
      * A specialization of DataSourceV2Relation with the streaming bit set to true. Otherwise identical
      * to the non-streaming relation.
      */
    -class StreamingDataSourceV2Relation(
    +case class StreamingDataSourceV2Relation(
         output: Seq[AttributeReference],
    -    reader: DataSourceReader) extends DataSourceV2Relation(output, reader) {
    +    reader: DataSourceReader)
    +    extends LeafNode with DataSourceReaderHolder with MultiInstanceRelation {
       override def isStreaming: Boolean = true
    +
    +  override def canEqual(other: Any): Boolean = other.isInstanceOf[StreamingDataSourceV2Relation]
    +
    +  override def newInstance(): LogicalPlan = copy(output = output.map(_.newInstance()))
     }
     
     object DataSourceV2Relation {
    -  def apply(reader: DataSourceReader): DataSourceV2Relation = {
    -    new DataSourceV2Relation(reader.readSchema().toAttributes, reader)
    +  private implicit class SourceHelpers(source: DataSourceV2) {
    +    def asReadSupport: ReadSupport = {
    +      source match {
    +        case support: ReadSupport =>
    +          support
    +        case _: ReadSupportWithSchema =>
    +          // this method is only called if there is no user-supplied schema. if there is no
    +          // user-supplied schema and ReadSupport was not implemented, throw a helpful exception.
    +          throw new AnalysisException(s"Data source requires a user-supplied schema: $name")
    +        case _ =>
    +          throw new AnalysisException(s"Data source is not readable: $name")
    +      }
    +    }
    +
    +    def asReadSupportWithSchema: ReadSupportWithSchema = {
    +      source match {
    +        case support: ReadSupportWithSchema =>
    +          support
    +        case _: ReadSupport =>
    --- End diff --
    
    For your second concern about checking ASAP: this will be done when the relation is first created because projection is required and is always based on the schema returned by a reader. To be more clear about when this should happen, I think that the requirement is for this to happen during job planning and, ideally, before filter push-down.
    
    For the case where the user supplies a schema that is identical to the source's schema: I think this might be a bad idea because it will cause confusion when source schemas change. Plus, I can't think of a situation where it is a good idea to pass a schema that is ignored.
    
    Here's an example of how this will be confusing: think of a job that supplies a schema identical to the table's schema and runs fine, so it goes into production. What happens when the table's schema changes? If someone adds a column to the table, then the job will start failing and report that the source doesn't support user-supplied schemas, even though it had previously worked just fine with a user-supplied schema. In addition, the change to the table is actually compatible with the old job because the new column will be removed by a projection.
    
    To fix this situation, it may be tempting to use the user-supplied schema as an initial projection. But that doesn't make sense because we don't need two projection mechanisms. If we used this as a second way to project, it would be confusing that you can't actually leave out columns (at least for CSV) and it would be odd that using this path you can coerce types, which should usually be done by Spark.
    
    I think it is best not to allow a user-supplied schema when it isn't supported by a source.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Us...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r163734444
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,15 +17,149 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
     import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceV2, DataSourceV2Options, ReadSupport, ReadSupportWithSchema, WriteSupport}
     import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.sources.v2.writer.DataSourceV2Writer
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    fullOutput: Seq[AttributeReference],
    -    reader: DataSourceV2Reader) extends LeafNode with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    path: Option[String] = None,
    +    table: Option[TableIdentifier] = None,
    +    projection: Option[Seq[AttributeReference]] = None,
    +    filters: Option[Seq[Expression]] = None,
    --- End diff --
    
    so every time we add a new push down interface, we need to add parameters here too?


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    **[Test build #87123 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87123/testReport)** for PR 20387 at commit [`7ef90cb`](https://github.com/apache/spark/commit/7ef90cb7e20b903b3569470ae0e3c26a03cb6a2a).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/882/
    Test PASSed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    **[Test build #87274 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87274/testReport)** for PR 20387 at commit [`ce5f40d`](https://github.com/apache/spark/commit/ce5f40d6a512874e2dd45bab9256f77ff74e628b).


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/674/
    Test PASSed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/87427/
    Test FAILed.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/87268/
    Test FAILed.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r166997241
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,17 +17,130 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
     import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    -import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
    +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema, WriteSupport}
    +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics}
    +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    output: Seq[AttributeReference],
    -    reader: DataSourceReader)
    -  extends LeafNode with MultiInstanceRelation with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    projection: Option[Seq[AttributeReference]] = None,
    +    filters: Option[Seq[Expression]] = None,
    +    userSchema: Option[StructType] = None) extends LeafNode with MultiInstanceRelation {
    +
    +  override def simpleString: String = {
    +    s"DataSourceV2Relation(source=$sourceName, " +
    +      s"schema=[${output.map(a => s"$a ${a.dataType.simpleString}").mkString(", ")}], " +
    +      s"filters=[${pushedFilters.mkString(", ")}], options=$options)"
    +  }
    +
    +  override lazy val schema: StructType = reader.readSchema()
    +
    +  override lazy val output: Seq[AttributeReference] = {
    +    projection match {
    +      case Some(attrs) =>
    +        // use the projection attributes to avoid assigning new ids. fields that are not projected
    +        // will be assigned new ids, which is okay because they are not projected.
    +        val attrMap = attrs.map(a => a.name -> a).toMap
    +        schema.map(f => attrMap.getOrElse(f.name,
    +          AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()))
    +      case _ =>
    +        schema.toAttributes
    +    }
    +  }
    +
    +  private lazy val v2Options: DataSourceOptions = {
    +    // ensure path and table options are set correctly
    +    val updatedOptions = new mutable.HashMap[String, String]
    +    updatedOptions ++= options
    +
    +    new DataSourceOptions(options.asJava)
    --- End diff --
    
    As we've already discussed at length, I think it is a bad idea to create `DataSourceOptions` and pass it to the relation.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    > Sorry, what do you want to change?
    
    Nothing, just a potential use case to support creating `DataSourceOptions` in `DataSourceV2Relation`. If there are a lot of places like this, it's painful to duplicate the logic of merging extra entries to `DataSourceOptions`.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark pull request #20387: [SPARK-23203][SPARK-23204][SQL]: DataSourceV2: Us...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20387#discussion_r164034244
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala ---
    @@ -17,15 +17,149 @@
     
     package org.apache.spark.sql.execution.datasources.v2
     
    -import org.apache.spark.sql.catalyst.expressions.AttributeReference
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
     import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
    +import org.apache.spark.sql.execution.datasources.DataSourceStrategy
    +import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
    +import org.apache.spark.sql.sources.v2.{DataSourceV2, DataSourceV2Options, ReadSupport, ReadSupportWithSchema, WriteSupport}
     import org.apache.spark.sql.sources.v2.reader._
    +import org.apache.spark.sql.sources.v2.writer.DataSourceV2Writer
    +import org.apache.spark.sql.types.StructType
     
     case class DataSourceV2Relation(
    -    fullOutput: Seq[AttributeReference],
    -    reader: DataSourceV2Reader) extends LeafNode with DataSourceReaderHolder {
    +    source: DataSourceV2,
    +    options: Map[String, String],
    +    path: Option[String] = None,
    +    table: Option[TableIdentifier] = None,
    --- End diff --
    
    But not all data sources have path and table name, if you feel strongly about it, we can add 2 methods that exact path and table from options.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    **[Test build #87004 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87004/testReport)** for PR 20387 at commit [`ab945a1`](https://github.com/apache/spark/commit/ab945a19efe666c41deae9c044002f3455220c1d).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20387
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/87274/
    Test FAILed.


---

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