Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-28554][SQL] Adds a v1 fallback writer implementation for v2 data source codepaths #25348

Closed
wants to merge 25 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
25 commits
Select commit Hold shift + click to select a range
d5798fd
Adds a v1 fallback writer implementation for v2 data source codepaths
brkyvz Aug 3, 2019
59094e9
Update WriteToDataSourceV2Exec.scala
brkyvz Aug 4, 2019
1587d31
Merge branch 'master' of github.com:apache/spark into v1WriteFallback
brkyvz Aug 5, 2019
5aed803
some changes but doubtful
brkyvz Aug 5, 2019
bcdf8c5
Merge branch 'master' of github.com:apache/spark into v1WriteFallback
brkyvz Aug 7, 2019
a1284a1
Revert "some changes but doubtful"
brkyvz Aug 7, 2019
ef2ec72
Address comments and separate whatever's possible
brkyvz Aug 7, 2019
335a92d
update docs
brkyvz Aug 7, 2019
3e35d5c
minor move for better separation
brkyvz Aug 7, 2019
41c4c0a
use implicit class
brkyvz Aug 7, 2019
138f2b9
use insertable relation instead
brkyvz Aug 7, 2019
078d0f1
Merge branch 'master' of github.com:apache/spark into v1WriteFallback
brkyvz Aug 8, 2019
d816824
address comments
brkyvz Aug 8, 2019
c62a60d
Merge branch 'master' of github.com:apache/spark into v1WriteFallback
brkyvz Aug 12, 2019
cfdb036
Added basic tests
brkyvz Aug 12, 2019
442836b
Update V1WriteBuilder.scala
brkyvz Aug 12, 2019
7396f24
fix tests
brkyvz Aug 14, 2019
20b906d
Merge branch 'v1WriteFallback' of github.com:brkyvz/spark into v1Writ…
brkyvz Aug 14, 2019
a93bf8c
Update V1WriteBuilder.scala
brkyvz Aug 14, 2019
83dbd78
Merge branch 'master' into v1WriteFallback
brkyvz Aug 14, 2019
9bfb76e
Merge branch 'master' of github.com:apache/spark into v1WriteFallback
brkyvz Aug 20, 2019
00347ee
Add table capability to do V1_BATCH_WRITE
brkyvz Aug 20, 2019
d4d6276
test refactor
brkyvz Aug 20, 2019
749ae85
Update V1WriteFallbackSuite.scala
brkyvz Aug 20, 2019
27598ce
address nits
brkyvz Aug 21, 2019
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Expand Up @@ -89,5 +89,14 @@ public enum TableCapability {
/**
* Signals that the table accepts input of any schema in a write operation.
*/
ACCEPT_ANY_SCHEMA
ACCEPT_ANY_SCHEMA,

/**
* Signals that the table supports append writes using the V1 InsertableRelation interface.
* <p>
* Tables that return this capability must create a V1WriteBuilder and may also support additional
* write modes, like {@link #TRUNCATE}, and {@link #OVERWRITE_BY_FILTER}, but cannot support
* {@link #OVERWRITE_DYNAMIC}.
*/
V1_BATCH_WRITE
}
Expand Up @@ -17,6 +17,8 @@

package org.apache.spark.sql.execution.datasources.v2

import java.util.UUID

import scala.collection.JavaConverters._
import scala.collection.mutable

Expand All @@ -29,8 +31,10 @@ import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SparkPlan}
import org.apache.spark.sql.execution.datasources.DataSourceStrategy
import org.apache.spark.sql.execution.streaming.continuous.{ContinuousCoalesceExec, WriteToContinuousDataSource, WriteToContinuousDataSourceExec}
import org.apache.spark.sql.sources
import org.apache.spark.sql.sources.v2.TableCapability
import org.apache.spark.sql.sources.v2.reader._
import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousStream, MicroBatchStream}
import org.apache.spark.sql.sources.v2.writer.V1WriteBuilder
import org.apache.spark.sql.util.CaseInsensitiveStringMap

