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-33492][SQL] DSv2: Append/Overwrite/ReplaceTable should invalidate cache #30429

Closed
wants to merge 3 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -147,6 +147,7 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat
catalog match {
case staging: StagingTableCatalog =>
AtomicReplaceTableAsSelectExec(
session,
staging,
ident,
parts,
Expand All @@ -157,6 +158,7 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat
orCreate = orCreate) :: Nil
case _ =>
ReplaceTableAsSelectExec(
session,
catalog,
ident,
parts,
Expand All @@ -170,9 +172,9 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat
case AppendData(r: DataSourceV2Relation, query, writeOptions, _) =>
r.table.asWritable match {
case v1 if v1.supports(TableCapability.V1_BATCH_WRITE) =>
AppendDataExecV1(v1, writeOptions.asOptions, query) :: Nil
AppendDataExecV1(v1, writeOptions.asOptions, query, r) :: Nil
case v2 =>
AppendDataExec(v2, writeOptions.asOptions, planLater(query)) :: Nil
AppendDataExec(session, v2, r, writeOptions.asOptions, planLater(query)) :: Nil
}

case OverwriteByExpression(r: DataSourceV2Relation, deleteExpr, query, writeOptions, _) =>
Expand All @@ -184,14 +186,15 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat
}.toArray
r.table.asWritable match {
case v1 if v1.supports(TableCapability.V1_BATCH_WRITE) =>
OverwriteByExpressionExecV1(v1, filters, writeOptions.asOptions, query) :: Nil
OverwriteByExpressionExecV1(v1, filters, writeOptions.asOptions, query, r) :: Nil
case v2 =>
OverwriteByExpressionExec(v2, filters, writeOptions.asOptions, planLater(query)) :: Nil
OverwriteByExpressionExec(session, v2, r, filters,
writeOptions.asOptions, planLater(query)) :: Nil
}

case OverwritePartitionsDynamic(r: DataSourceV2Relation, query, writeOptions, _) =>
OverwritePartitionsDynamicExec(
r.table.asWritable, writeOptions.asOptions, planLater(query)) :: Nil
session, r.table.asWritable, r, writeOptions.asOptions, planLater(query)) :: Nil

