- Aggregated Number Of State Rows Dropped By Watermark {SparkUIUtils.tooltip("Aggregated number of state rows dropped by watermark.", "right")}
+ Aggregated Number Of Rows Dropped By Watermark {SparkUIUtils.tooltip("Accumulates all input rows being dropped in stateful operators by watermark. 'Inputs' are relative to operators.", "right")}
|
- {graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)} |
- {graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)} |
+ {graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)} |
+ {graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)} |
// scalastyle:on
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala
index ddafa1bb5070a..90df4ee08bfc0 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala
@@ -36,6 +36,7 @@ import org.apache.spark.sql.internal.connector.SimpleTableProvider
import org.apache.spark.sql.sources.SimpleScanSource
import org.apache.spark.sql.types.{BooleanType, LongType, StringType, StructField, StructType}
import org.apache.spark.sql.util.CaseInsensitiveStringMap
+import org.apache.spark.unsafe.types.UTF8String
import org.apache.spark.util.Utils
class DataSourceV2SQLSuite
@@ -43,7 +44,6 @@ class DataSourceV2SQLSuite
with AlterTableTests with DatasourceV2SQLBase {
import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
- import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._
private val v2Source = classOf[FakeV2Provider].getName
override protected val v2Format = v2Source
@@ -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) {
@@ -1980,57 +2058,6 @@ class DataSourceV2SQLSuite
}
}
- test("ALTER TABLE RECOVER PARTITIONS") {
- val t = "testcat.ns1.ns2.tbl"
- withTable(t) {
- spark.sql(s"CREATE TABLE $t (id bigint, data string) USING foo")
- val e = intercept[AnalysisException] {
- sql(s"ALTER TABLE $t RECOVER PARTITIONS")
- }
- assert(e.message.contains("ALTER TABLE RECOVER PARTITIONS is only supported with v1 tables"))
- }
- }
-
- test("ALTER TABLE ADD PARTITION") {
- val t = "testpart.ns1.ns2.tbl"
- withTable(t) {
- spark.sql(s"CREATE TABLE $t (id bigint, data string) USING foo PARTITIONED BY (id)")
- spark.sql(s"ALTER TABLE $t ADD PARTITION (id=1) LOCATION 'loc'")
-
- val partTable = catalog("testpart").asTableCatalog
- .loadTable(Identifier.of(Array("ns1", "ns2"), "tbl")).asInstanceOf[InMemoryPartitionTable]
- assert(partTable.partitionExists(InternalRow.fromSeq(Seq(1))))
-
- val partMetadata = partTable.loadPartitionMetadata(InternalRow.fromSeq(Seq(1)))
- assert(partMetadata.containsKey("location"))
- assert(partMetadata.get("location") == "loc")
- }
- }
-
- test("ALTER TABLE RENAME PARTITION") {
- val t = "testcat.ns1.ns2.tbl"
- withTable(t) {
- spark.sql(s"CREATE TABLE $t (id bigint, data string) USING foo PARTITIONED BY (id)")
- val e = intercept[AnalysisException] {
- sql(s"ALTER TABLE $t PARTITION (id=1) RENAME TO PARTITION (id=2)")
- }
- assert(e.message.contains("ALTER TABLE RENAME PARTITION is only supported with v1 tables"))
- }
- }
-
- test("ALTER TABLE DROP PARTITION") {
- val t = "testpart.ns1.ns2.tbl"
- withTable(t) {
- spark.sql(s"CREATE TABLE $t (id bigint, data string) USING foo PARTITIONED BY (id)")
- spark.sql(s"ALTER TABLE $t ADD PARTITION (id=1) LOCATION 'loc'")
- spark.sql(s"ALTER TABLE $t DROP PARTITION (id=1)")
-
- val partTable =
- catalog("testpart").asTableCatalog.loadTable(Identifier.of(Array("ns1", "ns2"), "tbl"))
- assert(!partTable.asPartitionable.partitionExists(InternalRow.fromSeq(Seq(1))))
- }
- }
-
test("ALTER TABLE SerDe properties") {
val t = "testcat.ns1.ns2.tbl"
withTable(t) {
@@ -2513,6 +2540,25 @@ class DataSourceV2SQLSuite
}
}
+ test("SPARK-33505: insert into partitioned table") {
+ val t = "testpart.ns1.ns2.tbl"
+ withTable(t) {
+ sql(s"""
+ |CREATE TABLE $t (id bigint, city string, data string)
+ |USING foo
+ |PARTITIONED BY (id, city)""".stripMargin)
+ val partTable = catalog("testpart").asTableCatalog
+ .loadTable(Identifier.of(Array("ns1", "ns2"), "tbl")).asInstanceOf[InMemoryPartitionTable]
+ val expectedPartitionIdent = InternalRow.fromSeq(Seq(1, UTF8String.fromString("NY")))
+ assert(!partTable.partitionExists(expectedPartitionIdent))
+ sql(s"INSERT INTO $t PARTITION(id = 1, city = 'NY') SELECT 'abc'")
+ assert(partTable.partitionExists(expectedPartitionIdent))
+ // Insert into the existing partition must not fail
+ sql(s"INSERT INTO $t PARTITION(id = 1, city = 'NY') SELECT 'def'")
+ assert(partTable.partitionExists(expectedPartitionIdent))
+ }
+ }
+
private def testNotSupportedV2Command(sqlCommand: String, sqlParams: String): Unit = {
val e = intercept[AnalysisException] {
sql(s"$sqlCommand $sqlParams")
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/V1WriteFallbackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/V1WriteFallbackSuite.scala
index 4b52a4cbf4116..cba7dd35fb3bc 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/connector/V1WriteFallbackSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/V1WriteFallbackSuite.scala
@@ -24,14 +24,17 @@ import scala.collection.mutable
import org.scalatest.BeforeAndAfter
+import org.apache.spark.rdd.RDD
import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row, SaveMode, SparkSession, SQLContext}
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.catalyst.trees.TreeNodeTag
-import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapability}
+import org.apache.spark.sql.connector.catalog.{Identifier, SupportsRead, SupportsWrite, Table, TableCapability}
import org.apache.spark.sql.connector.expressions.{FieldReference, IdentityTransform, Transform}
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, V1Scan}
import org.apache.spark.sql.connector.write.{LogicalWriteInfo, LogicalWriteInfoImpl, SupportsOverwrite, SupportsTruncate, V1WriteBuilder, WriteBuilder}
import org.apache.spark.sql.execution.datasources.DataSourceUtils
+import org.apache.spark.sql.functions.lit
import org.apache.spark.sql.internal.SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION
import org.apache.spark.sql.internal.connector.SimpleTableProvider
import org.apache.spark.sql.sources._
@@ -145,6 +148,52 @@ class V1WriteFallbackSuite extends QueryTest with SharedSparkSession with Before
SparkSession.setDefaultSession(spark)
}
}
+
+ test("SPARK-33492: append fallback should refresh cache") {
+ SparkSession.clearActiveSession()
+ SparkSession.clearDefaultSession()
+ try {
+ val session = SparkSession.builder()
+ .master("local[1]")
+ .config(V2_SESSION_CATALOG_IMPLEMENTATION.key, classOf[V1FallbackTableCatalog].getName)
+ .getOrCreate()
+ val df = session.createDataFrame(Seq((1, "x")))
+ df.write.mode("append").option("name", "t1").format(v2Format).saveAsTable("test")
+ session.catalog.cacheTable("test")
+ checkAnswer(session.read.table("test"), Row(1, "x") :: Nil)
+
+ val df2 = session.createDataFrame(Seq((2, "y")))
+ df2.writeTo("test").append()
+ checkAnswer(session.read.table("test"), Row(1, "x") :: Row(2, "y") :: Nil)
+
+ } finally {
+ SparkSession.setActiveSession(spark)
+ SparkSession.setDefaultSession(spark)
+ }
+ }
+
+ test("SPARK-33492: overwrite fallback should refresh cache") {
+ SparkSession.clearActiveSession()
+ SparkSession.clearDefaultSession()
+ try {
+ val session = SparkSession.builder()
+ .master("local[1]")
+ .config(V2_SESSION_CATALOG_IMPLEMENTATION.key, classOf[V1FallbackTableCatalog].getName)
+ .getOrCreate()
+ val df = session.createDataFrame(Seq((1, "x")))
+ df.write.mode("append").option("name", "t1").format(v2Format).saveAsTable("test")
+ session.catalog.cacheTable("test")
+ checkAnswer(session.read.table("test"), Row(1, "x") :: Nil)
+
+ val df2 = session.createDataFrame(Seq((2, "y")))
+ df2.writeTo("test").overwrite(lit(true))
+ checkAnswer(session.read.table("test"), Row(2, "y") :: Nil)
+
+ } finally {
+ SparkSession.setActiveSession(spark)
+ SparkSession.setDefaultSession(spark)
+ }
+ }
}
class V1WriteFallbackSessionCatalogSuite
@@ -177,6 +226,7 @@ class V1FallbackTableCatalog extends TestV2SessionCatalogBase[InMemoryTableWithV
properties: util.Map[String, String]): InMemoryTableWithV1Fallback = {
val t = new InMemoryTableWithV1Fallback(name, schema, partitions, properties)
InMemoryV1Provider.tables.put(name, t)
+ tables.put(Identifier.of(Array("default"), name), t)
t
}
}
@@ -272,7 +322,7 @@ class InMemoryTableWithV1Fallback(
override val partitioning: Array[Transform],
override val properties: util.Map[String, String])
extends Table
- with SupportsWrite {
+ with SupportsWrite with SupportsRead {
partitioning.foreach { t =>
if (!t.isInstanceOf[IdentityTransform]) {
@@ -281,6 +331,7 @@ class InMemoryTableWithV1Fallback(
}
override def capabilities: util.Set[TableCapability] = Set(
+ TableCapability.BATCH_READ,
TableCapability.V1_BATCH_WRITE,
TableCapability.OVERWRITE_BY_FILTER,
TableCapability.TRUNCATE).asJava
@@ -338,6 +389,30 @@ class InMemoryTableWithV1Fallback(
}
}
}
+
+ override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder =
+ new V1ReadFallbackScanBuilder(schema)
+
+ private class V1ReadFallbackScanBuilder(schema: StructType) extends ScanBuilder {
+ override def build(): Scan = new V1ReadFallbackScan(schema)
+ }
+
+ private class V1ReadFallbackScan(schema: StructType) extends V1Scan {
+ override def readSchema(): StructType = schema
+ override def toV1TableScan[T <: BaseRelation with TableScan](context: SQLContext): T =
+ new V1TableScan(context, schema).asInstanceOf[T]
+ }
+
+ private class V1TableScan(
+ context: SQLContext,
+ requiredSchema: StructType) extends BaseRelation with TableScan {
+ override def sqlContext: SQLContext = context
+ override def schema: StructType = requiredSchema
+ override def buildScan(): RDD[Row] = {
+ val data = InMemoryV1Provider.getTableData(context.sparkSession, name).collect()
+ context.sparkContext.makeRDD(data)
+ }
+ }
}
/** A rule that fails if a query plan is analyzed twice. */
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/UISeleniumSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/UISeleniumSuite.scala
index 2aaeb67d30538..94844c4e87a84 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/UISeleniumSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/UISeleniumSuite.scala
@@ -141,7 +141,7 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B
summaryText should contain ("Aggregated Number Of Total State Rows (?)")
summaryText should contain ("Aggregated Number Of Updated State Rows (?)")
summaryText should contain ("Aggregated State Memory Used In Bytes (?)")
- summaryText should contain ("Aggregated Number Of State Rows Dropped By Watermark (?)")
+ summaryText should contain ("Aggregated Number Of Rows Dropped By Watermark (?)")
summaryText should contain ("Aggregated Custom Metric stateOnCurrentVersionSizeBytes" +
" (?)")
summaryText should not contain ("Aggregated Custom Metric loadedMapCacheHitCount (?)")