object DataSourceV2Strategy extends Strategy with PredicateHelper {
Expand Down Expand Up @@ -169,10 +173,10 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper {
catalog match {
case staging: StagingTableCatalog =>
AtomicCreateTableAsSelectExec(
staging, ident, parts, planLater(query), props, writeOptions, ifNotExists) :: Nil
staging, ident, parts, query, planLater(query), props, writeOptions, ifNotExists) :: Nil
case _ =>
CreateTableAsSelectExec(
catalog, ident, parts, planLater(query), props, writeOptions, ifNotExists) :: Nil
catalog, ident, parts, query, planLater(query), props, writeOptions, ifNotExists) :: Nil
}

case ReplaceTable(catalog, ident, schema, parts, props, orCreate) =>
Expand All @@ -191,6 +195,7 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper {
staging,
ident,
parts,
query,
planLater(query),
props,
writeOptions,
Expand All @@ -200,24 +205,33 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper {
catalog,
ident,
parts,
query,
planLater(query),
props,
writeOptions,
orCreate = orCreate) :: Nil
}

case AppendData(r: DataSourceV2Relation, query, _) =>
AppendDataExec(r.table.asWritable, r.options, planLater(query)) :: Nil
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why we reuse the v2 CreateTableAsSelectExec for v1 fallback, but not AppendDataExec?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

because table creation happens after certain operations with side effects in CTAS and RTAS.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this is because CTAS and RTAS plans don't have a table instance. CTAS and RTAS create a table at runtime and then write to it.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This explains why we must reuse CreateTableAsSelectExec for v1 fallback, but why can't we reuse AppendDataExec for v1 fallback as well?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It's just cleaner to have a separation of which code path was used. (Shows up separately in the SQL tab, etc)

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If end-users look at the SQL tab and see AppendDataExecV1, they would expect to see v1 version of CTAS physical plan as well, and may report a bug if they don't see it.

BTW I think there are other ways to implement this feature (users know if v1 fallback is triggered from SQL tab), e.g. we can use SQLMetrics to report it, which can be updated at runtime and support CTAS as well.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think most (98%) of Spark users already don't know what the physical nodes stand for, etc. I was thinking of using metrics as well. We just wanted to keep the nodes separate, because the semantics are a bit different for things like OverwriteByExpression, where the source's implementation of the Overwrite + Append (v2 behavior) may not be atomic, etc

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

good point on having different physical nodes for different semantics. It seems to me that the semantic of append is the same between v1 and v2. Shall we reuse AppendDataExec? Then we can make a good story: If semantic is the same, reuse the v2 plan (CREATE TABLE, CTAS, APPEND). Otherwise, create a new physical node (OVERWRITE).

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I like the idea to use the same physical plan node when the semantics match, but I'm not sure that it is possible to make strong guarantees about not changing v1 if we do. The problem is that there are multiple v1 plan nodes for the same operation, which could have slightly different behavior.

We could take the time to inspect the v1 implementations and convert, but that adds risk and takes time. It also isn't needed to migrate to v2, and wouldn't speed up the migration, unless I'm missing something. So it probably doesn't provide enough value to make doing it worth while.

r.table.asWritable match {
case v1 if v1.supports(TableCapability.V1_BATCH_WRITE) =>
AppendDataExecV1(v1, r.options, query) :: Nil
case v2 =>
AppendDataExec(v2, r.options, planLater(query)) :: Nil
}

case OverwriteByExpression(r: DataSourceV2Relation, deleteExpr, query, _) =>
// fail if any filter cannot be converted. correctness depends on removing all matching data.
val filters = splitConjunctivePredicates(deleteExpr).map {
filter => DataSourceStrategy.translateFilter(deleteExpr).getOrElse(
throw new AnalysisException(s"Cannot translate expression to source filter: $filter"))
}.toArray

OverwriteByExpressionExec(
r.table.asWritable, filters, r.options, planLater(query)) :: Nil
r.table.asWritable match {
case v1 if v1.supports(TableCapability.V1_BATCH_WRITE) =>
OverwriteByExpressionExecV1(v1, filters, r.options, query) :: Nil
case v2 =>
OverwriteByExpressionExec(v2, filters, r.options, planLater(query)) :: Nil
}

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