case DeleteFromTable(relation, condition) =>
relation match {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,10 +37,11 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap
case class AppendDataExecV1(
table: SupportsWrite,
writeOptions: CaseInsensitiveStringMap,
plan: LogicalPlan) extends V1FallbackWriters {
plan: LogicalPlan,
v2Relation: DataSourceV2Relation) extends V1FallbackWriters {

override protected def run(): Seq[InternalRow] = {
writeWithV1(newWriteBuilder().buildForV1Write())
writeWithV1(newWriteBuilder().buildForV1Write(), Some(v2Relation))
}
}

Expand All @@ -59,7 +60,8 @@ case class OverwriteByExpressionExecV1(
table: SupportsWrite,
deleteWhere: Array[Filter],
writeOptions: CaseInsensitiveStringMap,
plan: LogicalPlan) extends V1FallbackWriters {
plan: LogicalPlan,
v2Relation: DataSourceV2Relation) extends V1FallbackWriters {

private def isTruncate(filters: Array[Filter]): Boolean = {
filters.length == 1 && filters(0).isInstanceOf[AlwaysTrue]
Expand All @@ -68,10 +70,10 @@ case class OverwriteByExpressionExecV1(
override protected def run(): Seq[InternalRow] = {
newWriteBuilder() match {
case builder: SupportsTruncate if isTruncate(deleteWhere) =>
writeWithV1(builder.truncate().asV1Builder.buildForV1Write())
writeWithV1(builder.truncate().asV1Builder.buildForV1Write(), Some(v2Relation))

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

case _ =>
throw new SparkException(s"Table does not support overwrite by expression: $table")
Expand Down Expand Up @@ -112,9 +114,14 @@ sealed trait V1FallbackWriters extends V2CommandExec with SupportsV1Write {
trait SupportsV1Write extends SparkPlan {
def plan: LogicalPlan

protected def writeWithV1(relation: InsertableRelation): Seq[InternalRow] = {
protected def writeWithV1(
relation: InsertableRelation,
v2Relation: Option[DataSourceV2Relation] = None): Seq[InternalRow] = {
val session = sqlContext.sparkSession
// The `plan` is already optimized, we should not analyze and optimize it again.
relation.insert(AlreadyOptimized.dataFrame(sqlContext.sparkSession, plan), overwrite = false)
relation.insert(AlreadyOptimized.dataFrame(session, plan), overwrite = false)
v2Relation.foreach(r => session.sharedState.cacheManager.recacheByPlan(session, r))

Nil
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import org.apache.spark.{SparkEnv, SparkException, TaskContext}
import org.apache.spark.executor.CommitDeniedException
import org.apache.spark.internal.Logging
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchTableException, TableAlreadyExistsException}
import org.apache.spark.sql.catalyst.expressions.Attribute
Expand Down Expand Up @@ -127,6 +128,7 @@ case class AtomicCreateTableAsSelectExec(
* ReplaceTableAsSelectStagingExec.
*/
case class ReplaceTableAsSelectExec(
session: SparkSession,
catalog: TableCatalog,
ident: Identifier,
partitioning: Seq[Transform],
Expand All @@ -146,6 +148,8 @@ case class ReplaceTableAsSelectExec(
// 2. Writing to the new table fails,
// 3. The table returned by catalog.createTable doesn't support writing.
if (catalog.tableExists(ident)) {
val table = catalog.loadTable(ident)
uncacheTable(session, catalog, table, ident)
catalog.dropTable(ident)
} else if (!orCreate) {
throw new CannotReplaceMissingTableException(ident)
Expand All @@ -169,6 +173,7 @@ case class ReplaceTableAsSelectExec(
* is left untouched.
*/
case class AtomicReplaceTableAsSelectExec(
session: SparkSession,
catalog: StagingTableCatalog,
ident: Identifier,
partitioning: Seq[Transform],
Expand All @@ -180,6 +185,10 @@ case class AtomicReplaceTableAsSelectExec(

override protected def run(): Seq[InternalRow] = {
val schema = query.schema.asNullable
if (catalog.tableExists(ident)) {
val table = catalog.loadTable(ident)
uncacheTable(session, catalog, table, ident)
}
val staged = if (orCreate) {
catalog.stageCreateOrReplace(
ident, schema, partitioning.toArray, properties.asJava)
Expand All @@ -204,12 +213,16 @@ case class AtomicReplaceTableAsSelectExec(
* Rows in the output data set are appended.
*/
case class AppendDataExec(
session: SparkSession,
table: SupportsWrite,
relation: DataSourceV2Relation,
writeOptions: CaseInsensitiveStringMap,
query: SparkPlan) extends V2TableWriteExec with BatchWriteHelper {

override protected def run(): Seq[InternalRow] = {
writeWithV2(newWriteBuilder().buildForBatch())
val writtenRows = writeWithV2(newWriteBuilder().buildForBatch())
session.sharedState.cacheManager.recacheByPlan(session, relation)
writtenRows
}
}

Expand All @@ -224,7 +237,9 @@ case class AppendDataExec(
* AlwaysTrue to delete all rows.
*/
case class OverwriteByExpressionExec(
session: SparkSession,
table: SupportsWrite,
relation: DataSourceV2Relation,
deleteWhere: Array[Filter],
writeOptions: CaseInsensitiveStringMap,
query: SparkPlan) extends V2TableWriteExec with BatchWriteHelper {
Expand All @@ -234,7 +249,7 @@ case class OverwriteByExpressionExec(
}

override protected def run(): Seq[InternalRow] = {
newWriteBuilder() match {
val writtenRows = newWriteBuilder() match {
case builder: SupportsTruncate if isTruncate(deleteWhere) =>
writeWithV2(builder.truncate().buildForBatch())

Expand All @@ -244,9 +259,12 @@ case class OverwriteByExpressionExec(
case _ =>
throw new SparkException(s"Table does not support overwrite by expression: $table")
}
session.sharedState.cacheManager.recacheByPlan(session, relation)
writtenRows
}
}


/**
* Physical plan node for dynamic partition overwrite into a v2 table.
*
Expand All @@ -257,18 +275,22 @@ case class OverwriteByExpressionExec(
* are not modified.
*/
case class OverwritePartitionsDynamicExec(
session: SparkSession,
table: SupportsWrite,
relation: DataSourceV2Relation,
writeOptions: CaseInsensitiveStringMap,
query: SparkPlan) extends V2TableWriteExec with BatchWriteHelper {

override protected def run(): Seq[InternalRow] = {
newWriteBuilder() match {
val writtenRows = newWriteBuilder() match {
case builder: SupportsDynamicOverwrite =>
writeWithV2(builder.overwriteDynamicPartitions().buildForBatch())

case _ =>
throw new SparkException(s"Table does not support dynamic partition overwrite: $table")
}
session.sharedState.cacheManager.recacheByPlan(session, relation)
writtenRows
}
}

Expand Down Expand Up @@ -370,6 +392,15 @@ trait V2TableWriteExec extends V2CommandExec with UnaryExecNode {

Nil
}

protected def uncacheTable(
session: SparkSession,
catalog: TableCatalog,
table: Table,
ident: Identifier): Unit = {
val plan = DataSourceV2Relation.create(table, Some(catalog), Some(ident))
session.sharedState.cacheManager.uncacheQuery(session, plan, cascade = true)
}
}

object DataWritingSparkTask extends Logging {
Expand Down Expand Up @@ -484,3 +515,4 @@ private[v2] case class DataWritingSparkTaskResult(
* Sink progress information collected after commit.
*/
private[sql] case class StreamWriterCommitProgress(numOutputRows: Long)

Original file line number Diff line number Diff line change
Expand Up @@ -782,6 +782,84 @@ class DataSourceV2SQLSuite
}
}

test("SPARK-33492: ReplaceTableAsSelect (atomic or non-atomic) should invalidate cache") {
Seq("testcat.ns.t", "testcat_atomic.ns.t").foreach { t =>
val view = "view"
withTable(t) {
withTempView(view) {
sql(s"CREATE TABLE $t USING foo AS SELECT id, data FROM source")
sql(s"CACHE TABLE $view AS SELECT id FROM $t")
checkAnswer(sql(s"SELECT * FROM $t"), spark.table("source"))
checkAnswer(sql(s"SELECT * FROM $view"), spark.table("source").select("id"))

sql(s"REPLACE TABLE $t USING foo AS SELECT id FROM source")
assert(spark.sharedState.cacheManager.lookupCachedData(spark.table(view)).isEmpty)
}
}
}
}

test("SPARK-33492: AppendData should refresh cache") {
import testImplicits._

val t = "testcat.ns.t"
val view = "view"
withTable(t) {
withTempView(view) {
Seq((1, "a")).toDF("i", "j").write.saveAsTable(t)
sql(s"CACHE TABLE $view AS SELECT i FROM $t")
checkAnswer(sql(s"SELECT * FROM $t"), Row(1, "a") :: Nil)
checkAnswer(sql(s"SELECT * FROM $view"), Row(1) :: Nil)

Seq((2, "b")).toDF("i", "j").write.mode(SaveMode.Append).saveAsTable(t)

assert(spark.sharedState.cacheManager.lookupCachedData(spark.table(view)).isDefined)
checkAnswer(sql(s"SELECT * FROM $t"), Row(1, "a") :: Row(2, "b") :: Nil)
checkAnswer(sql(s"SELECT * FROM $view"), Row(1) :: Row(2) :: Nil)
}
}
}

test("SPARK-33492: OverwriteByExpression should refresh cache") {
val t = "testcat.ns.t"
val view = "view"
withTable(t) {
withTempView(view) {
sql(s"CREATE TABLE $t USING foo AS SELECT id, data FROM source")
sql(s"CACHE TABLE $view AS SELECT id FROM $t")
checkAnswer(sql(s"SELECT * FROM $t"), spark.table("source"))
checkAnswer(sql(s"SELECT * FROM $view"), spark.table("source").select("id"))

sql(s"INSERT OVERWRITE TABLE $t VALUES (1, 'a')")

assert(spark.sharedState.cacheManager.lookupCachedData(spark.table(view)).isDefined)
checkAnswer(sql(s"SELECT * FROM $t"), Row(1, "a") :: Nil)
checkAnswer(sql(s"SELECT * FROM $view"), Row(1) :: Nil)
}
}
}

test("SPARK-33492: OverwritePartitionsDynamic should refresh cache") {
import testImplicits._

val t = "testcat.ns.t"
val view = "view"
withTable(t) {
withTempView(view) {
Seq((1, "a", 1)).toDF("i", "j", "k").write.partitionBy("k") saveAsTable(t)
sql(s"CACHE TABLE $view AS SELECT i FROM $t")
checkAnswer(sql(s"SELECT * FROM $t"), Row(1, "a", 1) :: Nil)
checkAnswer(sql(s"SELECT * FROM $view"), Row(1) :: Nil)

Seq((2, "b", 1)).toDF("i", "j", "k").writeTo(t).overwritePartitions()

assert(spark.sharedState.cacheManager.lookupCachedData(spark.table(view)).isDefined)
checkAnswer(sql(s"SELECT * FROM $t"), Row(2, "b", 1) :: Nil)
checkAnswer(sql(s"SELECT * FROM $view"), Row(2) :: Nil)
}
}
}

test("Relation: basic") {
val t1 = "testcat.ns1.ns2.tbl"
withTable(t1) {
Expand Down
Loading