package org.apache.spark.sql.execution.datasources.v2

import java.util.UUID

import scala.collection.JavaConverters._

import org.apache.spark.SparkException
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.{Dataset, SaveMode}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.execution.SparkPlan
import org.apache.spark.sql.sources.{AlwaysTrue, CreatableRelationProvider, Filter, InsertableRelation}
import org.apache.spark.sql.sources.v2.{SupportsWrite, Table}
import org.apache.spark.sql.sources.v2.writer._
import org.apache.spark.sql.util.CaseInsensitiveStringMap

/**
* Physical plan node for append into a v2 table using V1 write interfaces.
*
* Rows in the output data set are appended.
*/
case class AppendDataExecV1(
brkyvz marked this conversation as resolved.
Show resolved Hide resolved
table: SupportsWrite,
writeOptions: CaseInsensitiveStringMap,
plan: LogicalPlan) extends V1FallbackWriters {
brkyvz marked this conversation as resolved.
Show resolved Hide resolved

override protected def doExecute(): RDD[InternalRow] = {
writeWithV1(newWriteBuilder().buildForV1Write())
}
}

/**
* Physical plan node for overwrite into a v2 table with V1 write interfaces. Note that when this
* interface is used, the atomicity of the operation depends solely on the target data source.
*
* Overwrites data in a table matched by a set of filters. Rows matching all of the filters will be
* deleted and rows in the output data set are appended.
*
* This plan is used to implement SaveMode.Overwrite. The behavior of SaveMode.Overwrite is to
* truncate the table -- delete all rows -- and append the output data set. This uses the filter
* AlwaysTrue to delete all rows.
*/
case class OverwriteByExpressionExecV1(
table: SupportsWrite,
deleteWhere: Array[Filter],
writeOptions: CaseInsensitiveStringMap,
plan: LogicalPlan) extends V1FallbackWriters {

private def isTruncate(filters: Array[Filter]): Boolean = {
filters.length == 1 && filters(0).isInstanceOf[AlwaysTrue]
}

override protected def doExecute(): RDD[InternalRow] = {
newWriteBuilder() match {
case builder: SupportsTruncate if isTruncate(deleteWhere) =>
writeWithV1(builder.truncate().asV1Builder.buildForV1Write())
brkyvz marked this conversation as resolved.
Show resolved Hide resolved

case builder: SupportsOverwrite =>
writeWithV1(builder.overwrite(deleteWhere).asV1Builder.buildForV1Write())

case _ =>
throw new SparkException(s"Table does not support overwrite by expression: $table")
}
}
}

/** Some helper interfaces that use V2 write semantics through the V1 writer interface. */
sealed trait V1FallbackWriters extends SupportsV1Write {
override def output: Seq[Attribute] = Nil
override final def children: Seq[SparkPlan] = Nil

def table: SupportsWrite
def writeOptions: CaseInsensitiveStringMap

protected implicit class toV1WriteBuilder(builder: WriteBuilder) {
def asV1Builder: V1WriteBuilder = builder match {
case v1: V1WriteBuilder => v1
case other => throw new IllegalStateException(
s"The returned writer ${other} was no longer a V1WriteBuilder.")
}
}

protected def newWriteBuilder(): V1WriteBuilder = {
val writeBuilder = table.newWriteBuilder(writeOptions)
.withInputDataSchema(plan.schema)
.withQueryId(UUID.randomUUID().toString)
writeBuilder.asV1Builder
}
}

/**
* A trait that allows Tables that use V1 Writer interfaces to append data.
*/
trait SupportsV1Write extends SparkPlan {
// TODO: We should be able to work on SparkPlans at this point.
def plan: LogicalPlan

protected def writeWithV1(relation: InsertableRelation): RDD[InternalRow] = {
relation.insert(Dataset.ofRows(sqlContext.sparkSession, plan), overwrite = false)
brkyvz marked this conversation as resolved.
Show resolved Hide resolved
sparkContext.emptyRDD
}
}