(spark) branch master updated: [SPARK-47920][DOCS][SS][PYTHON] Add doc for python streaming data source API

2024-05-22 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 366a052fe106 [SPARK-47920][DOCS][SS][PYTHON] Add doc for python 
streaming data source API
366a052fe106 is described below

commit 366a052fe10662379ab7f636ccf14ae53a46d8ed
Author: Chaoqin Li 
AuthorDate: Wed May 22 16:35:48 2024 +0900

[SPARK-47920][DOCS][SS][PYTHON] Add doc for python streaming data source API

### What changes were proposed in this pull request?
add doc for python streaming data source API

### Why are the changes needed?

Add user guide to help user develop python streaming data source.

Closes #46139 from chaoqin-li1123/python_ds_doc.

Authored-by: Chaoqin Li 
Signed-off-by: Jungtaek Lim 
---
 .../source/user_guide/sql/python_data_source.rst   | 208 -
 1 file changed, 205 insertions(+), 3 deletions(-)

diff --git a/python/docs/source/user_guide/sql/python_data_source.rst 
b/python/docs/source/user_guide/sql/python_data_source.rst
index 19ed016b82c2..01eddd5566ea 100644
--- a/python/docs/source/user_guide/sql/python_data_source.rst
+++ b/python/docs/source/user_guide/sql/python_data_source.rst
@@ -33,9 +33,23 @@ To create a custom Python data source, you'll need to 
subclass the :class:`DataS
 
 This example demonstrates creating a simple data source to generate synthetic 
data using the `faker` library. Ensure the `faker` library is installed and 
accessible in your Python environment.
 
-**Step 1: Define the Data Source**
+**Define the Data Source**
 
-Start by creating a new subclass of :class:`DataSource`. Define the source 
name, schema, and reader logic as follows:
+Start by creating a new subclass of :class:`DataSource` with the source name, 
schema.
+
+In order to be used as source or sink in batch or streaming query, 
corresponding method of DataSource needs to be implemented.
+
+Method that needs to be implemented for a capability:
+
+++--+--+
+||   source |  sink|
+++==+==+
+| batch  | reader() | writer() |
+++--+--+
+|| streamReader()   |  |
+| streaming  | or   | streamWriter()   |
+|| simpleStreamReader() |  |
+++--+--+
 
 .. code-block:: python
 
@@ -59,8 +73,19 @@ Start by creating a new subclass of :class:`DataSource`. 
Define the source name,
 def reader(self, schema: StructType):
 return FakeDataSourceReader(schema, self.options)
 
+def streamReader(self, schema: StructType):
+return FakeStreamReader(schema, self.options)
+
+# Please skip the implementation of this method if streamReader has 
been implemented.
+def simpleStreamReader(self, schema: StructType):
+return SimpleStreamReader()
 
-**Step 2: Implement the Reader**
+def streamWriter(self, schema: StructType, overwrite: bool):
+return FakeStreamWriter(self.options)
+
+Implementing Reader for Python Data Source
+--
+**Implement the Reader**
 
 Define the reader logic to generate synthetic data. Use the `faker` library to 
populate each field in the schema.
 
@@ -84,9 +109,157 @@ Define the reader logic to generate synthetic data. Use 
the `faker` library to p
 row.append(value)
 yield tuple(row)
 
+Implementing Streaming Reader and Writer for Python Data Source
+---
+**Implement the Stream Reader**
+
+This is a dummy streaming data reader that generate 2 rows in every 
microbatch. The streamReader instance has a integer offset that increase by 2 
in every microbatch.
+
+.. code-block:: python
+
+class RangePartition(InputPartition):
+def __init__(self, start, end):
+self.start = start
+self.end = end
+
+class FakeStreamReader(DataSourceStreamReader):
+def __init__(self, schema, options):
+self.current = 0
+
+def initialOffset(self) -> dict:
+"""
+Return the initial start offset of the reader.
+"""
+return {"offset": 0}
+
+def latestOffset(self) -> dict:
+"""
+Return the current latest offset that the next microbatch will 
read to.
+"""
+self.current += 2
+return {"offset": self.current}
+
+def partitions(self, start: dict, end: dict):
+""

(spark) branch master updated: [SPARK-48314][SS] Don't double cache files for FileStreamSource using Trigger.AvailableNow

2024-05-21 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new e702b32656bc [SPARK-48314][SS] Don't double cache files for 
FileStreamSource using Trigger.AvailableNow
e702b32656bc is described below

commit e702b32656bcbe194be19876990954a4be457734
Author: Adam Binford 
AuthorDate: Wed May 22 10:58:02 2024 +0900

[SPARK-48314][SS] Don't double cache files for FileStreamSource using 
Trigger.AvailableNow

### What changes were proposed in this pull request?

Files don't need to be cached for reuse in `FileStreamSource` when using 
`Trigger.AvailableNow` because all files are already cached for the lifetime of 
the query in `allFilesForTriggerAvailableNow`.

### Why are the changes needed?

As reported in https://issues.apache.org/jira/browse/SPARK-44924 (with a PR 
to address https://github.com/apache/spark/pull/45362), the hard coded cap of 
10k files being cached can cause problems when using a maxFilesPerTrigger > 
10k. It causes every other batch to be 10k files, which can greatly limit the 
throughput of a new streaming trying to catch up.

### Does this PR introduce _any_ user-facing change?

Every other streaming batch won't be 10k files if using 
Trigger.AvailableNow and maxFilesPerTrigger greater than 10k.

### How was this patch tested?

New UT

### Was this patch authored or co-authored using generative AI tooling?

No

Closes #46627 from Kimahriman/available-now-no-cache.

Authored-by: Adam Binford 
Signed-off-by: Jungtaek Lim 
---
 .../sql/execution/streaming/FileStreamSource.scala | 10 +++--
 .../sql/streaming/FileStreamSourceSuite.scala  | 45 ++
 2 files changed, 51 insertions(+), 4 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala
index 373a122e0001..4a9b2d11b7e0 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala
@@ -184,9 +184,11 @@ class FileStreamSource(
   }
 }
 
+val shouldCache = !sourceOptions.latestFirst && 
allFilesForTriggerAvailableNow == null
+
 // Obey user's setting to limit the number of files in this batch trigger.
 val (batchFiles, unselectedFiles) = limit match {
-  case files: ReadMaxFiles if !sourceOptions.latestFirst =>
+  case files: ReadMaxFiles if shouldCache =>
 // we can cache and reuse remaining fetched list of files in further 
batches
 val (bFiles, usFiles) = newFiles.splitAt(files.maxFiles())
 if (usFiles.size < files.maxFiles() * discardCachedInputRatio) {
@@ -200,10 +202,10 @@ class FileStreamSource(
 }
 
   case files: ReadMaxFiles =>
-// implies "sourceOptions.latestFirst = true" which we want to refresh 
the list per batch
+// don't use the cache, just take files for the next batch
 (newFiles.take(files.maxFiles()), null)
 
-  case files: ReadMaxBytes if !sourceOptions.latestFirst =>
+  case files: ReadMaxBytes if shouldCache =>
 // we can cache and reuse remaining fetched list of files in further 
batches
 val (FilesSplit(bFiles, _), FilesSplit(usFiles, rSize)) =
   takeFilesUntilMax(newFiles, files.maxBytes())
@@ -218,8 +220,8 @@ class FileStreamSource(
 }
 
   case files: ReadMaxBytes =>
+// don't use the cache, just take files for the next batch
 val (FilesSplit(bFiles, _), _) = takeFilesUntilMax(newFiles, 
files.maxBytes())
-// implies "sourceOptions.latestFirst = true" which we want to refresh 
the list per batch
 (bFiles, null)
 
   case _: ReadAllAvailable => (newFiles, null)
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala
index ff3cc5c247df..ca4f2a7f26ce 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala
@@ -2448,6 +2448,51 @@ class FileStreamSourceSuite extends FileStreamSourceTest 
{
 }
   }
 
+  test("SPARK-48314: Don't cache unread files when using 
Trigger.AvailableNow") {
+withCountListingLocalFileSystemAsLocalFileSystem {
+  withThreeTempDirs { case (src, meta, tmp) =>
+val options = Map("latestFirst" -> "false", "maxFilesPerTrigger" -> 
"5",
+  "maxCachedF

(spark) branch master updated (f5ffb74f170e -> 0393ab432389)

2024-05-20 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


from f5ffb74f170e [SPARK-48328][BUILD] Upgrade `Arrow` to 16.1.0
 add 0393ab432389 [SPARK-48330][SS][PYTHON] Fix the python streaming data 
source timeout issue for large trigger interval

No new revisions were added by this update.

Summary of changes:
 .../streaming/python_streaming_source_runner.py|   4 +-
 .../sql/worker/python_streaming_sink_runner.py |  67 --
 .../python/PythonStreamingSinkCommitRunner.scala   | 102 +
 .../v2/python/PythonStreamingWrite.scala   |  19 +++-
 .../python/PythonStreamingDataSourceSuite.scala|  76 +++
 5 files changed, 154 insertions(+), 114 deletions(-)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



(spark) branch master updated: [SPARK-44924][SS] Add config for FileStreamSource cached files

2024-05-19 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 74b42fd1ce7d [SPARK-44924][SS] Add config for FileStreamSource cached 
files
74b42fd1ce7d is described below

commit 74b42fd1ce7dd9353d08ea1b096b91b487953fd3
Author: ragnarok56 
AuthorDate: Mon May 20 10:51:02 2024 +0900

[SPARK-44924][SS] Add config for FileStreamSource cached files

### What changes were proposed in this pull request?
This change adds configuration options for the streaming input File Source 
for `maxCachedFiles` and `discardCachedInputRatio`.  These values were 
originally introduced with https://github.com/apache/spark/pull/27620 but were 
hardcoded to 10,000 and 0.2, respectively.

### Why are the changes needed?
Under certain workloads with large `maxFilesPerTrigger` settings, the 
performance gain from caching the input files capped at 10,000 can cause a 
cluster to be underutilized and jobs to take longer to finish if each batch 
takes a while to finish.  For example, a job with `maxFilesPerTrigger` set to 
100,000 would do all 100k in batch 1, then only 10k in batch 2, but both 
batches could take just as long since some of the files cause skewed processing 
times.  This results in a cluster spe [...]

### Does this PR introduce _any_ user-facing change?
Updated documentation for structured streaming sources to describe new 
configurations options

### How was this patch tested?
New and existing unit tests.

### Was this patch authored or co-authored using generative AI tooling?
No

Closes #45362 from ragnarok56/filestream-cached-files-config.

Authored-by: ragnarok56 
Signed-off-by: Jungtaek Lim 
---
 docs/structured-streaming-programming-guide.md |  4 +
 .../execution/streaming/FileStreamOptions.scala| 24 ++
 .../sql/execution/streaming/FileStreamSource.scala | 16 ++--
 .../sql/streaming/FileStreamSourceSuite.scala  | 93 +-
 4 files changed, 129 insertions(+), 8 deletions(-)

diff --git a/docs/structured-streaming-programming-guide.md 
b/docs/structured-streaming-programming-guide.md
index f3a8a0a40694..fabe7f17b78b 100644
--- a/docs/structured-streaming-programming-guide.md
+++ b/docs/structured-streaming-programming-guide.md
@@ -574,6 +574,10 @@ Here are the details of all the sources in Spark.
 
 maxFileAge: Maximum age of a file that can be found in 
this directory, before it is ignored. For the first batch all files will be 
considered valid. If latestFirst is set to `true` and 
maxFilesPerTrigger or maxBytesPerTrigger is set, then 
this parameter will be ignored, because old files that are valid, and should be 
processed, may be ignored. The max age is specified with respect to the 
timestamp of the latest file, and not the [...]
 
+maxCachedFiles: maximum number of files to cache to be 
processed in subsequent batches (default: 1).  If files are available in 
the cache, they will be read from first before listing from the input source.
+
+discardCachedInputRatio: ratio of cached files/bytes to 
max files/bytes to allow for listing from input source when there is less 
cached input than could be available to be read (default: 0.2).  For example, 
if there are only 10 cached files remaining for a batch but the 
maxFilesPerTrigger is set to 100, the 10 cached files would be 
discarded and a new listing would be performed instead. Similarly, if there are 
cached files that are 10 MB remaining for a [...]
+
 cleanSource: option to clean up completed files after 
processing.
 Available options are "archive", "delete", "off". If the option is not 
provided, the default value is "off".
 When "archive" is provided, additional option 
sourceArchiveDir must be provided as well. The value of 
"sourceArchiveDir" must not match with source pattern in depth (the number of 
directories from the root directory), where the depth is minimum of depth on 
both paths. This will ensure archived files are never included as new source 
files.
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamOptions.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamOptions.scala
index 07c1ccc432cd..b259f9dbcdcb 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamOptions.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamOptions.scala
@@ -125,6 +125,30 @@ class FileStreamOptions(parameters: 
CaseInsensitiveMap[String]) extends Logging
 matchedMode
   }
 
+  /**
+   * maximum number of files to cache to be processed in subsequent batches
+   */
+  val maxC

(spark) branch branch-3.4 updated: [SPARK-48105][SS][3.5] Fix the race condition between state store unloading and snapshotting

2024-05-17 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch branch-3.4
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.4 by this push:
 new 090022d475d6 [SPARK-48105][SS][3.5] Fix the race condition between 
state store unloading and snapshotting
090022d475d6 is described below

commit 090022d475d671ee345f22eb661f644b29ca28c5
Author: Huanli Wang 
AuthorDate: Wed May 15 14:52:04 2024 +0900

[SPARK-48105][SS][3.5] Fix the race condition between state store unloading 
and snapshotting

* When we close the hdfs state store, we should only remove the entry from 
`loadedMaps` rather than doing the active data cleanup. JVM GC should be able 
to help us GC those objects.
* we should wait for the maintenance thread to stop before unloading the 
providers.

There are two race conditions between state store snapshotting and state 
store unloading which could result in query failure and potential data 
corruption.

Case 1:
1. the maintenance thread pool encounters some issues and call the 
[stopMaintenanceTask,](https://github.com/apache/spark/blob/d9d79a54a3cd487380039c88ebe9fa708e0dcf23/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala#L774)
 this function further calls 
[threadPool.stop.](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala#L587)
 However, this function doesn't wait for th [...]
2. the provider unload will [close the state 
store](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala#L719-L721)
 which [clear the values of 
loadedMaps](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala#L353-L355)
 for HDFS backed state store.
3. if the not-yet-stop maintenance thread is still running and trying to do 
the snapshot, but the data in the underlying `HDFSBackedStateStoreMap` has been 
removed. if this snapshot process completes successfully, then we will write 
corrupted data and the following batches will consume this corrupted data.

Case 2:

1. In executor_1, the maintenance thread is going to do the snapshot for 
state_store_1, it retrieves the `HDFSBackedStateStoreMap` object from the 
loadedMaps, after this, the maintenance thread [releases the lock of the 
loadedMaps](https://github.com/apache/spark/blob/c6696cdcd611a682ebf5b7a183e2970ecea3b58c/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala#L750-L751).
2. state_store_1 is loaded in another executor, e.g. executor_2.
3. another state store, state_store_2, is loaded on executor_1 and 
[reportActiveStoreInstance](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala#L854-L871)
 to driver.
4. executor_1 does the 
[unload](https://github.com/apache/spark/blob/c6696cdcd611a682ebf5b7a183e2970ecea3b58c/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala#L713)
 for those no longer active state store which clears the data entries in the 
`HDFSBackedStateStoreMap`
5. the snapshotting thread is terminated and uploads the incomplete 
snapshot to cloud because the [iterator doesn't have next 
element](https://github.com/apache/spark/blob/c6696cdcd611a682ebf5b7a183e2970ecea3b58c/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala#L634)
 after doing the clear.
6. future batches are consuming the corrupted data.

No

```
[info] Run completed in 2 minutes, 55 seconds.
[info] Total number of tests run: 153
[info] Suites: completed 1, aborted 0
[info] Tests: succeeded 153, failed 0, canceled 0, ignored 0, pending 0
[info] All tests passed.
[success] Total time: 271 s (04:31), completed May 2, 2024, 6:26:33 PM
```
before this change

```
[info] - state store unload/close happens during the maintenance *** FAILED 
*** (648 milliseconds)
[info]   Vector("a1", "a10", "a11", "a12", "a13", "a14", "a15", "a16", 
"a17", "a18", "a19", "a2", "a20", "a3", "a4", "a5", "a6", "a7", "a8", "a9") did 
not equal ArrayBuffer("a8") (StateStoreSuite.scala:414)
[info]   Analysis:
[info]   Vector1(0: "a1" -> "a8", 1: "a10" -> , 2: "a11" -> , 3: "a12" -> , 
4: "a13" -> , 5: "a14" -> , 6: "a15" -> , 7: 

(spark) branch master updated (97717363abae -> 0ba8ddc9ce5b)

2024-05-16 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


from 97717363abae [SPARK-48264][BUILD] Upgrade `datasketches-java` to 6.0.0
 add 0ba8ddc9ce5b [SPARK-48293][SS] Add test for when 
ForeachBatchUserFuncException wraps interrupted exception due to query stop

No new revisions were added by this update.

Summary of changes:
 .../streaming/sources/ForeachBatchSink.scala   |  2 +-
 .../apache/spark/sql/streaming/StreamSuite.scala   | 25 --
 2 files changed, 15 insertions(+), 12 deletions(-)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



(spark) branch branch-3.5 updated (74724d61c3d0 -> 07e08c00b32f)

2024-05-14 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a change to branch branch-3.5
in repository https://gitbox.apache.org/repos/asf/spark.git


from 74724d61c3d0 Revert "[SPARK-48172][SQL] Fix escaping issues in JDBC 
Dialects"
 add 07e08c00b32f [SPARK-48105][SS][3.5] Fix the race condition between 
state store unloading and snapshotting

No new revisions were added by this update.

Summary of changes:
 .../streaming/state/HDFSBackedStateStoreMap.scala  |  8 -
 .../state/HDFSBackedStateStoreProvider.scala   |  5 ++-
 .../streaming/state/StateStoreSuite.scala  | 38 ++
 3 files changed, 42 insertions(+), 9 deletions(-)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



(spark) branch master updated: [SPARK-48233][SS][TESTS] Tests for streaming on columns with non-default collations

2024-05-14 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 7ec37e47d13e [SPARK-48233][SS][TESTS] Tests for streaming on columns 
with non-default collations
7ec37e47d13e is described below

commit 7ec37e47d13e7f0ad51e295f99853e9bdee5b7a9
Author: Aleksandar Tomic 
AuthorDate: Wed May 15 14:42:44 2024 +0900

[SPARK-48233][SS][TESTS] Tests for streaming on columns with non-default 
collations

### What changes were proposed in this pull request?

This change covers tests for streaming operations under columns of string 
type that are collated with non-utf8-binary collations. PR introduces following 
tests:
1) Non-stateful streaming for non-binary collated columns. We use 
`UTF8_BINARY_LCASE` non-binary collation as the input and assert that streaming 
propagates collation and that filtering behaves under rules of given collation.
2) Stateful streaming for binary collations. We use `UNICODE` collation as 
source and make sure that stateful operations (deduplication as taken as the 
example) work.
3) More tests that assert that stateful operations in combination with 
non-binary collations throw proper exception.

### Why are the changes needed?

You can find more information about collation effort in document attached 
to root jira ticket.

This PR adds tests for basic non-stateful streaming operations with 
collations (e.g. filtering).

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

PR is test only.

### Was this patch authored or co-authored using generative AI tooling?

No

Closes #46247 from dbatomic/streaming_and_collations.

Authored-by: Aleksandar Tomic 
Signed-off-by: Jungtaek Lim 
---
 .../streaming/StreamingDeduplicationSuite.scala| 48 ++
 .../spark/sql/streaming/StreamingQuerySuite.scala  | 29 +
 2 files changed, 77 insertions(+)

diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala
index 5c3d8d877f39..5c816c5cddc7 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala
@@ -21,11 +21,13 @@ import java.io.File
 
 import org.apache.commons.io.FileUtils
 
+import org.apache.spark.SparkUnsupportedOperationException
 import org.apache.spark.sql.DataFrame
 import org.apache.spark.sql.catalyst.streaming.InternalOutputModes._
 import org.apache.spark.sql.execution.streaming.MemoryStream
 import org.apache.spark.sql.functions._
 import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StringType
 import org.apache.spark.tags.SlowSQLTest
 import org.apache.spark.util.Utils
 
@@ -484,6 +486,52 @@ class StreamingDeduplicationSuite extends 
StateStoreMetricsTest {
   CheckLastBatch(("c", 9, "c"))
 )
   }
+
+  test("collation aware deduplication") {
+val inputData = MemoryStream[(String, Int)]
+val result = inputData.toDF()
+  .select(col("_1")
+.try_cast(StringType("UNICODE")).as("str"),
+col("_2").as("int"))
+  .dropDuplicates("str")
+
+testStream(result, Append)(
+  AddData(inputData, "a" -> 1),
+  CheckLastBatch("a" -> 1),
+  assertNumStateRows(total = 1, updated = 1, droppedByWatermark = 0),
+  AddData(inputData, "a" -> 2), // Dropped
+  CheckLastBatch(),
+  assertNumStateRows(total = 1, updated = 0, droppedByWatermark = 0),
+  // scalastyle:off
+  AddData(inputData, "ä" -> 1),
+  CheckLastBatch("ä" -> 1),
+  // scalastyle:on
+  assertNumStateRows(total = 2, updated = 1, droppedByWatermark = 0)
+)
+  }
+
+  test("non-binary collation aware deduplication not supported") {
+val inputData = MemoryStream[(String)]
+val result = inputData.toDF()
+  .select(col("value")
+.try_cast(StringType("UTF8_BINARY_LCASE")).as("str"))
+  .dropDuplicates("str")
+
+val ex = intercept[StreamingQueryException] {
+  testStream(result, Append)(
+AddData(inputData, "a"),
+CheckLastBatch("a"))
+}
+
+checkError(
+  ex.getCause.asInstanceOf[SparkUnsupportedOperationException],
+  errorClass = "STATE_STORE_UNSUPPORTED_OPERATION_BINARY_INEQUALITY",
+  parameters = Map(
+"schema" -> ".+\"type\":\"string collate UTF8_B

(spark) branch branch-3.5 updated: [SPARK-48267][SS] Regression e2e test with SPARK-47305

2024-05-14 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch branch-3.5
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.5 by this push:
 new 172a23f780ae [SPARK-48267][SS] Regression e2e test with SPARK-47305
172a23f780ae is described below

commit 172a23f780ae2a603908421b49683aff6748e419
Author: Jungtaek Lim 
AuthorDate: Tue May 14 15:40:51 2024 +0900

[SPARK-48267][SS] Regression e2e test with SPARK-47305

### What changes were proposed in this pull request?

This PR proposes to add a regression test (e2e) with SPARK-47305.

As of commit cae2248bc13 (pre-Spark 4.0), the query in new unit test is 
represented as below logical plans:

> Batch 0

>> analyzed plan

```
WriteToMicroBatchDataSource MemorySink, 
5067923b-e1d0-484c-914c-b111c9e60aac, Append, 0
+- Project [value#1]
   +- Join Inner, (cast(code#5 as bigint) = ref_code#14L)
  :- Union false, false
  :  :- Project [value#1, 1 AS code#5]
  :  :  +- StreamingDataSourceV2ScanRelation[value#1] 
MemoryStreamDataSource
  :  +- Project [value#3, cast(code#9 as int) AS code#16]
  : +- Project [value#3, null AS code#9]
  :+- LocalRelation , [value#3]
  +- Project [id#12L AS ref_code#14L]
 +- Range (1, 5, step=1, splits=Some(2))
```

>> optimized plan

```
WriteToDataSourceV2 MicroBatchWrite[epoch: 0, writer: ...]
+- Join Inner
   :- StreamingDataSourceV2ScanRelation[value#1] MemoryStreamDataSource
   +- Project
  +- Filter (1 = id#12L)
 +- Range (1, 5, step=1, splits=Some(2))
```

> Batch 1

>> analyzed plan

```
WriteToMicroBatchDataSource MemorySink, 
d1c8be66-88e7-437a-9f25-6b87db8efe17, Append, 1
+- Project [value#1]
   +- Join Inner, (cast(code#5 as bigint) = ref_code#14L)
  :- Union false, false
  :  :- Project [value#1, 1 AS code#5]
  :  :  +- LocalRelation , [value#1]
  :  +- Project [value#3, cast(code#9 as int) AS code#16]
  : +- Project [value#3, null AS code#9]
  :+- StreamingDataSourceV2ScanRelation[value#3] 
MemoryStreamDataSource
  +- Project [id#12L AS ref_code#14L]
 +- Range (1, 5, step=1, splits=Some(2))
```

>> optimized plan

```
WriteToDataSourceV2 MicroBatchWrite[epoch: 1, writer: ...]
+- Join Inner
   :- StreamingDataSourceV2ScanRelation[value#3] MemoryStreamDataSource
   +- LocalRelation 
```

Notice the difference in optimized plan between batch 0 and batch 1. In 
optimized plan for batch 1, the batch side is pruned out, which goes with the 
path of PruneFilters. The sequence of optimization is,

1) left stream side is collapsed with empty local relation
2) union is replaced with subtree for right stream side as left stream side 
is simply an empty local relation
3) the value of 'code' column is now known to be 'null' and it's propagated 
to the join criteria (`null = ref_code`)
4) join criteria is extracted out from join, and being pushed to the batch 
side
5) the value of 'ref_code' column can never be null, hence the filter is 
optimized as `filter false`
6) `filter false` triggers PruneFilters (where we fix a bug in SPARK-47305)

Before SPARK-47305, a new empty local relation was incorrectly marked as 
streaming.

NOTE: I intentionally didn't put the detail like above as code comment, as 
optimization result is subject to change for Spark versions.

### Why are the changes needed?

In the PR of SPARK-47305 we only added an unit test to verify the fix, but 
it wasn't e2e about the workload we encountered an issue. Given the complexity 
of QO, it'd be ideal to put an e2e reproducer (despite simplified) as 
regression test.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

New UT.

### Was this patch authored or co-authored using generative AI tooling?

No.

Closes #46569 from HeartSaVioR/SPARK-48267.

Authored-by: Jungtaek Lim 
Signed-off-by: Jungtaek Lim 
---
 ...treamingQueryOptimizationCorrectnessSuite.scala | 37 +-
 1 file changed, 36 insertions(+), 1 deletion(-)

diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryOptimizationCorrectnessSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryOptimizationCorrectnessSuite.scala
index efc84c8e4c7c..d17da5d31edd 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryOptimizationCorrectnessSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryOptimizationCorrect

(spark) branch master updated: [SPARK-48267][SS] Regression e2e test with SPARK-47305

2024-05-14 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new c0982621f46b [SPARK-48267][SS] Regression e2e test with SPARK-47305
c0982621f46b is described below

commit c0982621f46b696c7c4c6a805ae0c7d101570929
Author: Jungtaek Lim 
AuthorDate: Tue May 14 15:40:51 2024 +0900

[SPARK-48267][SS] Regression e2e test with SPARK-47305

### What changes were proposed in this pull request?

This PR proposes to add a regression test (e2e) with SPARK-47305.

As of commit cae2248bc13 (pre-Spark 4.0), the query in new unit test is 
represented as below logical plans:

> Batch 0

>> analyzed plan

```
WriteToMicroBatchDataSource MemorySink, 
5067923b-e1d0-484c-914c-b111c9e60aac, Append, 0
+- Project [value#1]
   +- Join Inner, (cast(code#5 as bigint) = ref_code#14L)
  :- Union false, false
  :  :- Project [value#1, 1 AS code#5]
  :  :  +- StreamingDataSourceV2ScanRelation[value#1] 
MemoryStreamDataSource
  :  +- Project [value#3, cast(code#9 as int) AS code#16]
  : +- Project [value#3, null AS code#9]
  :+- LocalRelation , [value#3]
  +- Project [id#12L AS ref_code#14L]
 +- Range (1, 5, step=1, splits=Some(2))
```

>> optimized plan

```
WriteToDataSourceV2 MicroBatchWrite[epoch: 0, writer: ...]
+- Join Inner
   :- StreamingDataSourceV2ScanRelation[value#1] MemoryStreamDataSource
   +- Project
  +- Filter (1 = id#12L)
 +- Range (1, 5, step=1, splits=Some(2))
```

> Batch 1

>> analyzed plan

```
WriteToMicroBatchDataSource MemorySink, 
d1c8be66-88e7-437a-9f25-6b87db8efe17, Append, 1
+- Project [value#1]
   +- Join Inner, (cast(code#5 as bigint) = ref_code#14L)
  :- Union false, false
  :  :- Project [value#1, 1 AS code#5]
  :  :  +- LocalRelation , [value#1]
  :  +- Project [value#3, cast(code#9 as int) AS code#16]
  : +- Project [value#3, null AS code#9]
  :+- StreamingDataSourceV2ScanRelation[value#3] 
MemoryStreamDataSource
  +- Project [id#12L AS ref_code#14L]
 +- Range (1, 5, step=1, splits=Some(2))
```

>> optimized plan

```
WriteToDataSourceV2 MicroBatchWrite[epoch: 1, writer: ...]
+- Join Inner
   :- StreamingDataSourceV2ScanRelation[value#3] MemoryStreamDataSource
   +- LocalRelation 
```

Notice the difference in optimized plan between batch 0 and batch 1. In 
optimized plan for batch 1, the batch side is pruned out, which goes with the 
path of PruneFilters. The sequence of optimization is,

1) left stream side is collapsed with empty local relation
2) union is replaced with subtree for right stream side as left stream side 
is simply an empty local relation
3) the value of 'code' column is now known to be 'null' and it's propagated 
to the join criteria (`null = ref_code`)
4) join criteria is extracted out from join, and being pushed to the batch 
side
5) the value of 'ref_code' column can never be null, hence the filter is 
optimized as `filter false`
6) `filter false` triggers PruneFilters (where we fix a bug in SPARK-47305)

Before SPARK-47305, a new empty local relation was incorrectly marked as 
streaming.

NOTE: I intentionally didn't put the detail like above as code comment, as 
optimization result is subject to change for Spark versions.

### Why are the changes needed?

In the PR of SPARK-47305 we only added an unit test to verify the fix, but 
it wasn't e2e about the workload we encountered an issue. Given the complexity 
of QO, it'd be ideal to put an e2e reproducer (despite simplified) as 
regression test.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

New UT.

### Was this patch authored or co-authored using generative AI tooling?

No.

Closes #46569 from HeartSaVioR/SPARK-48267.

Authored-by: Jungtaek Lim 
Signed-off-by: Jungtaek Lim 
---
 ...treamingQueryOptimizationCorrectnessSuite.scala | 37 +-
 1 file changed, 36 insertions(+), 1 deletion(-)

diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryOptimizationCorrectnessSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryOptimizationCorrectnessSuite.scala
index efc84c8e4c7c..d17da5d31edd 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryOptimizationCorrectnessSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryOptimizationCorrectnessSuite.

(spark) branch master updated: [SPARK-48208][SS] Skip providing memory usage metrics from RocksDB if bounded memory usage is enabled

2024-05-09 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 045ec6a166c8 [SPARK-48208][SS] Skip providing memory usage metrics 
from RocksDB if bounded memory usage is enabled
045ec6a166c8 is described below

commit 045ec6a166c8d2bdf73585fc4160c136e5f2888a
Author: Anish Shrigondekar 
AuthorDate: Thu May 9 17:10:01 2024 +0900

[SPARK-48208][SS] Skip providing memory usage metrics from RocksDB if 
bounded memory usage is enabled

### What changes were proposed in this pull request?
Skip providing memory usage metrics from RocksDB if bounded memory usage is 
enabled

### Why are the changes needed?
Without this, we are providing memory usage that is the max usage per node 
at a partition level.
For eg - if we report this
```
"allRemovalsTimeMs" : 93,
"commitTimeMs" : 32240,
"memoryUsedBytes" : 15956211724278,
"numRowsDroppedByWatermark" : 0,
"numShufflePartitions" : 200,
"numStateStoreInstances" : 200,
```

We have 200 partitions in this case.
So the memory usage per partition / state store would be ~78GB. However, 
this node has 256GB memory total and we have 2 such nodes. We have configured 
our cluster to use 30% of available memory on each node for RocksDB which is 
~77GB.
So the memory being reported here is actually per node rather than per 
partition which could be confusing for users.

### Does this PR introduce _any_ user-facing change?
No - only a metrics reporting change

### How was this patch tested?
Added unit tests

```
[info] Run completed in 10 seconds, 878 milliseconds.
[info] Total number of tests run: 24
[info] Suites: completed 1, aborted 0
[info] Tests: succeeded 24, failed 0, canceled 0, ignored 0, pending 0
[info] All tests passed.
```

### Was this patch authored or co-authored using generative AI tooling?
No

Closes #46491 from anishshri-db/task/SPARK-48208.

Authored-by: Anish Shrigondekar 
Signed-off-by: Jungtaek Lim 
---
 .../apache/spark/sql/execution/streaming/state/RocksDB.scala  | 11 ++-
 .../spark/sql/execution/streaming/state/RocksDBSuite.scala| 11 +++
 2 files changed, 21 insertions(+), 1 deletion(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala
index caecf817c12f..151695192281 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala
@@ -777,10 +777,19 @@ class RocksDB(
   .keys.filter(checkInternalColumnFamilies(_)).size
 val numExternalColFamilies = colFamilyNameToHandleMap.keys.size - 
numInternalColFamilies
 
+// if bounded memory usage is enabled, we share the block cache across all 
state providers
+// running on the same node and account the usage to this single cache. In 
this case, its not
+// possible to provide partition level or query level memory usage.
+val memoryUsage = if (conf.boundedMemoryUsage) {
+  0L
+} else {
+  readerMemUsage + memTableMemUsage + blockCacheUsage
+}
+
 RocksDBMetrics(
   numKeysOnLoadedVersion,
   numKeysOnWritingVersion,
-  readerMemUsage + memTableMemUsage + blockCacheUsage,
+  memoryUsage,
   pinnedBlocksMemUsage,
   totalSSTFilesBytes,
   nativeOpsLatencyMicros,
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala
index ab2afa1b8a61..6086fd43846f 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala
@@ -1699,6 +1699,11 @@ class RocksDBSuite extends 
AlsoTestWithChangelogCheckpointingEnabled with Shared
   db.load(0)
   db.put("a", "1")
   db.commit()
+  if (boundedMemoryUsage == "true") {
+assert(db.metricsOpt.get.totalMemUsageBytes === 0)
+  } else {
+assert(db.metricsOpt.get.totalMemUsageBytes > 0)
+  }
   db.getWriteBufferManagerAndCache()
 }
 
@@ -1709,6 +1714,11 @@ class RocksDBSuite extends 
AlsoTestWithChangelogCheckpointingEnabled with Shared
   db.load(0)
   db.put("a", "1")
   db.commit()
+  if (boundedM

(spark) branch master updated: [SPARK-47960][SS] Allow chaining other stateful operators after transformWithState operator

2024-05-07 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 5e49665ac39b [SPARK-47960][SS] Allow chaining other stateful operators 
after transformWithState operator
5e49665ac39b is described below

commit 5e49665ac39b49b875d6970f93df59aedd830fa5
Author: Bhuwan Sahni 
AuthorDate: Wed May 8 09:20:01 2024 +0900

[SPARK-47960][SS] Allow chaining other stateful operators after 
transformWithState operator

### What changes were proposed in this pull request?

This PR adds support to define event time column in the output dataset of 
`TransformWithState` operator. The new event time column will be used to 
evaluate watermark expressions in downstream operators.

1. Note that the transformWithState operator does not enforce that values 
generated by user's computation adhere to the watermark semantics. (no output 
rows are generated which have event time less than watermark).
2. Updated the watermark value passed in TimerInfo as evictionWatermark, 
rather than lateEventsWatermark.
3. Ensure that event time column can only be defined in output if a 
watermark has been defined previously.

### Why are the changes needed?

This change is required to support chaining of stateful operators after 
`transformWithState`. Event time column is required to evaluate watermark 
expressions in downstream stateful operators.

### Does this PR introduce _any_ user-facing change?

Yes. Adds a new version of transformWithState API which allows redefining 
the event time column.

### How was this patch tested?

Added unit test cases.

### Was this patch authored or co-authored using generative AI tooling?

No

Closes #45376 from sahnib/tws-chaining-stateful-operators.

Authored-by: Bhuwan Sahni 
Signed-off-by: Jungtaek Lim 
---
 .../src/main/resources/error/error-conditions.json |  14 +
 .../spark/sql/catalyst/analysis/Analyzer.scala |   3 +
 .../ResolveUpdateEventTimeWatermarkColumn.scala|  52 +++
 .../plans/logical/EventTimeWatermark.scala |  79 +++-
 .../spark/sql/catalyst/plans/logical/object.scala  |   2 +-
 .../sql/catalyst/rules/RuleIdCollection.scala  |   1 +
 .../spark/sql/catalyst/trees/TreePatterns.scala|   1 +
 .../spark/sql/errors/QueryCompilationErrors.scala  |   7 +
 .../spark/sql/errors/QueryExecutionErrors.scala|  11 +
 .../apache/spark/sql/KeyValueGroupedDataset.scala  | 178 -
 .../spark/sql/execution/SparkStrategies.scala  |  12 +
 .../streaming/EventTimeWatermarkExec.scala |  88 -
 .../execution/streaming/IncrementalExecution.scala |  24 +-
 .../streaming/TransformWithStateExec.scala |  32 +-
 .../execution/streaming/statefulOperators.scala|   2 +-
 .../TransformWithStateChainingSuite.scala  | 411 +
 16 files changed, 866 insertions(+), 51 deletions(-)

diff --git a/common/utils/src/main/resources/error/error-conditions.json 
b/common/utils/src/main/resources/error/error-conditions.json
index bae94a0ab97e..8a64c4c590e8 100644
--- a/common/utils/src/main/resources/error/error-conditions.json
+++ b/common/utils/src/main/resources/error/error-conditions.json
@@ -125,6 +125,12 @@
 ],
 "sqlState" : "428FR"
   },
+  "CANNOT_ASSIGN_EVENT_TIME_COLUMN_WITHOUT_WATERMARK" : {
+"message" : [
+  "Watermark needs to be defined to reassign event time column. Failed to 
find watermark definition in the streaming query."
+],
+"sqlState" : "42611"
+  },
   "CANNOT_CAST_DATATYPE" : {
 "message" : [
   "Cannot cast  to ."
@@ -1057,6 +1063,14 @@
 },
 "sqlState" : "4274K"
   },
+  "EMITTING_ROWS_OLDER_THAN_WATERMARK_NOT_ALLOWED" : {
+"message" : [
+  "Previous node emitted a row with eventTime= which 
is older than current_watermark_value=",
+  "This can lead to correctness issues in the stateful operators 
downstream in the execution pipeline.",
+  "Please correct the operator logic to emit rows after current global 
watermark value."
+],
+"sqlState" : "42815"
+  },
   "EMPTY_JSON_FIELD_VALUE" : {
 "message" : [
   "Failed to parse an empty string for data type ."
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index c29432c916f9..55b6f1af7fd8 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/cata

(spark) branch master updated: [SPARK-48105][SS] Fix the race condition between state store unloading and snapshotting

2024-05-06 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 9dc5599b01b1 [SPARK-48105][SS] Fix the race condition between state 
store unloading and snapshotting
9dc5599b01b1 is described below

commit 9dc5599b01b197b6f703d93486ff960a67e4e25c
Author: Huanli Wang 
AuthorDate: Tue May 7 09:39:13 2024 +0900

[SPARK-48105][SS] Fix the race condition between state store unloading and 
snapshotting

### What changes were proposed in this pull request?

* When we close the hdfs state store, we should only remove the entry from 
`loadedMaps` rather than doing the active data cleanup. JVM GC should be able 
to help us GC those objects.
* we should wait for the maintenance thread to stop before unloading the 
providers.

### Why are the changes needed?

There are two race conditions between state store snapshotting and state 
store unloading which could result in query failure and potential data 
corruption.

Case 1:
1. the maintenance thread pool encounters some issues and call the 
[stopMaintenanceTask,](https://github.com/apache/spark/blob/d9d79a54a3cd487380039c88ebe9fa708e0dcf23/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala#L774)
 this function further calls 
[threadPool.stop.](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala#L587)
 However, this function doesn't wait for th [...]
2. the provider unload will [close the state 
store](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala#L719-L721)
 which [clear the values of 
loadedMaps](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala#L353-L355)
 for HDFS backed state store.
3. if the not-yet-stop maintenance thread is still running and trying to do 
the snapshot, but the data in the underlying `HDFSBackedStateStoreMap` has been 
removed. if this snapshot process completes successfully, then we will write 
corrupted data and the following batches will consume this corrupted data.

Case 2:

1. In executor_1, the maintenance thread is going to do the snapshot for 
state_store_1, it retrieves the `HDFSBackedStateStoreMap` object from the 
loadedMaps, after this, the maintenance thread [releases the lock of the 
loadedMaps](https://github.com/apache/spark/blob/c6696cdcd611a682ebf5b7a183e2970ecea3b58c/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala#L750-L751).
2. state_store_1 is loaded in another executor, e.g. executor_2.
3. another state store, state_store_2, is loaded on executor_1 and 
[reportActiveStoreInstance](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala#L854-L871)
 to driver.
4. executor_1 does the 
[unload](https://github.com/apache/spark/blob/c6696cdcd611a682ebf5b7a183e2970ecea3b58c/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala#L713)
 for those no longer active state store which clears the data entries in the 
`HDFSBackedStateStoreMap`
5. the snapshotting thread is terminated and uploads the incomplete 
snapshot to cloud because the [iterator doesn't have next 
element](https://github.com/apache/spark/blob/c6696cdcd611a682ebf5b7a183e2970ecea3b58c/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala#L634)
 after doing the clear.
6. future batches are consuming the corrupted data.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?
```
[info] Run completed in 2 minutes, 55 seconds.
[info] Total number of tests run: 153
[info] Suites: completed 1, aborted 0
[info] Tests: succeeded 153, failed 0, canceled 0, ignored 0, pending 0
[info] All tests passed.
[success] Total time: 271 s (04:31), completed May 2, 2024, 6:26:33 PM
```
before this change

```
[info] - state store unload/close happens during the maintenance *** FAILED 
*** (648 milliseconds)
[info]   Vector("a1", "a10", "a11", "a12", "a13", "a14", "a15", "a16", 
"a17", "a18", "a19", "a2", "a20", "a3", "a4", "a5", "a6", "a7", "a8", "a9") did 
not equal ArrayBuffer("a8") (StateStoreSuite.scala:414)
[info]   Analysis:
[info]   Vector1(0: "a1" -> "

(spark) branch master updated: [SPARK-48102][SS] Track duration for acquiring source/sink metrics while reporting streaming query progress

2024-05-02 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new d9d79a54a3cd [SPARK-48102][SS] Track duration for acquiring 
source/sink metrics while reporting streaming query progress
d9d79a54a3cd is described below

commit d9d79a54a3cd487380039c88ebe9fa708e0dcf23
Author: Anish Shrigondekar 
AuthorDate: Fri May 3 11:30:58 2024 +0900

[SPARK-48102][SS] Track duration for acquiring source/sink metrics while 
reporting streaming query progress

### What changes were proposed in this pull request?
Track duration for acquiring source/sink metrics while reporting streaming 
query progress

### Why are the changes needed?
Change needed to help us understand how long the source/sink progress 
metrics calculation is taking. Also need to understand distribution if multiple 
sources are used

Sample log:
```
17:26:14.769 INFO 
org.apache.spark.sql.execution.streaming.MicroBatchExecutionContext: Extracting 
source progress metrics for source=MemoryStream[value#636] took duration_ms=0
17:26:14.769 INFO 
org.apache.spark.sql.execution.streaming.MicroBatchExecutionContext: Extracting 
sink progress metrics for sink=MemorySink took duration_ms=0
```

Existing test:
```
[info] Run completed in 9 seconds, 995 milliseconds.
[info] Total number of tests run: 11
[info] Suites: completed 1, aborted 0
[info] Tests: succeeded 11, failed 0, canceled 0, ignored 1, pending 0
[info] All tests passed.
```

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Existing unit tests

### Was this patch authored or co-authored using generative AI tooling?
No

Closes #46350 from anishshri-db/task/SPARK-48102.

Authored-by: Anish Shrigondekar 
Signed-off-by: Jungtaek Lim 
---
 .../sql/execution/streaming/ProgressReporter.scala | 54 +-
 1 file changed, 32 insertions(+), 22 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala
index 6ef6f0eb7118..3842ed574355 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala
@@ -37,7 +37,7 @@ import org.apache.spark.sql.execution.QueryExecution
 import org.apache.spark.sql.execution.datasources.v2.{MicroBatchScanExec, 
StreamingDataSourceV2ScanRelation, StreamWriterCommitProgress}
 import org.apache.spark.sql.streaming._
 import org.apache.spark.sql.streaming.StreamingQueryListener.{QueryIdleEvent, 
QueryProgressEvent}
-import org.apache.spark.util.Clock
+import org.apache.spark.util.{Clock, Utils}
 
 /**
  * Responsible for continually reporting statistics about the amount of data 
processed as well
@@ -334,33 +334,43 @@ abstract class ProgressContext(
   inputTimeSec: Double,
   processingTimeSec: Double): Seq[SourceProgress] = {
 sources.distinct.map { source =>
-  val numRecords = execStats.flatMap(_.inputRows.get(source)).getOrElse(0L)
-  val sourceMetrics = source match {
-case withMetrics: ReportsSourceMetrics =>
-  
withMetrics.metrics(Optional.ofNullable(latestStreamProgress.get(source).orNull))
-case _ => Map[String, String]().asJava
+  val (result, duration) = Utils.timeTakenMs {
+val numRecords = 
execStats.flatMap(_.inputRows.get(source)).getOrElse(0L)
+val sourceMetrics = source match {
+  case withMetrics: ReportsSourceMetrics =>
+
withMetrics.metrics(Optional.ofNullable(latestStreamProgress.get(source).orNull))
+  case _ => Map[String, String]().asJava
+}
+new SourceProgress(
+  description = source.toString,
+  startOffset = currentTriggerStartOffsets.get(source).orNull,
+  endOffset = currentTriggerEndOffsets.get(source).orNull,
+  latestOffset = currentTriggerLatestOffsets.get(source).orNull,
+  numInputRows = numRecords,
+  inputRowsPerSecond = numRecords / inputTimeSec,
+  processedRowsPerSecond = numRecords / processingTimeSec,
+  metrics = sourceMetrics
+)
   }
-  new SourceProgress(
-description = source.toString,
-startOffset = currentTriggerStartOffsets.get(source).orNull,
-endOffset = currentTriggerEndOffsets.get(source).orNull,
-latestOffset = currentTriggerLatestOffsets.get(source).orNull,
-numInputRows = numRecords,
-inputRowsPerSecond = numRecords / inputTimeSec,
-processedRowsPerSecond = numRecords / processingTimeSec,
-metrics 

(spark) branch master updated: [SPARK-47793][SS][PYTHON] Implement SimpleDataSourceStreamReader for python streaming data source

2024-04-30 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new c8c249204178 [SPARK-47793][SS][PYTHON] Implement 
SimpleDataSourceStreamReader for python streaming data source
c8c249204178 is described below

commit c8c2492041782b9be7f10647191dcd0d5f6a5a8a
Author: Chaoqin Li 
AuthorDate: Tue Apr 30 22:08:32 2024 +0900

[SPARK-47793][SS][PYTHON] Implement SimpleDataSourceStreamReader for python 
streaming data source

### What changes were proposed in this pull request?
SimpleDataSourceStreamReader is a simplified version of the 
DataSourceStreamReader interface.

There are 3 functions that needs to be defined

1. Read data and return the end offset.
_def read(self, start: Offset) -> (Iterator[Tuple], Offset)_

2. Read data between start and end offset, this is required for exactly 
once read.
_def readBetweenOffset(self, start: Offset, end: Offset) -> Iterator[Tuple]_

3. initial start offset of the streaming query.
_def initialOffset() -> dict_

The implementation wrap the SimpleDataSourceStreamReader instance in a 
DataSourceStreamReader that prefetch and cache data in latestOffset. The record 
prefetched in python process will be sent to JVM as arrow record batches in 
planInputPartitions() and cached by block manager and read by partition reader 
from executor later..

### Why are the changes needed?
Compared to DataSourceStreamReader interface, the simplified interface has 
some advantages.
It doesn’t require developers to reason about data partitioning.
It doesn’t require getting the latest offset before reading data.

### Does this PR introduce _any_ user-facing change?
No.

### How was this patch tested?
Add unit test and integration test.

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #45977 from chaoqin-li1123/simple_reader_impl.

Lead-authored-by: Chaoqin Li 
Co-authored-by: chaoqin-li1123 
<55518381+chaoqin-li1...@users.noreply.github.com>
Signed-off-by: Jungtaek Lim 
---
 .../scala/org/apache/spark/storage/BlockId.scala   |   8 +
 python/pyspark/sql/datasource.py   | 129 -
 python/pyspark/sql/datasource_internal.py  | 146 ++
 .../streaming/python_streaming_source_runner.py|  58 +++-
 python/pyspark/sql/worker/plan_data_source_read.py | 142 +-
 .../v2/python/PythonMicroBatchStream.scala |  61 +++-
 .../datasources/v2/python/PythonScan.scala |   3 +
 .../PythonStreamingPartitionReaderFactory.scala|  89 ++
 .../python/PythonStreamingSourceRunner.scala   |  57 +++-
 .../python/PythonStreamingDataSourceSuite.scala| 307 -
 10 files changed, 911 insertions(+), 89 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/storage/BlockId.scala 
b/core/src/main/scala/org/apache/spark/storage/BlockId.scala
index 585d9a886b47..6eb015d56b2c 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala
@@ -170,6 +170,11 @@ case class StreamBlockId(streamId: Int, uniqueId: Long) 
extends BlockId {
   override def name: String = "input-" + streamId + "-" + uniqueId
 }
 
+@DeveloperApi
+case class PythonStreamBlockId(streamId: Int, uniqueId: Long) extends BlockId {
+  override def name: String = "python-stream-" + streamId + "-" + uniqueId
+}
+
 /** Id associated with temporary local data managed as blocks. Not 
serializable. */
 private[spark] case class TempLocalBlockId(id: UUID) extends BlockId {
   override def name: String = "temp_local_" + id
@@ -213,6 +218,7 @@ object BlockId {
   val BROADCAST = "broadcast_([0-9]+)([_A-Za-z0-9]*)".r
   val TASKRESULT = "taskresult_([0-9]+)".r
   val STREAM = "input-([0-9]+)-([0-9]+)".r
+  val PYTHON_STREAM = "python-stream-([0-9]+)-([0-9]+)".r
   val TEMP_LOCAL = "temp_local_([-A-Fa-f0-9]+)".r
   val TEMP_SHUFFLE = "temp_shuffle_([-A-Fa-f0-9]+)".r
   val TEST = "test_(.*)".r
@@ -250,6 +256,8 @@ object BlockId {
   TaskResultBlockId(taskId.toLong)
 case STREAM(streamId, uniqueId) =>
   StreamBlockId(streamId.toInt, uniqueId.toLong)
+case PYTHON_STREAM(streamId, uniqueId) =>
+  PythonStreamBlockId(streamId.toInt, uniqueId.toLong)
 case TEMP_LOCAL(uuid) =>
   TempLocalBlockId(UUID.fromString(uuid))
 case TEMP_SHUFFLE(uuid) =>
diff --git a/python/pyspark/sql/datasource.py b/python/pyspark/sql/datasource.py
index c08b5b7af77f..6cac7e35ff41 100644
--- a/python/pyspark/sql/datasource.py
+++ b/python/pyspark/sql/datasource.py
@@ -183,11 +183,36 @@ cl

(spark) branch master updated (332570f42203 -> 94763438943e)

2024-04-30 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


from 332570f42203 [SPARK-48052][PYTHON][CONNECT] Recover `pyspark-connect` 
CI by parent classes
 add 94763438943e [SPARK-48050][SS] Log logical plan at query start

No new revisions were added by this update.

Summary of changes:
 .../org/apache/spark/sql/execution/streaming/StreamExecution.scala| 4 
 1 file changed, 4 insertions(+)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



(spark) branch master updated (2b2a33cc35a8 -> d5712cea88cd)

2024-04-26 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


from 2b2a33cc35a8 [SPARK-48011][CORE] Store LogKey name as a value to avoid 
generating new string instances
 add d5712cea88cd [SPARK-48018][SS] Fix null groupId causing missing param 
error when throwing KafkaException.couldNotReadOffsetRange

No new revisions were added by this update.

Summary of changes:
 .../src/main/scala/org/apache/spark/sql/kafka010/KafkaExceptions.scala  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



(spark) branch master updated (033ca3e7dd5b -> b0e03a193531)

2024-04-26 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


from 033ca3e7dd5b [SPARK-47922][SQL] Implement the try_parse_json expression
 add b0e03a193531 [SPARK-47999][SS] Improve logging around snapshot 
creation and adding/removing entries from state cache map in HDFS backed state 
store provider

No new revisions were added by this update.

Summary of changes:
 .../state/HDFSBackedStateStoreProvider.scala   | 31 ++
 1 file changed, 26 insertions(+), 5 deletions(-)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



(spark) branch master updated: [SPARK-47805][SS] Implementing TTL for MapState

2024-04-23 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 61ac3421651c [SPARK-47805][SS] Implementing TTL for MapState
61ac3421651c is described below

commit 61ac3421651c2d42127a4b3f62c9df51e85cbb85
Author: Eric Marnadi 
AuthorDate: Tue Apr 23 16:57:16 2024 +0900

[SPARK-47805][SS] Implementing TTL for MapState

### What changes were proposed in this pull request?

This PR adds support for expiring state based on TTL for MapState. Using 
this functionality, Spark users can specify a TTL Mode for transformWithState 
operator, and provide a ttlDuration for each value in MapState. Once the 
ttlDuration has expired, the value will not be returned as part of get() and 
would be cleaned up at the end of the micro-batch.

### Why are the changes needed?

These changes are needed to support TTL for MapState. The PR supports 
specifying ttl for processing time.

### Does this PR introduce _any_ user-facing change?

Yes, modifies the MapState interface for specifying ttlDuration

### How was this patch tested?

Added the TransformWithMapStateTTLSuite, MapStateSuite, 
StatefulProcessorHandleSuite
### Was this patch authored or co-authored using generative AI tooling?

No

Closes #45991 from ericm-db/map-state-ttl.

Authored-by: Eric Marnadi 
Signed-off-by: Jungtaek Lim 
---
 .../sql/streaming/StatefulProcessorHandle.scala|  23 ++
 .../execution/streaming/ListStateImplWithTTL.scala |   1 +
 .../execution/streaming/MapStateImplWithTTL.scala  | 252 
 .../streaming/StateTypesEncoderUtils.scala |  25 ++
 .../streaming/StatefulProcessorHandleImpl.scala|  17 ++
 .../spark/sql/execution/streaming/TTLState.scala   | 151 --
 .../streaming/TransformWithStateExec.scala |   2 +
 .../streaming/ValueStateImplWithTTL.scala  |   1 +
 .../execution/streaming/state/MapStateSuite.scala  |  92 +-
 .../state/StatefulProcessorHandleSuite.scala   |  19 ++
 .../streaming/TransformWithMapStateTTLSuite.scala  | 322 +
 11 files changed, 886 insertions(+), 19 deletions(-)

diff --git 
a/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala
 
b/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala
index f662b685c4e4..4dc2ca875ef0 100644
--- 
a/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala
+++ 
b/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala
@@ -108,6 +108,29 @@ private[sql] trait StatefulProcessorHandle extends 
Serializable {
   userKeyEnc: Encoder[K],
   valEncoder: Encoder[V]): MapState[K, V]
 
+  /**
+   * Function to create new or return existing map state variable of given type
+   * with ttl. State values will not be returned past ttlDuration, and will be 
eventually removed
+   * from the state store. Any values in mapState which have expired after 
ttlDuration will not
+   * returned on get() and will be eventually removed from the state.
+   *
+   * The user must ensure to call this function only within the `init()` 
method of the
+   * StatefulProcessor.
+   *
+   * @param stateName  - name of the state variable
+   * @param userKeyEnc  - spark sql encoder for the map key
+   * @param valEncoder - SQL encoder for state variable
+   * @param ttlConfig  - the ttl configuration (time to live duration etc.)
+   * @tparam K - type of key for map state variable
+   * @tparam V - type of value for map state variable
+   * @return - instance of MapState of type [K,V] that can be used to store 
state persistently
+   */
+  def getMapState[K, V](
+ stateName: String,
+ userKeyEnc: Encoder[K],
+ valEncoder: Encoder[V],
+ ttlConfig: TTLConfig): MapState[K, V]
+
   /** Function to return queryInfo for currently running task */
   def getQueryInfo(): QueryInfo
 
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ListStateImplWithTTL.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ListStateImplWithTTL.scala
index 32bc21cea6ed..dc72f8bcd560 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ListStateImplWithTTL.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ListStateImplWithTTL.scala
@@ -137,6 +137,7 @@ class ListStateImplWithTTL[S](
   /** Remove this state. */
   override def clear(): Unit = {
 store.remove(stateTypesEncoder.encodeGroupingKey(), stateName)
+clearTTLState()
   }
 
   private def validateNewState(newState: Array[S]): Unit = {
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MapStateImplWithTTL.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution

(spark) branch branch-3.5 updated: [SPARK-47840][SS] Disable foldable propagation across Streaming Aggregate/Join nodes

2024-04-15 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch branch-3.5
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.5 by this push:
 new 6c67c61bfd21 [SPARK-47840][SS] Disable foldable propagation across 
Streaming Aggregate/Join nodes
6c67c61bfd21 is described below

commit 6c67c61bfd21ebe68837f889502368ab9d99ebc5
Author: Bhuwan Sahni 
AuthorDate: Tue Apr 16 12:36:08 2024 +0900

[SPARK-47840][SS] Disable foldable propagation across Streaming 
Aggregate/Join nodes

### What changes were proposed in this pull request?

Streaming queries with Union of 2 data streams followed by an Aggregate 
(groupBy) can produce incorrect results if the grouping key is a constant 
literal for micro-batch duration.

The query produces incorrect results because the query optimizer recognizes 
the literal value in the grouping key as foldable and replaces the grouping key 
expression with the actual literal value. This optimization is correct for 
batch queries. However Streaming queries also read information from StateStore, 
and the output contains both the results from StateStore (computed in previous 
microbatches) and data from input sources (computed in this microbatch). The 
HashAggregate node aft [...]

See an example logical and physical plan below for a query performing a 
union on 2 data streams, followed by a groupBy. Note that the name#4 expression 
has been optimized to ds1. The Streaming query Aggregate adds StateStoreSave 
node as child of HashAggregate, however any grouping key read from StateStore 
will still be read as ds1 due to the optimization.

### Optimized Logical Plan

```
=== Applying Rule 
org.apache.spark.sql.catalyst.optimizer.FoldablePropagation ===

=== Old Plan ===

WriteToMicroBatchDataSource MemorySink, 
eb67645e-30fc-41a8-8006-35bb7649c202, Complete, 0
+- Aggregate [name#4], [name#4, count(1) AS count#31L]
   +- Project [ds1 AS name#4]
+- StreamingDataSourceV2ScanRelation[value#1] MemoryStreamDataSource

=== New Plan ===

WriteToMicroBatchDataSource MemorySink, 
eb67645e-30fc-41a8-8006-35bb7649c202, Complete, 0
+- Aggregate [ds1], [ds1 AS name#4, count(1) AS count#31L]
   +- Project [ds1 AS name#4]
+- StreamingDataSourceV2ScanRelation[value#1] MemoryStreamDataSource


```

### Corresponding Physical Plan

```
WriteToDataSourceV2 MicroBatchWrite[epoch: 0, writer: 
org.apache.spark.sql.execution.streaming.sources.MemoryStreamingWrite2b4c6242], 
org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy$$Lambda$3143/185907563435709d26
+- HashAggregate(keys=[ds1#39], functions=[finalmerge_count(merge 
count#38L) AS count(1)#30L], output=[name#4, count#31L])
   +- StateStoreSave [ds1#39], state info [ checkpoint = 
file:/tmp/streaming.metadata-e470782a-18a3-463c-9e61-3a10d0bdf180/state, runId 
= 4dedecca-910c-4518-855e-456702617414, opId = 0, ver = 0, numPartitions = 5], 
Complete, 0, 0, 2
+- HashAggregate(keys=[ds1#39], functions=[merge_count(merge 
count#38L) AS count#38L], output=[ds1#39, count#38L])
+- StateStoreRestore [ds1#39], state info [ checkpoint = 
file:/tmp/streaming.metadata-e470782a-18a3-463c-9e61-3a10d0bdf180/state, runId 
= 4dedecca-910c-4518-855e-456702617414, opId = 0, ver = 0, numPartitions = 5], 2
+- HashAggregate(keys=[ds1#39], 
functions=[merge_count(merge count#38L) AS count#38L], output=[ds1#39, 
count#38L])
+- HashAggregate(keys=[ds1 AS ds1#39], 
functions=[partial_count(1) AS count#38L], output=[ds1#39, count#38L])
+- Project
+- MicroBatchScan[value#1] MemoryStreamDataSource

```

This PR disables foldable propagation across Streaming Aggregate/Join nodes 
in the logical plan.

### Why are the changes needed?

Changes are needed to ensure that Streaming queries with literal value for 
grouping key/join key produce correct results.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

Added 
`sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryOptimizationCorrectnessSuite.scala`
 testcase.

```

[info] Run completed in 54 seconds, 150 milliseconds.
[info] Total number of tests run: 9
[info] Suites: completed 1, aborted 0
[info] Tests: succeeded 9, failed 0, canceled 0, ignored 0, pending 0
[info] All tests passed.

```

### Was this patch authored or co-authored using generative AI tooling?

No

Closes #46035 from sahnib/SPARK-47840.

Authored-by: Bhuwan Sahni 
Signed-off-by: Jungtaek Lim 
(cherry picked from commit f21719346eb0492cf9de47495853a4efad37dbab)
Signed

(spark) branch master updated (51d3efcead5b -> f21719346eb0)

2024-04-15 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


from 51d3efcead5b [SPARK-47233][CONNECT][SS][2/2] Client & Server logic for 
Client side streaming query listener
 add f21719346eb0 [SPARK-47840][SS] Disable foldable propagation across 
Streaming Aggregate/Join nodes

No new revisions were added by this update.

Summary of changes:
 .../spark/sql/catalyst/optimizer/expressions.scala |  18 +-
 ...treamingQueryOptimizationCorrectnessSuite.scala | 419 +
 2 files changed, 435 insertions(+), 2 deletions(-)
 create mode 100644 
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryOptimizationCorrectnessSuite.scala


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



(spark) branch master updated: [SPARK-47673][SS] Implementing TTL for ListState

2024-04-15 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new be080703688f [SPARK-47673][SS] Implementing TTL for ListState
be080703688f is described below

commit be080703688f8c59f8e7a0b24ce747d9ba14264e
Author: Eric Marnadi 
AuthorDate: Tue Apr 16 11:04:15 2024 +0900

[SPARK-47673][SS] Implementing TTL for ListState

### What changes were proposed in this pull request?

This PR adds support for expiring state based on TTL for ListState. Using 
this functionality, Spark users can specify a TTL Mode for transformWithState 
operator, and provide a ttlDuration for each value in ListState. TTL support 
for Map State will be added in future PRs. Once the ttlDuration has expired, 
the value will not be returned as part of get() and would be cleaned up at the 
end of the micro-batch.

### Why are the changes needed?

These changes are needed to support TTL for ListState. The PR supports 
specifying ttl for processing time.

### Does this PR introduce _any_ user-facing change?

Yes, modifies the ListState interface for specifying ttlDuration

### How was this patch tested?

Added the TransformWithListStateTTLSuite, ListStateSuite, 
StatefulProcessorHandleSuite
### Was this patch authored or co-authored using generative AI tooling?

No

Closes #45932 from ericm-db/ls-ttl.

Authored-by: Eric Marnadi 
Signed-off-by: Jungtaek Lim 
---
 .../sql/streaming/StatefulProcessorHandle.scala|  20 +
 .../execution/streaming/ListStateImplWithTTL.scala | 220 ++
 .../streaming/StateTypesEncoderUtils.scala |   5 +
 .../streaming/StatefulProcessorHandleImpl.scala|  32 ++
 .../spark/sql/execution/streaming/TTLState.scala   |  32 +-
 .../streaming/TransformWithStateExec.scala |   2 +
 .../streaming/ValueStateImplWithTTL.scala  |  46 +--
 .../execution/streaming/state/ListStateSuite.scala |  90 +++-
 .../state/StatefulProcessorHandleSuite.scala   |  25 +-
 .../streaming/state/ValueStateSuite.scala  |   4 +-
 .../streaming/TransformWithListStateTTLSuite.scala | 454 +
 ...Suite.scala => TransformWithStateTTLTest.scala} | 220 +-
 .../TransformWithValueStateTTLSuite.scala  | 270 +---
 13 files changed, 911 insertions(+), 509 deletions(-)

diff --git 
a/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala
 
b/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala
index e65667206ded..f662b685c4e4 100644
--- 
a/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala
+++ 
b/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala
@@ -72,6 +72,26 @@ private[sql] trait StatefulProcessorHandle extends 
Serializable {
*/
   def getListState[T](stateName: String, valEncoder: Encoder[T]): ListState[T]
 
+  /**
+   * Function to create new or return existing list state variable of given 
type
+   * with ttl. State values will not be returned past ttlDuration, and will be 
eventually removed
+   * from the state store. Any values in listState which have expired after 
ttlDuration will not
+   * be returned on get() and will be eventually removed from the state.
+   *
+   * The user must ensure to call this function only within the `init()` 
method of the
+   * StatefulProcessor.
+   *
+   * @param stateName  - name of the state variable
+   * @param valEncoder - SQL encoder for state variable
+   * @param ttlConfig  - the ttl configuration (time to live duration etc.)
+   * @tparam T - type of state variable
+   * @return - instance of ListState of type T that can be used to store state 
persistently
+   */
+  def getListState[T](
+  stateName: String,
+  valEncoder: Encoder[T],
+  ttlConfig: TTLConfig): ListState[T]
+
   /**
* Creates new or returns existing map state associated with stateName.
* The MapState persists Key-Value pairs of type [K, V].
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ListStateImplWithTTL.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ListStateImplWithTTL.scala
new file mode 100644
index ..32bc21cea6ed
--- /dev/null
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ListStateImplWithTTL.scala
@@ -0,0 +1,220 @@
+/*
+ * 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 

(spark) branch master updated (f86dc2b7e5c3 -> e6b7950f553c)

2024-04-15 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


from f86dc2b7e5c3 [SPARK-47848][SS] Fix thread safety issue around access 
to loadedMaps in close function for hdfs store provider
 add e6b7950f553c [SPARK-47788][SS] Ensure the same hash partitioning for 
streaming stateful ops

No new revisions were added by this update.

Summary of changes:
 .../partition-tests/randomSchemas  |   1 +
 .../partition-tests/rowsAndPartIds | Bin 0 -> 4862115 bytes
 .../StreamingQueryHashPartitionVerifySuite.scala   | 230 +
 3 files changed, 231 insertions(+)
 create mode 100644 
sql/core/src/test/resources/structured-streaming/partition-tests/randomSchemas
 create mode 100644 
sql/core/src/test/resources/structured-streaming/partition-tests/rowsAndPartIds
 create mode 100644 
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryHashPartitionVerifySuite.scala


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



(spark) branch master updated: [SPARK-47848][SS] Fix thread safety issue around access to loadedMaps in close function for hdfs store provider

2024-04-15 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new f86dc2b7e5c3 [SPARK-47848][SS] Fix thread safety issue around access 
to loadedMaps in close function for hdfs store provider
f86dc2b7e5c3 is described below

commit f86dc2b7e5c3aa4a94888feb499bc495877f36a9
Author: Anish Shrigondekar 
AuthorDate: Mon Apr 15 14:59:03 2024 +0900

[SPARK-47848][SS] Fix thread safety issue around access to loadedMaps in 
close function for hdfs store provider

### What changes were proposed in this pull request?
Fix thread safety issue around access to loadedMaps in close function for 
hdfs store provider

### Why are the changes needed?
To ensure thread safe access to `loadedMaps` which is the critical section 
for the hdfs backed state store provider in structured streaming

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Existing unit tests

```
[info] Run completed in 2 minutes, 51 seconds.
[info] Total number of tests run: 152
[info] Suites: completed 1, aborted 0
[info] Tests: succeeded 152, failed 0, canceled 0, ignored 0, pending 0
[info] All tests passed.
```

### Was this patch authored or co-authored using generative AI tooling?
No

Closes #46048 from anishshri-db/task/SPARK-47848.

Authored-by: Anish Shrigondekar 
Signed-off-by: Jungtaek Lim 
---
 .../sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala| 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala
index 850329e1ec69..2ecfa0931042 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala
@@ -346,7 +346,7 @@ private[sql] class HDFSBackedStateStoreProvider extends 
StateStoreProvider with
   }
 
   override def close(): Unit = {
-loadedMaps.values.asScala.foreach(_.clear())
+synchronized { loadedMaps.values.asScala.foreach(_.clear()) }
   }
 
   override def supportedCustomMetrics: Seq[StateStoreCustomMetric] = {


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



(spark) branch master updated (27987536be38 -> ae93b46b8c2b)

2024-04-12 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


from 27987536be38 [SPARK-47800][SQL] Create new method for identifier to 
tableIdentifier conversion
 add ae93b46b8c2b [SPARK-47733][SS] Add custom metrics for 
transformWithState operator part of query progress

No new revisions were added by this update.

Summary of changes:
 .../streaming/StatefulProcessorHandleImpl.scala| 28 +-
 .../spark/sql/execution/streaming/TTLState.scala   | 16 +
 .../streaming/TransformWithStateExec.scala | 27 +++--
 .../streaming/ValueStateImplWithTTL.scala  |  5 +++-
 .../state/HDFSBackedStateStoreProvider.scala   |  2 +-
 .../sql/execution/streaming/state/RocksDB.scala| 14 ++-
 .../state/RocksDBStateStoreProvider.scala  | 19 +++
 .../sql/execution/streaming/state/StateStore.scala |  2 +-
 .../streaming/state/MemoryStateStore.scala |  2 +-
 .../state/RocksDBStateStoreIntegrationSuite.scala  |  3 ++-
 .../execution/streaming/state/RocksDBSuite.scala   |  2 ++
 .../streaming/TransformWithListStateSuite.scala|  5 +++-
 .../sql/streaming/TransformWithMapStateSuite.scala |  5 +++-
 .../sql/streaming/TransformWithStateSuite.scala| 26 ++--
 .../TransformWithValueStateTTLSuite.scala  | 12 +-
 15 files changed, 140 insertions(+), 28 deletions(-)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



(spark) branch master updated: [SPARK-47784][SS] Merge TTLMode and TimeoutMode into a single TimeMode

2024-04-11 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 9ffdbc65029a [SPARK-47784][SS] Merge TTLMode and TimeoutMode into a 
single TimeMode
9ffdbc65029a is described below

commit 9ffdbc65029a08ce621ca50037683db05dc55761
Author: Bhuwan Sahni 
AuthorDate: Fri Apr 12 13:47:07 2024 +0900

[SPARK-47784][SS] Merge TTLMode and TimeoutMode into a single TimeMode

### What changes were proposed in this pull request?

This PR merges the `TimeoutMode` and `TTLMode` parameter for 
`transformWithState` into a single `TimeMode`. Currently, users need to specify 
the notion of time (ProcessingTime/EventTime) for timers and ttl separately. 
This allows users to use a single parameter.

We do not expect users to use mix/match EventTime/ProcessingTime for timers 
and ttl in a single query because it makes hard to reason about the time 
semantics (when will timer be fired?, when will the state be evicted? etc.). 
Its simpler to stick to one notion of time throughout timers and ttl.

### Why are the changes needed?

Changes are needed to simplify Arbitrary State API `transformWithState` 
interface by merging TTLMode/TimeoutMode into a single TimeMode.

### Does this PR introduce _any_ user-facing change?

Yes, this PR changes the API parameters for `transformWithState`.

### How was this patch tested?

All existing testcases for `transformWithState` API pass.

### Was this patch authored or co-authored using generative AI tooling?

No

Closes #45960 from sahnib/introduce-timeMode.

Authored-by: Bhuwan Sahni 
Signed-off-by: Jungtaek Lim 
---
 .../src/main/resources/error/error-classes.json| 16 +++---
 .../apache/spark/sql/KeyValueGroupedDataset.scala  | 38 +
 dev/checkstyle-suppressions.xml|  4 +-
 docs/sql-error-conditions.md   | 16 +++---
 .../sql/streaming/{TTLMode.java => TimeMode.java}  | 34 ++-
 .../apache/spark/sql/streaming/TimeoutMode.java| 51 -
 .../logical/{TTLMode.scala => TimeMode.scala}  | 10 ++--
 .../logical/TransformWithStateTimeoutModes.scala   | 24 
 .../spark/sql/streaming/StatefulProcessor.scala|  5 +-
 .../spark/sql/catalyst/plans/logical/object.scala  | 17 ++
 .../apache/spark/sql/KeyValueGroupedDataset.scala  | 36 +---
 .../spark/sql/execution/SparkStrategies.scala  |  9 ++-
 .../execution/streaming/ExpiredTimerInfoImpl.scala |  4 +-
 .../streaming/StatefulProcessorHandleImpl.scala| 17 +++---
 .../sql/execution/streaming/TimerStateImpl.scala   | 14 ++---
 .../streaming/TransformWithStateExec.scala | 65 +-
 .../streaming/state/StateStoreErrors.scala | 37 +---
 .../org/apache/spark/sql/JavaDatasetSuite.java |  6 +-
 .../apache/spark/sql/TestStatefulProcessor.java|  3 +-
 .../sql/TestStatefulProcessorWithInitialState.java |  3 +-
 .../execution/streaming/state/ListStateSuite.scala | 14 ++---
 .../execution/streaming/state/MapStateSuite.scala  | 11 ++--
 .../state/StatefulProcessorHandleSuite.scala   | 64 ++---
 .../sql/execution/streaming/state/TimerSuite.scala | 42 +++---
 .../streaming/state/ValueStateSuite.scala  | 35 
 .../streaming/TransformWithListStateSuite.scala| 30 --
 .../sql/streaming/TransformWithMapStateSuite.scala | 18 ++
 .../TransformWithStateInitialStateSuite.scala  | 25 +++--
 .../sql/streaming/TransformWithStateSuite.scala| 61 +++-
 .../TransformWithValueStateTTLSuite.scala  | 21 +++
 30 files changed, 267 insertions(+), 463 deletions(-)

diff --git a/common/utils/src/main/resources/error/error-classes.json 
b/common/utils/src/main/resources/error/error-classes.json
index 62581116000b..7b13fa4278e4 100644
--- a/common/utils/src/main/resources/error/error-classes.json
+++ b/common/utils/src/main/resources/error/error-classes.json
@@ -3591,21 +3591,15 @@
 ],
 "sqlState" : "0A000"
   },
-  "STATEFUL_PROCESSOR_CANNOT_ASSIGN_TTL_IN_NO_TTL_MODE" : {
-"message" : [
-  "Cannot use TTL for state= in NoTTL() mode."
-],
-"sqlState" : "42802"
-  },
   "STATEFUL_PROCESSOR_CANNOT_PERFORM_OPERATION_WITH_INVALID_HANDLE_STATE" : {
 "message" : [
   "Failed to perform stateful processor operation= with 
invalid handle state=."
 ],
 "sqlState" : "42802"
   },
-  "STATEFUL_PROCESSOR_CANNOT_PERFORM_OPERATION_WITH_INVALID_TIMEOUT_MODE" : {
+  "STATEFUL_PROCESSOR_CANNOT_PERFORM_OPERATION_WITH_INVALID_TIME_MODE" : {
 "message" : [
- 

(spark) branch master updated: [SPARK-47776][SS] Disallow binary inequality collation be used in key schema of stateful operator

2024-04-09 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 2f7cdc9cdaf4 [SPARK-47776][SS] Disallow binary inequality collation be 
used in key schema of stateful operator
2f7cdc9cdaf4 is described below

commit 2f7cdc9cdaf4122ccd41e9f9b3296f4b190fee05
Author: Jungtaek Lim 
AuthorDate: Wed Apr 10 13:38:07 2024 +0900

[SPARK-47776][SS] Disallow binary inequality collation be used in key 
schema of stateful operator

### What changes were proposed in this pull request?

This PR proposes to disallow using binary inequality collation column in 
the key schema of stateful operator. Worth noting that changing the collation 
for the same string column during the query restart was already disallowed at 
the time of introduction of collation.

### Why are the changes needed?

state store API is heavily relying on the fact that provider implementation 
performs O(1)-like get and put operation. While the actual implementation would 
be dependent on the state store provider, it is intuitive to assume that these 
providers only do lookup of the key based on binary format (implying binary 
equality).

That said, even though the column spec is case insensitive, state store API 
wouldn't take this into consideration, and could lead to produce the wrong 
result. e.g. Determiniing 'a' and 'A' differently while the column is case 
insensitive.

### Does this PR introduce _any_ user-facing change?

No, as it wasn't released yet.

### How was this patch tested?

New UTs.

### Was this patch authored or co-authored using generative AI tooling?

No.

Closes #45951 from HeartSaVioR/SPARK-47776.

Authored-by: Jungtaek Lim 
Signed-off-by: Jungtaek Lim 
---
 .../src/main/resources/error/error-classes.json|  6 
 docs/sql-error-conditions.md   |  6 
 .../sql/execution/streaming/state/StateStore.scala | 22 +++-
 .../StateSchemaCompatibilityCheckerSuite.scala | 24 +
 .../spark/sql/streaming/StreamingQuerySuite.scala  | 41 +-
 5 files changed, 97 insertions(+), 2 deletions(-)

diff --git a/common/utils/src/main/resources/error/error-classes.json 
b/common/utils/src/main/resources/error/error-classes.json
index c3a01e9dcd90..45a1ec5e1e84 100644
--- a/common/utils/src/main/resources/error/error-classes.json
+++ b/common/utils/src/main/resources/error/error-classes.json
@@ -3652,6 +3652,12 @@
 ],
 "sqlState" : "XXKST"
   },
+  "STATE_STORE_UNSUPPORTED_OPERATION_BINARY_INEQUALITY" : {
+"message" : [
+  "Binary inequality column is not supported with state store. Provided 
schema: ."
+],
+"sqlState" : "XXKST"
+  },
   "STATE_STORE_UNSUPPORTED_OPERATION_ON_MISSING_COLUMN_FAMILY" : {
 "message" : [
   "State store operation= not supported on missing column 
family=."
diff --git a/docs/sql-error-conditions.md b/docs/sql-error-conditions.md
index 1887af2e814b..bb25a4c7f9f0 100644
--- a/docs/sql-error-conditions.md
+++ b/docs/sql-error-conditions.md
@@ -2256,6 +2256,12 @@ Null type ordering column with name=`` at 
index=`` is not supp
 
 `` operation not supported with ``
 
+### STATE_STORE_UNSUPPORTED_OPERATION_BINARY_INEQUALITY
+
+[SQLSTATE: XXKST](sql-error-conditions-sqlstates.html#class-XX-internal-error)
+
+Binary inequality column is not supported with state store. Provided schema: 
``.
+
 ### STATE_STORE_UNSUPPORTED_OPERATION_ON_MISSING_COLUMN_FAMILY
 
 [SQLSTATE: 
42802](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation)
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala
index 959cbbaef8b0..69c9e0ed85be 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala
@@ -29,7 +29,7 @@ import scala.util.control.NonFatal
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.Path
 
-import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.{SparkContext, SparkEnv, 
SparkUnsupportedOperationException}
 import org.apache.spark.internal.Logging
 import org.apache.spark.sql.catalyst.expressions.UnsafeRow
 import org.apache.spark.sql.catalyst.util.UnsafeRowUtils
@@ -635,6 +635,15 @@ object StateStore extends Logging {
 storeProvider.getStore(version)
   }
 
+  private def disallowBinaryInequalityColumn(schema: StructType): Unit = {
+if (!UnsafeRowUtil

(spark) branch master updated: [SPARK-47746] Implement ordinal-based range encoding in the RocksDBStateEncoder

2024-04-08 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 60806c63d97b [SPARK-47746] Implement ordinal-based range encoding in 
the RocksDBStateEncoder
60806c63d97b is described below

commit 60806c63d97bc35f62a049b2185eb921217904c4
Author: Neil Ramaswamy 
AuthorDate: Mon Apr 8 17:39:18 2024 +0900

[SPARK-47746] Implement ordinal-based range encoding in the 
RocksDBStateEncoder

### What changes were proposed in this pull request?

The RocksDBStateEncoder now implements range projection by reading a list 
of ordering ordinals, and using that to project certain columns, in big-endian, 
to the front of the `Array[Byte]` encoded rows returned by the encoder.

### Why are the changes needed?

StateV2 implementations (and other state-related operators) project certain 
columns to the front of `UnsafeRow`s, and then rely on the RocksDBStateEncoder 
to range-encode those columns. We can avoid the initial projection by just 
passing the RocksDBStateEncoder the ordinals to encode at the front. This 
should avoid any GC or codegen overheads associated with projection.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

New UTs. All existing UTs should pass.

### Was this patch authored or co-authored using generative AI tooling?

Yes

Closes #45905 from neilramaswamy/spark-47746.

Authored-by: Neil Ramaswamy 
Signed-off-by: Jungtaek Lim 
---
 .../src/main/resources/error/error-classes.json|   2 +-
 docs/sql-error-conditions.md   |   2 +-
 .../spark/sql/execution/streaming/TTLState.scala   |   2 +-
 .../sql/execution/streaming/TimerStateImpl.scala   |   2 +-
 .../streaming/state/RocksDBStateEncoder.scala  |  87 ++
 .../sql/execution/streaming/state/StateStore.scala |   7 +-
 .../streaming/state/RocksDBStateStoreSuite.scala   | 184 ++---
 .../streaming/state/StateStoreSuite.scala  |   2 +-
 8 files changed, 228 insertions(+), 60 deletions(-)

diff --git a/common/utils/src/main/resources/error/error-classes.json 
b/common/utils/src/main/resources/error/error-classes.json
index f28adaf40230..c3a01e9dcd90 100644
--- a/common/utils/src/main/resources/error/error-classes.json
+++ b/common/utils/src/main/resources/error/error-classes.json
@@ -3630,7 +3630,7 @@
   },
   "STATE_STORE_INCORRECT_NUM_ORDERING_COLS_FOR_RANGE_SCAN" : {
 "message" : [
-  "Incorrect number of ordering columns= for range scan 
encoder. Ordering columns cannot be zero or greater than num of schema columns."
+  "Incorrect number of ordering ordinals= for range scan 
encoder. The number of ordering ordinals cannot be zero or greater than number 
of schema columns."
 ],
 "sqlState" : "42802"
   },
diff --git a/docs/sql-error-conditions.md b/docs/sql-error-conditions.md
index d8261b8c2765..1887af2e814b 100644
--- a/docs/sql-error-conditions.md
+++ b/docs/sql-error-conditions.md
@@ -2236,7 +2236,7 @@ Please only use the StatefulProcessor within the 
transformWithState operator.
 
 [SQLSTATE: 
42802](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation)
 
-Incorrect number of ordering columns=`` for range scan 
encoder. Ordering columns cannot be zero or greater than num of schema columns.
+Incorrect number of ordering ordinals=`` for range scan 
encoder. The number of ordering ordinals cannot be zero or greater than number 
of schema columns.
 
 ### STATE_STORE_INCORRECT_NUM_PREFIX_COLS_FOR_PREFIX_SCAN
 
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TTLState.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TTLState.scala
index 0ae93549b731..f64c8cc44555 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TTLState.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TTLState.scala
@@ -93,7 +93,7 @@ abstract class SingleKeyTTLStateImpl(
 
UnsafeProjection.create(Array[DataType](NullType)).apply(InternalRow.apply(null))
 
   store.createColFamilyIfAbsent(ttlColumnFamilyName, TTL_KEY_ROW_SCHEMA, 
TTL_VALUE_ROW_SCHEMA,
-RangeKeyScanStateEncoderSpec(TTL_KEY_ROW_SCHEMA, 1), isInternal = true)
+RangeKeyScanStateEncoderSpec(TTL_KEY_ROW_SCHEMA, Seq(0)), isInternal = 
true)
 
   def upsertTTLForStateKey(
   expirationMs: Long,
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TimerStateImpl.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TimerStateImpl.scala
index 8d410b677c84..55acc4953c50 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TimerStateIm

(spark) branch master updated: [SPARK-47558][SS] State TTL support for ValueState

2024-04-07 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new d55bb617a135 [SPARK-47558][SS] State TTL support for ValueState
d55bb617a135 is described below

commit d55bb617a13561f0eb9f301089a4e4fb06e06228
Author: Bhuwan Sahni 
AuthorDate: Mon Apr 8 12:22:04 2024 +0900

[SPARK-47558][SS] State TTL support for ValueState

**Note**: This change has been co-authored by ericm-db  and sahnib

**Authors: ericm-db sahnib**

### What changes were proposed in this pull request?

This PR adds support for expiring state based on TTL for ValueState. Using 
this functionality, Spark users can specify a TTL Mode for transformWithState 
operator, and provide a ttlDuration/expirationTImeInMs for each value in 
ValueState. TTL support for List/Map State will be added in future PRs. Once 
the ttlDuration has expired, the value will not be returned as part of `get()` 
and would be cleaned up at the end of the micro-batch.

### Why are the changes needed?

These changes are needed to support TTL for ValueState. The PR supports 
specifying ttl for processing time or event time. Processing time ttl is 
calculated by adding ttlDuration to `batchTimestamp`, and event time ttl is 
specified using absolute expiration time (`expirationTimeInMs`).

### Does this PR introduce _any_ user-facing change?

Yes, modifies the ValueState interface for specifying `ttlDuration`, and 
adds `ttlMode` to `transformWithState` API.

### How was this patch tested?

Added unit test cases for both event time and processing time in 
`ValueStateWithTTLSuite`.

```
WARNING: Using incubator modules: jdk.incubator.foreign, 
jdk.incubator.vector
[info] TransformWithStateTTLSuite:
11:56:54.590 WARN org.apache.hadoop.util.NativeCodeLoader: Unable to load 
native-hadoop library for your platform... using builtin-java classes where 
applicable
11:56:56.054 WARN 
org.apache.spark.sql.execution.streaming.ResolveWriteToStream: 
spark.sql.adaptive.enabled is not supported in streaming DataFrames/Datasets 
and will be disabled.
[info] - validate state is evicted at ttl expiry - processing time ttl (6 
seconds, 244 milliseconds)
11:57:01.188 WARN 
org.apache.spark.sql.execution.streaming.ResolveWriteToStream: 
spark.sql.adaptive.enabled is not supported in streaming DataFrames/Datasets 
and will be disabled.
[info] - validate ttl update updates the expiration timestamp - processing 
time ttl (4 seconds, 465 milliseconds)
11:57:05.641 WARN 
org.apache.spark.sql.execution.streaming.ResolveWriteToStream: 
spark.sql.adaptive.enabled is not supported in streaming DataFrames/Datasets 
and will be disabled.
[info] - validate ttl removal keeps value in state - processing time ttl (4 
seconds, 407 milliseconds)
11:57:10.041 WARN 
org.apache.spark.sql.execution.streaming.ResolveWriteToStream: 
spark.sql.adaptive.enabled is not supported in streaming DataFrames/Datasets 
and will be disabled.
[info] - validate multiple value states - with and without ttl - processing 
time ttl (3 seconds, 131 milliseconds)
11:57:13.175 WARN 
org.apache.spark.sql.execution.streaming.ResolveWriteToStream: 
spark.sql.adaptive.enabled is not supported in streaming DataFrames/Datasets 
and will be disabled.
[info] - validate state is evicted at ttl expiry - event time ttl (4 
seconds, 186 milliseconds)
11:57:17.355 WARN 
org.apache.spark.sql.execution.streaming.ResolveWriteToStream: 
spark.sql.adaptive.enabled is not supported in streaming DataFrames/Datasets 
and will be disabled.
[info] - validate ttl update updates the expiration timestamp - event time 
ttl (4 seconds, 28 milliseconds)
11:57:21.391 WARN 
org.apache.spark.sql.execution.streaming.ResolveWriteToStream: 
spark.sql.adaptive.enabled is not supported in streaming DataFrames/Datasets 
and will be disabled.
[info] - validate ttl removal keeps value in state - event time ttl (4 
seconds, 428 milliseconds)
11:57:25.838 WARN org.apache.spark.sql.streaming.TransformWithStateTTLSuite:

[info] Run completed in 32 seconds, 433 milliseconds.
[info] Total number of tests run: 7
[info] Suites: completed 1, aborted 0
[info] Tests: succeeded 7, failed 0, canceled 0, ignored 0, pending 0
[info] All tests passed.

```

### Was this patch authored or co-authored using generative AI tooling?

No

Closes #45674 from sahnib/state-ttl.

Authored-by: Bhuwan Sahni 
Signed-off-by: Jungtaek Lim 
---
 .../src/main/resources/error/error-classes.json|  17 +
 .../apache/spark/sql/KeyValueGroupedDataset.scala  |  14 +-
 dev/checkstyle-suppressions.xml|   2 +
 ...r-conditions-unsupported-feature-error-class.md |   4 +
 docs/sql

(spark) branch branch-3.5 updated: [SPARK-47299][PYTHON][DOCS] Use the same `versions.json` in the dropdown of different versions of PySpark documents

2024-04-07 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch branch-3.5
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.5 by this push:
 new 850ec0b4adcb [SPARK-47299][PYTHON][DOCS] Use the same `versions.json` 
in the dropdown of different versions of PySpark documents
850ec0b4adcb is described below

commit 850ec0b4adcb219d048bed003a7cb42cfc731f33
Author: panbingkun 
AuthorDate: Mon Apr 8 10:19:26 2024 +0900

[SPARK-47299][PYTHON][DOCS] Use the same `versions.json` in the dropdown of 
different versions of PySpark documents

### What changes were proposed in this pull request?
The pr aims to use the same `versions.json` in the dropdown of `different 
versions` of PySpark documents.

### Why are the changes needed?
As discussed in the email group, using this approach can avoid `maintenance 
difficulties` and `inconsistencies` that may arise when `multi active release 
version lines` are released in the future.
https://github.com/apache/spark/assets/15246973/8a08a4fe-e1fb-4334-a3f9-c6dffb01cbd6;>

### Does this PR introduce _any_ user-facing change?
Yes, only for pyspark docs.

### How was this patch tested?
- Manually test.
- Pass GA.

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #45400 from panbingkun/SPARK-47299.

Authored-by: panbingkun 
Signed-off-by: Jungtaek Lim 
(cherry picked from commit b299b2bc06a91db630ab39b9c35663342931bb56)
Signed-off-by: Jungtaek Lim 
---
 python/docs/source/_static/versions.json | 22 --
 python/docs/source/conf.py   |  6 +-
 2 files changed, 5 insertions(+), 23 deletions(-)

diff --git a/python/docs/source/_static/versions.json 
b/python/docs/source/_static/versions.json
deleted file mode 100644
index 3d0bd1481806..
--- a/python/docs/source/_static/versions.json
+++ /dev/null
@@ -1,22 +0,0 @@
-[
-{
-"name": "3.4.1",
-"version": "3.4.1"
-},
-{
-"name": "3.4.0",
-"version": "3.4.0"
-},
-{
-"name": "3.3.2",
-"version": "3.3.2"
-},
-{
-"name": "3.3.1",
-"version": "3.3.1"
-},
-{
-"name": "3.3.0",
-"version": "3.3.0"
-}
-]
diff --git a/python/docs/source/conf.py b/python/docs/source/conf.py
index 08a25c5dd071..1b5cf3474465 100644
--- a/python/docs/source/conf.py
+++ b/python/docs/source/conf.py
@@ -182,7 +182,11 @@ autosummary_generate = True
 html_theme = 'pydata_sphinx_theme'
 
 html_context = {
-"switcher_json_url": "_static/versions.json",
+# When releasing a new Spark version, please update the file
+# "site/static/versions.json" under the code repository "spark-website"
+# (item should be added in order), and also set the local environment
+# variable "RELEASE_VERSION".
+"switcher_json_url": "https://spark.apache.org/static/versions.json;,
 "switcher_template_url": 
"https://spark.apache.org/docs/{version}/api/python/index.html;,
 }
 


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



(spark) branch master updated (0c992b205946 -> b299b2bc06a9)

2024-04-07 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


from 0c992b205946 [SPARK-47755][CONNECT] Pivot should fail when the number 
of distinct values is too large
 add b299b2bc06a9 [SPARK-47299][PYTHON][DOCS] Use the same `versions.json` 
in the dropdown of different versions of PySpark documents

No new revisions were added by this update.

Summary of changes:
 python/docs/source/_static/versions.json | 22 --
 python/docs/source/conf.py   |  6 +-
 2 files changed, 5 insertions(+), 23 deletions(-)
 delete mode 100644 python/docs/source/_static/versions.json


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



(spark) branch branch-3.4 updated: [SPARK-47734][PYTHON][TESTS][3.4] Fix flaky DataFrame.writeStream doctest by stopping streaming query

2024-04-07 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch branch-3.4
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.4 by this push:
 new 1f66a40e3b85 [SPARK-47734][PYTHON][TESTS][3.4] Fix flaky 
DataFrame.writeStream doctest by stopping streaming query
1f66a40e3b85 is described below

commit 1f66a40e3b85ea2153c021d65be8124920091fa7
Author: Josh Rosen 
AuthorDate: Mon Apr 8 07:05:45 2024 +0900

[SPARK-47734][PYTHON][TESTS][3.4] Fix flaky DataFrame.writeStream doctest 
by stopping streaming query

### What changes were proposed in this pull request?

Backport of https://github.com/apache/spark/pull/45885.

This PR deflakes the `pyspark.sql.dataframe.DataFrame.writeStream` doctest.

PR https://github.com/apache/spark/pull/45298 aimed to fix that test but 
misdiagnosed the root issue. The problem is not that concurrent tests were 
colliding on a temporary directory. Rather, the issue is specific to the 
`DataFrame.writeStream` test's logic: that test is starting a streaming query 
that writes files to the temporary directory, the exits the temp directory 
context manager without first stopping the streaming query. That creates a race 
condition where the context manager [...]

```
File "/__w/spark/spark/python/pyspark/sql/dataframe.py", line ?, in 
pyspark.sql.dataframe.DataFrame.writeStream
Failed example:
with tempfile.TemporaryDirectory() as d:
# Create a table with Rate source.
df.writeStream.toTable(
"my_table", checkpointLocation=d)
Exception raised:
Traceback (most recent call last):
  File "/usr/lib/python3.11/doctest.py", line 1353, in __run
exec(compile(example.source, filename, "single",
  File "", line 
1, in 
with tempfile.TemporaryDirectory() as d:
  File "/usr/lib/python3.11/tempfile.py", line 1043, in __exit__
self.cleanup()
  File "/usr/lib/python3.11/tempfile.py", line 1047, in cleanup
self._rmtree(self.name, ignore_errors=self._ignore_cleanup_errors)
  File "/usr/lib/python3.11/tempfile.py", line 1029, in _rmtree
_rmtree(name, onerror=onerror)
  File "/usr/lib/python3.11/shutil.py", line 738, in rmtree
onerror(os.rmdir, path, sys.exc_info())
  File "/usr/lib/python3.11/shutil.py", line 736, in rmtree
os.rmdir(path, dir_fd=dir_fd)
OSError: [Errno 39] Directory not empty: 
'/__w/spark/spark/python/target/4f062b09-213f-4ac2-a10a-2d704990141b/tmp29irqweq'
```

In this PR, I update the doctest to properly stop the streaming query.

### Why are the changes needed?

Fix flaky test.

### Does this PR introduce _any_ user-facing change?

No, test-only. Small user-facing doc change, but one that is consistent 
with other doctest examples.

### How was this patch tested?

Manually ran updated test.

### Was this patch authored or co-authored using generative AI tooling?

No.

Closes #45908 from JoshRosen/fix-flaky-writestream-doctest-3.4.

Authored-by: Josh Rosen 
Signed-off-by: Jungtaek Lim 
---
 python/pyspark/sql/dataframe.py | 8 +---
 1 file changed, 5 insertions(+), 3 deletions(-)

diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py
index 14426c514392..f69d74ad5002 100644
--- a/python/pyspark/sql/dataframe.py
+++ b/python/pyspark/sql/dataframe.py
@@ -527,6 +527,7 @@ class DataFrame(PandasMapOpsMixin, PandasConversionMixin):
 
 Examples
 
+>>> import time
 >>> import tempfile
 >>> df = spark.readStream.format("rate").load()
 >>> type(df.writeStream)
@@ -534,9 +535,10 @@ class DataFrame(PandasMapOpsMixin, PandasConversionMixin):
 
 >>> with tempfile.TemporaryDirectory() as d:
 ... # Create a table with Rate source.
-... df.writeStream.toTable(
-... "my_table", checkpointLocation=d) # doctest: +ELLIPSIS
-
+... query = df.writeStream.toTable(
+... "my_table", checkpointLocation=d)
+... time.sleep(3)
+... query.stop()
 """
 return DataStreamWriter(self)
 


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



(spark) branch master updated: [SPARK-47744] Add support for negative-valued bytes in range encoder

2024-04-07 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new e92e8f5441a7 [SPARK-47744] Add support for negative-valued bytes in 
range encoder
e92e8f5441a7 is described below

commit e92e8f5441a702021e3cbcb282c172f6697f7118
Author: Neil Ramaswamy 
AuthorDate: Mon Apr 8 05:48:51 2024 +0900

[SPARK-47744] Add support for negative-valued bytes in range encoder

### What changes were proposed in this pull request?

The RocksDBStateEncoder now encodes negative-valued bytes correctly.

### Why are the changes needed?

Components that use the state encoder might want to use the full-range of 
values of the Scala (signed) `Byte` type.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Existing UT was modified. All existing UTs should pass.

### Was this patch authored or co-authored using generative AI tooling?

No

Closes #45906 from neilramaswamy/spark-47744.

Authored-by: Neil Ramaswamy 
Signed-off-by: Jungtaek Lim 
---
 .../sql/execution/streaming/state/RocksDBStateEncoder.scala| 10 --
 .../sql/execution/streaming/state/RocksDBStateStoreSuite.scala |  2 +-
 2 files changed, 9 insertions(+), 3 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala
index 06c3940af127..e9b910a76148 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala
@@ -323,8 +323,14 @@ class RangeKeyScanStateEncoder(
 field.dataType match {
   case BooleanType =>
   case ByteType =>
-bbuf.put(positiveValMarker)
-bbuf.put(value.asInstanceOf[Byte])
+val byteVal = value.asInstanceOf[Byte]
+val signCol = if (byteVal < 0) {
+  negativeValMarker
+} else {
+  positiveValMarker
+}
+bbuf.put(signCol)
+bbuf.put(byteVal)
 writer.write(idx, bbuf.array())
 
   case ShortType =>
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreSuite.scala
index 1e5f664c980c..16a5935e04f4 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreSuite.scala
@@ -625,7 +625,7 @@ class RocksDBStateStoreSuite extends 
StateStoreSuiteBase[RocksDBStateStoreProvid
   val timerTimestamps: Seq[(Byte, Int)] = Seq((0x33, 10), (0x1A, 40), 
(0x1F, 1), (0x01, 68),
 (0x7F, 2000), (0x01, 27), (0x01, 394), (0x01, 5), (0x03, 980), (0x35, 
2112),
 (0x11, -190), (0x1A, -69), (0x01, -344245), (0x31, -901),
-(0x06, 90118), (0x09, 95118), (0x06, 87210))
+(-0x01, 90118), (-0x7F, 95118), (-0x80, 87210))
   timerTimestamps.foreach { ts =>
 // order by byte col first and then by int col
 val keyRow = schemaProj.apply(new GenericInternalRow(Array[Any](ts._1, 
ts._2,


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



(spark) branch master updated: [SPARK-47310][SS] Add micro-benchmark for merge operations for multiple values in value portion of state store

2024-04-05 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 1efbf43160aa [SPARK-47310][SS] Add micro-benchmark for merge 
operations for multiple values in value portion of state store
1efbf43160aa is described below

commit 1efbf43160aa4e36710a4668f05fe61534f49648
Author: Anish Shrigondekar 
AuthorDate: Sat Apr 6 06:10:18 2024 +0900

[SPARK-47310][SS] Add micro-benchmark for merge operations for multiple 
values in value portion of state store

### What changes were proposed in this pull request?
Add microbenchmark for merge operations for multiple values in value 
portion of state store

### Why are the changes needed?
Micro-benchmark to understand performance with/without rows tracking around 
merge operations

As shown in the results, merge without tracking is consistently 3x faster

```
merging 1 rows with 10 values per key (1 rows to overwrite - rate 
100):  Best Time(ms)   Avg Time(ms)   Stdev(ms)Rate(M/s)   Per Row(ns)   
Relative

--
RocksDB (trackTotalNumberOfRows: true)  
  519533   7  0.0   51916.6   
1.0X
RocksDB (trackTotalNumberOfRows: false) 
  171177   3  0.1   17083.9   
3.0X
```

GH Actions here:
- https://github.com/anishshri-db/spark/actions/runs/8559698160
- https://github.com/anishshri-db/spark/actions/runs/8559694994

Difference is even more running locally (> 7x faster without tracking)

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Test only change

### Was this patch authored or co-authored using generative AI tooling?
No

Closes #45865 from anishshri-db/task/SPARK-47310.

Authored-by: Anish Shrigondekar 
Signed-off-by: Jungtaek Lim 
---
 ...StoreBasicOperationsBenchmark-jdk21-results.txt | 107 +++--
 .../StateStoreBasicOperationsBenchmark-results.txt | 107 +++--
 .../StateStoreBasicOperationsBenchmark.scala   | 130 -
 3 files changed, 265 insertions(+), 79 deletions(-)

diff --git 
a/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-jdk21-results.txt 
b/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-jdk21-results.txt
index d3b3aafc21e5..0317e6116375 100644
--- a/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-jdk21-results.txt
@@ -6,33 +6,66 @@ OpenJDK 64-Bit Server VM 21.0.2+13-LTS on Linux 
6.5.0-1016-azure
 AMD EPYC 7763 64-Core Processor
 putting 1 rows (1 rows to overwrite - rate 100):  Best Time(ms)   Avg 
Time(ms)   Stdev(ms)Rate(M/s)   Per Row(ns)   Relative
 
---
-In-memory9 
10   1  1.1 894.7   1.0X
-RocksDB (trackTotalNumberOfRows: true)  41 
42   2  0.24064.6   0.2X
-RocksDB (trackTotalNumberOfRows: false) 15 
15   1  0.71466.8   0.6X
+In-memory9 
10   1  1.1 936.2   1.0X
+RocksDB (trackTotalNumberOfRows: true)  41 
42   1  0.24068.9   0.2X
+RocksDB (trackTotalNumberOfRows: false) 15 
16   1  0.71500.4   0.6X
 
 OpenJDK 64-Bit Server VM 21.0.2+13-LTS on Linux 6.5.0-1016-azure
 AMD EPYC 7763 64-Core Processor
 putting 1 rows (5000 rows to overwrite - rate 50):  Best Time(ms)   Avg 
Time(ms)   Stdev(ms)Rate(M/s)   Per Row(ns)   Relative
 
-
-In-memory  9   
  10   0  1.1 893.1   1.0X
-RocksDB (trackTotalNumberOfRows: true)40   
  40   1  0.33959.7   0.2X
-RocksDB (trackTotalNumberOfRows: false)   15   
  16   1  0.71510.8   0.6X
+In-mem

(spark) branch master updated (a427a4586177 -> 1515c567e4f0)

2024-04-03 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


from a427a4586177 [SPARK-47710][SQL][DOCS] Postgres: Document Mapping Spark 
SQL Data Types from PostgreSQL
 add 1515c567e4f0 [SPARK-47553][SS] Add Java support for transformWithState 
operator APIs

No new revisions were added by this update.

Summary of changes:
 .../apache/spark/sql/KeyValueGroupedDataset.scala  | 100 +++-
 .../spark/sql/streaming/StatefulProcessor.scala|   4 +-
 .../apache/spark/sql/KeyValueGroupedDataset.scala  |  64 -
 .../org/apache/spark/sql/JavaDatasetSuite.java |  55 ++-
 .../apache/spark/sql/TestStatefulProcessor.java|  95 +++
 .../sql/TestStatefulProcessorWithInitialState.java |  78 
 .../TransformWithStateInitialStateSuite.scala  | 103 +++--
 7 files changed, 479 insertions(+), 20 deletions(-)
 create mode 100644 
sql/core/src/test/java/test/org/apache/spark/sql/TestStatefulProcessor.java
 create mode 100644 
sql/core/src/test/java/test/org/apache/spark/sql/TestStatefulProcessorWithInitialState.java


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



(spark) branch master updated: [SPARK-47653][SS] Add support for negative numeric types and range scan key encoder

2024-04-03 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new d16183d044a6 [SPARK-47653][SS] Add support for negative numeric types 
and range scan key encoder
d16183d044a6 is described below

commit d16183d044a6207cb6e71cbaa1c942621fec5c12
Author: Anish Shrigondekar 
AuthorDate: Wed Apr 3 16:54:23 2024 +0900

[SPARK-47653][SS] Add support for negative numeric types and range scan key 
encoder

### What changes were proposed in this pull request?
Add support for negative numeric types and range scan key encoder

### Why are the changes needed?
Without this change, sort ordering for `-ve` numbers is not maintained on 
iteration. Negative numbers would appear last previously. Note that only 
non-floating integer types such as `short, integer, long` are supported for 
signed values. For float/double, we cannot simply prepend a sign byte given the 
way floating point values are stored in the IEEE 754 floating point 
representation. Additionally we also need to flip all the bits and convert them 
back to the original value on read, in [...]

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Added unit tests

```
[info] - rocksdb range scan - with prefix scan - with 
colFamiliesEnabled=true (with changelog checkpointing) (164 milliseconds)
[info] - rocksdb range scan - with prefix scan - with 
colFamiliesEnabled=true (without changelog checkpointing) (95 milliseconds)
[info] - rocksdb range scan - with prefix scan - with 
colFamiliesEnabled=false (with changelog checkpointing) (155 milliseconds)
[info] - rocksdb range scan - with prefix scan - with 
colFamiliesEnabled=false (without changelog checkpointing) (82 milliseconds)
12:55:54.184 WARN 
org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreSuite:

= POSSIBLE THREAD LEAK IN SUITE 
o.a.s.sql.execution.streaming.state.RocksDBStateStoreSuite, threads: 
rpc-boss-3-1 (daemon=true), ForkJoinPool.commonPool-worker-2 (daemon=true), 
shuffle-boss-6-1 (daemon=true), ForkJoinPool.commonPool-worker-1 (daemon=true) 
=
[info] Run completed in 8 seconds, 888 milliseconds.
[info] Total number of tests run: 44
[info] Suites: completed 1, aborted 0
[info] Tests: succeeded 44, failed 0, canceled 0, ignored 0, pending 0
[info] All tests passed.
[success] Total time: 21 s, completed Mar 29, 2024, 12:55:54 PM
```

### Was this patch authored or co-authored using generative AI tooling?
No

Closes #45778 from anishshri-db/task/SPARK-47653.

Authored-by: Anish Shrigondekar 
Signed-off-by: Jungtaek Lim 
---
 .../streaming/state/RocksDBStateEncoder.scala  | 116 ++---
 .../sql/execution/streaming/state/StateStore.scala |   6 +-
 .../streaming/state/RocksDBStateStoreSuite.scala   |  92 +---
 3 files changed, 182 insertions(+), 32 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala
index f342853514d8..06c3940af127 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala
@@ -17,6 +17,8 @@
 
 package org.apache.spark.sql.execution.streaming.state
 
+import java.lang.Double.{doubleToRawLongBits, longBitsToDouble}
+import java.lang.Float.{floatToRawIntBits, intBitsToFloat}
 import java.nio.{ByteBuffer, ByteOrder}
 
 import org.apache.spark.internal.Logging
@@ -206,19 +208,25 @@ class PrefixKeyScanStateEncoder(
  * for the range scan into an UnsafeRow; we then rewrite that UnsafeRow's 
fields in BIG_ENDIAN
  * to allow for scanning keys in sorted order using the byte-wise comparison 
method that
  * RocksDB uses.
+ *
  * Then, for the rest of the fields, we project those into another UnsafeRow.
  * We then effectively join these two UnsafeRows together, and finally take 
those bytes
  * to get the resulting row.
+ *
  * We cannot support variable sized fields given the UnsafeRow format which 
stores variable
  * sized fields as offset and length pointers to the actual values, thereby 
changing the required
  * ordering.
+ *
  * Note that we also support "null" values being passed for these fixed size 
fields. We prepend
  * a single byte to indicate whether the column value is null or not. We 
cannot change the
  * nullability on the UnsafeRow itself as the expected ordering would change 
if non-first
  * columns are marked as null. If the first col is null, those entries will 
appear last in
  * the iterator. If non-first column

(spark) branch master updated (49b7b6b9fe6b -> 2c2a2adc3275)

2024-04-02 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


from 49b7b6b9fe6b [SPARK-47691][SQL] Postgres: Support multi dimensional 
array on the write side
 add 2c2a2adc3275 [SPARK-47655][SS] Integrate timer with Initial State 
handling for state-v2

No new revisions were added by this update.

Summary of changes:
 .../spark/sql/streaming/StatefulProcessor.scala|   7 +-
 .../streaming/TransformWithStateExec.scala |   3 +-
 .../TransformWithStateInitialStateSuite.scala  | 124 -
 .../sql/streaming/TransformWithStateSuite.scala|  60 ++
 4 files changed, 164 insertions(+), 30 deletions(-)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



(spark) branch master updated: [SPARK-47568][SS] Fix race condition between maintenance thread and load/commit for snapshot files

2024-03-28 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 0b844e52b35b [SPARK-47568][SS] Fix race condition between maintenance 
thread and load/commit for snapshot files
0b844e52b35b is described below

commit 0b844e52b35b0491717ba9f0ae8fe2e0cf45e88d
Author: Bhuwan Sahni 
AuthorDate: Fri Mar 29 13:23:15 2024 +0900

[SPARK-47568][SS] Fix race condition between maintenance thread and 
load/commit for snapshot files

### What changes were proposed in this pull request?

This PR fixes a race condition between the maintenance thread and task 
thread when change-log checkpointing is enabled, and ensure all snapshots are 
valid.

1. The maintenance thread currently relies on class variable lastSnapshot 
to find the latest checkpoint and uploads it to DFS. This checkpoint can be 
modified at commit time by Task thread if a new snapshot is created.
2. The task thread was not resetting the lastSnapshot at load time, which 
can result in newer snapshots (if a old version is loaded) being considered 
valid and uploaded to DFS. This results in VersionIdMismatch errors.

### Why are the changes needed?

These are logical bugs which can cause `VersionIdMismatch` errors causing 
user to discard the snapshot and restart the query.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

Added unit test cases.

### Was this patch authored or co-authored using generative AI tooling?

No

Closes #45724 from sahnib/rocks-db-fix.

Authored-by: Bhuwan Sahni 
Signed-off-by: Jungtaek Lim 
---
 .../sql/execution/streaming/state/RocksDB.scala| 66 ++
 .../streaming/state/RocksDBFileManager.scala   |  3 +-
 .../execution/streaming/state/RocksDBSuite.scala   | 37 
 3 files changed, 82 insertions(+), 24 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala
index 1817104a5c22..fcefc1666f3a 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala
@@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.streaming.state
 
 import java.io.File
 import java.util.Locale
+import java.util.concurrent.TimeUnit
 import javax.annotation.concurrent.GuardedBy
 
 import scala.collection.{mutable, Map}
@@ -49,6 +50,7 @@ case object RollbackStore extends 
RocksDBOpType("rollback_store")
 case object CloseStore extends RocksDBOpType("close_store")
 case object ReportStoreMetrics extends RocksDBOpType("report_store_metrics")
 case object StoreTaskCompletionListener extends 
RocksDBOpType("store_task_completion_listener")
+case object StoreMaintenance extends RocksDBOpType("store_maintenance")
 
 /**
  * Class representing a RocksDB instance that checkpoints version of data to 
DFS.
@@ -184,19 +186,23 @@ class RocksDB(
 loadedVersion = latestSnapshotVersion
 
 // reset last snapshot version
-lastSnapshotVersion = 0L
+if (lastSnapshotVersion > latestSnapshotVersion) {
+  // discard any newer snapshots
+  lastSnapshotVersion = 0L
+  latestSnapshot = None
+}
 openDB()
 
 numKeysOnWritingVersion = if (!conf.trackTotalNumberOfRows) {
-  // we don't track the total number of rows - discard the number 
being track
-  -1L
-} else if (metadata.numKeys < 0) {
-  // we track the total number of rows, but the snapshot doesn't have 
tracking number
-  // need to count keys now
-  countKeys()
-} else {
-  metadata.numKeys
-}
+// we don't track the total number of rows - discard the number 
being track
+-1L
+  } else if (metadata.numKeys < 0) {
+// we track the total number of rows, but the snapshot doesn't 
have tracking number
+// need to count keys now
+countKeys()
+  } else {
+metadata.numKeys
+  }
 if (loadedVersion != version) replayChangelog(version)
 // After changelog replay the numKeysOnWritingVersion will be updated 
to
 // the correct number of keys in the loaded version.
@@ -571,16 +577,14 @@ class RocksDB(
   // background operations.
   val cp = Checkpoint.create(db)
   cp.createCheckpoint(checkpointDir.toString)
-  synchronized {
-// if changelog checkpointing is disabled, the snapshot is 
uploaded synchronous

(spark) branch master updated: [SPARK-47363][SS] Initial State without state reader implementation for State API v2

2024-03-27 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 4d72be3abdc4 [SPARK-47363][SS] Initial State without state reader 
implementation for State API v2
4d72be3abdc4 is described below

commit 4d72be3abdc4c651da029bdbd24a574099d45e7c
Author: jingz-db 
AuthorDate: Thu Mar 28 14:50:46 2024 +0900

[SPARK-47363][SS] Initial State without state reader implementation for 
State API v2

### What changes were proposed in this pull request?

This PR adds support for users to provide a Dataframe that can be used to 
instantiate state for the query in the first batch for arbitrary state API v2.

Note that populating the initial state will only happen for the first batch 
of the new streaming query. Trying to re-initialize state for the same grouping 
key will result in an error.

### Why are the changes needed?

These changes are needed to support initial state. The changes are part of 
the work around adding new stateful streaming operator for arbitrary state mgmt 
that provides a bunch of new features listed in the SPIP JIRA here - 
https://issues.apache.org/jira/browse/SPARK-45939

### Does this PR introduce _any_ user-facing change?

Yes.
This PR introduces a new function:
```
def transformWithState(
  statefulProcessor: StatefulProcessorWithInitialState[K, V, U, S],
  timeoutMode: TimeoutMode,
  outputMode: OutputMode,
  initialState: KeyValueGroupedDataset[K, S]): Dataset[U]
```

### How was this patch tested?

Unit tests in `TransformWithStateWithInitialStateSuite`

### Was this patch authored or co-authored using generative AI tooling?

No

Closes #45467 from jingz-db/initial-state-state-v2.

Lead-authored-by: jingz-db 
Co-authored-by: Jing Zhan <135738831+jingz...@users.noreply.github.com>
Signed-off-by: Jungtaek Lim 
---
 .../src/main/resources/error/error-classes.json|   6 +
 docs/sql-error-conditions.md   |   6 +
 .../spark/sql/streaming/StatefulProcessor.scala|  19 ++
 .../spark/sql/catalyst/plans/logical/object.scala  |  55 +++-
 .../apache/spark/sql/KeyValueGroupedDataset.scala  |  38 ++-
 .../spark/sql/execution/SparkStrategies.scala  |  20 +-
 .../execution/streaming/IncrementalExecution.scala |   4 +-
 .../streaming/TransformWithStateExec.scala | 254 ++
 .../streaming/state/StateStoreErrors.scala |  10 +
 .../sql/streaming/TransformWithMapStateSuite.scala |   5 +-
 .../TransformWithStateInitialStateSuite.scala  | 293 +
 .../sql/streaming/TransformWithStateSuite.scala|  20 ++
 12 files changed, 661 insertions(+), 69 deletions(-)

diff --git a/common/utils/src/main/resources/error/error-classes.json 
b/common/utils/src/main/resources/error/error-classes.json
index 185e86853dfd..11c8204d2c93 100644
--- a/common/utils/src/main/resources/error/error-classes.json
+++ b/common/utils/src/main/resources/error/error-classes.json
@@ -3553,6 +3553,12 @@
 ],
 "sqlState" : "42802"
   },
+  "STATEFUL_PROCESSOR_CANNOT_REINITIALIZE_STATE_ON_KEY" : {
+"message" : [
+  "Cannot re-initialize state on the same grouping key during initial 
state handling for stateful processor. Invalid grouping key=."
+],
+"sqlState" : "42802"
+  },
   "STATE_STORE_CANNOT_CREATE_COLUMN_FAMILY_WITH_RESERVED_CHARS" : {
 "message" : [
   "Failed to create column family with unsupported starting character and 
name=."
diff --git a/docs/sql-error-conditions.md b/docs/sql-error-conditions.md
index 838ca2fa33c9..85b9e85ac420 100644
--- a/docs/sql-error-conditions.md
+++ b/docs/sql-error-conditions.md
@@ -2162,6 +2162,12 @@ Failed to perform stateful processor 
operation=`` with invalid ha
 
 Failed to perform stateful processor operation=`` with invalid 
timeoutMode=``
 
+### STATEFUL_PROCESSOR_CANNOT_REINITIALIZE_STATE_ON_KEY
+
+[SQLSTATE: 
42802](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation)
+
+Cannot re-initialize state on the same grouping key during initial state 
handling for stateful processor. Invalid grouping key=``.
+
 ### STATE_STORE_CANNOT_CREATE_COLUMN_FAMILY_WITH_RESERVED_CHARS
 
 [SQLSTATE: 
42802](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation)
diff --git 
a/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessor.scala 
b/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessor.scala
index ad9b807ddf5a..1a61972f0ed0 100644
--- 
a/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessor.scala
+++ 
b/sql/api/src/main/scala/org/apach

(spark) branch master updated: [SPARK-47107][SS][PYTHON] Implement partition reader for python streaming data source

2024-03-27 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new e3e7135af4df [SPARK-47107][SS][PYTHON] Implement partition reader for 
python streaming data source
e3e7135af4df is described below

commit e3e7135af4df3427f4c61cccfe189f702844e1f5
Author: Chaoqin Li 
AuthorDate: Thu Mar 28 06:33:49 2024 +0900

[SPARK-47107][SS][PYTHON] Implement partition reader for python streaming 
data source

### What changes were proposed in this pull request?
Piggy back the PythonPartitionReaderFactory to implement reading a data 
partition for python streaming data source. Add test case to verify that python 
streaming data source can read and process data end to end.

### Why are the changes needed?
This is part of the effort to support developing streaming data source in 
python interface.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Add integration test to verify data are read and metrics are emitted 
correctly.

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #45485 from chaoqin-li1123/python_stream_read.

Authored-by: Chaoqin Li 
Signed-off-by: Jungtaek Lim 
---
 .../streaming/python_streaming_source_runner.py|   2 +-
 python/pyspark/sql/worker/plan_data_source_read.py |  75 +
 .../v2/python/PythonMicroBatchStream.scala |  16 +-
 .../datasources/v2/python/PythonScan.scala |   3 +-
 .../datasources/v2/python/PythonTable.scala|   4 +-
 .../v2/python/UserDefinedPythonDataSource.scala|  16 +-
 .../spark/sql/streaming/DataStreamReader.scala |   5 +
 .../python/PythonStreamingDataSourceSuite.scala| 182 ++---
 8 files changed, 238 insertions(+), 65 deletions(-)

diff --git a/python/pyspark/sql/streaming/python_streaming_source_runner.py 
b/python/pyspark/sql/streaming/python_streaming_source_runner.py
index 8dbac431a8ba..512191866a16 100644
--- a/python/pyspark/sql/streaming/python_streaming_source_runner.py
+++ b/python/pyspark/sql/streaming/python_streaming_source_runner.py
@@ -141,7 +141,7 @@ def main(infile: IO, outfile: IO) -> None:
 error_msg = "data source {} throw exception: 
{}".format(data_source.name, e)
 raise PySparkRuntimeError(
 error_class="PYTHON_STREAMING_DATA_SOURCE_RUNTIME_ERROR",
-message_parameters={"error": error_msg},
+message_parameters={"msg": error_msg},
 )
 finally:
 reader.stop()
diff --git a/python/pyspark/sql/worker/plan_data_source_read.py 
b/python/pyspark/sql/worker/plan_data_source_read.py
index 8f1fc1e59a61..3e5105996ed4 100644
--- a/python/pyspark/sql/worker/plan_data_source_read.py
+++ b/python/pyspark/sql/worker/plan_data_source_read.py
@@ -25,6 +25,7 @@ from pyspark.accumulators import _accumulatorRegistry
 from pyspark.errors import PySparkAssertionError, PySparkRuntimeError
 from pyspark.java_gateway import local_connect_and_auth
 from pyspark.serializers import (
+read_bool,
 read_int,
 write_int,
 SpecialLengths,
@@ -127,33 +128,14 @@ def main(infile: IO, outfile: IO) -> None:
 f"'{max_arrow_batch_size}'"
 )
 
-# Instantiate data source reader.
-reader = data_source.reader(schema=schema)
+is_streaming = read_bool(infile)
 
-# Get the partitions if any.
-try:
-partitions = reader.partitions()
-if not isinstance(partitions, list):
-raise PySparkRuntimeError(
-error_class="DATA_SOURCE_TYPE_MISMATCH",
-message_parameters={
-"expected": "'partitions' to return a list",
-"actual": f"'{type(partitions).__name__}'",
-},
-)
-if not all(isinstance(p, InputPartition) for p in partitions):
-partition_types = ", ".join([f"'{type(p).__name__}'" for p in 
partitions])
-raise PySparkRuntimeError(
-error_class="DATA_SOURCE_TYPE_MISMATCH",
-message_parameters={
-"expected": "all elements in 'partitions' to be of 
type 'InputPartition'",
-"actual": partition_types,
-},
-)
-if len(partitions) == 0:
-partitions = [None]  # type: ignore
-except NotImplementedError:
-partitions = [None]  # type: ignore
+# Instantiate data source reader.
+reader = (
+   

(spark) branch master updated: [SPARK-47570][SS] Integrate range scan encoder changes with timer implementation

2024-03-26 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 88b29c5076d4 [SPARK-47570][SS] Integrate range scan encoder changes 
with timer implementation
88b29c5076d4 is described below

commit 88b29c5076d48f4ecbed402a693a8ccce57cd7d0
Author: jingz-db 
AuthorDate: Wed Mar 27 13:37:48 2024 +0900

[SPARK-47570][SS] Integrate range scan encoder changes with timer 
implementation

### What changes were proposed in this pull request?

Previously timer state implementation was using No prefix rocksdb state 
encoder. When doing `iterator()` or `prefix()`, the returned iterator is not 
sorted on timestamp value. After Anish's PR for supporting range scan encoder, 
we could integrate it with `TimerStateImpl` such that we will use range scan 
encoder on `timer to key`.

### Why are the changes needed?

The changes are part of the work around adding new stateful streaming 
operator for arbitrary state mgmt that provides a bunch of new features listed 
in the SPIP JIRA here - https://issues.apache.org/jira/browse/SPARK-45939

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

Added unit tests in `TimerSuite`

### Was this patch authored or co-authored using generative AI tooling?

No

Closes #45709 from jingz-db/integrate-range-scan.

Authored-by: jingz-db 
Signed-off-by: Jungtaek Lim 
---
 .../streaming/StatefulProcessorHandleImpl.scala|  8 ++-
 .../sql/execution/streaming/TimerStateImpl.scala   | 19 --
 .../streaming/TransformWithStateExec.scala | 16 ++---
 .../sql/execution/streaming/state/TimerSuite.scala | 69 +++---
 4 files changed, 85 insertions(+), 27 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala
index 9b905ad5235d..5f3b794fd117 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala
@@ -163,12 +163,14 @@ class StatefulProcessorHandleImpl(
   }
 
   /**
-   * Function to retrieve all registered timers for all grouping keys
+   * Function to retrieve all expired registered timers for all grouping keys
+   * @param expiryTimestampMs Threshold for expired timestamp in milliseconds, 
this function
+   *  will return all timers that have timestamp less 
than passed threshold
* @return - iterator of registered timers for all grouping keys
*/
-  def getExpiredTimers(): Iterator[(Any, Long)] = {
+  def getExpiredTimers(expiryTimestampMs: Long): Iterator[(Any, Long)] = {
 verifyTimerOperations("get_expired_timers")
-timerState.getExpiredTimers()
+timerState.getExpiredTimers(expiryTimestampMs)
   }
 
   /**
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TimerStateImpl.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TimerStateImpl.scala
index 6166374d25e9..af321eecb4db 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TimerStateImpl.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TimerStateImpl.scala
@@ -91,7 +91,7 @@ class TimerStateImpl(
 
   val tsToKeyCFName = timerCFName + TimerStateUtils.TIMESTAMP_TO_KEY_CF
   store.createColFamilyIfAbsent(tsToKeyCFName, keySchemaForSecIndex,
-schemaForValueRow, NoPrefixKeyStateEncoderSpec(keySchemaForSecIndex),
+schemaForValueRow, RangeKeyScanStateEncoderSpec(keySchemaForSecIndex, 1),
 useMultipleValuesPerKey = false, isInternal = true)
 
   private def getGroupingKey(cfName: String): Any = {
@@ -110,7 +110,6 @@ class TimerStateImpl(
 
   // We maintain a secondary index that inverts the ordering of the timestamp
   // and grouping key
-  // TODO: use range scan encoder to encode the secondary index key
   private def encodeSecIndexKey(groupingKey: Any, expiryTimestampMs: Long): 
UnsafeRow = {
 val keyByteArr = 
keySerializer.apply(groupingKey).asInstanceOf[UnsafeRow].getBytes()
 val keyRow = secIndexKeyEncoder(InternalRow(expiryTimestampMs, keyByteArr))
@@ -187,10 +186,15 @@ class TimerStateImpl(
   }
 
   /**
-   * Function to get all the registered timers for all grouping keys
+   * Function to get all the expired registered timers for all grouping keys.
+   * Perform a range scan on timestamp and will stop iterating once the key 
row timestamp equals or
+   * exceeds the limit (as timestamp key is increasingly sorted).
+   * @param expiryTimestampMs Threshold for expired timestamp in mi

(spark) branch master updated: [SPARK-47273][SS][PYTHON] implement Python data stream writer interface

2024-03-26 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 5ac39181fe87 [SPARK-47273][SS][PYTHON] implement Python data stream 
writer interface
5ac39181fe87 is described below

commit 5ac39181fe87aba4eab66ff2590bbc16349c0bab
Author: Chaoqin Li 
AuthorDate: Wed Mar 27 12:51:36 2024 +0900

[SPARK-47273][SS][PYTHON] implement Python data stream writer interface

### What changes were proposed in this pull request?
Reuse PythonPartitionWriter to implement the serialization and execution of 
write callback in executor.

Implement python worker process to run python streaming data sink committer 
and communicate with JVM through socket in spark driver. For each python 
streaming data sink instance there will be a long live python worker process 
created. Inside the python process, the python write committer will receive 
abort or commit function call and send back result through socket.

### Why are the changes needed?
In order to support developing spark streaming sink in python, we need to 
implement python stream writer interface.

### Does this PR introduce _any_ user-facing change?
No.

### How was this patch tested?
Unit and integration test.

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #45305 from chaoqin-li1123/python_stream_writer.

Authored-by: Chaoqin Li 
Signed-off-by: Jungtaek Lim 
---
 python/pyspark/sql/datasource.py   |  94 
 .../sql/worker/python_streaming_sink_runner.py | 140 +++
 .../pyspark/sql/worker/write_into_data_source.py   |  10 +-
 .../python/PythonStreamingSinkCommitRunner.scala   | 133 +++
 .../v2/python/PythonStreamingWrite.scala   |  84 +++
 .../datasources/v2/python/PythonTable.scala|   4 +-
 .../datasources/v2/python/PythonWrite.scala|  12 +-
 .../v2/python/UserDefinedPythonDataSource.scala|  11 +-
 .../spark/sql/streaming/DataStreamWriter.scala |   5 +
 .../python/PythonStreamingDataSourceSuite.scala| 261 -
 10 files changed, 744 insertions(+), 10 deletions(-)

diff --git a/python/pyspark/sql/datasource.py b/python/pyspark/sql/datasource.py
index 803765e83093..c08b5b7af77f 100644
--- a/python/pyspark/sql/datasource.py
+++ b/python/pyspark/sql/datasource.py
@@ -160,6 +160,29 @@ class DataSource(ABC):
 message_parameters={"feature": "writer"},
 )
 
+def streamWriter(self, schema: StructType, overwrite: bool) -> 
"DataSourceStreamWriter":
+"""
+Returns a :class:`DataSourceStreamWriter` instance for writing data 
into a streaming sink.
+
+The implementation is required for writable streaming data sources.
+
+Parameters
+--
+schema : :class:`StructType`
+The schema of the data to be written.
+overwrite : bool
+A flag indicating whether to overwrite existing data when writing 
current microbatch.
+
+Returns
+---
+writer : :class:`DataSourceStreamWriter`
+A writer instance for writing data into a streaming sink.
+"""
+raise PySparkNotImplementedError(
+error_class="NOT_IMPLEMENTED",
+message_parameters={"feature": "streamWriter"},
+)
+
 def streamReader(self, schema: StructType) -> "DataSourceStreamReader":
 """
 Returns a :class:`DataSourceStreamReader` instance for reading 
streaming data.
@@ -513,6 +536,77 @@ class DataSourceWriter(ABC):
 ...
 
 
+class DataSourceStreamWriter(ABC):
+"""
+A base class for data stream writers. Data stream writers are responsible 
for writing
+the data to the streaming sink.
+
+.. versionadded: 4.0.0
+"""
+
+@abstractmethod
+def write(self, iterator: Iterator[Row]) -> "WriterCommitMessage":
+"""
+Writes data into the streaming sink.
+
+This method is called on executors to write data to the streaming data 
sink in
+each microbatch. It accepts an iterator of input data and returns a 
single row
+representing a commit message, or None if there is no commit message.
+
+The driver collects commit messages, if any, from all executors and 
passes them
+to the ``commit`` method if all tasks run successfully. If any task 
fails, the
+``abort`` method will be called with the collected commit messages.
+
+Parameters
+--
+iterator : Iterator[Row]
+An iterator of input data.
+
+Returns
+-

(spark) branch master updated: [SPARK-47469][SS][TESTS] Add `Trigger.AvailableNow` tests for `transformWithState` operator

2024-03-26 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 3e8343205c4d [SPARK-47469][SS][TESTS] Add `Trigger.AvailableNow` tests 
for `transformWithState` operator
3e8343205c4d is described below

commit 3e8343205c4d434076c013acd14cbfd8736241d4
Author: jingz-db 
AuthorDate: Tue Mar 26 18:24:13 2024 +0900

[SPARK-47469][SS][TESTS] Add `Trigger.AvailableNow` tests for 
`transformWithState` operator

### What changes were proposed in this pull request?

Add tests for AvailableNow for TransformWithState operator.
### Why are the changes needed?

Compliance with state-v2 test plan.

### Does this PR introduce _any_ user-facing change?

No.
### How was this patch tested?

Added unit test suites.

### Was this patch authored or co-authored using generative AI tooling?

No.

Closes #45596 from jingz-db/avaiNow-tests-state-v2.

Authored-by: jingz-db 
Signed-off-by: Jungtaek Lim 
---
 .../sql/streaming/TransformWithStateSuite.scala| 101 -
 1 file changed, 100 insertions(+), 1 deletion(-)

diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala
index 0fd2ef055ffc..24b0d59c45c5 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala
@@ -17,9 +17,13 @@
 
 package org.apache.spark.sql.streaming
 
+import java.io.File
+import java.util.UUID
+
 import org.apache.spark.SparkRuntimeException
 import org.apache.spark.internal.Logging
-import org.apache.spark.sql.Encoders
+import org.apache.spark.sql.{Dataset, Encoders}
+import org.apache.spark.sql.catalyst.util.stringToFile
 import org.apache.spark.sql.execution.streaming._
 import 
org.apache.spark.sql.execution.streaming.state.{AlsoTestWithChangelogCheckpointingEnabled,
 RocksDBStateStoreProvider, 
StatefulProcessorCannotPerformOperationWithInvalidHandleState, 
StateStoreMultipleColumnFamiliesNotSupportedException}
 import org.apache.spark.sql.functions.timestamp_seconds
@@ -650,6 +654,101 @@ class TransformWithStateSuite extends 
StateStoreMetricsTest
   )
 }
   }
+
+  /** Create a text file with a single data item */
+  private def createFile(data: String, srcDir: File): File =
+stringToFile(new File(srcDir, s"${UUID.randomUUID()}.txt"), data)
+
+  private def createFileStream(srcDir: File): Dataset[(String, String)] = {
+spark
+  .readStream
+  .option("maxFilesPerTrigger", "1")
+  .text(srcDir.getCanonicalPath)
+  .select("value").as[String]
+  .groupByKey(x => x)
+  .transformWithState(new RunningCountStatefulProcessor(),
+TimeoutMode.NoTimeouts(),
+OutputMode.Update())
+  }
+
+  test("transformWithState - availableNow trigger mode, rate limit is 
respected") {
+withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+  classOf[RocksDBStateStoreProvider].getName) {
+  withTempDir { srcDir =>
+
+Seq("a", "b", "c").foreach(createFile(_, srcDir))
+
+// Set up a query to read text files one at a time
+val df = createFileStream(srcDir)
+
+testStream(df)(
+  StartStream(trigger = Trigger.AvailableNow()),
+  ProcessAllAvailable(),
+  CheckNewAnswer(("a", "1"), ("b", "1"), ("c", "1")),
+  StopStream,
+  Execute { _ =>
+createFile("a", srcDir)
+  },
+  StartStream(trigger = Trigger.AvailableNow()),
+  ProcessAllAvailable(),
+  CheckNewAnswer(("a", "2"))
+)
+
+var index = 0
+val foreachBatchDf = df.writeStream
+  .foreachBatch((_: Dataset[(String, String)], _: Long) => {
+index += 1
+  })
+  .trigger(Trigger.AvailableNow())
+  .start()
+
+try {
+  foreachBatchDf.awaitTermination()
+  assert(index == 4)
+} finally {
+  foreachBatchDf.stop()
+}
+  }
+}
+  }
+
+  test("transformWithState - availableNow trigger mode, multiple restarts") {
+withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+  classOf[RocksDBStateStoreProvider].getName) {
+  withTempDir { srcDir =>
+Seq("a", "b", "c").foreach(createFile(_, srcDir))
+val df = createFileStream(srcDir)
+
+var index = 0
+
+def startTriggerAvailableNowQueryAndCheck(expectedIdx: Int): Unit =

(spark) branch master updated: [SPARK-47372][SS] Add support for range scan based key state encoder for use with state store provider

2024-03-25 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new ff38378d7e42 [SPARK-47372][SS] Add support for range scan based key 
state encoder for use with state store provider
ff38378d7e42 is described below

commit ff38378d7e425c3e810e6556a3916f4594f2aec0
Author: Anish Shrigondekar 
AuthorDate: Tue Mar 26 13:48:40 2024 +0900

[SPARK-47372][SS] Add support for range scan based key state encoder for 
use with state store provider

### What changes were proposed in this pull request?
Add support for range scan based key state encoder for use with state store 
provider

### Why are the changes needed?
Changes are needed to allow range scan of fixed size initial cols 
especially with RocksDB state store provider. Earlier we had tried to use the 
existing key state encoder to encapsulate state for ordering columns using 
BIG_ENDIAN encoding. However, that model does not work with variable sized 
non-ordering columns part of the grouping key, since in that case the variable 
length portion gets encoded upfront in the `UnsafeRow` thereby breaking the 
range scan/sorting functionality. In ord [...]
- create the state store instance or any column family with a key schema 
and the `RangeScanKeyStateEncoder` encoder type and the num of ordering cols 
being specified
- with the new encoder, user can perform range scan in sorted order using 
ordering cols
- with the new encoder, user can perform prefix scan using ordering cols
- ordering cols have to be > 0 and <= num_key_schema_cols which is a more 
lenient requirement than `PrefixScanKeyStateEncoder`
- only fixed size (primitive type) ordering cols can be used with this 
encoder

Internally, we will convert the passed `UnsafeRow` into a new one that has 
the BIG_ENDIAN encoded byte arrays that are eventually written out to RocksDB.

Note that the existing - `NoPrefixKeyStateEncoder` and 
`PrefixScanKeyStateEncoder` will also continue to be supported. The user can 
decide which encoder best fits the need of their store/col family schema/use.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Added unit tests

```
...
[info] - rocksdb range scan validation - variable sized columns - with 
colFamiliesEnabled=true (without changelog checkpointing) (1 millisecond)
[info] - rocksdb range scan validation - variable sized columns - with 
colFamiliesEnabled=false (with changelog checkpointing) (1 millisecond)
[info] - rocksdb range scan validation - variable sized columns - with 
colFamiliesEnabled=false (without changelog checkpointing) (1 millisecond)
[info] - rocksdb range scan - fixed size non-ordering columns - with 
colFamiliesEnabled=true (with changelog checkpointing) (782 milliseconds)
[info] - rocksdb range scan - fixed size non-ordering columns - with 
colFamiliesEnabled=true (without changelog checkpointing) (125 milliseconds)
[info] - rocksdb range scan - fixed size non-ordering columns - with 
colFamiliesEnabled=false (with changelog checkpointing) (177 milliseconds)
[info] - rocksdb range scan - fixed size non-ordering columns - with 
colFamiliesEnabled=false (without changelog checkpointing) (89 milliseconds)
[info] - rocksdb range scan - variable size non-ordering columns - with 
colFamiliesEnabled=true (with changelog checkpointing) (192 milliseconds)
[info] - rocksdb range scan - variable size non-ordering columns - with 
colFamiliesEnabled=true (without changelog checkpointing) (107 milliseconds)
[info] - rocksdb range scan - variable size non-ordering columns - with 
colFamiliesEnabled=false (with changelog checkpointing) (185 milliseconds)
[info] - rocksdb range scan - variable size non-ordering columns - with 
colFamiliesEnabled=false (without changelog checkpointing) (96 milliseconds)
[info] - rocksdb range scan - ordering cols and key schema cols are same - 
with colFamiliesEnabled=true (with changelog checkpointing) (195 milliseconds)
[info] - rocksdb range scan - ordering cols and key schema cols are same - 
with colFamiliesEnabled=true (without changelog checkpointing) (106 
milliseconds)
[info] - rocksdb range scan - ordering cols and key schema cols are same - 
with colFamiliesEnabled=false (with changelog checkpointing) (161 milliseconds)
[info] - rocksdb range scan - ordering cols and key schema cols are same - 
with colFamiliesEnabled=false (without changelog checkpointing) (88 
milliseconds)
[info] - rocksdb range scan - with prefix scan - with 
colFamiliesEnabled=true (with changelog checkpointing) (169 milliseconds)
[info] - rocksdb range scan - with prefix scan - with 
colFamiliesEnabled=true (without changelog checkpointing) (105 millis

(spark) branch master updated: [SPARK-47512][SS] Tag operation type used with RocksDB state store instance lock acquisition/release

2024-03-21 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 057acf986d78 [SPARK-47512][SS] Tag operation type used with RocksDB 
state store instance lock acquisition/release
057acf986d78 is described below

commit 057acf986d7877fbb1e8b435e7417330ab18b81e
Author: Anish Shrigondekar 
AuthorDate: Fri Mar 22 11:32:10 2024 +0900

[SPARK-47512][SS] Tag operation type used with RocksDB state store instance 
lock acquisition/release

### What changes were proposed in this pull request?
Tag operation type used with RocksDB state store instance lock 
acquisition/release

### Why are the changes needed?
Currently its impossible to tell which operation/thread tried to 
acquire/release the RocksDB instance lock which is crucial in debugging issues 
around RocksDB instance lock acquisition

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Added unit tests

```
[info] RocksDBSuite:
12:34:41.365 WARN org.apache.hadoop.util.NativeCodeLoader: Unable to load 
native-hadoop library for your platform... using builtin-java classes where 
applicable
[info] - disallow concurrent updates to the same RocksDB instance - with 
colFamiliesEnabled=true (with changelog checkpointing) (1 second, 634 
milliseconds)
[info] - disallow concurrent updates to the same RocksDB instance - with 
colFamiliesEnabled=true (without changelog checkpointing) (654 milliseconds)
[info] - disallow concurrent updates to the same RocksDB instance - with 
colFamiliesEnabled=false (with changelog checkpointing) (609 milliseconds)
[info] - disallow concurrent updates to the same RocksDB instance - with 
colFamiliesEnabled=false (without changelog checkpointing) (584 milliseconds)
12:34:45.319 WARN 
org.apache.spark.sql.execution.streaming.state.RocksDBSuite:
```

Verified updated logs
```
sql/core/target/unit-tests.log:1958:13:38:41.698 concurrent-test-thread-3 
INFO RocksDB [Thread-17]: RocksDB instance was acquired by [ThreadId: 
Some(158)] for opType=load_store
sql/core/target/unit-tests.log:2235:13:38:41.886 concurrent-test-thread-3 
INFO RocksDB [Thread-17]: RocksDB instance was released by [ThreadId: 
Some(158)] for opType=load_store
sql/core/target/unit-tests.log:2236:13:38:41.886 
pool-1-thread-1-ScalaTest-running-RocksDBSuite INFO RocksDB [Thread-17]: 
RocksDB instance was acquired by [ThreadId: Some(17)] for opType=close_store
sql/core/target/unit-tests.log:2239:13:38:41.889 
pool-1-thread-1-ScalaTest-running-RocksDBSuite INFO RocksDB [Thread-17]: 
RocksDB instance was released by [ThreadId: Some(17)] for opType=close_store
sql/core/target/unit-tests.log:2249:13:38:41.896 
pool-1-thread-1-ScalaTest-running-RocksDBSuite INFO RocksDB [Thread-17]: 
RocksDB instance was acquired by [ThreadId: Some(17)] for opType=load_store
sql/core/target/unit-tests.log:2518:13:38:41.950 
pool-1-thread-1-ScalaTest-running-RocksDBSuite INFO RocksDB [Thread-17]: 
RocksDB instance was acquired by [ThreadId: Some(17)] for opType=load_store
sql/core/target/unit-tests.log:2562:13:38:42.139 
pool-1-thread-1-ScalaTest-running-RocksDBSuite INFO RocksDB [Thread-17]: 
RocksDB instance was released by [ThreadId: Some(17)] for opType=load_store
sql/core/target/unit-tests.log:2563:13:38:42.139 concurrent-test-thread-2 
INFO RocksDB [Thread-17]: RocksDB instance was acquired by [ThreadId: 
Some(181)] for opType=load_store
sql/core/target/unit-tests.log:2607:13:38:42.293 concurrent-test-thread-2 
INFO RocksDB [Thread-17]: RocksDB instance was released by [ThreadId: 
Some(181)] for opType=load_store
sql/core/target/unit-tests.log:2608:13:38:42.293 
pool-1-thread-1-ScalaTest-running-RocksDBSuite INFO RocksDB [Thread-17]: 
RocksDB instance was acquired by [ThreadId: Some(17)] for opType=load_store
sql/core/target/unit-tests.log:2611:13:38:42.319 
pool-1-thread-1-ScalaTest-running-RocksDBSuite INFO RocksDB [Thread-17]: 
RocksDB instance was acquired by [ThreadId: Some(17)] for opType=rollback_store
sql/core/target/unit-tests.log:2612:13:38:42.319 
pool-1-thread-1-ScalaTest-running-RocksDBSuite INFO RocksDB [Thread-17]: 
RocksDB instance was released by [ThreadId: Some(17)] for opType=rollback_store
sql/core/target/unit-tests.log:2614:13:38:42.319 concurrent-test-thread-3 
INFO RocksDB [Thread-17]: RocksDB instance was acquired by [ThreadId: 
Some(193)] for opType=load_store
sql/core/target/unit-tests.log:2942:13:38:42.525 concurrent-test-thread-3 
INFO RocksDB [Thread-17]: RocksDB instance was released by [ThreadId: 
Some(193)] for opType=load_store
```

### Was this patch authored or co-authored using generative AI tooling?
No

Closes #45651 from anishshri-db/task/SPARK-47512

(spark) branch master updated: [SPARK-47449][SS] Refactor and split list/timer unit tests

2024-03-19 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new f0061dbe856a [SPARK-47449][SS] Refactor and split list/timer unit tests
f0061dbe856a is described below

commit f0061dbe856a55295cc95835aff5dc717aa19431
Author: jingz-db 
AuthorDate: Wed Mar 20 09:21:04 2024 +0900

[SPARK-47449][SS] Refactor and split list/timer unit tests

### What changes were proposed in this pull request?

Refactor StatefulProcessorHandle unit test suites. Add List state and timer 
state unit tests.
As planned in test plan for state-v2, list/timer should be tested in both 
integration and unit tests. Currently StatefulProcessorHandle related tests 
could be refactored to use base suite class in `ValueStateSuite`, and 
list/timer state unit tests are needed in addition to integration tests.

### Why are the changes needed?

Compliance with test plan for state-v2 project.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

Test suites refactored and added.

### Was this patch authored or co-authored using generative AI tooling?

No

Closes #45573 from jingz-db/split-timer-list-state-v2.

Authored-by: jingz-db 
Signed-off-by: Jungtaek Lim 
---
 .../execution/streaming/state/ListStateSuite.scala | 163 +
 .../execution/streaming/state/MapStateSuite.scala  |   2 +-
 .../state/StatefulProcessorHandleSuite.scala   |  69 +
 .../sql/execution/streaming/state/TimerSuite.scala | 113 ++
 .../streaming/state/ValueStateSuite.scala  |   8 +-
 5 files changed, 289 insertions(+), 66 deletions(-)

diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ListStateSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ListStateSuite.scala
new file mode 100644
index ..e895e475b74d
--- /dev/null
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ListStateSuite.scala
@@ -0,0 +1,163 @@
+/*
+ * 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.streaming.state
+
+import java.util.UUID
+
+import org.apache.spark.SparkIllegalArgumentException
+import org.apache.spark.sql.Encoders
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
+import org.apache.spark.sql.execution.streaming.{ImplicitGroupingKeyTracker, 
StatefulProcessorHandleImpl}
+import org.apache.spark.sql.streaming.{ListState, TimeoutMode, ValueState}
+
+/**
+ * Class that adds unit tests for ListState types used in arbitrary stateful
+ * operators such as transformWithState
+ */
+class ListStateSuite extends StateVariableSuiteBase {
+  // overwrite useMultipleValuesPerKey in base suite to be true for list state
+  override def useMultipleValuesPerKey: Boolean = true
+
+  private def testMapStateWithNullUserKey()(runListOps: ListState[Long] => 
Unit): Unit = {
+tryWithProviderResource(newStoreProviderWithStateVariable(true)) { 
provider =>
+  val store = provider.getStore(0)
+  val handle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(),
+Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]], 
TimeoutMode.NoTimeouts())
+
+  val listState: ListState[Long] = handle.getListState[Long]("listState", 
Encoders.scalaLong)
+
+  ImplicitGroupingKeyTracker.setImplicitKey("test_key")
+  val e = intercept[SparkIllegalArgumentException] {
+runListOps(listState)
+  }
+
+  checkError(
+exception = e.asInstanceOf[SparkIllegalArgumentException],
+errorClass = "ILLEGAL_STATE_STORE_VALUE.NULL_VALUE",
+sqlState = Some("42601"),
+parameters = Map("stateName" -> "listState")
+  )
+}
+  }
+
+  Seq("appendList", "put").foreach { listImplFunc =>
+test(s"Test list operation($listImplFunc) with null") {
+  testMapSta

(spark) branch master updated (acf17fd67217 -> 9f8147c2a8d2)

2024-03-18 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


from acf17fd67217 [SPARK-47450][INFRA][R] Use R 4.3.3 in `windows` R GitHub 
Action job
 add 9f8147c2a8d2 [SPARK-47329][SS][DOCS] Add note to persist dataframe 
while using foreachbatch and stateful streaming query to prevent state from 
being re-loaded in each batch

No new revisions were added by this update.

Summary of changes:
 docs/structured-streaming-programming-guide.md |  6 +-
 .../sql/execution/streaming/sources/ForeachBatchSink.scala | 10 ++
 2 files changed, 15 insertions(+), 1 deletion(-)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



(spark) branch master updated: [SPARK-46913][SS] Add support for processing/event time based timers with transformWithState operator

2024-03-13 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 839bc9f9c264 [SPARK-46913][SS] Add support for processing/event time 
based timers with transformWithState operator
839bc9f9c264 is described below

commit 839bc9f9c264671aa75795d714558c61bd6f64b0
Author: Anish Shrigondekar 
AuthorDate: Thu Mar 14 05:27:28 2024 +0900

[SPARK-46913][SS] Add support for processing/event time based timers with 
transformWithState operator

### What changes were proposed in this pull request?
Add support for processing/event time based timers with 
`transformWithState` operator

### Why are the changes needed?
Changes are required to add event-driven timer based support for stateful 
streaming applications based on arbitrary state  API with the 
`transformWithState` operator

As part of this change - we introduce a bunch of functions that users can 
use within the `StatefulProcessor` logic. Using the `StatefulProcessorHandle`, 
users can do the following:
- register timer at a given timestamp
- delete timer at a given timestamp
- list timers

Note that all the above operations are tied to the implicit grouping key.

In terms of the implementation, we make use of additional column families 
to support the operations mentioned above. For registered timers, we maintain a 
primary index (as a col family) that keeps the mapping between the grouping key 
and expiry timestamp. This col family is used to add and delete timers with 
direct access to the key and also for listing registered timers for a given 
grouping key using `prefix scan`. We also maintain a secondary index that 
inverts the ordering of the t [...]

Few additional constraints:
- only registered timers are tracked and occupy storage (locally and 
remotely)
- col families starting with `_` are reserved and cannot be used as state 
variables
- timers are checkpointed as before
- users have to provide a `timeoutMode` to the operator. Currently, they 
can choose to not register timeouts or register timeouts that are 
processing-time based or event-time based. However, this mode has to be 
declared upfront within the operator arguments.

### Does this PR introduce _any_ user-facing change?
Yes

### How was this patch tested?
Added unit tests as well as pseudo-integration tests

StatefulProcessorHandleSuite
```
13:58:42.463 WARN 
org.apache.spark.sql.execution.streaming.state.StatefulProcessorHandleSuite:

= POSSIBLE THREAD LEAK IN SUITE 
o.a.s.sql.execution.streaming.state.StatefulProcessorHandleSuite, threads: 
rpc-boss-3-1 (daemon=true), shuffle-boss-6-1 (daemon=true) =
[info] Run completed in 4 seconds, 559 milliseconds.
[info] Total number of tests run: 8
[info] Suites: completed 1, aborted 0
[info] Tests: succeeded 8, failed 0, canceled 0, ignored 0, pending 0
[info] All tests passed.
```

TransformWithStateSuite
```
13:48:41.858 WARN org.apache.spark.sql.streaming.TransformWithStateSuite:

= POSSIBLE THREAD LEAK IN SUITE 
o.a.s.sql.streaming.TransformWithStateSuite, threads: QueryStageCreator-0 
(daemon=true), state-store-maintenance-thread-0 (daemon=true), 
ForkJoinPool.commonPool-worker-4 (daemon=true), 
state-store-maintenance-thread-1 (daemon=true), QueryStageCreator-1 
(daemon=true), rpc-boss-3-1 (daemon=true), F
orkJoinPool.commonPool-worker-3 (daemon=true), QueryStageCreator-2 
(daemon=true), QueryStageCreator-3 (daemon=true), state-store-maintenance-task 
(daemon=true), ForkJoinPool.com...
[info] Run completed in 1 minute, 32 seconds.
[info] Total number of tests run: 20
[info] Suites: completed 1, aborted 0
[info] Tests: succeeded 20, failed 0, canceled 0, ignored 0, pending 0
[info] All tests passed.
```

### Was this patch authored or co-authored using generative AI tooling?
No

Closes #45051 from anishshri-db/task/SPARK-46913.

Authored-by: Anish Shrigondekar 
Signed-off-by: Jungtaek Lim 
---
 .../src/main/resources/error/error-classes.json|  18 ++
 docs/sql-error-conditions.md   |  18 ++
 .../apache/spark/sql/streaming/TimeoutMode.java|  14 +
 .../logical/TransformWithStateTimeoutModes.scala   |   4 +-
 .../spark/sql/streaming/ExpiredTimerInfo.scala}|  25 +-
 .../spark/sql/streaming/StatefulProcessor.scala|  10 +-
 .../sql/streaming/StatefulProcessorHandle.scala|  23 ++
 .../streaming/ExpiredTimerInfoImpl.scala}  |  31 +-
 .../streaming/StatefulProcessorHandleImpl.scala|  80 -
 .../sql/execution/streaming/TimerStateImpl.scala   | 214 +
 .../streaming/TransformWithStateExec.scala | 123 +++-
 .../state

(spark) branch master updated: [SPARK-47272][SS] Add MapState implementation for State API v2.

2024-03-12 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 29e91d0e48a3 [SPARK-47272][SS] Add MapState implementation for State 
API v2.
29e91d0e48a3 is described below

commit 29e91d0e48a342ceae1259a9b0d10cb27244a14a
Author: jingz-db 
AuthorDate: Wed Mar 13 07:18:25 2024 +0900

[SPARK-47272][SS] Add MapState implementation for State API v2.

### What changes were proposed in this pull request?

This PR adds changes for MapState implementation in State Api v2. This 
implementation adds a new encoder/decoder to encode grouping key and user key 
into a composite key to be put into RocksDB so that we could retrieve key-value 
pair by user specified user key by one rocksdb get.

### Why are the changes needed?

These changes are needed to support map values in the State Store. The 
changes are part of the work around adding new stateful streaming operator for 
arbitrary state mgmt that provides a bunch of new features listed in the SPIP 
JIRA here - https://issues.apache.org/jira/browse/SPARK-45939

### Does this PR introduce _any_ user-facing change?

Yes
This PR introduces a new state type (MapState) that users can use in their 
Spark streaming queries.

### How was this patch tested?

Unit tests in `TransforWithMapStateSuite`.

### Was this patch authored or co-authored using generative AI tooling?

No

Closes #45341 from jingz-db/map-state-impl.

Lead-authored-by: jingz-db 
Co-authored-by: Jing Zhan <135738831+jingz...@users.noreply.github.com>
Signed-off-by: Jungtaek Lim 
---
 .../org/apache/spark/sql/streaming/MapState.scala  |  54 +
 .../sql/streaming/StatefulProcessorHandle.scala|  16 ++
 .../sql/execution/streaming/MapStateImpl.scala | 110 ++
 .../streaming/StateTypesEncoderUtils.scala |  53 -
 .../streaming/StatefulProcessorHandleImpl.scala|  12 +-
 .../execution/streaming/state/MapStateSuite.scala  | 170 
 .../streaming/state/ValueStateSuite.scala  | 117 ++-
 .../sql/streaming/TransformWithMapStateSuite.scala | 226 +
 8 files changed, 701 insertions(+), 57 deletions(-)

diff --git 
a/sql/api/src/main/scala/org/apache/spark/sql/streaming/MapState.scala 
b/sql/api/src/main/scala/org/apache/spark/sql/streaming/MapState.scala
new file mode 100644
index ..030c3ee989c6
--- /dev/null
+++ b/sql/api/src/main/scala/org/apache/spark/sql/streaming/MapState.scala
@@ -0,0 +1,54 @@
+/*
+ * 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.streaming
+
+import org.apache.spark.annotation.{Evolving, Experimental}
+
+@Experimental
+@Evolving
+/**
+ * Interface used for arbitrary stateful operations with the v2 API to capture
+ * map value state.
+ */
+trait MapState[K, V] extends Serializable {
+  /** Whether state exists or not. */
+  def exists(): Boolean
+
+  /** Get the state value if it exists */
+  def getValue(key: K): V
+
+  /** Check if the user key is contained in the map */
+  def containsKey(key: K): Boolean
+
+  /** Update value for given user key */
+  def updateValue(key: K, value: V) : Unit
+
+  /** Get the map associated with grouping key */
+  def iterator(): Iterator[(K, V)]
+
+  /** Get the list of keys present in map associated with grouping key */
+  def keys(): Iterator[K]
+
+  /** Get the list of values present in map associated with grouping key */
+  def values(): Iterator[V]
+
+  /** Remove user key from map state */
+  def removeKey(key: K): Unit
+
+  /** Remove this state. */
+  def clear(): Unit
+}
diff --git 
a/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala
 
b/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala
index 86bf1e85f90c..c26d0d806b86 100644
--- 
a/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala
+++ 
b/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHa

(spark) branch master updated: [SPARK-47250][SS] Add additional validations and NERF changes for RocksDB state provider and use of column families

2024-03-11 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new e778ce689dcb [SPARK-47250][SS] Add additional validations and NERF 
changes for RocksDB state provider and use of column families
e778ce689dcb is described below

commit e778ce689dcbe5e75ce5781a03cf9d8466910cd2
Author: Anish Shrigondekar 
AuthorDate: Tue Mar 12 14:27:50 2024 +0900

[SPARK-47250][SS] Add additional validations and NERF changes for RocksDB 
state provider and use of column families

### What changes were proposed in this pull request?
Add additional validations and NERF changes for RocksDB state provider and 
use of col families

### Why are the changes needed?
Improve error handling and migrating errors to NERF.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Added new unit tests
StateStoreSuite
```
= POSSIBLE THREAD LEAK IN SUITE 
o.a.s.sql.execution.streaming.state.StateStoreSuite, threads: shuffle-boss-36-1 
(daemon=true), ForkJoinPool.commonPool-worker-1 (daemon=true), 
ForkJoinPool.commonPool-worker-3 (daemon=true), rpc-boss-33-1 (daemon=true), 
ForkJoinPool.commonPool-worker-2 (daemon=true) =
[info] Run completed in 2 minutes, 57 seconds.
[info] Total number of tests run: 151
[info] Suites: completed 1, aborted 0
[info] Tests: succeeded 151, failed 0, canceled 0, ignored 0, pending 0
[info] All tests passed.
```

RocksDBSuite
```
[info] Run completed in 4 minutes, 54 seconds.
[info] Total number of tests run: 188
[info] Suites: completed 1, aborted 0
[info] Tests: succeeded 188, failed 0, canceled 0, ignored 0, pending 0
[info] All tests passed.
```

### Was this patch authored or co-authored using generative AI tooling?
No

Closes #45360 from anishshri-db/task/SPARK-47250.

Authored-by: Anish Shrigondekar 
Signed-off-by: Jungtaek Lim 
---
 .../src/main/resources/error/error-classes.json|  10 +-
 docs/sql-error-conditions.md   |  10 +-
 .../state/HDFSBackedStateStoreProvider.scala   |  33 --
 .../sql/execution/streaming/state/RocksDB.scala|  92 +++
 .../state/RocksDBStateStoreProvider.scala  |   4 +-
 .../streaming/state/StateStoreErrors.scala |  49 
 .../execution/streaming/state/RocksDBSuite.scala   | 124 +++--
 .../streaming/state/StateStoreSuite.scala  |  63 +++
 .../streaming/state/ValueStateSuite.scala  |   2 +-
 9 files changed, 318 insertions(+), 69 deletions(-)

diff --git a/common/utils/src/main/resources/error/error-classes.json 
b/common/utils/src/main/resources/error/error-classes.json
index 3d130fdce301..99fbc585f981 100644
--- a/common/utils/src/main/resources/error/error-classes.json
+++ b/common/utils/src/main/resources/error/error-classes.json
@@ -3371,9 +3371,9 @@
 ],
 "sqlState" : "0A000"
   },
-  "STATE_STORE_CANNOT_REMOVE_DEFAULT_COLUMN_FAMILY" : {
+  "STATE_STORE_CANNOT_USE_COLUMN_FAMILY_WITH_INVALID_NAME" : {
 "message" : [
-  "Failed to remove default column family with reserved 
name=."
+  "Failed to perform column family operation= with invalid 
name=. Column family name cannot be empty or include 
leading/trailing spaces or use the reserved keyword=default"
 ],
 "sqlState" : "42802"
   },
@@ -3396,6 +3396,12 @@
 ],
 "sqlState" : "XXKST"
   },
+  "STATE_STORE_UNSUPPORTED_OPERATION_ON_MISSING_COLUMN_FAMILY" : {
+"message" : [
+  "State store operation= not supported on missing column 
family=."
+],
+"sqlState" : "42802"
+  },
   "STATIC_PARTITION_COLUMN_IN_INSERT_COLUMN_LIST" : {
 "message" : [
   "Static partition column  is also specified in the column 
list."
diff --git a/docs/sql-error-conditions.md b/docs/sql-error-conditions.md
index 2cddb6a94c14..b6b159f277c0 100644
--- a/docs/sql-error-conditions.md
+++ b/docs/sql-error-conditions.md
@@ -2105,11 +2105,11 @@ The SQL config `` cannot be found. Please 
verify that the config exists
 
 Star (*) is not allowed in a select list when GROUP BY an ordinal position is 
used.
 
-### STATE_STORE_CANNOT_REMOVE_DEFAULT_COLUMN_FAMILY
+### STATE_STORE_CANNOT_USE_COLUMN_FAMILY_WITH_INVALID_NAME
 
 [SQLSTATE: 
42802](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation)
 
-Failed to remove default column family with reserved name=``.
+Failed to perform column family operation=`` with invalid 
name=``. Column family name cannot be empty or include 
leading/trail

(spark) branch master updated: [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source

2024-03-11 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new f53dc08230b7 [SPARK-46962][SS][PYTHON] Add interface for python 
streaming data source API and implement python worker to run python streaming 
data source
f53dc08230b7 is described below

commit f53dc08230b7a758227f02fd75d2b446721c139f
Author: Chaoqin Li 
AuthorDate: Tue Mar 12 12:35:18 2024 +0900

[SPARK-46962][SS][PYTHON] Add interface for python streaming data source 
API and implement python worker to run python streaming data source

### What changes were proposed in this pull request?
This is the first PR the implement the support to implement streaming data 
source through python API.
Implement python worker to run python streaming data source and communicate 
with JVM through socket. Create a PythonMicrobatchStream to invoke RPC function 
call. This happens in the spark driver. For each python streaming data source 
instance there will be a long live python worker process created. Inside the 
python process, the python streaming reader will receive function call and 
parameter from JVM PythonMicroBatchStream and send back result through socket.

### Why are the changes needed?
In preparation for support of development of streaming data source in 
Python.

### Does this PR introduce _any_ user-facing change?
No.

### How was this patch tested?
Unit test. PythonMicroBatchStream plan offset and partitions by invoking 
function call through socket correctly and handle error correctly.

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #45023 from chaoqin-li1123/python_table.

Lead-authored-by: Chaoqin Li 
Co-authored-by: Hyukjin Kwon 
Co-authored-by: chaoqin-li1123 
<55518381+chaoqin-li1...@users.noreply.github.com>
Signed-off-by: Jungtaek Lim 
---
 .../src/main/resources/error/error-classes.json|   6 +
 docs/sql-error-conditions.md   |   6 +
 python/pyspark/errors/error_classes.py |   5 +
 python/pyspark/sql/datasource.py   | 153 ++
 .../streaming/python_streaming_source_runner.py| 167 +++
 .../spark/sql/errors/QueryExecutionErrors.scala|   9 +
 .../v2/python/PythonMicroBatchStream.scala |  68 ++
 .../datasources/v2/python/PythonScan.scala |  32 ++-
 .../v2/python/UserDefinedPythonDataSource.scala|   2 +-
 .../python/PythonStreamingSourceRunner.scala   | 202 ++
 .../execution/python/PythonDataSourceSuite.scala   |  20 +-
 .../python/PythonStreamingDataSourceSuite.scala| 233 +
 12 files changed, 883 insertions(+), 20 deletions(-)

diff --git a/common/utils/src/main/resources/error/error-classes.json 
b/common/utils/src/main/resources/error/error-classes.json
index afe81b8e9bea..3d130fdce301 100644
--- a/common/utils/src/main/resources/error/error-classes.json
+++ b/common/utils/src/main/resources/error/error-classes.json
@@ -3197,6 +3197,12 @@
 ],
 "sqlState" : "38000"
   },
+  "PYTHON_STREAMING_DATA_SOURCE_RUNTIME_ERROR" : {
+"message" : [
+  "Failed when Python streaming data source perform : "
+],
+"sqlState" : "38000"
+  },
   "RECURSIVE_PROTOBUF_SCHEMA" : {
 "message" : [
   "Found recursive reference in Protobuf schema, which can not be 
processed by Spark by default: . try setting the option 
`recursive.fields.max.depth` 0 to 10. Going beyond 10 levels of recursion is 
not allowed."
diff --git a/docs/sql-error-conditions.md b/docs/sql-error-conditions.md
index 0695ed28b7fc..2cddb6a94c14 100644
--- a/docs/sql-error-conditions.md
+++ b/docs/sql-error-conditions.md
@@ -1931,6 +1931,12 @@ Protobuf type not yet supported: ``.
 
 Failed to `` Python data source ``: ``
 
+### PYTHON_STREAMING_DATA_SOURCE_RUNTIME_ERROR
+
+[SQLSTATE: 
38000](sql-error-conditions-sqlstates.html#class-38-external-routine-exception)
+
+Failed when Python streaming data source perform ``: ``
+
 ### RECURSIVE_PROTOBUF_SCHEMA
 
 [SQLSTATE: 
42K0G](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation)
diff --git a/python/pyspark/errors/error_classes.py 
b/python/pyspark/errors/error_classes.py
index c9a7cfbf356e..1e21ad3543e9 100644
--- a/python/pyspark/errors/error_classes.py
+++ b/python/pyspark/errors/error_classes.py
@@ -812,6 +812,11 @@ ERROR_CLASSES_JSON = '''
   "Randomness of hash of string should be disabled via PYTHONHASHSEED."
 ]
   },
+  "PYTHON_STREAMING_DATA_SOURCE_RUNTIME_ERROR": {
+"message": [
+  "Failed when running Python streaming data source: "
+]
+

(spark) branch master updated: [SPARK-47331][SS] Serialization using case classes/primitives/POJO based on SQL encoder for Arbitrary State API v2

2024-03-10 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new afbebfbadc4b [SPARK-47331][SS] Serialization using case 
classes/primitives/POJO based on SQL encoder for Arbitrary State API v2
afbebfbadc4b is described below

commit afbebfbadc4b5e927df7c568a8afb08fc4407f58
Author: jingz-db 
AuthorDate: Mon Mar 11 09:20:44 2024 +0900

[SPARK-47331][SS] Serialization using case classes/primitives/POJO based on 
SQL encoder for Arbitrary State API v2

### What changes were proposed in this pull request?

In the new operator for arbitrary state-v2, we cannot rely on the 
session/encoder being available since the initialization for the various state 
instances happens on the executors. Hence, for the state serialization, we 
propose to let user explicitly pass in encoder for state variable and serialize 
primitives/case classes/POJO with SQL encoder. Leveraging SQL encoder can speed 
up the serialization.

### Why are the changes needed?

These changes are needed for providing a dedicated serializer for state-v2.
The changes are part of the work around adding new stateful streaming 
operator for arbitrary state mgmt that provides a bunch of new features listed 
in the SPIP JIRA here - https://issues.apache.org/jira/browse/SPARK-45939

### Does this PR introduce _any_ user-facing change?

Users will need to specify the SQL encoder for their state variable:
`def getValueState[T](stateName: String, valEncoder: Encoder[T]): 
ValueState[T]`
`def getListState[T](stateName: String, valEncoder: Encoder[T]): 
ListState[T]`

For primitive type, Encoder is something as: `Encoders.scalaLong`; for case 
class, `Encoders.product[CaseClass]`; for POJO, 
`Encoders.bean(classOf[POJOClass])`

### How was this patch tested?

Unit tests for primitives, case classes, POJO separately in 
`ValueStateSuite`

### Was this patch authored or co-authored using generative AI tooling?

No

Closes #45447 from jingz-db/sql-encoder-state-v2.

Authored-by: jingz-db 
Signed-off-by: Jungtaek Lim 
---
 .../sql/streaming/StatefulProcessorHandle.scala|   7 +-
 .../sql/execution/streaming/ListStateImpl.scala|   8 +-
 .../streaming/StateTypesEncoderUtils.scala |  41 +---
 .../streaming/StatefulProcessorHandleImpl.scala|   9 +-
 .../sql/execution/streaming/ValueStateImpl.scala   |   9 +-
 .../execution/streaming/state/POJOTestClass.java   |  78 ++
 .../streaming/state/ValueStateSuite.scala  | 117 -
 .../streaming/TransformWithListStateSuite.scala|   7 +-
 .../sql/streaming/TransformWithStateSuite.scala|  11 +-
 9 files changed, 250 insertions(+), 37 deletions(-)

diff --git 
a/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala
 
b/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala
index 5d3390f80f6d..86bf1e85f90c 100644
--- 
a/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala
+++ 
b/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala
@@ -19,6 +19,7 @@ package org.apache.spark.sql.streaming
 import java.io.Serializable
 
 import org.apache.spark.annotation.{Evolving, Experimental}
+import org.apache.spark.sql.Encoder
 
 /**
  * Represents the operation handle provided to the stateful processor used in 
the
@@ -33,20 +34,22 @@ private[sql] trait StatefulProcessorHandle extends 
Serializable {
* The user must ensure to call this function only within the `init()` 
method of the
* StatefulProcessor.
* @param stateName - name of the state variable
+   * @param valEncoder - SQL encoder for state variable
* @tparam T - type of state variable
* @return - instance of ValueState of type T that can be used to store 
state persistently
*/
-  def getValueState[T](stateName: String): ValueState[T]
+  def getValueState[T](stateName: String, valEncoder: Encoder[T]): 
ValueState[T]
 
   /**
* Creates new or returns existing list state associated with stateName.
* The ListState persists values of type T.
*
* @param stateName  - name of the state variable
+   * @param valEncoder - SQL encoder for state variable
* @tparam T - type of state variable
* @return - instance of ListState of type T that can be used to store state 
persistently
*/
-  def getListState[T](stateName: String): ListState[T]
+  def getListState[T](stateName: String, valEncoder: Encoder[T]): ListState[T]
 
   /** Function to return queryInfo for currently running task */
   def getQueryInfo(): QueryInfo
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ListStateImpl.scala
 
b/sql/core/src/main/scala/org/apache/spark

(spark) branch master updated: [SPARK-47200][SS] Make Foreach batch sink user function error handling backward compatible

2024-03-10 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new edb970b8a73e [SPARK-47200][SS] Make Foreach batch sink user function 
error handling backward compatible
edb970b8a73e is described below

commit edb970b8a73e5b1e08b01f9370dadb05a3e231e3
Author: micheal-o 
AuthorDate: Mon Mar 11 08:44:30 2024 +0900

[SPARK-47200][SS] Make Foreach batch sink user function error handling 
backward compatible

### What changes were proposed in this pull request?
I checked in a previous PR (https://github.com/apache/spark/pull/45299), 
that handles and classifies exceptions thrown in user provided functions for 
foreach batch sink. This change is to make it backward compatible in order not 
to break current users, since users may be depending on getting the user code 
error from the `StreamingQueryException.cause` instead of 
`StreamingQueryException.cause.cause`

### Why are the changes needed?
To prevent breaking existing usage pattern.

### Does this PR introduce _any_ user-facing change?
Yes, better error message with error class for ForeachBatchSink user 
function failures.

### How was this patch tested?
updated existing tests

### Was this patch authored or co-authored using generative AI tooling?
No

Closes #45449 from micheal-o/ForeachBatchExBackwardCompat.

Authored-by: micheal-o 
Signed-off-by: Jungtaek Lim 
---
 .../src/main/resources/error/error-classes.json|  2 +-
 docs/sql-error-conditions.md   |  2 +-
 .../sql/execution/streaming/StreamExecution.scala  | 29 +++---
 .../streaming/sources/ForeachBatchSink.scala   | 14 ---
 .../sql/errors/QueryExecutionErrorsSuite.scala |  2 +-
 .../streaming/sources/ForeachBatchSinkSuite.scala  | 17 +++--
 6 files changed, 43 insertions(+), 23 deletions(-)

diff --git a/common/utils/src/main/resources/error/error-classes.json 
b/common/utils/src/main/resources/error/error-classes.json
index 57746d6dbf1e..9717ff2ed49c 100644
--- a/common/utils/src/main/resources/error/error-classes.json
+++ b/common/utils/src/main/resources/error/error-classes.json
@@ -1297,7 +1297,7 @@
   },
   "FOREACH_BATCH_USER_FUNCTION_ERROR" : {
 "message" : [
-  "An error occurred in the user provided function in foreach batch sink."
+  "An error occurred in the user provided function in foreach batch sink. 
Reason: "
 ],
 "sqlState" : "39000"
   },
diff --git a/docs/sql-error-conditions.md b/docs/sql-error-conditions.md
index 7be01f8cb513..0be75cde968f 100644
--- a/docs/sql-error-conditions.md
+++ b/docs/sql-error-conditions.md
@@ -778,7 +778,7 @@ The operation `` is not allowed on the 
``: ``
 
 [SQLSTATE: 
39000](sql-error-conditions-sqlstates.html#class-39-external-routine-invocation-exception)
 
-An error occurred in the user provided function in foreach batch sink.
+An error occurred in the user provided function in foreach batch sink. Reason: 
``
 
 ### FOUND_MULTIPLE_DATA_SOURCES
 
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala
index 859fce8b1154..50a73082a8c4 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala
@@ -40,6 +40,7 @@ import org.apache.spark.sql.connector.catalog.{SupportsWrite, 
Table}
 import org.apache.spark.sql.connector.read.streaming.{Offset => OffsetV2, 
ReadLimit, SparkDataStream}
 import org.apache.spark.sql.connector.write.{LogicalWriteInfoImpl, 
SupportsTruncate, Write}
 import org.apache.spark.sql.execution.command.StreamingExplainCommand
+import 
org.apache.spark.sql.execution.streaming.sources.ForeachBatchUserFuncException
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.internal.connector.SupportsStreamingUpdateAsAppend
 import org.apache.spark.sql.streaming._
@@ -279,6 +280,7 @@ abstract class StreamExecution(
* `start()` method returns.
*/
   private def runStream(): Unit = {
+var errorClassOpt: Option[String] = None
 try {
   sparkSession.sparkContext.setJobGroup(runId.toString, 
getBatchDescriptionString,
 interruptOnCancel = true)
@@ -330,9 +332,17 @@ abstract class StreamExecution(
 getLatestExecutionContext().updateStatusMessage("Stopped")
   case e: Throwable =>
 val message = if (e.getMessage == null) "" else e.getMessage
+val cause = if (e.isInstanceOf[ForeachBatchUserFuncException]) {
+  // We want to maintain the current way users get the cau

(spark) branch branch-3.4 updated: [SPARK-47305][SQL] Fix PruneFilters to tag the isStreaming flag of LocalRelation correctly when the plan has both batch and streaming

2024-03-06 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch branch-3.4
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.4 by this push:
 new 7e5d59289870 [SPARK-47305][SQL] Fix PruneFilters to tag the 
isStreaming flag of LocalRelation correctly when the plan has both batch and 
streaming
7e5d59289870 is described below

commit 7e5d5928987069e255da94f8dd8b0cd7696a773b
Author: Jungtaek Lim 
AuthorDate: Thu Mar 7 15:11:15 2024 +0900

[SPARK-47305][SQL] Fix PruneFilters to tag the isStreaming flag of 
LocalRelation correctly when the plan has both batch and streaming

### What changes were proposed in this pull request?

This PR proposes to fix PruneFilters to tag the isStreaming flag of 
LocalRelation correctly when the plan has both batch and streaming.

### Why are the changes needed?

When filter is evaluated to be always false, PruneFilters replaces the 
filter with empty LocalRelation, which effectively prunes filter. The logic 
cares about migration of the isStreaming flag, but incorrectly migrated in some 
case, via picking up the value of isStreaming flag from root node rather than 
filter (or child).

isStreaming flag is true if the value of isStreaming flag from any of 
children is true. Flipping the coin, some children might have isStreaming flag 
as "false". If the filter being pruned is a descendant to such children (in 
other word, ancestor of streaming node), LocalRelation is incorrectly tagged as 
streaming where it should be batch.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

New UT verifying the fix. The new UT fails without this PR and passes with 
this PR.

### Was this patch authored or co-authored using generative AI tooling?

No.

Closes #45406 from HeartSaVioR/SPARK-47305.

Authored-by: Jungtaek Lim 
Signed-off-by: Jungtaek Lim 
(cherry picked from commit 8d6bd9bbd29da6023e5740b622e12c7e1f8581ce)
Signed-off-by: Jungtaek Lim 
---
 .../spark/sql/catalyst/optimizer/Optimizer.scala   |  4 +-
 .../optimizer/PropagateEmptyRelationSuite.scala| 43 +-
 2 files changed, 43 insertions(+), 4 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
index 7aebf7c28f11..3d774af1ce33 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
@@ -1636,9 +1636,9 @@ object PruneFilters extends Rule[LogicalPlan] with 
PredicateHelper {
 // If the filter condition always evaluate to null or false,
 // replace the input with an empty relation.
 case Filter(Literal(null, _), child) =>
-  LocalRelation(child.output, data = Seq.empty, isStreaming = 
plan.isStreaming)
+  LocalRelation(child.output, data = Seq.empty, isStreaming = 
child.isStreaming)
 case Filter(Literal(false, BooleanType), child) =>
-  LocalRelation(child.output, data = Seq.empty, isStreaming = 
plan.isStreaming)
+  LocalRelation(child.output, data = Seq.empty, isStreaming = 
child.isStreaming)
 // If any deterministic condition is guaranteed to be true given the 
constraints on the child's
 // output, remove the condition
 case f @ Filter(fc, p: LogicalPlan) =>
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelationSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelationSuite.scala
index fe45e02c67fa..a1132eadcc6f 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelationSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelationSuite.scala
@@ -21,10 +21,10 @@ import org.apache.spark.sql.Row
 import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow}
 import org.apache.spark.sql.catalyst.dsl.expressions._
 import org.apache.spark.sql.catalyst.dsl.plans._
-import org.apache.spark.sql.catalyst.expressions.{Literal, UnspecifiedFrame}
+import org.apache.spark.sql.catalyst.expressions.{EqualTo, Literal, 
UnspecifiedFrame}
 import org.apache.spark.sql.catalyst.expressions.Literal.FalseLiteral
 import org.apache.spark.sql.catalyst.plans._
-import org.apache.spark.sql.catalyst.plans.logical.{Expand, LocalRelation, 
LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.plans.logical.{Expand, Filter, 
LocalRelation, LogicalPlan, Project}
 import org.apache.spark.sql.catalyst.rules.RuleExecutor
 import org.apache.spark.sql.types.{IntegerType, MetadataBuilder, StructType}
 
@@ -221

(spark) branch branch-3.5 updated: [SPARK-47305][SQL] Fix PruneFilters to tag the isStreaming flag of LocalRelation correctly when the plan has both batch and streaming

2024-03-06 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch branch-3.5
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.5 by this push:
 new 679f3b1e5e96 [SPARK-47305][SQL] Fix PruneFilters to tag the 
isStreaming flag of LocalRelation correctly when the plan has both batch and 
streaming
679f3b1e5e96 is described below

commit 679f3b1e5e965a6be12823faf012d0680771a5e2
Author: Jungtaek Lim 
AuthorDate: Thu Mar 7 15:11:15 2024 +0900

[SPARK-47305][SQL] Fix PruneFilters to tag the isStreaming flag of 
LocalRelation correctly when the plan has both batch and streaming

### What changes were proposed in this pull request?

This PR proposes to fix PruneFilters to tag the isStreaming flag of 
LocalRelation correctly when the plan has both batch and streaming.

### Why are the changes needed?

When filter is evaluated to be always false, PruneFilters replaces the 
filter with empty LocalRelation, which effectively prunes filter. The logic 
cares about migration of the isStreaming flag, but incorrectly migrated in some 
case, via picking up the value of isStreaming flag from root node rather than 
filter (or child).

isStreaming flag is true if the value of isStreaming flag from any of 
children is true. Flipping the coin, some children might have isStreaming flag 
as "false". If the filter being pruned is a descendant to such children (in 
other word, ancestor of streaming node), LocalRelation is incorrectly tagged as 
streaming where it should be batch.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

New UT verifying the fix. The new UT fails without this PR and passes with 
this PR.

### Was this patch authored or co-authored using generative AI tooling?

No.

Closes #45406 from HeartSaVioR/SPARK-47305.

Authored-by: Jungtaek Lim 
Signed-off-by: Jungtaek Lim 
(cherry picked from commit 8d6bd9bbd29da6023e5740b622e12c7e1f8581ce)
Signed-off-by: Jungtaek Lim 
---
 .../spark/sql/catalyst/optimizer/Optimizer.scala   |  4 +-
 .../optimizer/PropagateEmptyRelationSuite.scala| 43 +-
 2 files changed, 43 insertions(+), 4 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
index 04d3eb962ed4..239682ab1f84 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
@@ -1668,9 +1668,9 @@ object PruneFilters extends Rule[LogicalPlan] with 
PredicateHelper {
 // If the filter condition always evaluate to null or false,
 // replace the input with an empty relation.
 case Filter(Literal(null, _), child) =>
-  LocalRelation(child.output, data = Seq.empty, isStreaming = 
plan.isStreaming)
+  LocalRelation(child.output, data = Seq.empty, isStreaming = 
child.isStreaming)
 case Filter(Literal(false, BooleanType), child) =>
-  LocalRelation(child.output, data = Seq.empty, isStreaming = 
plan.isStreaming)
+  LocalRelation(child.output, data = Seq.empty, isStreaming = 
child.isStreaming)
 // If any deterministic condition is guaranteed to be true given the 
constraints on the child's
 // output, remove the condition
 case f @ Filter(fc, p: LogicalPlan) =>
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelationSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelationSuite.scala
index e8d2ca1ff75d..5aeb27f7ee6b 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelationSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelationSuite.scala
@@ -21,10 +21,10 @@ import org.apache.spark.sql.Row
 import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow}
 import org.apache.spark.sql.catalyst.dsl.expressions._
 import org.apache.spark.sql.catalyst.dsl.plans._
-import org.apache.spark.sql.catalyst.expressions.{Literal, UnspecifiedFrame}
+import org.apache.spark.sql.catalyst.expressions.{EqualTo, Literal, 
UnspecifiedFrame}
 import org.apache.spark.sql.catalyst.expressions.Literal.FalseLiteral
 import org.apache.spark.sql.catalyst.plans._
-import org.apache.spark.sql.catalyst.plans.logical.{Expand, LocalRelation, 
LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.plans.logical.{Expand, Filter, 
LocalRelation, LogicalPlan, Project}
 import org.apache.spark.sql.catalyst.rules.RuleExecutor
 import org.apache.spark.sql.catalyst.types.DataTypeUtils
 import org.apache.spark.sql.types.{IntegerTy

(spark) branch master updated (3ec79a9df06f -> 8d6bd9bbd29d)

2024-03-06 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


from 3ec79a9df06f [SPARK-47238][SQL] Reduce executor memory usage by making 
generated code in WSCG a broadcast variable
 add 8d6bd9bbd29d [SPARK-47305][SQL] Fix PruneFilters to tag the 
isStreaming flag of LocalRelation correctly when the plan has both batch and 
streaming

No new revisions were added by this update.

Summary of changes:
 .../spark/sql/catalyst/optimizer/Optimizer.scala   |  4 +-
 .../optimizer/PropagateEmptyRelationSuite.scala| 43 +-
 2 files changed, 43 insertions(+), 4 deletions(-)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



(spark-website) branch asf-site updated: MINOR: Update the link of latest to 3.5.1

2024-02-29 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch asf-site
in repository https://gitbox.apache.org/repos/asf/spark-website.git


The following commit(s) were added to refs/heads/asf-site by this push:
 new 53982b743a MINOR: Update the link of latest to 3.5.1
53982b743a is described below

commit 53982b743a2834472f2f028e821a543355b359da
Author: Jungtaek Lim 
AuthorDate: Fri Mar 1 13:23:42 2024 +0900

MINOR: Update the link of latest to 3.5.1
---
 site/docs/latest | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/site/docs/latest b/site/docs/latest
index e5b820341f..3c8ff8c36b 12
--- a/site/docs/latest
+++ b/site/docs/latest
@@ -1 +1 @@
-3.5.0
\ No newline at end of file
+3.5.1
\ No newline at end of file


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



(spark) branch master updated (9ce43c85a5d2 -> 7447172ffd5c)

2024-02-29 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


from 9ce43c85a5d2 [SPARK-47229][CORE][SQL][SS][YARN][CONNECT] Change the 
never changed `var` to `val`
 add 7447172ffd5c [SPARK-47200][SS] Error class for Foreach batch sink user 
function error

No new revisions were added by this update.

Summary of changes:
 .../src/main/resources/error/error-classes.json|  6 +
 docs/sql-error-conditions.md   |  6 +
 .../streaming/test_streaming_foreach_batch.py  |  5 -
 .../streaming/sources/ForeachBatchSink.scala   | 18 ++-
 .../sql/errors/QueryExecutionErrorsSuite.scala |  2 +-
 .../streaming/sources/ForeachBatchSinkSuite.scala  | 26 ++
 6 files changed, 60 insertions(+), 3 deletions(-)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



(spark) branch master updated: [SPARK-47135][SS] Implement error classes for Kafka data loss exceptions

2024-02-28 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 36df0a63a139 [SPARK-47135][SS] Implement error classes for Kafka data 
loss exceptions
36df0a63a139 is described below

commit 36df0a63a139704ccd2a344d057e430430b11ad8
Author: micheal-o 
AuthorDate: Thu Feb 29 09:26:47 2024 +0900

[SPARK-47135][SS] Implement error classes for Kafka data loss exceptions

### What changes were proposed in this pull request?
In the kafka connector code, we have several code that throws the java 
**IllegalStateException** to report data loss, while reading from Kafka. This 
change is to properly classify those exceptions using the new error framework. 
Adds a new exception type `SparkIllegalStateException` that can receive error 
class. New error classes are introduced for Kafka data loss errors.

### Why are the changes needed?
New error framework for better error messages

### Does this PR introduce _any_ user-facing change?
Yes, better error message with error class

### How was this patch tested?
Updated existing tests

### Was this patch authored or co-authored using generative AI tooling?
No

Closes #45221 from micheal-o/bmo/IllegalStateEx.

Lead-authored-by: micheal-o 
Co-authored-by: micheal-o 
Signed-off-by: Jungtaek Lim 
---
 .../main/resources/error/kafka-error-classes.json  |  56 +++
 .../spark/sql/kafka010/KafkaContinuousStream.scala |  26 ++---
 .../spark/sql/kafka010/KafkaExceptions.scala   | 109 +++--
 .../spark/sql/kafka010/KafkaMicroBatchStream.scala |   6 +-
 .../spark/sql/kafka010/KafkaOffsetReader.scala |   4 +-
 .../sql/kafka010/KafkaOffsetReaderAdmin.scala  |  42 +---
 .../sql/kafka010/KafkaOffsetReaderConsumer.scala   |  46 ++---
 .../apache/spark/sql/kafka010/KafkaSource.scala|   6 +-
 .../spark/sql/kafka010/KafkaSourceProvider.scala   |   8 --
 .../sql/kafka010/consumer/KafkaDataConsumer.scala  |  85 
 .../sql/kafka010/KafkaMicroBatchSourceSuite.scala  |   2 +-
 .../sql/kafka010/KafkaOffsetReaderSuite.scala  |   2 +-
 .../kafka010/consumer/KafkaDataConsumerSuite.scala |  11 ++-
 13 files changed, 290 insertions(+), 113 deletions(-)

diff --git 
a/connector/kafka-0-10-sql/src/main/resources/error/kafka-error-classes.json 
b/connector/kafka-0-10-sql/src/main/resources/error/kafka-error-classes.json
index ea7ffb592a55..a7b22e1370fd 100644
--- a/connector/kafka-0-10-sql/src/main/resources/error/kafka-error-classes.json
+++ b/connector/kafka-0-10-sql/src/main/resources/error/kafka-error-classes.json
@@ -22,5 +22,61 @@
   "Some of partitions in Kafka topic(s) report available offset which is 
less than end offset during running query with Trigger.AvailableNow. The error 
could be transient - restart your query, and report if you still see the same 
issue.",
   "latest offset: , end offset: "
 ]
+  },
+  "KAFKA_DATA_LOSS" : {
+"message" : [
+  "Some data may have been lost because they are not available in Kafka 
any more;",
+  "either the data was aged out by Kafka or the topic may have been 
deleted before all the data in the",
+  "topic was processed.",
+  "If you don't want your streaming query to fail on such cases, set the 
source option failOnDataLoss to false.",
+  "Reason:"
+],
+"subClass" : {
+  "ADDED_PARTITION_DOES_NOT_START_FROM_OFFSET_ZERO" : {
+"message" : [
+  "Added partition  starts from  instead 
of 0."
+]
+  },
+  "COULD_NOT_READ_OFFSET_RANGE" : {
+"message" : [
+  "Could not read records in offset [, ) for 
topic partition ",
+  "with consumer group ."
+]
+  },
+  "INITIAL_OFFSET_NOT_FOUND_FOR_PARTITIONS" : {
+"message" : [
+  "Cannot find initial offsets for partitions . They may 
have been deleted."
+]
+  },
+  "PARTITIONS_DELETED" : {
+"message" : [
+  "Partitions  have been deleted."
+]
+  },
+  "PARTITIONS_DELETED_AND_GROUP_ID_CONFIG_PRESENT" : {
+"message" : [
+  "Partitions  have been deleted.",
+  "Kafka option 'kafka.' has been set on this query, it 
is",
+  "not recommended to set this option. This option is unsafe to use 
since multiple concurrent",
+  "queries or sources using the same group id will interfere with each 
other as they are part",
+  "of the same consumer group. Restarted queries may also s

(spark-website) branch asf-site updated: Slight correction of introduction for 3.5.1

2024-02-28 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch asf-site
in repository https://gitbox.apache.org/repos/asf/spark-website.git


The following commit(s) were added to refs/heads/asf-site by this push:
 new 5cf6d34d00 Slight correction of introduction for 3.5.1
5cf6d34d00 is described below

commit 5cf6d34d005332717ff3a65f5645beb9a3383032
Author: Jungtaek Lim 
AuthorDate: Wed Feb 28 18:47:54 2024 +0900

Slight correction of introduction for 3.5.1



This PR fixes a silly mistake on introducing 3.5.1, it's the **first** 
maintenance release of 3.5 version line, not **last**. Directly modified the 
html as the change is just several characters.

Author: Jungtaek Lim 

Closes #506 from HeartSaVioR/3.5.1-slight-correction.
---
 releases/_posts/2024-02-23-spark-release-3-5-1.md | 2 +-
 site/releases/spark-release-3-5-1.html| 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)

diff --git a/releases/_posts/2024-02-23-spark-release-3-5-1.md 
b/releases/_posts/2024-02-23-spark-release-3-5-1.md
index f01f910a2d..43e97d77ca 100644
--- a/releases/_posts/2024-02-23-spark-release-3-5-1.md
+++ b/releases/_posts/2024-02-23-spark-release-3-5-1.md
@@ -11,7 +11,7 @@ meta:
   _wpas_done_all: '1'
 ---
 
-Spark 3.5.1 is the last maintenance release containing security and 
correctness fixes. This release is based on the branch-3.5 maintenance branch 
of Spark. We strongly recommend all 3.5 users to upgrade to this stable release.
+Spark 3.5.1 is the first maintenance release containing security and 
correctness fixes. This release is based on the branch-3.5 maintenance branch 
of Spark. We strongly recommend all 3.5 users to upgrade to this stable release.
 
 ### Notable changes
 
diff --git a/site/releases/spark-release-3-5-1.html 
b/site/releases/spark-release-3-5-1.html
index 93fd0b846a..16360dff7c 100644
--- a/site/releases/spark-release-3-5-1.html
+++ b/site/releases/spark-release-3-5-1.html
@@ -142,7 +142,7 @@
   Spark Release 3.5.1
 
 
-Spark 3.5.1 is the last maintenance release containing security and 
correctness fixes. This release is based on the branch-3.5 maintenance branch 
of Spark. We strongly recommend all 3.5 users to upgrade to this stable 
release.
+Spark 3.5.1 is the first maintenance release containing security and 
correctness fixes. This release is based on the branch-3.5 maintenance branch 
of Spark. We strongly recommend all 3.5 users to upgrade to this stable 
release.
 
 Notable changes
 


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



(spark) branch branch-3.5 updated: [SPARK-47036][SS][3.5] Cleanup RocksDB file tracking for previously uploaded files if files were deleted from local directory

2024-02-21 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch branch-3.5
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.5 by this push:
 new 3f4425b4880d [SPARK-47036][SS][3.5] Cleanup RocksDB file tracking for 
previously uploaded files if files were deleted from local directory
3f4425b4880d is described below

commit 3f4425b4880dfe3e494a894da18b412ecdba4fb1
Author: Bhuwan Sahni 
AuthorDate: Thu Feb 22 10:59:15 2024 +0900

[SPARK-47036][SS][3.5] Cleanup RocksDB file tracking for previously 
uploaded files if files were deleted from local directory

Backports PR https://github.com/apache/spark/pull/45092 to Spark 3.5

### What changes were proposed in this pull request?

This change cleans up any dangling files tracked as being previously 
uploaded if they were cleaned up from the filesystem. The cleaning can happen 
due to a compaction racing in parallel with commit, where compaction completes 
after commit and a older version is loaded on the same executor.

### Why are the changes needed?

The changes are needed to prevent RocksDB versionId mismatch errors (which 
require users to clean the checkpoint directory and retry the query).

A particular scenario where this can happen is provided below:

1. Version V1 is loaded on executor A, RocksDB State Store has 195.sst, 
196.sst, 197.sst and 198.sst files.
2. State changes are made, which result in creation of a new table file 
200.sst.
3. State store is committed as version V2. The SST file 200.sst (as 
000200-8c80161a-bc23-4e3b-b175-cffe38e427c7.sst) is uploaded to DFS, and 
previous 4 files are reused. A new metadata file is created to track the exact 
SST files with unique IDs, and uploaded with RocksDB Manifest as part of V1.zip.
4. Rocks DB compaction is triggered at the same time. The compaction 
creates a new L1 file (201.sst), and deletes existing 5 SST files.
5. Spark Stage is retried.
6. Version V1 is reloaded on the same executor. The local files are 
inspected, and 201.sst is deleted. The 4 SST files in version V1 are downloaded 
again to local file system.
7. Any local files which are deleted (as part of version load) are also 
removed from local → DFS file upload tracking. **However, the files already 
deleted as a result of compaction are not removed from tracking. This is the 
bug which resulted in the failure.**
8. State store is committed as version V1. However, the local mapping of 
SST files to DFS file path still has 200.sst in its tracking, hence the SST 
file is not re-uploaded. A new metadata file is created to track the exact SST 
files with unique IDs, and uploaded with the new RocksDB Manifest as part of 
V2.zip. (The V2.zip file is overwritten here atomically)
9. A new executor tried to load version V2. However, the SST files in (1) 
are now incompatible with Manifest file in (6) resulting in the version Id 
mismatch failure.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

Added unit test cases to cover the scenario where some files were deleted 
on the file system.

The test case fails with the existing master with error `Mismatch in unique 
ID on table file 16`, and is successful with changes in this PR.

### Was this patch authored or co-authored using generative AI tooling?

No

Closes #45206 from sahnib/spark-3.5-rocks-db-fix.

Authored-by: Bhuwan Sahni 
Signed-off-by: Jungtaek Lim 
---
 .../streaming/state/RocksDBFileManager.scala   | 41 +++---
 .../execution/streaming/state/RocksDBSuite.scala   | 91 +-
 2 files changed, 119 insertions(+), 13 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala
index 300a3b8137b4..3089de7127e7 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala
@@ -496,16 +496,12 @@ class RocksDBFileManager(
   s" DFS for version $version. $filesReused files reused without copying.")
 versionToRocksDBFiles.put(version, immutableFiles)
 
-// clean up deleted SST files from the localFilesToDfsFiles Map
-val currentLocalFiles = localFiles.map(_.getName).toSet
-val mappingsToClean = localFilesToDfsFiles.asScala
-  .keys
-  .filterNot(currentLocalFiles.contains)
-
-mappingsToClean.foreach { f =>
-  logInfo(s"cleaning $f from the localFilesToDfsFiles map")
-  localFilesToDfsFiles.remove(f)
-}
+// Cleanup locally deleted files from the localFilesToDfsFiles map
+  

(spark) branch master updated: [SPARK-47036][SS] Cleanup RocksDB file tracking for previously uploaded files if files were deleted from local directory

2024-02-21 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new af28aebfa17c [SPARK-47036][SS] Cleanup RocksDB file tracking for 
previously uploaded files if files were deleted from local directory
af28aebfa17c is described below

commit af28aebfa17cb51bbb5397dab06cce5a7768b572
Author: Bhuwan Sahni 
AuthorDate: Thu Feb 22 05:33:13 2024 +0900

[SPARK-47036][SS] Cleanup RocksDB file tracking for previously uploaded 
files if files were deleted from local directory

### What changes were proposed in this pull request?

This change cleans up any dangling files tracked as being previously 
uploaded if they were cleaned up from the filesystem. The cleaning can happen 
due to a compaction racing in parallel with commit, where compaction completes 
after commit and a older version is loaded on the same executor.

### Why are the changes needed?

The changes are needed to prevent RocksDB versionId mismatch errors (which 
require users to clean the checkpoint directory and retry the query).

A particular scenario where this can happen is provided below:

1. Version V1 is loaded on executor A, RocksDB State Store has 195.sst, 
196.sst, 197.sst and 198.sst files.
2. State changes are made, which result in creation of a new table file 
200.sst.
3. State store is committed as version V2. The SST file 200.sst (as 
000200-8c80161a-bc23-4e3b-b175-cffe38e427c7.sst) is uploaded to DFS, and 
previous 4 files are reused. A new metadata file is created to track the exact 
SST files with unique IDs, and uploaded with RocksDB Manifest as part of V1.zip.
4. Rocks DB compaction is triggered at the same time. The compaction 
creates a new L1 file (201.sst), and deletes existing 5 SST files.
5. Spark Stage is retried.
6. Version V1 is reloaded on the same executor. The local files are 
inspected, and 201.sst is deleted. The 4 SST files in version V1 are downloaded 
again to local file system.
7. Any local files which are deleted (as part of version load) are also 
removed from local → DFS file upload tracking. **However, the files already 
deleted as a result of compaction are not removed from tracking. This is the 
bug which resulted in the failure.**
8. State store is committed as version V1. However, the local mapping of 
SST files to DFS file path still has 200.sst in its tracking, hence the SST 
file is not re-uploaded. A new metadata file is created to track the exact SST 
files with unique IDs, and uploaded with the new RocksDB Manifest as part of 
V2.zip. (The V2.zip file is overwritten here atomically)
9. A new executor tried to load version V2. However, the SST files in (1) 
are now incompatible with Manifest file in (6) resulting in the version Id 
mismatch failure.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

Added unit test cases to cover the scenario where some files were deleted 
on the file system.

The test case fails with the existing master with error `Mismatch in unique 
ID on table file 16`, and is successful with changes in this PR.

### Was this patch authored or co-authored using generative AI tooling?

No

Closes #45092 from sahnib/rocksdb-compaction-file-tracking-fix.

Authored-by: Bhuwan Sahni 
Signed-off-by: Jungtaek Lim 
---
 .../streaming/state/RocksDBFileManager.scala   | 41 +++---
 .../execution/streaming/state/RocksDBSuite.scala   | 91 +-
 2 files changed, 119 insertions(+), 13 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala
index f86b683a9058..18c769bed350 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala
@@ -518,16 +518,12 @@ class RocksDBFileManager(
   s" DFS for version $version. $filesReused files reused without copying.")
 versionToRocksDBFiles.put(version, immutableFiles)
 
-// clean up deleted SST files from the localFilesToDfsFiles Map
-val currentLocalFiles = localFiles.map(_.getName).toSet
-val mappingsToClean = localFilesToDfsFiles.asScala
-  .keys
-  .filterNot(currentLocalFiles.contains)
-
-mappingsToClean.foreach { f =>
-  logInfo(s"cleaning $f from the localFilesToDfsFiles map")
-  localFilesToDfsFiles.remove(f)
-}
+// Cleanup locally deleted files from the localFilesToDfsFiles map
+// Locally, SST Files can be deleted due to RocksDB compaction. These 
files nee

(spark) branch master updated: [SPARK-46928][SS] Add support for ListState in Arbitrary State API v2

2024-02-20 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 0b907ed11e6e [SPARK-46928][SS] Add support for ListState in Arbitrary 
State API v2
0b907ed11e6e is described below

commit 0b907ed11e6ec6bc4c7d07926ed352806636d58a
Author: Bhuwan Sahni 
AuthorDate: Wed Feb 21 14:43:27 2024 +0900

[SPARK-46928][SS] Add support for ListState in Arbitrary State API v2

### What changes were proposed in this pull request?

This PR adds changes for ListState implementation in State Api v2. As a 
list contains multiple values for a single key, we utilize RocksDB merge 
operator to persist multiple values.

Changes include

1. A new encoder/decoder to encode multiple values inside a single byte[] 
array (stored in RocksDB). The encoding scheme is compatible with RocksDB 
StringAppendOperator merge operator.
2. Support merge operations in ChangelogCheckpointing v2.
3. Extend StateStore to support merge operation, and read multiple values 
for a single key (via a Iterator). Note that these changes are only supported 
for RocksDB currently.

### Why are the changes needed?

These changes are needed to support list values in the State Store. The 
changes are part of the work around adding new stateful streaming operator for 
arbitrary state mgmt that provides a bunch of new features listed in the SPIP 
JIRA here - https://issues.apache.org/jira/browse/SPARK-45939

### Does this PR introduce _any_ user-facing change?

Yes
This PR introduces a new state type (ListState) that users can use in their 
Spark streaming queries.

### How was this patch tested?

1. Added a new test suite for ListState to ensure the state produces 
correct results.
2. Added additional testcases for input validation.
3. Added tests for merge operator with RocksDB.
4. Added tests for changelog checkpointing merge operator.
5. Added tests for reading merged values in RocksDBStateStore.

### Was this patch authored or co-authored using generative AI tooling?

No

Closes #44961 from sahnib/state-api-v2-list-state.

Authored-by: Bhuwan Sahni 
Signed-off-by: Jungtaek Lim 
---
 .../src/main/resources/error/error-classes.json|  18 ++
 ...itions-illegal-state-store-value-error-class.md |  41 +++
 docs/sql-error-conditions.md   |   8 +
 .../{ValueState.scala => ListState.scala}  |  30 +-
 .../sql/streaming/StatefulProcessorHandle.scala|  10 +
 .../apache/spark/sql/streaming/ValueState.scala|   2 +-
 .../v2/state/StatePartitionReader.scala|   2 +-
 .../sql/execution/streaming/ListStateImpl.scala| 121 
 .../streaming/StateTypesEncoderUtils.scala |  88 ++
 .../streaming/StatefulProcessorHandleImpl.scala|   8 +-
 .../streaming/TransformWithStateExec.scala |   6 +-
 .../sql/execution/streaming/ValueStateImpl.scala   |  61 +---
 .../state/HDFSBackedStateStoreProvider.scala   |  27 +-
 .../sql/execution/streaming/state/RocksDB.scala|  37 +++
 .../streaming/state/RocksDBStateEncoder.scala  |  96 +-
 .../state/RocksDBStateStoreProvider.scala  |  53 +++-
 .../sql/execution/streaming/state/StateStore.scala |  53 +++-
 .../streaming/state/StateStoreChangelog.scala  |  48 ++-
 .../streaming/state/StateStoreErrors.scala |  12 +
 .../execution/streaming/state/StateStoreRDD.scala  |   5 +-
 .../state/SymmetricHashJoinStateManager.scala  |   3 +-
 .../sql/execution/streaming/state/package.scala|   6 +-
 .../streaming/state/MemoryStateStore.scala |  11 +-
 .../streaming/state/RocksDBStateStoreSuite.scala   |  56 +++-
 .../execution/streaming/state/RocksDBSuite.scala   |  84 ++
 .../streaming/state/StateStoreSuite.scala  |   7 +-
 .../streaming/state/ValueStateSuite.scala  |  12 +-
 .../apache/spark/sql/streaming/StreamSuite.scala   |   3 +-
 .../streaming/TransformWithListStateSuite.scala| 328 +
 .../sql/streaming/TransformWithStateSuite.scala|   2 +-
 30 files changed, 1120 insertions(+), 118 deletions(-)

diff --git a/common/utils/src/main/resources/error/error-classes.json 
b/common/utils/src/main/resources/error/error-classes.json
index c1b1171b5dc8..b30b1d60bb4a 100644
--- a/common/utils/src/main/resources/error/error-classes.json
+++ b/common/utils/src/main/resources/error/error-classes.json
@@ -1380,6 +1380,24 @@
 ],
 "sqlState" : "42601"
   },
+  "ILLEGAL_STATE_STORE_VALUE" : {
+"message" : [
+  "Illegal value provided to the State Store"
+],
+"subClass" : {
+  "EMPTY_LIST_VALUE" : {
+"message" : [
+  "Cannot write em

(spark) branch master updated: [SPARK-47052][SS] Separate state tracking variables from MicroBatchExecution/StreamExecution

2024-02-20 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new bffa92c838d6 [SPARK-47052][SS] Separate state tracking variables from 
MicroBatchExecution/StreamExecution
bffa92c838d6 is described below

commit bffa92c838d6650249a6e71bb0ef8189cf970383
Author: Jerry Peng 
AuthorDate: Wed Feb 21 12:58:48 2024 +0900

[SPARK-47052][SS] Separate state tracking variables from 
MicroBatchExecution/StreamExecution

### What changes were proposed in this pull request?

To improve code clarity and maintainability, I propose that we move all the 
variables that track mutable state and metrics for a streaming query into a 
separate class.  With this refactor, it would be easy to track and find all the 
mutable state a microbatch can have.

### Why are the changes needed?

To improve code clarity and maintainability.  All the state and metrics 
that is needed for the execution lifecycle of a microbatch is consolidated into 
one class.  If we decide to modify or add additional state to a streaming 
query, it will be easier to determine 1) where to add it 2) what existing state 
are there.

### Does this PR introduce _any_ user-facing change?

No
### How was this patch tested?

Existing tests should suffice

### Was this patch authored or co-authored using generative AI tooling?

No

Closes #45109 from jerrypeng/SPARK-47052.

Authored-by: Jerry Peng 
Signed-off-by: Jungtaek Lim 
---
 .../sql/execution/streaming/AsyncLogPurge.scala|  11 +-
 .../AsyncProgressTrackingMicroBatchExecution.scala |  30 +-
 .../execution/streaming/MicroBatchExecution.scala  | 422 ++---
 .../sql/execution/streaming/ProgressReporter.scala | 521 +
 .../sql/execution/streaming/StreamExecution.scala  | 112 +++--
 .../streaming/StreamExecutionContext.scala | 233 +
 .../sql/execution/streaming/TriggerExecutor.scala  |  24 +-
 .../streaming/continuous/ContinuousExecution.scala |  56 ++-
 .../streaming/ProcessingTimeExecutorSuite.scala|   6 +-
 9 files changed, 945 insertions(+), 470 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AsyncLogPurge.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AsyncLogPurge.scala
index b3729dbc7b45..aa393211a1c1 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AsyncLogPurge.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AsyncLogPurge.scala
@@ -29,11 +29,8 @@ import org.apache.spark.util.ThreadUtils
  */
 trait AsyncLogPurge extends Logging {
 
-  protected var currentBatchId: Long
-
   protected val minLogEntriesToMaintain: Int
 
-
   protected[sql] val errorNotifier: ErrorNotifier
 
   protected val sparkSession: SparkSession
@@ -47,15 +44,11 @@ trait AsyncLogPurge extends Logging {
 
   protected lazy val useAsyncPurge: Boolean = 
sparkSession.conf.get(SQLConf.ASYNC_LOG_PURGE)
 
-  protected def purgeAsync(): Unit = {
+  protected def purgeAsync(batchId: Long): Unit = {
 if (purgeRunning.compareAndSet(false, true)) {
-  // save local copy because currentBatchId may get updated.  There are 
not really
-  // any concurrency issues here in regards to calculating the purge 
threshold
-  // but for the sake of defensive coding lets make a copy
-  val currentBatchIdCopy: Long = currentBatchId
   asyncPurgeExecutorService.execute(() => {
 try {
-  purge(currentBatchIdCopy - minLogEntriesToMaintain)
+  purge(batchId - minLogEntriesToMaintain)
 } catch {
   case throwable: Throwable =>
 logError("Encountered error while performing async log purge", 
throwable)
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AsyncProgressTrackingMicroBatchExecution.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AsyncProgressTrackingMicroBatchExecution.scala
index 206efb9a5450..ec24ec0fd335 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AsyncProgressTrackingMicroBatchExecution.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AsyncProgressTrackingMicroBatchExecution.scala
@@ -110,12 +110,12 @@ class AsyncProgressTrackingMicroBatchExecution(
 }
   }
 
-  override def markMicroBatchExecutionStart(): Unit = {
+  override def markMicroBatchExecutionStart(execCtx: 
MicroBatchExecutionContext): Unit = {
 // check if streaming query is stateful
 checkNotStatefulStreamingQuery
   }
 
-  override def cleanUpLastExecutedMicroBatch(): Unit = {
+  override def cleanUpLastExecutedMicroBatch(execCtx: 
MicroBatchExecutionContext): Unit = {
 // this is a no op for async p

(spark) branch master updated (eb71268a6a38 -> 8ede494ad6c7)

2024-02-20 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


from eb71268a6a38 [SPARK-47108][CORE] Set 
`derby.connection.requireAuthentication` to `false` explicitly in CLIs
 add 8ede494ad6c7 [SPARK-46906][SS] Add a check for stateful operator 
change for streaming

No new revisions were added by this update.

Summary of changes:
 .../src/main/resources/error/error-classes.json|  7 ++
 docs/sql-error-conditions.md   |  7 ++
 .../spark/sql/errors/QueryExecutionErrors.scala| 13 
 .../v2/state/metadata/StateMetadataSource.scala|  2 +-
 .../execution/streaming/IncrementalExecution.scala | 61 +++-
 .../execution/streaming/statefulOperators.scala|  2 +-
 .../state/OperatorStateMetadataSuite.scala | 81 +-
 7 files changed, 166 insertions(+), 7 deletions(-)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



svn commit: r67358 - in /dev/spark/v3.5.1-rc2-docs: ./ _site/ _site/api/ _site/api/R/ _site/api/R/articles/ _site/api/R/deps/ _site/api/R/deps/bootstrap-5.3.1/ _site/api/R/deps/bootstrap-5.3.1/fonts/

2024-02-15 Thread kabhwan
Author: kabhwan
Date: Thu Feb 15 14:08:37 2024
New Revision: 67358

Log:
Apache Spark v3.5.1-rc2 docs


[This commit notification would consist of 4713 parts, 
which exceeds the limit of 50 ones, so it was shortened to the summary.]

-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



svn commit: r67355 - /dev/spark/v3.5.1-rc2-bin/

2024-02-15 Thread kabhwan
Author: kabhwan
Date: Thu Feb 15 11:39:51 2024
New Revision: 67355

Log:
Apache Spark v3.5.1-rc2

Added:
dev/spark/v3.5.1-rc2-bin/
dev/spark/v3.5.1-rc2-bin/SparkR_3.5.1.tar.gz   (with props)
dev/spark/v3.5.1-rc2-bin/SparkR_3.5.1.tar.gz.asc
dev/spark/v3.5.1-rc2-bin/SparkR_3.5.1.tar.gz.sha512
dev/spark/v3.5.1-rc2-bin/pyspark-3.5.1.tar.gz   (with props)
dev/spark/v3.5.1-rc2-bin/pyspark-3.5.1.tar.gz.asc
dev/spark/v3.5.1-rc2-bin/pyspark-3.5.1.tar.gz.sha512
dev/spark/v3.5.1-rc2-bin/spark-3.5.1-bin-hadoop3-scala2.13.tgz   (with 
props)
dev/spark/v3.5.1-rc2-bin/spark-3.5.1-bin-hadoop3-scala2.13.tgz.asc
dev/spark/v3.5.1-rc2-bin/spark-3.5.1-bin-hadoop3-scala2.13.tgz.sha512
dev/spark/v3.5.1-rc2-bin/spark-3.5.1-bin-hadoop3.tgz   (with props)
dev/spark/v3.5.1-rc2-bin/spark-3.5.1-bin-hadoop3.tgz.asc
dev/spark/v3.5.1-rc2-bin/spark-3.5.1-bin-hadoop3.tgz.sha512
dev/spark/v3.5.1-rc2-bin/spark-3.5.1-bin-without-hadoop.tgz   (with props)
dev/spark/v3.5.1-rc2-bin/spark-3.5.1-bin-without-hadoop.tgz.asc
dev/spark/v3.5.1-rc2-bin/spark-3.5.1-bin-without-hadoop.tgz.sha512
dev/spark/v3.5.1-rc2-bin/spark-3.5.1.tgz   (with props)
dev/spark/v3.5.1-rc2-bin/spark-3.5.1.tgz.asc
dev/spark/v3.5.1-rc2-bin/spark-3.5.1.tgz.sha512

Added: dev/spark/v3.5.1-rc2-bin/SparkR_3.5.1.tar.gz
==
Binary file - no diff available.

Propchange: dev/spark/v3.5.1-rc2-bin/SparkR_3.5.1.tar.gz
--
svn:mime-type = application/octet-stream

Added: dev/spark/v3.5.1-rc2-bin/SparkR_3.5.1.tar.gz.asc
==
--- dev/spark/v3.5.1-rc2-bin/SparkR_3.5.1.tar.gz.asc (added)
+++ dev/spark/v3.5.1-rc2-bin/SparkR_3.5.1.tar.gz.asc Thu Feb 15 11:39:51 2024
@@ -0,0 +1,17 @@
+-BEGIN PGP SIGNATURE-
+
+iQJHBAABCgAxFiEE/T6ElC5eYQYjWh0lvTVqn4dA5P8FAmXN908THGthYmh3YW5A
+YXBhY2hlLm9yZwAKCRC9NWqfh0Dk/7gaD/43OxM8XC9NB/uf53WLbJr467fDRxuc
+LDCUwTsCC4mhX1DzAX9pt8sLQ6coWuLyqVHdaXyLCzbnHwonXpEV2euYITWWvIcy
+hK7TKSFFnWWB2QipZjRw8LcnR5ScLPKENX76XgnLOIMwKojVBItWE+01Z2klFopZ
+875bh2STo0/BQ3VyJEu935EXcfB7nAV5nTByhc6io39wcxvxlH25+hKi73KL4pPq
+uDTWanamwI/IQtTI05Oe4d/6f2WDjksVBVdSj/xgAxubKMMUU2aRQNKHwr2Aebe3
+CACQpGgpBRAa1LzzHgvkNnrZZ2uDzx1HHPPuGyheZrYIMM4vVVi2Lj5M3Sfh/zkQ
+ifR7Gn2XXFiypo5SPgaR5C7Zr5vn1yGPKxeWnaeBpeTkzoMr7wkyj6CJM5RlzGlo
+jDbAJwgpV/EZookonEVqDKAYHkbkjh2bmuK9l/YvjvrADWSH1eAntrmZH8laG3q2
+O4TcOgqKQhaf1fUErTXOciHc22fg62XauLLvRjnCmHNbHggh47m3TEAIc/WjW21t
+YAXZ9P7kdyRYoNWxxFJf32Ne1/JCYTu+7ev4TbHM2yugYWxdkVcs9XXeou/V+gwt
+AMngu6QItixJWtSaS/twpUB/GdUpHx36pL6QwqVysgHiuCrC+VOAwpg3oAoNaGaM
+U1XUkNeqKZm8Vw==
+=Ux6g
+-END PGP SIGNATURE-

Added: dev/spark/v3.5.1-rc2-bin/SparkR_3.5.1.tar.gz.sha512
==
--- dev/spark/v3.5.1-rc2-bin/SparkR_3.5.1.tar.gz.sha512 (added)
+++ dev/spark/v3.5.1-rc2-bin/SparkR_3.5.1.tar.gz.sha512 Thu Feb 15 11:39:51 2024
@@ -0,0 +1 @@
+1a57cf9dd6129e022c7bda178ab0baca2aeb8265b0eb6184c9bb2b1c30b9f4b118bef3bd7b165bcfff0b5b8d80ab7e8b57d6eba141e5a49fd47a6668550d0d14
  SparkR_3.5.1.tar.gz

Added: dev/spark/v3.5.1-rc2-bin/pyspark-3.5.1.tar.gz
==
Binary file - no diff available.

Propchange: dev/spark/v3.5.1-rc2-bin/pyspark-3.5.1.tar.gz
--
svn:mime-type = application/octet-stream

Added: dev/spark/v3.5.1-rc2-bin/pyspark-3.5.1.tar.gz.asc
==
--- dev/spark/v3.5.1-rc2-bin/pyspark-3.5.1.tar.gz.asc (added)
+++ dev/spark/v3.5.1-rc2-bin/pyspark-3.5.1.tar.gz.asc Thu Feb 15 11:39:51 2024
@@ -0,0 +1,17 @@
+-BEGIN PGP SIGNATURE-
+
+iQJHBAABCgAxFiEE/T6ElC5eYQYjWh0lvTVqn4dA5P8FAmXN91ETHGthYmh3YW5A
+YXBhY2hlLm9yZwAKCRC9NWqfh0Dk/+WMEAC5YH5rKEQQ4BuwaIXb9AAd0afzEOCj
+3nyYGHcT5BfIfs0fEicTvcs7rL5kxfzSa9Dhr8ly2FQbiv3DWm3m2UEqPY+HPxZH
+dI/jbaZlpx5k4K0fEpT4v5vDiHaN0z9pCqGZnbzcqggIML4u22/R4SctjuhfdCXC
+AKx6YJ0OpWixjblUmKixp/hg0bQJ5JaNmSMOJra0hNNgmZZQ/vYRcsltZFqsA/uh
+tN0gGeQ/o9uNYrSrD+zMlKx+SUZI6x6PcLZrPjVrn5g4QlwWb9qhrRrA37Nh6Z2K
+x9t6TjBIZjql8u6BzcLfgjrUXHA8+IDLEFVkSSssOuMUT8VbRDIqmIHY0m4bHNM+
+KORAJkKTzQ2PEAV5bYK+qsFrpiXOksMGCc9ia2NEHxrcZNJ0VnivsfJDEwbVYfj0
+4OIWTrJPkxwKUg38hkZemHKdz1SHxZRrXfofsVDkUnfszHdzyPyFuyHjFyzCoLMB
+6JcFZNf/ca2EVvvy1h323B0i09PbXKmxS8gdpDz7Mt1+lTFWTjaslAw6EqQPI+0I
+O7jbgJdAcRuyzXnGur4L24qh8zNUJH13fqPNFVtoANOgj4MKjeqYh0wQDkbSumpa
+4E0tnehKAm9jmbAkLl2Yusouv5kucMzRx5glG3+yWZ3MEOAxvlqokzu9Htf8qBQ6
+UgmPGPeUXF/Tgg==
+=+Er+
+-END PGP SIGNATURE-

Added: dev/spark/v3.5.1-rc2-bin/pyspark-3.5.1.tar.gz.sha512
==
--- dev/spark/v3.5.1-rc2-bin/pyspark-3.5.1.tar.gz.sha512 (added)
+++ dev/spark

(spark) 01/01: Preparing development version 3.5.2-SNAPSHOT

2024-02-15 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch branch-3.5
in repository https://gitbox.apache.org/repos/asf/spark.git

commit cacb6fa0868a8741cb67ef705375ac378adaeebb
Author: Jungtaek Lim 
AuthorDate: Thu Feb 15 10:56:51 2024 +

Preparing development version 3.5.2-SNAPSHOT
---
 R/pkg/DESCRIPTION  | 2 +-
 assembly/pom.xml   | 2 +-
 common/kvstore/pom.xml | 2 +-
 common/network-common/pom.xml  | 2 +-
 common/network-shuffle/pom.xml | 2 +-
 common/network-yarn/pom.xml| 2 +-
 common/sketch/pom.xml  | 2 +-
 common/tags/pom.xml| 2 +-
 common/unsafe/pom.xml  | 2 +-
 common/utils/pom.xml   | 2 +-
 connector/avro/pom.xml | 2 +-
 connector/connect/client/jvm/pom.xml   | 2 +-
 connector/connect/common/pom.xml   | 2 +-
 connector/connect/server/pom.xml   | 2 +-
 connector/docker-integration-tests/pom.xml | 2 +-
 connector/kafka-0-10-assembly/pom.xml  | 2 +-
 connector/kafka-0-10-sql/pom.xml   | 2 +-
 connector/kafka-0-10-token-provider/pom.xml| 2 +-
 connector/kafka-0-10/pom.xml   | 2 +-
 connector/kinesis-asl-assembly/pom.xml | 2 +-
 connector/kinesis-asl/pom.xml  | 2 +-
 connector/protobuf/pom.xml | 2 +-
 connector/spark-ganglia-lgpl/pom.xml   | 2 +-
 core/pom.xml   | 2 +-
 docs/_config.yml   | 6 +++---
 examples/pom.xml   | 2 +-
 graphx/pom.xml | 2 +-
 hadoop-cloud/pom.xml   | 2 +-
 launcher/pom.xml   | 2 +-
 mllib-local/pom.xml| 2 +-
 mllib/pom.xml  | 2 +-
 pom.xml| 2 +-
 python/pyspark/version.py  | 2 +-
 repl/pom.xml   | 2 +-
 resource-managers/kubernetes/core/pom.xml  | 2 +-
 resource-managers/kubernetes/integration-tests/pom.xml | 2 +-
 resource-managers/mesos/pom.xml| 2 +-
 resource-managers/yarn/pom.xml | 2 +-
 sql/api/pom.xml| 2 +-
 sql/catalyst/pom.xml   | 2 +-
 sql/core/pom.xml   | 2 +-
 sql/hive-thriftserver/pom.xml  | 2 +-
 sql/hive/pom.xml   | 2 +-
 streaming/pom.xml  | 2 +-
 tools/pom.xml  | 2 +-
 45 files changed, 47 insertions(+), 47 deletions(-)

diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION
index 66faa8031c45..89bee06852be 100644
--- a/R/pkg/DESCRIPTION
+++ b/R/pkg/DESCRIPTION
@@ -1,6 +1,6 @@
 Package: SparkR
 Type: Package
-Version: 3.5.1
+Version: 3.5.2
 Title: R Front End for 'Apache Spark'
 Description: Provides an R Front end for 'Apache Spark' 
<https://spark.apache.org>.
 Authors@R:
diff --git a/assembly/pom.xml b/assembly/pom.xml
index 47b54729bbd2..d1ef9b24afda 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -21,7 +21,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.5.1
+3.5.2-SNAPSHOT
 ../pom.xml
   
 
diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml
index 66e6bb473bf2..9df20f8facf5 100644
--- a/common/kvstore/pom.xml
+++ b/common/kvstore/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.5.1
+3.5.2-SNAPSHOT
 ../../pom.xml
   
 
diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml
index 98897b4424ae..27a53b0f9f3b 100644
--- a/common/network-common/pom.xml
+++ b/common/network-common/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.5.1
+3.5.2-SNAPSHOT
 ../../pom.xml
   
 
diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml
index 44531ea54cd5..93410815e6c0 100644
--- a/common/network-shuffle/pom.xml
+++ b/common/network-shuffle/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.5.1
+3.5.2-SNAPSHOT
 ../../pom.xml
   
 
diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml
index 8fcf20328e8e..a99b8b96402a 100644
--- a/common/network-yarn/pom.xml
+++ b/common/network-yarn/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-paren

(spark) branch branch-3.5 updated (9b4778fc1dc7 -> cacb6fa0868a)

2024-02-15 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a change to branch branch-3.5
in repository https://gitbox.apache.org/repos/asf/spark.git


from 9b4778fc1dc7 [SPARK-46906][INFRA][3.5] Bump python libraries (pandas, 
pyarrow) in Docker image for release script
 add fd86f85e181f Preparing Spark release v3.5.1-rc2
 new cacb6fa0868a Preparing development version 3.5.2-SNAPSHOT

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



(spark) tag v3.5.1-rc2 created (now fd86f85e181f)

2024-02-15 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a change to tag v3.5.1-rc2
in repository https://gitbox.apache.org/repos/asf/spark.git


  at fd86f85e181f (commit)
This tag includes the following new commits:

 new fd86f85e181f Preparing Spark release v3.5.1-rc2

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



(spark) 01/01: Preparing Spark release v3.5.1-rc2

2024-02-15 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to tag v3.5.1-rc2
in repository https://gitbox.apache.org/repos/asf/spark.git

commit fd86f85e181fc2dc0f50a096855acf83a6cc5d9c
Author: Jungtaek Lim 
AuthorDate: Thu Feb 15 10:56:47 2024 +

Preparing Spark release v3.5.1-rc2
---
 R/pkg/DESCRIPTION  | 2 +-
 assembly/pom.xml   | 2 +-
 common/kvstore/pom.xml | 2 +-
 common/network-common/pom.xml  | 2 +-
 common/network-shuffle/pom.xml | 2 +-
 common/network-yarn/pom.xml| 2 +-
 common/sketch/pom.xml  | 2 +-
 common/tags/pom.xml| 2 +-
 common/unsafe/pom.xml  | 2 +-
 common/utils/pom.xml   | 2 +-
 connector/avro/pom.xml | 2 +-
 connector/connect/client/jvm/pom.xml   | 2 +-
 connector/connect/common/pom.xml   | 2 +-
 connector/connect/server/pom.xml   | 2 +-
 connector/docker-integration-tests/pom.xml | 2 +-
 connector/kafka-0-10-assembly/pom.xml  | 2 +-
 connector/kafka-0-10-sql/pom.xml   | 2 +-
 connector/kafka-0-10-token-provider/pom.xml| 2 +-
 connector/kafka-0-10/pom.xml   | 2 +-
 connector/kinesis-asl-assembly/pom.xml | 2 +-
 connector/kinesis-asl/pom.xml  | 2 +-
 connector/protobuf/pom.xml | 2 +-
 connector/spark-ganglia-lgpl/pom.xml   | 2 +-
 core/pom.xml   | 2 +-
 docs/_config.yml   | 6 +++---
 examples/pom.xml   | 2 +-
 graphx/pom.xml | 2 +-
 hadoop-cloud/pom.xml   | 2 +-
 launcher/pom.xml   | 2 +-
 mllib-local/pom.xml| 2 +-
 mllib/pom.xml  | 2 +-
 pom.xml| 2 +-
 python/pyspark/version.py  | 2 +-
 repl/pom.xml   | 2 +-
 resource-managers/kubernetes/core/pom.xml  | 2 +-
 resource-managers/kubernetes/integration-tests/pom.xml | 2 +-
 resource-managers/mesos/pom.xml| 2 +-
 resource-managers/yarn/pom.xml | 2 +-
 sql/api/pom.xml| 2 +-
 sql/catalyst/pom.xml   | 2 +-
 sql/core/pom.xml   | 2 +-
 sql/hive-thriftserver/pom.xml  | 2 +-
 sql/hive/pom.xml   | 2 +-
 streaming/pom.xml  | 2 +-
 tools/pom.xml  | 2 +-
 45 files changed, 47 insertions(+), 47 deletions(-)

diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION
index 89bee06852be..66faa8031c45 100644
--- a/R/pkg/DESCRIPTION
+++ b/R/pkg/DESCRIPTION
@@ -1,6 +1,6 @@
 Package: SparkR
 Type: Package
-Version: 3.5.2
+Version: 3.5.1
 Title: R Front End for 'Apache Spark'
 Description: Provides an R Front end for 'Apache Spark' 
<https://spark.apache.org>.
 Authors@R:
diff --git a/assembly/pom.xml b/assembly/pom.xml
index d1ef9b24afda..47b54729bbd2 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -21,7 +21,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.5.2-SNAPSHOT
+3.5.1
 ../pom.xml
   
 
diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml
index 9df20f8facf5..66e6bb473bf2 100644
--- a/common/kvstore/pom.xml
+++ b/common/kvstore/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.5.2-SNAPSHOT
+3.5.1
 ../../pom.xml
   
 
diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml
index 27a53b0f9f3b..98897b4424ae 100644
--- a/common/network-common/pom.xml
+++ b/common/network-common/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.5.2-SNAPSHOT
+3.5.1
 ../../pom.xml
   
 
diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml
index 93410815e6c0..44531ea54cd5 100644
--- a/common/network-shuffle/pom.xml
+++ b/common/network-shuffle/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.5.2-SNAPSHOT
+3.5.1
 ../../pom.xml
   
 
diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml
index a99b8b96402a..8fcf20328e8e 100644
--- a/common/network-yarn/pom.xml
+++ b/common/network-yarn/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-   

(spark) branch branch-3.5 updated: [SPARK-46906][INFRA][3.5] Bump python libraries (pandas, pyarrow) in Docker image for release script

2024-02-14 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch branch-3.5
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.5 by this push:
 new 9b4778fc1dc7 [SPARK-46906][INFRA][3.5] Bump python libraries (pandas, 
pyarrow) in Docker image for release script
9b4778fc1dc7 is described below

commit 9b4778fc1dc7047635c9ec19c187d4e75d182590
Author: Jungtaek Lim 
AuthorDate: Thu Feb 15 14:49:09 2024 +0900

[SPARK-46906][INFRA][3.5] Bump python libraries (pandas, pyarrow) in Docker 
image for release script

### What changes were proposed in this pull request?

This PR proposes to bump python libraries (pandas to 2.0.3, pyarrow to 
4.0.0) in Docker image for release script.

### Why are the changes needed?

Without this change, release script (do-release-docker.sh) fails on docs 
phase. Changing this fixes the release process against branch-3.5.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Confirmed with dry-run of release script against branch-3.5.

`dev/create-release/do-release-docker.sh -d ~/spark-release -n -s docs`

```
Generating HTML files for SQL API documentation.
INFO-  Cleaning site directory
INFO-  Building documentation to directory: 
/opt/spark-rm/output/spark/sql/site
INFO-  Documentation built in 0.85 seconds
/opt/spark-rm/output/spark/sql
Moving back into docs dir.
Making directory api/sql
cp -r ../sql/site/. api/sql
Source: /opt/spark-rm/output/spark/docs
   Destination: /opt/spark-rm/output/spark/docs/_site
 Incremental build: disabled. Enable with --incremental
  Generating...
done in 7.469 seconds.
 Auto-regeneration: disabled. Use --watch to enable.
```

### Was this patch authored or co-authored using generative AI tooling?

No.

Closes #45111 from HeartSaVioR/SPARK-46906-3.5.

Authored-by: Jungtaek Lim 
Signed-off-by: Jungtaek Lim 
---
 dev/create-release/spark-rm/Dockerfile | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/dev/create-release/spark-rm/Dockerfile 
b/dev/create-release/spark-rm/Dockerfile
index cd57226f5e01..789915d018de 100644
--- a/dev/create-release/spark-rm/Dockerfile
+++ b/dev/create-release/spark-rm/Dockerfile
@@ -42,7 +42,7 @@ ARG APT_INSTALL="apt-get install --no-install-recommends -y"
 #   We should use the latest Sphinx version once this is fixed.
 # TODO(SPARK-35375): Jinja2 3.0.0+ causes error when building with Sphinx.
 #   See also https://issues.apache.org/jira/browse/SPARK-35375.
-ARG PIP_PKGS="sphinx==3.0.4 mkdocs==1.1.2 numpy==1.20.3 
pydata_sphinx_theme==0.8.0 ipython==7.19.0 nbsphinx==0.8.0 numpydoc==1.1.0 
jinja2==2.11.3 twine==3.4.1 sphinx-plotly-directive==0.1.3 
sphinx-copybutton==0.5.2 pandas==1.5.3 pyarrow==3.0.0 plotly==5.4.0 
markupsafe==2.0.1 docutils<0.17 grpcio==1.56.0 protobuf==4.21.6 
grpcio-status==1.56.0 googleapis-common-protos==1.56.4"
+ARG PIP_PKGS="sphinx==3.0.4 mkdocs==1.1.2 numpy==1.20.3 
pydata_sphinx_theme==0.8.0 ipython==7.19.0 nbsphinx==0.8.0 numpydoc==1.1.0 
jinja2==2.11.3 twine==3.4.1 sphinx-plotly-directive==0.1.3 
sphinx-copybutton==0.5.2 pandas==2.0.3 pyarrow==4.0.0 plotly==5.4.0 
markupsafe==2.0.1 docutils<0.17 grpcio==1.56.0 protobuf==4.21.6 
grpcio-status==1.56.0 googleapis-common-protos==1.56.4"
 ARG GEM_PKGS="bundler:2.3.8"
 
 # Install extra needed repos and refresh.


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



(spark) branch master updated: [SPARK-46979][SS] Add support for specifying key and value encoder separately and also for each col family in RocksDB state store provider

2024-02-13 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 63b97c6ad82a [SPARK-46979][SS] Add support for specifying key and 
value encoder separately and also for each col family in RocksDB state store 
provider
63b97c6ad82a is described below

commit 63b97c6ad82afac71afcd64117346b6e0bda72bb
Author: Anish Shrigondekar 
AuthorDate: Wed Feb 14 06:19:48 2024 +0900

[SPARK-46979][SS] Add support for specifying key and value encoder 
separately and also for each col family in RocksDB state store provider

### What changes were proposed in this pull request?
Add support for specifying key and value encoder separately and also for 
each col family in RocksDB state store provider

### Why are the changes needed?
This change allows us to specify encoder for key/values separately and 
avoid encoding additional bytes. Also, it allows us to set schemas/encoders for 
individual column families, which will be required for future changes related 
to transformWithState operator (listState/timer changes etc)

We are refactoring a bit here given the upcoming changes. so we are 
proposing to split key and value encoders.
Key encoders can be of 2 types:
- with prefix scan
- without prefix scan

Value encoders can also eventually be of 2 types:
- single value
- multiple values (used for list state)

And we now also allow setting schema and getting encoder for each column 
family.
So after the change, we can potentially allow something like this:
- col family 1 - with keySchema with prefix scan and valueSchema with 
single value and binary type
- col family 2 - with keySchema without prefix scan and valueSchema with 
multiple values

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Added unit tests
```
[info] Run completed in 3 minutes, 5 seconds.
[info] Total number of tests run: 286
[info] Suites: completed 1, aborted 0
[info] Tests: succeeded 286, failed 0, canceled 0, ignored 0, pending 0
[info] All tests passed.
```

### Was this patch authored or co-authored using generative AI tooling?
No

Closes #45038 from anishshri-db/task/SPARK-46979.

Authored-by: Anish Shrigondekar 
Signed-off-by: Jungtaek Lim 
---
 .../streaming/StatefulProcessorHandleImpl.scala|   1 -
 .../sql/execution/streaming/ValueStateImpl.scala   |  20 ++--
 .../state/HDFSBackedStateStoreProvider.scala   |   6 +-
 .../streaming/state/RocksDBStateEncoder.scala  | 106 +++--
 .../state/RocksDBStateStoreProvider.scala  |  58 ---
 .../sql/execution/streaming/state/StateStore.scala |   6 +-
 .../streaming/state/MemoryStateStore.scala |   7 +-
 .../streaming/state/RocksDBStateStoreSuite.scala   |  24 +
 8 files changed, 151 insertions(+), 77 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala
index fed18fc7e458..62c97d11c926 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala
@@ -115,7 +115,6 @@ class StatefulProcessorHandleImpl(
   override def getValueState[T](stateName: String): ValueState[T] = {
 verify(currState == CREATED, s"Cannot create state variable with 
name=$stateName after " +
   "initialization is complete")
-store.createColFamilyIfAbsent(stateName)
 val resultState = new ValueStateImpl[T](store, stateName, keyEncoder)
 resultState
   }
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImpl.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImpl.scala
index 11ae7f65b43d..c1d807144df6 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImpl.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImpl.scala
@@ -32,7 +32,6 @@ import org.apache.spark.sql.types._
  * @param store - reference to the StateStore instance to be used for storing 
state
  * @param stateName - name of logical state partition
  * @param keyEnc - Spark SQL encoder for key
- * @tparam K - data type of key
  * @tparam S - data type of object that will be stored
  */
 class ValueStateImpl[S](
@@ -40,6 +39,16 @@ class ValueStateImpl[S](
 stateName: String,
 keyExprEnc: ExpressionEncoder[Any]) extends ValueState[S] with Logging {
 
+  private val schemaForKeyRow: StructType = new StructType().add("key"

svn commit: r67285 - /dev/spark/v3.5.1-rc1-bin/

2024-02-11 Thread kabhwan
Author: kabhwan
Date: Mon Feb 12 04:25:21 2024
New Revision: 67285

Log:
Apache Spark v3.5.1-rc1

Added:
dev/spark/v3.5.1-rc1-bin/
dev/spark/v3.5.1-rc1-bin/SparkR_3.5.1.tar.gz   (with props)
dev/spark/v3.5.1-rc1-bin/SparkR_3.5.1.tar.gz.asc
dev/spark/v3.5.1-rc1-bin/SparkR_3.5.1.tar.gz.sha512
dev/spark/v3.5.1-rc1-bin/pyspark-3.5.1.tar.gz   (with props)
dev/spark/v3.5.1-rc1-bin/pyspark-3.5.1.tar.gz.asc
dev/spark/v3.5.1-rc1-bin/pyspark-3.5.1.tar.gz.sha512
dev/spark/v3.5.1-rc1-bin/spark-3.5.1-bin-hadoop3-scala2.13.tgz   (with 
props)
dev/spark/v3.5.1-rc1-bin/spark-3.5.1-bin-hadoop3-scala2.13.tgz.asc
dev/spark/v3.5.1-rc1-bin/spark-3.5.1-bin-hadoop3-scala2.13.tgz.sha512
dev/spark/v3.5.1-rc1-bin/spark-3.5.1-bin-hadoop3.tgz   (with props)
dev/spark/v3.5.1-rc1-bin/spark-3.5.1-bin-hadoop3.tgz.asc
dev/spark/v3.5.1-rc1-bin/spark-3.5.1-bin-hadoop3.tgz.sha512
dev/spark/v3.5.1-rc1-bin/spark-3.5.1-bin-without-hadoop.tgz   (with props)
dev/spark/v3.5.1-rc1-bin/spark-3.5.1-bin-without-hadoop.tgz.asc
dev/spark/v3.5.1-rc1-bin/spark-3.5.1-bin-without-hadoop.tgz.sha512
dev/spark/v3.5.1-rc1-bin/spark-3.5.1.tgz   (with props)
dev/spark/v3.5.1-rc1-bin/spark-3.5.1.tgz.asc
dev/spark/v3.5.1-rc1-bin/spark-3.5.1.tgz.sha512

Added: dev/spark/v3.5.1-rc1-bin/SparkR_3.5.1.tar.gz
==
Binary file - no diff available.

Propchange: dev/spark/v3.5.1-rc1-bin/SparkR_3.5.1.tar.gz
--
svn:mime-type = application/octet-stream

Added: dev/spark/v3.5.1-rc1-bin/SparkR_3.5.1.tar.gz.asc
==
--- dev/spark/v3.5.1-rc1-bin/SparkR_3.5.1.tar.gz.asc (added)
+++ dev/spark/v3.5.1-rc1-bin/SparkR_3.5.1.tar.gz.asc Mon Feb 12 04:25:21 2024
@@ -0,0 +1,17 @@
+-BEGIN PGP SIGNATURE-
+
+iQJHBAABCgAxFiEE/T6ElC5eYQYjWh0lvTVqn4dA5P8FAmXJnQQTHGthYmh3YW5A
+YXBhY2hlLm9yZwAKCRC9NWqfh0Dk/7+kD/9Unoqxcj5xez1j68n4tsYohuqNCGmy
+q9rs3yJB5fRPHu0Ky7A7GYpp95Zxl14ZiT7aKfWY3DMalldIXH6webUjAyiCUtRt
+8EnJDUNA82D6FOeGU7xv4cWebnk0byeQFwHSWABy8sZNSOmgr5PSxqSf3bo9YL/S
+mVIN2iNN7vHQ9Glv9HH/nPw5e6K+B9zbgtbazPRfGS+ft4y82xTu3SLVzF6OZIHU
+SzBzUgN1DJ9djHzP2v+1eQIqGPkJHC7q2HN1rTSFziQJSgqdd8s7f4sdBvKYacnb
+d+yV+S8yM/ZBPfKrCiKNJVgULMB761tsgsDjLoNag3TKz4Z3y9LRJ95TGoLoPtYg
+in0rIgJUjxlTcQhB9M8LXfA+6CxNv1sYi6sO1RORPeoRd9JYmrjPk7Aju6434/oM
+dvcPHHU3j/zuWPJ37F4X9WrXzf3Zhh++sQJRUvTSp/0YV9VRWTb0vkTiSRV66pqt
+nNqrTIeA0waGw5u8Bgb7oDvWx9A5oSSg2nA3DCURokraw8UbHeY5P9b9kSWyeFHr
+9Qfz9wBREKXKJKd8TQqEm+ORSsoqIfifyJ+hZztMJWRFOE7zjJrO6yHJTUOsHe8v
+rArnbAeOIS++R+ZKVlHwBmIABbn1zVxg9lRTrcVXPAIwOYHH+1KwXgxneN6ITLFA
+giyeBFaMKkLA8A==
+=3BBV
+-END PGP SIGNATURE-

Added: dev/spark/v3.5.1-rc1-bin/SparkR_3.5.1.tar.gz.sha512
==
--- dev/spark/v3.5.1-rc1-bin/SparkR_3.5.1.tar.gz.sha512 (added)
+++ dev/spark/v3.5.1-rc1-bin/SparkR_3.5.1.tar.gz.sha512 Mon Feb 12 04:25:21 2024
@@ -0,0 +1 @@
+c67ac72c59ae7e66ad05f324f5bedcac6e93851106890a9afc88b7f8b418ae9b1569c4a7e509ab7e1893e36fb947aa4b63a0c8dd8203a705a7a0970cfc6857b2
  SparkR_3.5.1.tar.gz

Added: dev/spark/v3.5.1-rc1-bin/pyspark-3.5.1.tar.gz
==
Binary file - no diff available.

Propchange: dev/spark/v3.5.1-rc1-bin/pyspark-3.5.1.tar.gz
--
svn:mime-type = application/octet-stream

Added: dev/spark/v3.5.1-rc1-bin/pyspark-3.5.1.tar.gz.asc
==
--- dev/spark/v3.5.1-rc1-bin/pyspark-3.5.1.tar.gz.asc (added)
+++ dev/spark/v3.5.1-rc1-bin/pyspark-3.5.1.tar.gz.asc Mon Feb 12 04:25:21 2024
@@ -0,0 +1,17 @@
+-BEGIN PGP SIGNATURE-
+
+iQJHBAABCgAxFiEE/T6ElC5eYQYjWh0lvTVqn4dA5P8FAmXJnQYTHGthYmh3YW5A
+YXBhY2hlLm9yZwAKCRC9NWqfh0Dk/8nMD/9w3YRs5ZAREOjaOAisAdfqC4U2wBDl
+fZexBdMTvmRQcGjVAjQBOCKpRaU+WXVdsBrJO3Nif4S/nbjYFsZupWUVFarPSjTK
+6Y12sZ/BP5e5ivZDwKhAxtOFVjygj9snlmrg5qKMs1Q8/9bxErE24U1S/G2T62WQ
+6D3gplvSl7+dk4z/WbkHuitwew5NlZnx14/xSNYkTFxamsbFqUss6z/gd/OwQ7P3
+CLmHd9kMkGv7oZb+5muWJvmyxzwEpu29fiY3bLVOK2OFuNCigAxL9bs7FADjvCed
+cqG88k+X3rDonxA25T50h8xIyTwtqFd0gH5y741mCwD5pMD0Mq51Hr6NsTcfssWY
+SL9iGzu80rOkzJuQwsx6j2O/bzzwsCITiIFoi5ykaAgXiIyeOSUHHIQ5RnLmEag5
+f4KNPOasmc6jUwtFbSGYRvBLLWjmGwpELSh/6ALjVaAGVpH74C54f5RRpAoHsEGN
+ybVL4VSU7Car+tw32rvDLut6BjjMxWJJ8Q/HqzOIzSm+TOFtb5OqjWAvOuKXaDN+
+QYfHdWHIRZKDWZOF3eQXYmkqVCLvgvVVzy01Im9eRJm+8HmpFF/tXKlHo93pVqPB
+bQpIN89y9BmGKqi6JAE9ygxwKLVpiZjGhcye2YK8IdJtsatDjUVhnt7ktsZiswXp
+af0JeyGo0aL1xA==
+=NfQZ
+-END PGP SIGNATURE-

Added: dev/spark/v3.5.1-rc1-bin/pyspark-3.5.1.tar.gz.sha512
==
--- dev/spark/v3.5.1-rc1-bin/pyspark-3.5.1.tar.gz.sha512 (added)
+++ dev/spark

(spark) 01/01: Preparing Spark release v3.5.1-rc1

2024-02-11 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to tag v3.5.1-rc1
in repository https://gitbox.apache.org/repos/asf/spark.git

commit 08fe67b9ebf656b6ae7c44163bffba247061aa42
Author: Jungtaek Lim 
AuthorDate: Mon Feb 12 03:39:11 2024 +

Preparing Spark release v3.5.1-rc1
---
 assembly/pom.xml   | 2 +-
 common/kvstore/pom.xml | 2 +-
 common/network-common/pom.xml  | 2 +-
 common/network-shuffle/pom.xml | 2 +-
 common/network-yarn/pom.xml| 2 +-
 common/sketch/pom.xml  | 2 +-
 common/tags/pom.xml| 2 +-
 common/unsafe/pom.xml  | 2 +-
 common/utils/pom.xml   | 2 +-
 connector/avro/pom.xml | 2 +-
 connector/connect/client/jvm/pom.xml   | 2 +-
 connector/connect/common/pom.xml   | 2 +-
 connector/connect/server/pom.xml   | 2 +-
 connector/docker-integration-tests/pom.xml | 2 +-
 connector/kafka-0-10-assembly/pom.xml  | 2 +-
 connector/kafka-0-10-sql/pom.xml   | 2 +-
 connector/kafka-0-10-token-provider/pom.xml| 2 +-
 connector/kafka-0-10/pom.xml   | 2 +-
 connector/kinesis-asl-assembly/pom.xml | 2 +-
 connector/kinesis-asl/pom.xml  | 2 +-
 connector/protobuf/pom.xml | 2 +-
 connector/spark-ganglia-lgpl/pom.xml   | 2 +-
 core/pom.xml   | 2 +-
 docs/_config.yml   | 2 +-
 examples/pom.xml   | 2 +-
 graphx/pom.xml | 2 +-
 hadoop-cloud/pom.xml   | 2 +-
 launcher/pom.xml   | 2 +-
 mllib-local/pom.xml| 2 +-
 mllib/pom.xml  | 2 +-
 pom.xml| 2 +-
 python/pyspark/version.py  | 2 +-
 repl/pom.xml   | 2 +-
 resource-managers/kubernetes/core/pom.xml  | 2 +-
 resource-managers/kubernetes/integration-tests/pom.xml | 2 +-
 resource-managers/mesos/pom.xml| 2 +-
 resource-managers/yarn/pom.xml | 2 +-
 sql/api/pom.xml| 2 +-
 sql/catalyst/pom.xml   | 2 +-
 sql/core/pom.xml   | 2 +-
 sql/hive-thriftserver/pom.xml  | 2 +-
 sql/hive/pom.xml   | 2 +-
 streaming/pom.xml  | 2 +-
 tools/pom.xml  | 2 +-
 44 files changed, 44 insertions(+), 44 deletions(-)

diff --git a/assembly/pom.xml b/assembly/pom.xml
index 45b68dd81cb9..47b54729bbd2 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -21,7 +21,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.5.1-SNAPSHOT
+3.5.1
 ../pom.xml
   
 
diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml
index 7dece9de699c..66e6bb473bf2 100644
--- a/common/kvstore/pom.xml
+++ b/common/kvstore/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.5.1-SNAPSHOT
+3.5.1
 ../../pom.xml
   
 
diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml
index 54c10a05eed2..98897b4424ae 100644
--- a/common/network-common/pom.xml
+++ b/common/network-common/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.5.1-SNAPSHOT
+3.5.1
 ../../pom.xml
   
 
diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml
index 92bf5bc07854..44531ea54cd5 100644
--- a/common/network-shuffle/pom.xml
+++ b/common/network-shuffle/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.5.1-SNAPSHOT
+3.5.1
 ../../pom.xml
   
 
diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml
index 3003927e713c..8fcf20328e8e 100644
--- a/common/network-yarn/pom.xml
+++ b/common/network-yarn/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.5.1-SNAPSHOT
+3.5.1
 ../../pom.xml
   
 
diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml
index 43982032a621..901214de77c9 100644
--- a/common/sketch/pom.xml
+++ b/common/sketch/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.5.1-SNAPSHOT
+3.5.1
 ../../pom.xml
   
 
diff --git a/common/tags/pom.xml b/common/tags/pom.xml
index a54382c0f4d0..6395454245ef 100644

(spark) 01/01: Preparing development version 3.5.2-SNAPSHOT

2024-02-11 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch branch-3.5
in repository https://gitbox.apache.org/repos/asf/spark.git

commit d27bdbeae9c5b634702ad20a58b9d3c68ac9d39d
Author: Jungtaek Lim 
AuthorDate: Mon Feb 12 03:39:15 2024 +

Preparing development version 3.5.2-SNAPSHOT
---
 R/pkg/DESCRIPTION  | 2 +-
 assembly/pom.xml   | 2 +-
 common/kvstore/pom.xml | 2 +-
 common/network-common/pom.xml  | 2 +-
 common/network-shuffle/pom.xml | 2 +-
 common/network-yarn/pom.xml| 2 +-
 common/sketch/pom.xml  | 2 +-
 common/tags/pom.xml| 2 +-
 common/unsafe/pom.xml  | 2 +-
 common/utils/pom.xml   | 2 +-
 connector/avro/pom.xml | 2 +-
 connector/connect/client/jvm/pom.xml   | 2 +-
 connector/connect/common/pom.xml   | 2 +-
 connector/connect/server/pom.xml   | 2 +-
 connector/docker-integration-tests/pom.xml | 2 +-
 connector/kafka-0-10-assembly/pom.xml  | 2 +-
 connector/kafka-0-10-sql/pom.xml   | 2 +-
 connector/kafka-0-10-token-provider/pom.xml| 2 +-
 connector/kafka-0-10/pom.xml   | 2 +-
 connector/kinesis-asl-assembly/pom.xml | 2 +-
 connector/kinesis-asl/pom.xml  | 2 +-
 connector/protobuf/pom.xml | 2 +-
 connector/spark-ganglia-lgpl/pom.xml   | 2 +-
 core/pom.xml   | 2 +-
 docs/_config.yml   | 6 +++---
 examples/pom.xml   | 2 +-
 graphx/pom.xml | 2 +-
 hadoop-cloud/pom.xml   | 2 +-
 launcher/pom.xml   | 2 +-
 mllib-local/pom.xml| 2 +-
 mllib/pom.xml  | 2 +-
 pom.xml| 2 +-
 python/pyspark/version.py  | 2 +-
 repl/pom.xml   | 2 +-
 resource-managers/kubernetes/core/pom.xml  | 2 +-
 resource-managers/kubernetes/integration-tests/pom.xml | 2 +-
 resource-managers/mesos/pom.xml| 2 +-
 resource-managers/yarn/pom.xml | 2 +-
 sql/api/pom.xml| 2 +-
 sql/catalyst/pom.xml   | 2 +-
 sql/core/pom.xml   | 2 +-
 sql/hive-thriftserver/pom.xml  | 2 +-
 sql/hive/pom.xml   | 2 +-
 streaming/pom.xml  | 2 +-
 tools/pom.xml  | 2 +-
 45 files changed, 47 insertions(+), 47 deletions(-)

diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION
index 66faa8031c45..89bee06852be 100644
--- a/R/pkg/DESCRIPTION
+++ b/R/pkg/DESCRIPTION
@@ -1,6 +1,6 @@
 Package: SparkR
 Type: Package
-Version: 3.5.1
+Version: 3.5.2
 Title: R Front End for 'Apache Spark'
 Description: Provides an R Front end for 'Apache Spark' 
<https://spark.apache.org>.
 Authors@R:
diff --git a/assembly/pom.xml b/assembly/pom.xml
index 47b54729bbd2..d1ef9b24afda 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -21,7 +21,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.5.1
+3.5.2-SNAPSHOT
 ../pom.xml
   
 
diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml
index 66e6bb473bf2..9df20f8facf5 100644
--- a/common/kvstore/pom.xml
+++ b/common/kvstore/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.5.1
+3.5.2-SNAPSHOT
 ../../pom.xml
   
 
diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml
index 98897b4424ae..27a53b0f9f3b 100644
--- a/common/network-common/pom.xml
+++ b/common/network-common/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.5.1
+3.5.2-SNAPSHOT
 ../../pom.xml
   
 
diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml
index 44531ea54cd5..93410815e6c0 100644
--- a/common/network-shuffle/pom.xml
+++ b/common/network-shuffle/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.5.1
+3.5.2-SNAPSHOT
 ../../pom.xml
   
 
diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml
index 8fcf20328e8e..a99b8b96402a 100644
--- a/common/network-yarn/pom.xml
+++ b/common/network-yarn/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-paren

(spark) branch branch-3.5 updated (4e4d9f07d095 -> d27bdbeae9c5)

2024-02-11 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a change to branch branch-3.5
in repository https://gitbox.apache.org/repos/asf/spark.git


from 4e4d9f07d095 [SPARK-47022][CONNECT][TESTS][3.5] Fix 
`connect/client/jvm` to have explicit `commons-(io|lang3)` test dependency
 add 08fe67b9ebf6 Preparing Spark release v3.5.1-rc1
 new d27bdbeae9c5 Preparing development version 3.5.2-SNAPSHOT

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 R/pkg/DESCRIPTION  | 2 +-
 assembly/pom.xml   | 2 +-
 common/kvstore/pom.xml | 2 +-
 common/network-common/pom.xml  | 2 +-
 common/network-shuffle/pom.xml | 2 +-
 common/network-yarn/pom.xml| 2 +-
 common/sketch/pom.xml  | 2 +-
 common/tags/pom.xml| 2 +-
 common/unsafe/pom.xml  | 2 +-
 common/utils/pom.xml   | 2 +-
 connector/avro/pom.xml | 2 +-
 connector/connect/client/jvm/pom.xml   | 2 +-
 connector/connect/common/pom.xml   | 2 +-
 connector/connect/server/pom.xml   | 2 +-
 connector/docker-integration-tests/pom.xml | 2 +-
 connector/kafka-0-10-assembly/pom.xml  | 2 +-
 connector/kafka-0-10-sql/pom.xml   | 2 +-
 connector/kafka-0-10-token-provider/pom.xml| 2 +-
 connector/kafka-0-10/pom.xml   | 2 +-
 connector/kinesis-asl-assembly/pom.xml | 2 +-
 connector/kinesis-asl/pom.xml  | 2 +-
 connector/protobuf/pom.xml | 2 +-
 connector/spark-ganglia-lgpl/pom.xml   | 2 +-
 core/pom.xml   | 2 +-
 docs/_config.yml   | 6 +++---
 examples/pom.xml   | 2 +-
 graphx/pom.xml | 2 +-
 hadoop-cloud/pom.xml   | 2 +-
 launcher/pom.xml   | 2 +-
 mllib-local/pom.xml| 2 +-
 mllib/pom.xml  | 2 +-
 pom.xml| 2 +-
 python/pyspark/version.py  | 2 +-
 repl/pom.xml   | 2 +-
 resource-managers/kubernetes/core/pom.xml  | 2 +-
 resource-managers/kubernetes/integration-tests/pom.xml | 2 +-
 resource-managers/mesos/pom.xml| 2 +-
 resource-managers/yarn/pom.xml | 2 +-
 sql/api/pom.xml| 2 +-
 sql/catalyst/pom.xml   | 2 +-
 sql/core/pom.xml   | 2 +-
 sql/hive-thriftserver/pom.xml  | 2 +-
 sql/hive/pom.xml   | 2 +-
 streaming/pom.xml  | 2 +-
 tools/pom.xml  | 2 +-
 45 files changed, 47 insertions(+), 47 deletions(-)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



(spark) tag v3.5.1-rc1 created (now 08fe67b9ebf6)

2024-02-11 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a change to tag v3.5.1-rc1
in repository https://gitbox.apache.org/repos/asf/spark.git


  at 08fe67b9ebf6 (commit)
This tag includes the following new commits:

 new 08fe67b9ebf6 Preparing Spark release v3.5.1-rc1

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



(spark) branch master updated: [SPARK-46994][PYTHON] Refactor PythonWrite to prepare for supporting python data source streaming write

2024-02-07 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new abf8770ffac7 [SPARK-46994][PYTHON] Refactor PythonWrite to prepare for 
supporting python data source streaming write
abf8770ffac7 is described below

commit abf8770ffac7ac5f4dcd5b7b94b744b0267b34d9
Author: Chaoqin Li 
AuthorDate: Thu Feb 8 12:16:49 2024 +0900

[SPARK-46994][PYTHON] Refactor PythonWrite to prepare for supporting python 
data source streaming write

### What changes were proposed in this pull request?
Move PythonBatchWrite out of PythonWrite.

### Why are the changes needed?
This is to prepare for supporting python data source streaming write in the 
future.

### Does this PR introduce _any_ user-facing change?
No.

### How was this patch tested?
Trivial code refactoring, existing test sufficient.

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #45049 from chaoqin-li1123/python_sink.

Authored-by: Chaoqin Li 
Signed-off-by: Jungtaek Lim 
---
 .../datasources/v2/python/PythonWrite.scala| 34 +-
 1 file changed, 21 insertions(+), 13 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/PythonWrite.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/PythonWrite.scala
index d216dfde9974..a10a18e43f64 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/PythonWrite.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/PythonWrite.scala
@@ -26,15 +26,32 @@ class PythonWrite(
 shortName: String,
 info: LogicalWriteInfo,
 isTruncate: Boolean
-  ) extends Write with BatchWrite {
-  private[this] val jobArtifactUUID = 
JobArtifactSet.getCurrentJobArtifactState.map(_.uuid)
+  ) extends Write {
+
+  override def toString: String = shortName
+
+  override def toBatch: BatchWrite = new PythonBatchWrite(ds, shortName, info, 
isTruncate)
+
+  override def description: String = "(Python)"
+
+  override def supportedCustomMetrics(): Array[CustomMetric] =
+ds.source.createPythonMetrics()
+}
+
+class PythonBatchWrite(
+ds: PythonDataSourceV2,
+shortName: String,
+info: LogicalWriteInfo,
+isTruncate: Boolean
+  ) extends BatchWrite {
 
   // Store the pickled data source writer instance.
   private var pythonDataSourceWriter: Array[Byte] = _
 
-  override def createBatchWriterFactory(
-  physicalInfo: PhysicalWriteInfo): DataWriterFactory = {
+  private[this] val jobArtifactUUID = 
JobArtifactSet.getCurrentJobArtifactState.map(_.uuid)
 
+  override def createBatchWriterFactory(physicalInfo: PhysicalWriteInfo): 
DataWriterFactory =
+  {
 val writeInfo = ds.source.createWriteInfoInPython(
   shortName,
   info.schema(),
@@ -53,13 +70,4 @@ class PythonWrite(
   override def abort(messages: Array[WriterCommitMessage]): Unit = {
 ds.source.commitWriteInPython(pythonDataSourceWriter, messages, abort = 
true)
   }
-
-  override def toString: String = shortName
-
-  override def toBatch: BatchWrite = this
-
-  override def description: String = "(Python)"
-
-  override def supportedCustomMetrics(): Array[CustomMetric] =
-ds.source.createPythonMetrics()
 }


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



(spark) branch master updated: [SPARK-46865][SS] Add Batch Support for TransformWithState Operator

2024-02-07 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new becfb94e1c71 [SPARK-46865][SS] Add Batch Support for 
TransformWithState Operator
becfb94e1c71 is described below

commit becfb94e1c713d10dac83300d096be490a912fd2
Author: Eric Marnadi 
AuthorDate: Thu Feb 8 12:15:20 2024 +0900

[SPARK-46865][SS] Add Batch Support for TransformWithState Operator

### What changes were proposed in this pull request?

We are allowing batch queries to use and define the `TransformWithState` 
operator, which was initially introduced for streaming.

### Why are the changes needed?

This is needed to keep up the parity between streaming and batch APIs, 
since we want everything supported in streaming to be supported in batch, as 
well.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

Added unit tests that use the TransformWithState operator with a batch 
query.

### Was this patch authored or co-authored using generative AI tooling?

No

Closes #44884 from ericm-db/tws-batch.

Lead-authored-by: Eric Marnadi 
Co-authored-by: ericm-db <132308037+ericm...@users.noreply.github.com>
Signed-off-by: Jungtaek Lim 
---
 .../analysis/UnsupportedOperationChecker.scala |   3 -
 .../spark/sql/execution/SparkStrategies.scala  |   9 +-
 .../execution/streaming/IncrementalExecution.scala |   2 +-
 .../streaming/StatefulProcessorHandleImpl.scala|  25 ++--
 .../streaming/TransformWithStateExec.scala | 138 ++---
 .../sql/streaming/TransformWithStateSuite.scala|  29 ++---
 6 files changed, 151 insertions(+), 55 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala
index 15a856b273ed..d57464fcefc0 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala
@@ -43,9 +43,6 @@ object UnsupportedOperationChecker extends Logging {
 throwError("dropDuplicatesWithinWatermark is not supported with batch 
" +
   "DataFrames/DataSets")(d)
 
-  case t: TransformWithState =>
-throwError("transformWithState is not supported with batch 
DataFrames/Datasets")(t)
-
   case _ =>
 }
   }
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
index f5c2f17f8826..65347fc9d237 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
@@ -723,7 +723,7 @@ abstract class SparkStrategies extends 
QueryPlanner[SparkPlan] {
* Strategy to convert [[TransformWithState]] logical operator to physical 
operator
* in streaming plans.
*/
-  object TransformWithStateStrategy extends Strategy {
+  object StreamingTransformWithStateStrategy extends Strategy {
 override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
   case TransformWithState(
 keyDeserializer, valueDeserializer, groupingAttributes,
@@ -892,6 +892,13 @@ abstract class SparkStrategies extends 
QueryPlanner[SparkPlan] {
   initialStateGroupAttrs, data, initialStateDataAttrs, output, timeout,
   hasInitialState, planLater(initialState), planLater(child)
 ) :: Nil
+  case logical.TransformWithState(keyDeserializer, valueDeserializer, 
groupingAttributes,
+  dataAttributes, statefulProcessor, timeoutMode, outputMode, 
keyEncoder,
+  outputObjAttr, child) =>
+
TransformWithStateExec.generateSparkPlanForBatchQueries(keyDeserializer, 
valueDeserializer,
+  groupingAttributes, dataAttributes, statefulProcessor, timeoutMode, 
outputMode,
+  keyEncoder, outputObjAttr, planLater(child)) :: Nil
+
   case _: FlatMapGroupsInPandasWithState =>
 // TODO(SPARK-40443): support applyInPandasWithState in batch query
 throw new SparkUnsupportedOperationException("_LEGACY_ERROR_TEMP_3176")
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala
index 08d41b840d04..4469d52618e8 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/s

(spark) branch master updated: [SPARK-46960][SS] Testing Multiple Input Streams with TransformWithState operator

2024-02-05 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 5100b2e5d1aa [SPARK-46960][SS] Testing Multiple Input Streams with 
TransformWithState operator
5100b2e5d1aa is described below

commit 5100b2e5d1aab081e6c5ac9cb3d9a46f5b2b6353
Author: Eric Marnadi 
AuthorDate: Tue Feb 6 12:28:46 2024 +0900

[SPARK-46960][SS] Testing Multiple Input Streams with TransformWithState 
operator

Adding unit tests to test multiple input streams with the 
TransformWithState operator
### What changes were proposed in this pull request?

Added unit tests in TransformWithStateSuite
### Why are the changes needed?

These changes are needed to ensure that we can union multiple input streams 
with the TWS operator, just like any other stateful operator
### Does this PR introduce _any_ user-facing change?

No
### How was this patch tested?

This change is just adding tests. No further tests needed.
### Was this patch authored or co-authored using generative AI tooling?

No

Closes #45004 from ericm-db/multiple-input-streams.

Authored-by: Eric Marnadi 
Signed-off-by: Jungtaek Lim 
---
 .../sql/streaming/TransformWithStateSuite.scala| 96 ++
 1 file changed, 96 insertions(+)

diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala
index 7a6c3f00fc7a..3efef3b37000 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala
@@ -233,6 +233,102 @@ class TransformWithStateSuite extends 
StateStoreMetricsTest
   }
 }
   }
+
+  test("transformWithState - two input streams") {
+withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+  classOf[RocksDBStateStoreProvider].getName,
+  SQLConf.SHUFFLE_PARTITIONS.key ->
+TransformWithStateSuiteUtils.NUM_SHUFFLE_PARTITIONS.toString) {
+  val inputData1 = MemoryStream[String]
+  val inputData2 = MemoryStream[String]
+
+  val result = inputData1.toDS()
+.union(inputData2.toDS())
+.groupByKey(x => x)
+.transformWithState(new RunningCountStatefulProcessor(),
+  TimeoutMode.NoTimeouts(),
+  OutputMode.Update())
+
+  testStream(result, OutputMode.Update())(
+AddData(inputData1, "a"),
+CheckNewAnswer(("a", "1")),
+AddData(inputData2, "a", "b"),
+CheckNewAnswer(("a", "2"), ("b", "1")),
+AddData(inputData1, "a", "b"), // should remove state for "a" and not 
return anything for a
+CheckNewAnswer(("b", "2")),
+AddData(inputData1, "d", "e"),
+AddData(inputData2, "a", "c"), // should recreate state for "a" and 
return count as 1
+CheckNewAnswer(("a", "1"), ("c", "1"), ("d", "1"), ("e", "1")),
+StopStream
+  )
+}
+  }
+
+  test("transformWithState - three input streams") {
+withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+  classOf[RocksDBStateStoreProvider].getName,
+  SQLConf.SHUFFLE_PARTITIONS.key ->
+TransformWithStateSuiteUtils.NUM_SHUFFLE_PARTITIONS.toString) {
+  val inputData1 = MemoryStream[String]
+  val inputData2 = MemoryStream[String]
+  val inputData3 = MemoryStream[String]
+
+  // union 3 input streams
+  val result = inputData1.toDS()
+.union(inputData2.toDS())
+.union(inputData3.toDS())
+.groupByKey(x => x)
+.transformWithState(new RunningCountStatefulProcessor(),
+  TimeoutMode.NoTimeouts(),
+  OutputMode.Update())
+
+  testStream(result, OutputMode.Update())(
+AddData(inputData1, "a"),
+CheckNewAnswer(("a", "1")),
+AddData(inputData2, "a", "b"),
+CheckNewAnswer(("a", "2"), ("b", "1")),
+AddData(inputData3, "a", "b"), // should remove state for "a" and not 
return anything for a
+CheckNewAnswer(("b", "2")),
+AddData(inputData1, "d", "e"),
+AddData(inputData2, "a", "c"), // should recreate state for "a" and 
return count as 1
+CheckNewAnswer(("a", "1"), ("c", "1"), (&qu

svn commit: r67146 - /dev/spark/KEYS

2024-02-03 Thread kabhwan
Author: kabhwan
Date: Sat Feb  3 11:42:23 2024
New Revision: 67146

Log:
Update KEYS

Modified:
dev/spark/KEYS

Modified: dev/spark/KEYS
==
--- dev/spark/KEYS (original)
+++ dev/spark/KEYS Sat Feb  3 11:42:23 2024
@@ -2021,3 +2021,61 @@ fRTcYJkfeano1n8Bmb8EDvwTdF3LNVZfEiTUPEFI
 3Y0blL+bi0NrD82NZvCKoY1RaGFaUO11D7wpmAqf20hDBgRCTvaS9p511YbE2g==
 =nkBh
 -END PGP PUBLIC KEY BLOCK-
+
+pub   rsa4096 2024-02-03 [SC]
+  FD3E84942E5E6106235A1D25BD356A9F8740E4FF
+uid   [ultimate] Jungtaek Lim (CODE SIGNING KEY for ASF) 

+sub   rsa4096 2024-02-03 [E]
+-BEGIN PGP PUBLIC KEY BLOCK-
+
+mQINBGW+JH0BEAC7PdRu3RkVeynRjbO9EJ8ZArSjNWOzvs0lbJ9rk++IgbTWnQtN
+FktO8vajVqpbtVbWoGJoeHt7ueG+CEJQLSmi21k1Hc+2k3c3aNjBIYsU3dFihut7
+kLhA65WOYN7VA6lHeeKSrV4NxgfH0OGIE0phvH8r6QdZQeu+2TmrlbX2STnPdFwu
+aOYJ6ZSQAmeBogISRRNOdBioYyYwREFcGrpErjIMGY7DNu9rR+cLstBmevwmJEBZ
+H3UYrb+pt5l9pOjIXZ+HtGSsvGbS+O7hifayKZc7/L103EKLra8Fuzofopc71C4y
+nnSLuuyubv9PlkuuHMJCFDG6BzVM4E/HPgn0SYwFVGYef6mVEEST2byLL7WDgUl8
+SvL+LB+B3P0evM7tmlldL+kW7TgxriYzKEIfakL/+GnyWmLD7uGxKEyAE8MSwi1+
+a86VA1yYDdA8sFknuIcsaZF+fmk90MwbP1FdirBC6NlguIP4u092Xj5wN6i5WuXn
+mNQj8EeQexQamEYuJQIT/PGw6eE17V3WacymKmPwVHglgvItMCym4lx/Ik8jmuPH
+d2+wUPb0UHlSTkl/CFPSpwmRNmxErBT9Xvghdhu3rv7Ql34cCFiIEy6QBWbsqQ1S
++ov3oZwHS6H2kYYzGZR8pzQGZPp9ibGomzAH2fMVqMYJzx683EkHuuRu9wARAQAB
+tDxKdW5ndGFlayBMaW0gKENPREUgU0lHTklORyBLRVkgZm9yIEFTRikgPGthYmh3
+YW5AYXBhY2hlLm9yZz6JAlEEEwEIADsWIQT9PoSULl5hBiNaHSW9NWqfh0Dk/wUC
+Zb4kfQIbAwULCQgHAgIiAgYVCgkICwIEFgIDAQIeBwIXgAAKCRC9NWqfh0Dk/3Jf
+EACOH4Jjgd0pWkxT9dqX2o1JXddeiawbv43MbwwZkVkT/oOm4bGLILntkuYqFXnq
+9rNP8wE+8wpAchIk3yuy5cPsONhtHxvkWKCnzRsHdt8QBhtdi0KLRMl/9t6280qc
+JUaZdOOmr8vXI0xZ8dy3ZdUyupB8PC3YRm+pTIFVFNwFkJF/+a6C6OQd6qk488BX
+Hi0XgMbTHDJ6pgzaXqR2rUrNDL8L6Per2omaZDbZVw/5GPhNf0LweOT0iXSJtRxR
+N5yvd4AR3M74fswfke7h4nsXx/T4ZctYjs6LAb832MjwZIxLfHFgawvYyVcaJ3/9
+77Gd5uzy9sGNBTmgzVGpmDL7G3AmWN0NQJraj29XWabjsUglgmJecagQGnWNbrvo
+wd64ZzTlFGhOZnMQeSjr2mIWVkgLkzLPCRodY0QTmwaG1ThkhC3APcgnoVdCP4lk
+CSO86NMSfWPT1El36FT5Wtek8jxFHcM6OAAj2RR+Ut1ykHwBE98ErrCMf4Qeq/9R
+awuVaLB9cqLc+PXczguoVuPlHMFp/lSJIS9ztaFsJcSu7a5G/j6iVAgTq/kCF0w4
+fcRkTg2ah5MSt6sroi9eiPhXU+MHPufBvROSozuLqjTHaFOfqIUevSC1MkCfQRjy
+OpoV5O5XyB7sP5XiskgxhJGnqDx3X6h37JLuiEtlgBk6TLkCDQRlviR9ARAAo9X5
+ZtcUima+5NwNCkKi21ZX/55uq97HQK5mrY9Plyw5n+r5WSKUWwa8Cq5qOlUhvo6B
+wJic6V6y7MoLH+GnouaPs98borbRjofkx87v2e7M2BvPJduKeYc1TWDZQMXBQYsY
+2y8sfrbXkL9P4JNfCN9nJ9pVtZ18Dpm4OXkL1cuFGkp+JfvrqnGN/dMXyKh5C/Cr
+TsYJTbQqrUUGgyniQoYXbKkegbfBAxBW1aM2Nd9OEbxHcnpttnZoZyvbDfYcr9R5
+LGirLKh4/MfF9LxWc/hIFaPSo2HqLnPcUv0N2OS5EotqfezqJTsQkk5vtrSmxcCs
+lBnAHjeDYesBZ3vEz6C2l45aF1zV5qJgGoSUy997JVf5cHp8sUz/tGO4SM0hKVi2
+0C2u5uTBI32X0DYi6gPpBECtyzDJA+k17/sZKeYhHs2ec/muC8siZJMeTLGquC7d
+RDrl6cQOgnbcxvhxvI4u6AY5xlXklESj6IBSLgUYDsoAMj0/EBgfvA4qJ5FBerEO
+S5dsZ9vjfcQfAmT2cwmslMWXd3iyu+bUk9GLG7Q9s0vufEoh4VPlv+4llQWoTdE7
+bOJoPGxiUwEo2IhBxLEZRt50FPKxHdqG2GGlkDvFl6hD6h6Gsn6MFniezu36dzG6
+HqvkOEffr1RYOL4QaLDYZXHP2VWnEzuNIN0Z298AEQEAAYkCNgQYAQgAIBYhBP0+
+hJQuXmEGI1odJb01ap+HQOT/BQJlviR9AhsMAAoJEL01ap+HQOT/2AIP/182BxBk
+oAqCaZWrSVx0h9ETpzB8MMk45n3aVia0N5Qv890D3douEdvV5fK8ANcQfcSB/wje
+gG/Uo8b4r0W4vOaWrXEBEybr2twtQgtE+I/LAkPQbAHLS3zOqhTnLpJoSUT3dqEy
+BZ/HvF7o/AyqSHp0k7uKyIVJXNdL/nPzrfUQiQQnxXN2jj3jTxx6woPNHw9EvLCc
+/jh03QfZyc6453BdXYlbZ1VT+1TSoT/LalQ/s8AEJM+m2Pkv+zIggzxpaSAXqGNB
+9LdwcCPcP1RL97veFkXNXs3dIhvfPzzriVNcVY0XkdXMrqjztp3jma0lUa5gB2wz
+b2QCC+zv8S7g70FnZwDSVe4bCfbjAubck/pI1yOaES77hnnQI6XEXLhabqZPwlA3
+OUmwxG3AYCb/c+Qc1nGYDsUTe3e8b9ewKB9/SpPlaf0WCMKTJrPMowWS31VmzCO+
+h3gw3QsCzarqE0xg7ffs5Q41+JIgvrK8GdCThtCBU+Bo6YJLBjV04SjOwaRgKk5T
+5ii6rCzOMtpbEXewrOChQmAV22bXIbwNDafBd0xOaBT6oGrqJ8JLYWOlgIsk+GlC
+QuYsshdUdZCg8yDzqGW2qQNtqfdP/J2jUMdZRwdYDQ08z+6L5Fy7JVVLqicukUPc
+ThVo7dEVoknhannfoULNv5ekjZ/LsFNGHRUZ
+=9cvL
+-END PGP PUBLIC KEY BLOCK-
+



-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



(spark) branch master updated: [SPARK-46911][SS] Adding deleteIfExists operator to StatefulProcessorHandleImpl

2024-02-02 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 25d96f7bacb4 [SPARK-46911][SS] Adding deleteIfExists operator to 
StatefulProcessorHandleImpl
25d96f7bacb4 is described below

commit 25d96f7bacb43a7d5a835454ecc075e40d4f3c93
Author: Eric Marnadi 
AuthorDate: Fri Feb 2 22:32:42 2024 +0900

[SPARK-46911][SS] Adding deleteIfExists operator to 
StatefulProcessorHandleImpl

### What changes were proposed in this pull request?

Adding the `deleteIfExists` method to the `StatefulProcessorHandle` in 
order to remove state variables from the State Store. Implemented only for 
RocksDBStateStoreProvider, as we do not currently support multiple column 
families for HDFS.

### Why are the changes needed?

This functionality is needed to so users can remove state from the state 
store from the StatefulProcessorHandleImpl

### Does this PR introduce _any_ user-facing change?

Yes - this functionality (removing column families) was previously not 
supported from our RocksDB client.

### How was this patch tested?

Added a unit test that creates two streams with the same checkpoint 
directory. The second stream removes state that was created in the first stream 
upon initialization. We ensure that the state from the previous stream isn't 
kept.

### Was this patch authored or co-authored using generative AI tooling?

Closes #44903 from ericm-db/deleteIfExists.

Authored-by: Eric Marnadi 
Signed-off-by: Jungtaek Lim 
---
 .../src/main/resources/error/error-classes.json|  11 +++
 ...r-conditions-unsupported-feature-error-class.md |   4 +
 docs/sql-error-conditions.md   |   6 ++
 .../sql/streaming/StatefulProcessorHandle.scala|   6 ++
 .../streaming/StatefulProcessorHandleImpl.scala|  12 +++
 .../state/HDFSBackedStateStoreProvider.scala   |   6 ++
 .../sql/execution/streaming/state/RocksDB.scala|  16 
 .../state/RocksDBStateStoreProvider.scala  |   5 +
 .../sql/execution/streaming/state/StateStore.scala |   6 ++
 .../streaming/state/StateStoreErrors.scala |  22 +
 .../streaming/state/MemoryStateStore.scala |   4 +
 .../sql/streaming/TransformWithStateSuite.scala| 104 +
 12 files changed, 202 insertions(+)

diff --git a/common/utils/src/main/resources/error/error-classes.json 
b/common/utils/src/main/resources/error/error-classes.json
index baefb05a7070..136825ab374d 100644
--- a/common/utils/src/main/resources/error/error-classes.json
+++ b/common/utils/src/main/resources/error/error-classes.json
@@ -3241,6 +3241,12 @@
 ],
 "sqlState" : "0A000"
   },
+  "STATE_STORE_CANNOT_REMOVE_DEFAULT_COLUMN_FAMILY" : {
+"message" : [
+  "Failed to remove default column family with reserved 
name=."
+],
+"sqlState" : "42802"
+  },
   "STATE_STORE_MULTIPLE_VALUES_PER_KEY" : {
 "message" : [
   "Store does not support multiple values per key"
@@ -3950,6 +3956,11 @@
   "Creating multiple column families with  is not 
supported."
 ]
   },
+  "STATE_STORE_REMOVING_COLUMN_FAMILIES" : {
+"message" : [
+  "Removing column families with  is not 
supported."
+]
+  },
   "TABLE_OPERATION" : {
 "message" : [
   "Table  does not support . Please check the 
current catalog and namespace to make sure the qualified table name is 
expected, and also check the catalog implementation which is configured by 
\"spark.sql.catalog\"."
diff --git a/docs/sql-error-conditions-unsupported-feature-error-class.md 
b/docs/sql-error-conditions-unsupported-feature-error-class.md
index 1b12c4bfc1b3..8d42ecdce790 100644
--- a/docs/sql-error-conditions-unsupported-feature-error-class.md
+++ b/docs/sql-error-conditions-unsupported-feature-error-class.md
@@ -194,6 +194,10 @@ set PROPERTIES and DBPROPERTIES at the same time.
 
 Creating multiple column families with `` is not supported.
 
+## STATE_STORE_REMOVING_COLUMN_FAMILIES
+
+Removing column families with `` is not supported.
+
 ## TABLE_OPERATION
 
 Table `` does not support ``. Please check the current 
catalog and namespace to make sure the qualified table name is expected, and 
also check the catalog implementation which is configured by 
"spark.sql.catalog".
diff --git a/docs/sql-error-conditions.md b/docs/sql-error-conditions.md
index 3a2c4d261352..c704b1c10c46 100644
--- a/docs/sql-error-conditions.md
+++ b/docs/sql-error-conditions.md
@@ -2025,6 +2025,12 @@ The SQL config `` cannot be found. Please 
verify that the config ex

(spark) branch master updated: [SPARK-46864][SS] Onboard Arbitrary StateV2 onto New Error Class Framework

2024-02-01 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 00e63d63f9af [SPARK-46864][SS] Onboard Arbitrary StateV2 onto New 
Error Class Framework
00e63d63f9af is described below

commit 00e63d63f9af6ef186e14159ddbe8bb8d1c8690b
Author: Eric Marnadi 
AuthorDate: Fri Feb 2 05:38:15 2024 +0900

[SPARK-46864][SS] Onboard Arbitrary StateV2 onto New Error Class Framework

### What changes were proposed in this pull request?

This PR proposes to apply error class framework to the new data source, 
State API V2.

### Why are the changes needed?

Error class framework is a standard to represent all exceptions in Spark.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

Refactored unit tests to check that the right error class was being thrown 
in certain situations

### Was this patch authored or co-authored using generative AI tooling?

No

Closes #44883 from ericm-db/state-v2-error-class.

Lead-authored-by: Eric Marnadi 
Co-authored-by: ericm-db <132308037+ericm...@users.noreply.github.com>
Signed-off-by: Jungtaek Lim 
---
 .../src/main/resources/error/error-classes.json| 29 +++
 ...r-conditions-unsupported-feature-error-class.md |  4 ++
 docs/sql-error-conditions.md   | 24 ++
 .../sql/execution/streaming/ValueStateImpl.scala   |  5 +-
 .../state/HDFSBackedStateStoreProvider.scala   |  3 +-
 .../streaming/state/StateStoreChangelog.scala  | 16 +++
 .../streaming/state/StateStoreErrors.scala | 56 ++
 .../streaming/state/MemoryStateStore.scala |  2 +-
 .../execution/streaming/state/RocksDBSuite.scala   | 37 +-
 .../streaming/state/ValueStateSuite.scala  | 43 +++--
 10 files changed, 199 insertions(+), 20 deletions(-)

diff --git a/common/utils/src/main/resources/error/error-classes.json 
b/common/utils/src/main/resources/error/error-classes.json
index 8e47490f5a61..baefb05a7070 100644
--- a/common/utils/src/main/resources/error/error-classes.json
+++ b/common/utils/src/main/resources/error/error-classes.json
@@ -1656,6 +1656,12 @@
 ],
 "sqlState" : "XX000"
   },
+  "INTERNAL_ERROR_TWS" : {
+"message" : [
+  ""
+],
+"sqlState" : "XX000"
+  },
   "INTERVAL_ARITHMETIC_OVERFLOW" : {
 "message" : [
   "."
@@ -3235,6 +3241,18 @@
 ],
 "sqlState" : "0A000"
   },
+  "STATE_STORE_MULTIPLE_VALUES_PER_KEY" : {
+"message" : [
+  "Store does not support multiple values per key"
+],
+"sqlState" : "42802"
+  },
+  "STATE_STORE_UNSUPPORTED_OPERATION" : {
+"message" : [
+  " operation not supported with "
+],
+"sqlState" : "XXKST"
+  },
   "STATIC_PARTITION_COLUMN_IN_INSERT_COLUMN_LIST" : {
 "message" : [
   "Static partition column  is also specified in the column 
list."
@@ -3388,6 +3406,12 @@
 ],
 "sqlState" : "428EK"
   },
+  "TWS_VALUE_SHOULD_NOT_BE_NULL" : {
+"message" : [
+  "New value should be non-null for "
+],
+"sqlState" : "22004"
+  },
   "UDTF_ALIAS_NUMBER_MISMATCH" : {
 "message" : [
   "The number of aliases supplied in the AS clause does not match the 
number of columns output by the UDTF.",
@@ -3921,6 +3945,11 @@
   " is a VARIABLE and cannot be updated using the SET 
statement. Use SET VARIABLE  = ... instead."
 ]
   },
+  "STATE_STORE_MULTIPLE_COLUMN_FAMILIES" : {
+"message" : [
+  "Creating multiple column families with  is not 
supported."
+]
+  },
   "TABLE_OPERATION" : {
 "message" : [
   "Table  does not support . Please check the 
current catalog and namespace to make sure the qualified table name is 
expected, and also check the catalog implementation which is configured by 
\"spark.sql.catalog\"."
diff --git a/docs/sql-error-conditions-unsupported-feature-error-class.md 
b/docs/sql-error-conditions-unsupported-feature-error-class.md
index d90d2b2a109f..1b12c4bfc1b3 100644
--- a/docs/sql-error-conditions-unsupported-feature-error-class.md
+++ b/docs/sql-error-conditions-unsupported-feature-error-class.md
@@ -190,6 +190,10 @@ set PROPERTIES and DBPROPERTIES at the same time.
 
 `` is a VARIABLE and cannot be updated using the SET statement. 
Use S

(spark) branch master updated: [SPARK-46852][SS] Remove use of explicit key encoder and pass it implicitly to the operator for transformWithState operator

2024-02-01 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new e610d1d8f79b [SPARK-46852][SS] Remove use of explicit key encoder and 
pass it implicitly to the operator for transformWithState operator
e610d1d8f79b is described below

commit e610d1d8f79b913cb9ee9236a6325202c58d8397
Author: Anish Shrigondekar 
AuthorDate: Thu Feb 1 22:31:07 2024 +0900

[SPARK-46852][SS] Remove use of explicit key encoder and pass it implicitly 
to the operator for transformWithState operator

### What changes were proposed in this pull request?
Remove use of explicit key encoder and pass it implicitly to the operator 
for transformWithState operator

### Why are the changes needed?
Changes needed to avoid asking users to provide explicit key encoder and we 
also might need them for subsequent timer related changes

### Does this PR introduce _any_ user-facing change?
Yes

### How was this patch tested?
Existing unit tests

### Was this patch authored or co-authored using generative AI tooling?
No

Closes #44974 from anishshri-db/task/SPARK-46852.

Authored-by: Anish Shrigondekar 
Signed-off-by: Jungtaek Lim 
---
 .../sql/streaming/StatefulProcessorHandle.scala|  5 +
 .../spark/sql/catalyst/plans/logical/object.scala  |  3 +++
 .../spark/sql/execution/SparkStrategies.scala  |  3 ++-
 .../streaming/StatefulProcessorHandleImpl.scala| 13 +
 .../streaming/TransformWithStateExec.scala |  6 +-
 .../sql/execution/streaming/ValueStateImpl.scala   | 12 +---
 .../streaming/state/ValueStateSuite.scala  | 22 +++---
 .../sql/streaming/TransformWithStateSuite.scala|  8 +++-
 8 files changed, 39 insertions(+), 33 deletions(-)

diff --git 
a/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala
 
b/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala
index 302de4a3c947..5eaccceb947c 100644
--- 
a/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala
+++ 
b/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala
@@ -19,7 +19,6 @@ package org.apache.spark.sql.streaming
 import java.io.Serializable
 
 import org.apache.spark.annotation.{Evolving, Experimental}
-import org.apache.spark.sql.Encoder
 
 /**
  * Represents the operation handle provided to the stateful processor used in 
the
@@ -34,12 +33,10 @@ private[sql] trait StatefulProcessorHandle extends 
Serializable {
* The user must ensure to call this function only within the `init()` 
method of the
* StatefulProcessor.
* @param stateName - name of the state variable
-   * @param keyEncoder - Spark SQL Encoder for key
-   * @tparam K - type of key
* @tparam T - type of state variable
* @return - instance of ValueState of type T that can be used to store 
state persistently
*/
-  def getValueState[K, T](stateName: String, keyEncoder: Encoder[K]): 
ValueState[T]
+  def getValueState[T](stateName: String): ValueState[T]
 
   /** Function to return queryInfo for currently running task */
   def getQueryInfo(): QueryInfo
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala
index 8f937dd5a777..cb8673d20ed3 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala
@@ -577,6 +577,7 @@ object TransformWithState {
   timeoutMode: TimeoutMode,
   outputMode: OutputMode,
   child: LogicalPlan): LogicalPlan = {
+val keyEncoder = encoderFor[K]
 val mapped = new TransformWithState(
   UnresolvedDeserializer(encoderFor[K].deserializer, groupingAttributes),
   UnresolvedDeserializer(encoderFor[V].deserializer, dataAttributes),
@@ -585,6 +586,7 @@ object TransformWithState {
   statefulProcessor.asInstanceOf[StatefulProcessor[Any, Any, Any]],
   timeoutMode,
   outputMode,
+  keyEncoder.asInstanceOf[ExpressionEncoder[Any]],
   CatalystSerde.generateObjAttr[U],
   child
 )
@@ -600,6 +602,7 @@ case class TransformWithState(
 statefulProcessor: StatefulProcessor[Any, Any, Any],
 timeoutMode: TimeoutMode,
 outputMode: OutputMode,
+keyEncoder: ExpressionEncoder[Any],
 outputObjAttr: Attribute,
 child: LogicalPlan) extends UnaryNode with ObjectProducer {
 
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
index 5d4063d125c8..f5c2f17f8826 100644
--- 
a/sql/core/src

(spark) branch master updated: [SPARK-46736][PROTOBUF] retain empty message field in protobuf connector

2024-01-29 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 3f7994217d5a [SPARK-46736][PROTOBUF] retain empty message field in 
protobuf connector
3f7994217d5a is described below

commit 3f7994217d5a8d2816165459c1ce10d9b31bc7fd
Author: Chaoqin Li 
AuthorDate: Tue Jan 30 11:02:35 2024 +0900

[SPARK-46736][PROTOBUF] retain empty message field in protobuf connector

### What changes were proposed in this pull request?
Since Spark doesn't allow empty StructType, empty proto message type as 
field will be dropped by default. introduce an option to allow retaining an 
empty message field by inserting a dummy column.

### Why are the changes needed?
In protobuf, it is common to have empty message type without any field as a 
place holder, in some case people may not want to drop these empty message 
field.

### Does this PR introduce _any_ user-facing change?
Yes. The default behavior is still dropping an empty message field. The new 
option will enable customer to keep the empty message field though they will 
observe a dummy column.

### How was this patch tested?
Unit test and integration test.

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #44643 from chaoqin-li1123/empty_proto.

Authored-by: Chaoqin Li 
Signed-off-by: Jungtaek Lim 
---
 .../spark/sql/protobuf/utils/ProtobufOptions.scala |  17 +++
 .../sql/protobuf/utils/SchemaConverters.scala  |  34 --
 .../test/resources/protobuf/functions_suite.proto  |   9 ++
 .../sql/protobuf/ProtobufFunctionsSuite.scala  | 123 -
 4 files changed, 171 insertions(+), 12 deletions(-)

diff --git 
a/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/utils/ProtobufOptions.scala
 
b/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/utils/ProtobufOptions.scala
index 5f8c42df365a..6644bce98293 100644
--- 
a/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/utils/ProtobufOptions.scala
+++ 
b/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/utils/ProtobufOptions.scala
@@ -207,6 +207,23 @@ private[sql] class ProtobufOptions(
   //nil => nil, Int32Value(0) => 0, Int32Value(100) => 100.
   val unwrapWellKnownTypes: Boolean =
 parameters.getOrElse("unwrap.primitive.wrapper.types", 
false.toString).toBoolean
+
+  // Since Spark doesn't allow writing empty StructType, empty proto message 
type will be
+  // dropped by default. Setting this option to true will insert a dummy 
column to empty proto
+  // message so that the empty message will be retained.
+  // For example, an empty message is used as field in another message:
+  //
+  // ```
+  // message A {}
+  // message B {A a = 1, string name = 2}
+  // ```
+  //
+  // By default, in the spark schema field a will be dropped, which result in 
schema
+  // b struct
+  // If retain.empty.message.types=true, field a will be retained by inserting 
a dummy column.
+  // b struct, name: string>
+  val retainEmptyMessage: Boolean =
+parameters.getOrElse("retain.empty.message.types", 
false.toString).toBoolean
 }
 
 private[sql] object ProtobufOptions {
diff --git 
a/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/utils/SchemaConverters.scala
 
b/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/utils/SchemaConverters.scala
index b35aa153aaa1..feb5aed03451 100644
--- 
a/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/utils/SchemaConverters.scala
+++ 
b/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/utils/SchemaConverters.scala
@@ -51,12 +51,13 @@ object SchemaConverters extends Logging {
   def toSqlTypeHelper(
   descriptor: Descriptor,
   protobufOptions: ProtobufOptions): SchemaType = {
-SchemaType(
-  StructType(descriptor.getFields.asScala.flatMap(
-structFieldFor(_,
-  Map(descriptor.getFullName -> 1),
-  protobufOptions: ProtobufOptions)).toArray),
-  nullable = true)
+val fields = descriptor.getFields.asScala.flatMap(
+  structFieldFor(_,
+Map(descriptor.getFullName -> 1),
+protobufOptions: ProtobufOptions)).toSeq
+if (fields.isEmpty && protobufOptions.retainEmptyMessage) {
+  
SchemaType(convertEmptyProtoToStructWithDummyField(descriptor.getFullName), 
nullable = true)
+} else SchemaType(StructType(fields), nullable = true)
   }
 
   // existingRecordNames: Map[String, Int] used to track the depth of 
recursive fields and to
@@ -212,11 +213,15 @@ object SchemaConverters extends Logging {
   ).toSeq
   fields match {
 case Nil =>
-  log.info(
-s"Dropping 

(spark) branch branch-3.5 updated: [SPARK-46796][SS] Ensure the correct remote files (mentioned in metadata.zip) are used on RocksDB version load

2024-01-24 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch branch-3.5
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.5 by this push:
 new ef33b9c50806 [SPARK-46796][SS] Ensure the correct remote files 
(mentioned in metadata.zip) are used on RocksDB version load
ef33b9c50806 is described below

commit ef33b9c50806475f287267c05278aeda3645abac
Author: Bhuwan Sahni 
AuthorDate: Wed Jan 24 21:35:33 2024 +0900

[SPARK-46796][SS] Ensure the correct remote files (mentioned in 
metadata.zip) are used on RocksDB version load

This PR ensures that RocksDB loads do not run into SST file Version ID 
mismatch issue. RocksDB has added validation to ensure exact same SST file is 
used during database load from snapshot. Current streaming state suffers from 
certain edge cases where this condition is violated resulting in state load 
failure.

The changes introduced are:

1. Ensure that the local SST file is exactly the same DFS file (as per 
mapping in metadata.zip). We keep track of the DFS file path for a local SST 
file, and re download the SST file in case DFS file has a different UUID in 
metadata zip.
2. Reset lastSnapshotVersion in RocksDB when Rocks DB is loaded. Changelog 
checkpoint relies on this version for future snapshots. Currently, if a older 
version is reloaded we were not uploading snapshots as lastSnapshotVersion was 
pointing to a higher snapshot of a cleanup database.

We need to ensure that the correct SST files are used on executor during 
RocksDB load as per mapping in metadata.zip. With current implementation, its 
possible that the executor uses a SST file (with a different UUID) from a older 
version which is not the exact file mapped in the metadata.zip. This can cause 
version Id mismatch errors while loading RocksDB leading to streaming query 
failures.

See https://issues.apache.org/jira/browse/SPARK-46796 for failure scenarios.

No

Added exhaustive unit testcases covering the scenarios.

No

Closes #44837 from sahnib/SPARK-46796.

Authored-by: Bhuwan Sahni 
Signed-off-by: Jungtaek Lim 
(cherry picked from commit f25ebe52b9b84ece9b3c5ae30b83eaaef52ec55b)
Signed-off-by: Jungtaek Lim 
---
 .../sql/execution/streaming/state/RocksDB.scala|   3 +
 .../streaming/state/RocksDBFileManager.scala   |  92 --
 .../execution/streaming/state/RocksDBSuite.scala   | 314 -
 3 files changed, 372 insertions(+), 37 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala
index 2398b7780726..0c9738a6b081 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala
@@ -151,6 +151,8 @@ class RocksDB(
 val metadata = 
fileManager.loadCheckpointFromDfs(latestSnapshotVersion, workingDir)
 loadedVersion = latestSnapshotVersion
 
+// reset last snapshot version
+lastSnapshotVersion = 0L
 openDB()
 
 numKeysOnWritingVersion = if (!conf.trackTotalNumberOfRows) {
@@ -191,6 +193,7 @@ class RocksDB(
*/
   private def replayChangelog(endVersion: Long): Unit = {
 for (v <- loadedVersion + 1 to endVersion) {
+  logInfo(s"replaying changelog from version $loadedVersion -> 
$endVersion")
   var changelogReader: StateStoreChangelogReader = null
   try {
 changelogReader = fileManager.getChangelogReader(v)
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala
index faf9cd701aec..300a3b8137b4 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala
@@ -132,6 +132,15 @@ class RocksDBFileManager(
   import RocksDBImmutableFile._
 
   private val versionToRocksDBFiles = new ConcurrentHashMap[Long, 
Seq[RocksDBImmutableFile]]
+
+
+  // used to keep a mapping of the exact Dfs file that was used to create a 
local SST file.
+  // The reason this is a separate map because versionToRocksDBFiles can 
contain multiple similar
+  // SST files to a particular local file (for example 1.sst can map to 
1-UUID1.sst in v1 and
+  // 1-UUID2.sst in v2). We need to capture the exact file used to ensure 
Version ID compatibility
+  // across SST files and RocksDB manifest.
+  private[sql] val localFilesToDfsFiles = new ConcurrentHashMap[String, 
RocksDBImmutableFile]
+
   private lazy val fm = CheckpointFileManager.create(new

(spark) branch master updated: [SPARK-46796][SS] Ensure the correct remote files (mentioned in metadata.zip) are used on RocksDB version load

2024-01-24 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new f25ebe52b9b8 [SPARK-46796][SS] Ensure the correct remote files 
(mentioned in metadata.zip) are used on RocksDB version load
f25ebe52b9b8 is described below

commit f25ebe52b9b84ece9b3c5ae30b83eaaef52ec55b
Author: Bhuwan Sahni 
AuthorDate: Wed Jan 24 21:35:33 2024 +0900

[SPARK-46796][SS] Ensure the correct remote files (mentioned in 
metadata.zip) are used on RocksDB version load

### What changes were proposed in this pull request?

This PR ensures that RocksDB loads do not run into SST file Version ID 
mismatch issue. RocksDB has added validation to ensure exact same SST file is 
used during database load from snapshot. Current streaming state suffers from 
certain edge cases where this condition is violated resulting in state load 
failure.

The changes introduced are:

1. Ensure that the local SST file is exactly the same DFS file (as per 
mapping in metadata.zip). We keep track of the DFS file path for a local SST 
file, and re download the SST file in case DFS file has a different UUID in 
metadata zip.
2. Reset lastSnapshotVersion in RocksDB when Rocks DB is loaded. Changelog 
checkpoint relies on this version for future snapshots. Currently, if a older 
version is reloaded we were not uploading snapshots as lastSnapshotVersion was 
pointing to a higher snapshot of a cleanup database.

### Why are the changes needed?

We need to ensure that the correct SST files are used on executor during 
RocksDB load as per mapping in metadata.zip. With current implementation, its 
possible that the executor uses a SST file (with a different UUID) from a older 
version which is not the exact file mapped in the metadata.zip. This can cause 
version Id mismatch errors while loading RocksDB leading to streaming query 
failures.

See https://issues.apache.org/jira/browse/SPARK-46796 for failure scenarios.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

Added exhaustive unit testcases covering the scenarios.

### Was this patch authored or co-authored using generative AI tooling?

No

Closes #44837 from sahnib/SPARK-46796.

Authored-by: Bhuwan Sahni 
Signed-off-by: Jungtaek Lim 
---
 .../sql/execution/streaming/state/RocksDB.scala|   3 +
 .../streaming/state/RocksDBFileManager.scala   |  92 --
 .../execution/streaming/state/RocksDBSuite.scala   | 314 -
 3 files changed, 372 insertions(+), 37 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala
index 0284d4c9d303..8997e8df6c89 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala
@@ -162,6 +162,8 @@ class RocksDB(
 val metadata = 
fileManager.loadCheckpointFromDfs(latestSnapshotVersion, workingDir)
 loadedVersion = latestSnapshotVersion
 
+// reset last snapshot version
+lastSnapshotVersion = 0L
 openDB()
 
 numKeysOnWritingVersion = if (!conf.trackTotalNumberOfRows) {
@@ -202,6 +204,7 @@ class RocksDB(
*/
   private def replayChangelog(endVersion: Long): Unit = {
 for (v <- loadedVersion + 1 to endVersion) {
+  logInfo(s"replaying changelog from version $loadedVersion -> 
$endVersion")
   var changelogReader: StateStoreChangelogReader = null
   try {
 changelogReader = fileManager.getChangelogReader(v)
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala
index 5e2b6afee68e..794e39e2bacc 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala
@@ -134,6 +134,15 @@ class RocksDBFileManager(
   import RocksDBImmutableFile._
 
   private val versionToRocksDBFiles = new ConcurrentHashMap[Long, 
Seq[RocksDBImmutableFile]]
+
+
+  // used to keep a mapping of the exact Dfs file that was used to create a 
local SST file.
+  // The reason this is a separate map because versionToRocksDBFiles can 
contain multiple similar
+  // SST files to a particular local file (for example 1.sst can map to 
1-UUID1.sst in v1 and
+  // 1-UUID2.sst in v2). We need to capture the exact file used to ensure 
Version ID compatibility
+  // across SST files and 

(spark) branch master updated: [SPARK-46777][SS] Refactor `StreamingDataSourceV2Relation` catalyst structure to be more on-par with the batch version

2024-01-22 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 02533d71806e [SPARK-46777][SS] Refactor 
`StreamingDataSourceV2Relation` catalyst structure to be more on-par with the 
batch version
02533d71806e is described below

commit 02533d71806ec0be97ec793d680189093c9a0ecb
Author: jackierwzhang 
AuthorDate: Mon Jan 22 18:58:55 2024 +0900

[SPARK-46777][SS] Refactor `StreamingDataSourceV2Relation` catalyst 
structure to be more on-par with the batch version

### What changes were proposed in this pull request?
This PR refactors `StreamingDataSourceV2Relation` into 
`StreamingDataSourceV2Relation` and `StreamingDataSourceV2ScanRelation` to 
achieve better parity with the batch version. This prepares the codebase to be 
able to extend certain V2 optimization rules (e.g. `V2ScanRelationPushDown`) to 
be applied to streaming in the future.

### Why are the changes needed?
As described above, we would like to start reuse certain V2 batch 
optimization rules to apply to streaming relations.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
This is a pure refactoring, existing tests should be sufficient.

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #44818 from jackierwzhang/spark-46777.

Authored-by: jackierwzhang 
Signed-off-by: Jungtaek Lim 
---
 .../sql/kafka010/KafkaMicroBatchSourceSuite.scala  |  7 +-
 .../catalyst/streaming/StreamingRelationV2.scala   |  4 +-
 .../datasources/v2/DataSourceV2Relation.scala  | 83 --
 .../datasources/v2/DataSourceV2Strategy.scala  |  4 +-
 .../execution/streaming/MicroBatchExecution.scala  | 12 ++--
 .../sql/execution/streaming/ProgressReporter.scala |  4 +-
 .../streaming/continuous/ContinuousExecution.scala | 14 ++--
 .../sources/RateStreamProviderSuite.scala  |  4 +-
 .../streaming/sources/TextSocketStreamSuite.scala  |  4 +-
 .../apache/spark/sql/streaming/StreamSuite.scala   |  8 +--
 .../apache/spark/sql/streaming/StreamTest.scala|  4 +-
 .../sql/streaming/StreamingQueryManagerSuite.scala |  4 +-
 .../streaming/test/DataStreamTableAPISuite.scala   |  2 +-
 13 files changed, 99 insertions(+), 55 deletions(-)

diff --git 
a/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
 
b/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
index cee0d9a3dd72..fb5e71a1e7b8 100644
--- 
a/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
+++ 
b/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
@@ -40,7 +40,7 @@ import org.apache.spark.TestUtils
 import org.apache.spark.sql.{Dataset, ForeachWriter, Row, SparkSession}
 import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap
 import org.apache.spark.sql.connector.read.streaming.SparkDataStream
-import 
org.apache.spark.sql.execution.datasources.v2.StreamingDataSourceV2Relation
+import 
org.apache.spark.sql.execution.datasources.v2.StreamingDataSourceV2ScanRelation
 import org.apache.spark.sql.execution.exchange.ReusedExchangeExec
 import org.apache.spark.sql.execution.streaming._
 import 
org.apache.spark.sql.execution.streaming.AsyncProgressTrackingMicroBatchExecution.{ASYNC_PROGRESS_TRACKING_CHECKPOINTING_INTERVAL_MS,
 ASYNC_PROGRESS_TRACKING_ENABLED}
@@ -125,7 +125,8 @@ abstract class KafkaSourceTest extends StreamTest with 
SharedSparkSession with K
   val sources: Seq[SparkDataStream] = {
 query.get.logicalPlan.collect {
   case StreamingExecutionRelation(source: KafkaSource, _, _) => source
-  case r: StreamingDataSourceV2Relation if 
r.stream.isInstanceOf[KafkaMicroBatchStream] ||
+  case r: StreamingDataSourceV2ScanRelation
+if r.stream.isInstanceOf[KafkaMicroBatchStream] ||
   r.stream.isInstanceOf[KafkaContinuousStream] =>
 r.stream
 }
@@ -1654,7 +1655,7 @@ class KafkaMicroBatchV2SourceSuite extends 
KafkaMicroBatchSourceSuiteBase {
   makeSureGetOffsetCalled,
   AssertOnQuery { query =>
 query.logicalPlan.exists {
-  case r: StreamingDataSourceV2Relation => 
r.stream.isInstanceOf[KafkaMicroBatchStream]
+  case r: StreamingDataSourceV2ScanRelation => 
r.stream.isInstanceOf[KafkaMicroBatchStream]
   case _ => false
 }
   }
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/streaming/StreamingRelationV2.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/streaming/StreamingRelationV2.scala
index ab0352b606e5..c1d7daa6cfcf 100644
--- 
a/sql/catalyst/

(spark) branch master updated: [SPARK-46731][SS] Manage state store provider instance by state data source - reader

2024-01-19 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 56730f6390a1 [SPARK-46731][SS] Manage state store provider instance by 
state data source - reader
56730f6390a1 is described below

commit 56730f6390a19aeada75b866e64115a957212877
Author: Jungtaek Lim 
AuthorDate: Sat Jan 20 08:12:02 2024 +0900

[SPARK-46731][SS] Manage state store provider instance by state data source 
- reader

### What changes were proposed in this pull request?

This PR proposes to change state data source - reader part to manage state 
store provider instance by itself.

### Why are the changes needed?

Currently, state data source initializes state store instance via 
StateStore.get() which also initializes state store provider instance and 
registers the provider instance to the coordinator. This involves unnecessary 
overheads e.g. maintenance task could be triggered for this provider.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Existing UTs.

### Was this patch authored or co-authored using generative AI tooling?

No.

Closes #44751 from HeartSaVioR/SPARK-46731.

Authored-by: Jungtaek Lim 
Signed-off-by: Jungtaek Lim 
---
 .../v2/state/StatePartitionReader.scala| 16 +++--
 .../StreamStreamJoinStatePartitionReader.scala |  3 ++-
 .../state/SymmetricHashJoinStateManager.scala  | 28 ++
 3 files changed, 35 insertions(+), 12 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StatePartitionReader.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StatePartitionReader.scala
index ef8d7bf628bf..b79079aca56e 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StatePartitionReader.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StatePartitionReader.scala
@@ -21,7 +21,7 @@ import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, 
UnsafeRow}
 import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, 
PartitionReaderFactory}
 import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
-import org.apache.spark.sql.execution.streaming.state.{ReadStateStore, 
StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.execution.streaming.state.{ReadStateStore, 
StateStoreConf, StateStoreId, StateStoreProvider, StateStoreProviderId}
 import org.apache.spark.sql.types.StructType
 import org.apache.spark.util.SerializableConfiguration
 
@@ -53,15 +53,13 @@ class StatePartitionReader(
   private val keySchema = SchemaUtil.getSchemaAsDataType(schema, 
"key").asInstanceOf[StructType]
   private val valueSchema = SchemaUtil.getSchemaAsDataType(schema, 
"value").asInstanceOf[StructType]
 
-  private lazy val store: ReadStateStore = {
+  private lazy val provider: StateStoreProvider = {
 val stateStoreId = 
StateStoreId(partition.sourceOptions.stateCheckpointLocation.toString,
   partition.sourceOptions.operatorId, partition.partition, 
partition.sourceOptions.storeName)
 val stateStoreProviderId = StateStoreProviderId(stateStoreId, 
partition.queryId)
-
 val allStateStoreMetadata = new StateMetadataPartitionReader(
   partition.sourceOptions.stateCheckpointLocation.getParent.toString, 
hadoopConf)
   .stateMetadata.toArray
-
 val stateStoreMetadata = allStateStoreMetadata.filter { entry =>
   entry.operatorId == partition.sourceOptions.operatorId &&
 entry.stateStoreName == partition.sourceOptions.storeName
@@ -78,9 +76,12 @@ class StatePartitionReader(
   stateStoreMetadata.head.numColsPrefixKey
 }
 
-StateStore.getReadOnly(stateStoreProviderId, keySchema, valueSchema,
-  numColsPrefixKey = numColsPrefixKey, version = 
partition.sourceOptions.batchId + 1,
-  storeConf = storeConf, hadoopConf = hadoopConf.value)
+StateStoreProvider.createAndInit(
+  stateStoreProviderId, keySchema, valueSchema, numColsPrefixKey, 
storeConf, hadoopConf.value)
+  }
+
+  private lazy val store: ReadStateStore = {
+provider.getReadStore(partition.sourceOptions.batchId + 1)
   }
 
   private lazy val iter: Iterator[InternalRow] = {
@@ -104,6 +105,7 @@ class StatePartitionReader(
   override def close(): Unit = {
 current = null
 store.abort()
+provider.close()
   }
 
   private def unifyStateRowPair(pair: (UnsafeRow, UnsafeRow)): InternalRow = {
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StreamStreamJoinStatePartitionReader.sc

(spark) branch branch-3.5 updated: [SPARK-46676][SS] dropDuplicatesWithinWatermark should not fail on canonicalization of the plan

2024-01-18 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch branch-3.5
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.5 by this push:
 new fa6bf22112b4 [SPARK-46676][SS] dropDuplicatesWithinWatermark should 
not fail on canonicalization of the plan
fa6bf22112b4 is described below

commit fa6bf22112b4300dae1e7617f1480c0d12124b90
Author: Jungtaek Lim 
AuthorDate: Fri Jan 19 11:38:53 2024 +0900

[SPARK-46676][SS] dropDuplicatesWithinWatermark should not fail on 
canonicalization of the plan

### What changes were proposed in this pull request?

This PR proposes to fix the bug on canonicalizing the plan which contains 
the physical node of dropDuplicatesWithinWatermark 
(`StreamingDeduplicateWithinWatermarkExec`).

### Why are the changes needed?

Canonicalization of the plan will replace the expressions (including 
attributes) to remove out cosmetic, including name, "and metadata", which 
denotes the event time column marker.

StreamingDeduplicateWithinWatermarkExec assumes that the input attributes 
of child node contain the event time column, and it is determined at the 
initialization of the node instance. Once canonicalization is being triggered, 
child node will lose the notion of event time column from its attributes, and 
copy of StreamingDeduplicateWithinWatermarkExec will be performed which 
instantiating a new node of `StreamingDeduplicateWithinWatermarkExec` with new 
child node, which no longer has an [...]

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

New UT added.

### Was this patch authored or co-authored using generative AI tooling?

No.

Closes #44688 from HeartSaVioR/SPARK-46676.

Authored-by: Jungtaek Lim 
Signed-off-by: Jungtaek Lim 
(cherry picked from commit c1ed3e60e67f53bb323e2b9fa47789fcde70a75a)
Signed-off-by: Jungtaek Lim 
---
 .../sql/execution/streaming/statefulOperators.scala | 10 +++---
 ...StreamingDeduplicationWithinWatermarkSuite.scala | 21 +
 2 files changed, 28 insertions(+), 3 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala
index b31f6151fce2..b597c9723f5c 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala
@@ -1037,10 +1037,14 @@ case class StreamingDeduplicateWithinWatermarkExec(
 
   protected val extraOptionOnStateStore: Map[String, String] = Map.empty
 
-  private val eventTimeCol: Attribute = 
WatermarkSupport.findEventTimeColumn(child.output,
+  // Below three variables are defined as lazy, as evaluating these variables 
does not work with
+  // canonicalized plan. Specifically, attributes in child won't have an event 
time column in
+  // the canonicalized plan. These variables are NOT referenced in 
canonicalized plan, hence
+  // defining these variables as lazy would avoid such error.
+  private lazy val eventTimeCol: Attribute = 
WatermarkSupport.findEventTimeColumn(child.output,
 allowMultipleEventTimeColumns = false).get
-  private val delayThresholdMs = 
eventTimeCol.metadata.getLong(EventTimeWatermark.delayKey)
-  private val eventTimeColOrdinal: Int = child.output.indexOf(eventTimeCol)
+  private lazy val delayThresholdMs = 
eventTimeCol.metadata.getLong(EventTimeWatermark.delayKey)
+  private lazy val eventTimeColOrdinal: Int = 
child.output.indexOf(eventTimeCol)
 
   protected def initializeReusedDupInfoRow(): Option[UnsafeRow] = {
 val timeoutToUnsafeRow = UnsafeProjection.create(schemaForValueRow)
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala
index 595fc1cb9cea..9a02ab3df7dd 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala
@@ -199,4 +199,25 @@ class StreamingDeduplicationWithinWatermarkSuite extends 
StateStoreMetricsTest {
   )
 }
   }
+
+  test("SPARK-46676: canonicalization of 
StreamingDeduplicateWithinWatermarkExec should work") {
+withTempDir { checkpoint =>
+  val dedupeInputData = MemoryStream[(String, Int)]
+  val dedupe = dedupeInputData.toDS()
+.withColumn("eventTime", timestamp_seconds($"_2"))
+.withWatermark("eventTime", "10 second")
+.dropDuplicatesWithin

(spark) branch master updated: [SPARK-46676][SS] dropDuplicatesWithinWatermark should not fail on canonicalization of the plan

2024-01-18 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new c1ed3e60e67f [SPARK-46676][SS] dropDuplicatesWithinWatermark should 
not fail on canonicalization of the plan
c1ed3e60e67f is described below

commit c1ed3e60e67f53bb323e2b9fa47789fcde70a75a
Author: Jungtaek Lim 
AuthorDate: Fri Jan 19 11:38:53 2024 +0900

[SPARK-46676][SS] dropDuplicatesWithinWatermark should not fail on 
canonicalization of the plan

### What changes were proposed in this pull request?

This PR proposes to fix the bug on canonicalizing the plan which contains 
the physical node of dropDuplicatesWithinWatermark 
(`StreamingDeduplicateWithinWatermarkExec`).

### Why are the changes needed?

Canonicalization of the plan will replace the expressions (including 
attributes) to remove out cosmetic, including name, "and metadata", which 
denotes the event time column marker.

StreamingDeduplicateWithinWatermarkExec assumes that the input attributes 
of child node contain the event time column, and it is determined at the 
initialization of the node instance. Once canonicalization is being triggered, 
child node will lose the notion of event time column from its attributes, and 
copy of StreamingDeduplicateWithinWatermarkExec will be performed which 
instantiating a new node of `StreamingDeduplicateWithinWatermarkExec` with new 
child node, which no longer has an [...]

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

New UT added.

### Was this patch authored or co-authored using generative AI tooling?

No.

Closes #44688 from HeartSaVioR/SPARK-46676.

Authored-by: Jungtaek Lim 
Signed-off-by: Jungtaek Lim 
---
 .../sql/execution/streaming/statefulOperators.scala | 10 +++---
 ...StreamingDeduplicationWithinWatermarkSuite.scala | 21 +
 2 files changed, 28 insertions(+), 3 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala
index 8cb99a162ab2..c8a55ed679d0 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala
@@ -1097,10 +1097,14 @@ case class StreamingDeduplicateWithinWatermarkExec(
 
   protected val extraOptionOnStateStore: Map[String, String] = Map.empty
 
-  private val eventTimeCol: Attribute = 
WatermarkSupport.findEventTimeColumn(child.output,
+  // Below three variables are defined as lazy, as evaluating these variables 
does not work with
+  // canonicalized plan. Specifically, attributes in child won't have an event 
time column in
+  // the canonicalized plan. These variables are NOT referenced in 
canonicalized plan, hence
+  // defining these variables as lazy would avoid such error.
+  private lazy val eventTimeCol: Attribute = 
WatermarkSupport.findEventTimeColumn(child.output,
 allowMultipleEventTimeColumns = false).get
-  private val delayThresholdMs = 
eventTimeCol.metadata.getLong(EventTimeWatermark.delayKey)
-  private val eventTimeColOrdinal: Int = child.output.indexOf(eventTimeCol)
+  private lazy val delayThresholdMs = 
eventTimeCol.metadata.getLong(EventTimeWatermark.delayKey)
+  private lazy val eventTimeColOrdinal: Int = 
child.output.indexOf(eventTimeCol)
 
   protected def initializeReusedDupInfoRow(): Option[UnsafeRow] = {
 val timeoutToUnsafeRow = UnsafeProjection.create(schemaForValueRow)
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala
index 595fc1cb9cea..9a02ab3df7dd 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala
@@ -199,4 +199,25 @@ class StreamingDeduplicationWithinWatermarkSuite extends 
StateStoreMetricsTest {
   )
 }
   }
+
+  test("SPARK-46676: canonicalization of 
StreamingDeduplicateWithinWatermarkExec should work") {
+withTempDir { checkpoint =>
+  val dedupeInputData = MemoryStream[(String, Int)]
+  val dedupe = dedupeInputData.toDS()
+.withColumn("eventTime", timestamp_seconds($"_2"))
+.withWatermark("eventTime", "10 second")
+.dropDuplicatesWithinWatermark("_1")
+.select($"_1", $"eventTime".cast("long").as[Long])
+
+

(spark) branch master updated: [SPARK-46722][CONNECT] Add a test regarding to backward compatibility check for StreamingQueryListener in Spark Connect (Scala/PySpark)

2024-01-15 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new a58362ecbbf7 [SPARK-46722][CONNECT] Add a test regarding to backward 
compatibility check for StreamingQueryListener in Spark Connect (Scala/PySpark)
a58362ecbbf7 is described below

commit a58362ecbbf7c4e5d5f848411834cf2a9ef298b3
Author: Jungtaek Lim 
AuthorDate: Tue Jan 16 12:23:02 2024 +0900

[SPARK-46722][CONNECT] Add a test regarding to backward compatibility check 
for StreamingQueryListener in Spark Connect (Scala/PySpark)

### What changes were proposed in this pull request?

This PR proposes to add a functionality to perform backward compatibility 
check for StreamingQueryListener in Spark Connect (both Scala and PySpark), 
specifically implementing onQueryIdle or not.

### Why are the changes needed?

We missed to add backward compatibility test when introducing onQueryIdle, 
and it led to an issue in PySpark 
(https://issues.apache.org/jira/browse/SPARK-45631). We added the compatibility 
test in PySpark but didn't add it in Spark Connect.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Modified UTs.

### Was this patch authored or co-authored using generative AI tooling?

No.

Closes #44736 from HeartSaVioR/SPARK-46722.

Authored-by: Jungtaek Lim 
Signed-off-by: Jungtaek Lim 
---
 .../sql/streaming/ClientStreamingQuerySuite.scala  |  88 ++
 .../connect/streaming/test_parity_listener.py  | 133 +
 2 files changed, 142 insertions(+), 79 deletions(-)

diff --git 
a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/streaming/ClientStreamingQuerySuite.scala
 
b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/streaming/ClientStreamingQuerySuite.scala
index 91c562c0f98b..fd989b5da35c 100644
--- 
a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/streaming/ClientStreamingQuerySuite.scala
+++ 
b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/streaming/ClientStreamingQuerySuite.scala
@@ -20,7 +20,6 @@ package org.apache.spark.sql.streaming
 import java.io.{File, FileWriter}
 import java.util.concurrent.TimeUnit
 
-import scala.collection.mutable
 import scala.jdk.CollectionConverters._
 
 import org.scalatest.concurrent.Eventually.eventually
@@ -32,7 +31,7 @@ import org.apache.spark.api.java.function.VoidFunction2
 import org.apache.spark.internal.Logging
 import org.apache.spark.sql.{DataFrame, ForeachWriter, Row, SparkSession}
 import org.apache.spark.sql.functions.{col, udf, window}
-import org.apache.spark.sql.streaming.StreamingQueryListener.{QueryIdleEvent, 
QueryStartedEvent, QueryTerminatedEvent}
+import org.apache.spark.sql.streaming.StreamingQueryListener.{QueryIdleEvent, 
QueryProgressEvent, QueryStartedEvent, QueryTerminatedEvent}
 import org.apache.spark.sql.test.{QueryTest, SQLHelper}
 import org.apache.spark.util.SparkFileUtils
 
@@ -354,9 +353,15 @@ class ClientStreamingQuerySuite extends QueryTest with 
SQLHelper with Logging {
   }
 
   test("streaming query listener") {
+testStreamingQueryListener(new EventCollectorV1, "_v1")
+testStreamingQueryListener(new EventCollectorV2, "_v2")
+  }
+
+  private def testStreamingQueryListener(
+  listener: StreamingQueryListener,
+  tablePostfix: String): Unit = {
 assert(spark.streams.listListeners().length == 0)
 
-val listener = new EventCollector
 spark.streams.addListener(listener)
 
 val q = spark.readStream
@@ -370,11 +375,21 @@ class ClientStreamingQuerySuite extends QueryTest with 
SQLHelper with Logging {
   q.processAllAvailable()
   eventually(timeout(30.seconds)) {
 assert(q.isActive)
-checkAnswer(spark.table("my_listener_table").toDF(), Seq(Row(1, 2), 
Row(4, 5)))
+
+
assert(!spark.table(s"listener_start_events$tablePostfix").toDF().isEmpty)
+
assert(!spark.table(s"listener_progress_events$tablePostfix").toDF().isEmpty)
   }
 } finally {
   q.stop()
-  spark.sql("DROP TABLE IF EXISTS my_listener_table")
+
+  eventually(timeout(30.seconds)) {
+assert(!q.isActive)
+
assert(!spark.table(s"listener_terminated_events$tablePostfix").toDF().isEmpty)
+  }
+
+  spark.sql(s"DROP TABLE IF EXISTS listener_start_events$tablePostfix")
+  spark.sql(s"DROP TABLE IF EXISTS listener_progress_events$tablePostfix")
+  spark.sql(s"DROP TABLE IF EXISTS 
listener_terminated_events$tablePostfix")
 }
 
 // List listeners after adding a new listener, length should be 1.
@@ -382,7 +397,7 @@ class ClientStreamingQuerySuite extends 

(spark) branch master updated: [SPARK-46711][SS] Fix RocksDB state provider race condition during rollback

2024-01-14 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 86bac8911a31 [SPARK-46711][SS] Fix RocksDB state provider race 
condition during rollback
86bac8911a31 is described below

commit 86bac8911a31a98eb6ad1f5365554f4f095c0376
Author: Anish Shrigondekar 
AuthorDate: Mon Jan 15 11:09:34 2024 +0900

[SPARK-46711][SS] Fix RocksDB state provider race condition during rollback

### What changes were proposed in this pull request?
Fix RocksDB state provider race condition during rollback

### Why are the changes needed?
The rollback() method in RocksDB is not properly synchronized, thus a race 
condition can be introduced during rollback when there are tasks trying to 
commit.

The symptom of the race condition is the following exception being thrown:
```
`Caused by: java.io.FileNotFoundException: No such file or directory: 
...state/0/54/10369.changelog
at 
shaded.databricks.org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:4069)
at 
shaded.databricks.org.apache.hadoop.fs.s3a.S3AFileSystem.innerGetFileStatus(S3AFileSystem.java:3907)
at 
shaded.databricks.org.apache.hadoop.fs.s3a.S3AFileSystem.getFileStatus(S3AFileSystem.java:3801)
at 
com.databricks.common.filesystem.LokiS3FS.getFileStatusNoCache(LokiS3FS.scala:91)
at 
com.databricks.common.filesystem.LokiS3FS.getFileStatus(LokiS3FS.scala:86)
at 
shaded.databricks.org.apache.hadoop.fs.s3a.S3AFileSystem.open(S3AFileSystem.java:1525)`
```

This race condition can happen for the following sequence of events
1. task A gets cancelled after releasing lock for rocksdb
2. task B starts and loads 10368
3. task A performs rocksdb rollback to -1
4. task B reads data from rocksdb and commits

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Existing unit tests

### Was this patch authored or co-authored using generative AI tooling?
No

Closes #44722 from anishshri-db/task/SPARK-46711.

Authored-by: Anish Shrigondekar 
Signed-off-by: Jungtaek Lim 
---
 .../scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala   | 1 +
 1 file changed, 1 insertion(+)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala
index 101a9e6b9199..0284d4c9d303 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala
@@ -452,6 +452,7 @@ class RocksDB(
* Drop uncommitted changes, and roll back to previous version.
*/
   def rollback(): Unit = {
+acquire()
 numKeysOnWritingVersion = numKeysOnLoadedVersion
 loadedVersion = -1L
 changelogWriter.foreach(_.abort())


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



(spark) branch master updated: [MINOR][SS] Fix indent for streaming aggregation operator

2024-01-14 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 173ba2c6327f [MINOR][SS] Fix indent for streaming aggregation operator
173ba2c6327f is described below

commit 173ba2c6327ff74bab74c1660c80c0c8b43707c9
Author: Anish Shrigondekar 
AuthorDate: Mon Jan 15 09:26:24 2024 +0900

[MINOR][SS] Fix indent for streaming aggregation operator

### What changes were proposed in this pull request?
Fix indent for streaming aggregation operator

### Why are the changes needed?
Indent/style change

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Existing unit tests

### Was this patch authored or co-authored using generative AI tooling?
No

Closes #44723 from anishshri-db/task/SPARK-46712.

Authored-by: Anish Shrigondekar 
Signed-off-by: Jungtaek Lim 
---
 .../execution/streaming/statefulOperators.scala| 30 +++---
 1 file changed, 15 insertions(+), 15 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala
index 80f5b3532c5e..8cb99a162ab2 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala
@@ -434,22 +434,22 @@ case class StateStoreRestoreExec(
   numColsPrefixKey = 0,
   session.sessionState,
   Some(session.streams.stateStoreCoordinator)) { case (store, iter) =>
-val hasInput = iter.hasNext
-if (!hasInput && keyExpressions.isEmpty) {
-  // If our `keyExpressions` are empty, we're getting a global 
aggregation. In that case
-  // the `HashAggregateExec` will output a 0 value for the partial 
merge. We need to
-  // restore the value, so that we don't overwrite our state with a 0 
value, but rather
-  // merge the 0 with existing state.
-  store.iterator().map(_.value)
-} else {
-  iter.flatMap { row =>
-val key = stateManager.getKey(row.asInstanceOf[UnsafeRow])
-val restoredRow = stateManager.get(store, key)
-val outputRows = Option(restoredRow).toSeq :+ row
-numOutputRows += outputRows.size
-outputRows
-  }
+  val hasInput = iter.hasNext
+  if (!hasInput && keyExpressions.isEmpty) {
+// If our `keyExpressions` are empty, we're getting a global 
aggregation. In that case
+// the `HashAggregateExec` will output a 0 value for the partial 
merge. We need to
+// restore the value, so that we don't overwrite our state with a 0 
value, but rather
+// merge the 0 with existing state.
+store.iterator().map(_.value)
+  } else {
+iter.flatMap { row =>
+  val key = stateManager.getKey(row.asInstanceOf[UnsafeRow])
+  val restoredRow = stateManager.get(store, key)
+  val outputRows = Option(restoredRow).toSeq :+ row
+  numOutputRows += outputRows.size
+  outputRows
 }
+  }
 }
   }
 


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



(spark) branch branch-3.5 updated: [SPARK-46547][SS] Swallow non-fatal exception in maintenance task to avoid deadlock between maintenance thread and streaming aggregation operator

2024-01-10 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch branch-3.5
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.5 by this push:
 new a4b184d8db28 [SPARK-46547][SS] Swallow non-fatal exception in 
maintenance task to avoid deadlock between maintenance thread and streaming 
aggregation operator
a4b184d8db28 is described below

commit a4b184d8db284db1c279896fb50ef111bf4c91d2
Author: Anish Shrigondekar 
AuthorDate: Wed Jan 10 23:18:04 2024 +0900

[SPARK-46547][SS] Swallow non-fatal exception in maintenance task to avoid 
deadlock between maintenance thread and streaming aggregation operator

### What changes were proposed in this pull request?
Swallow non-fatal exception in maintenance task to avoid deadlock between 
maintenance thread and streaming aggregation operator

### Why are the changes needed?
This change fixes a race condition that causes a deadlock between the task 
thread and the maintenance thread. This is primarily only possible with the 
streaming aggregation operator. In this case, we use 2 physical operators - 
`StateStoreRestoreExec` and `StateStoreSaveExec`. The first one opens the store 
in read-only mode and the 2nd one does the actual commit.

However, the following sequence of events creates an issue
1. Task thread runs the `StateStoreRestoreExec` and gets the store instance 
and thereby the DB instance lock
2. Maintenance thread fails with an error for some reason
3. Maintenance thread takes the `loadedProviders` lock and tries to call 
`close` on all the loaded providers
4. Task thread tries to execute the StateStoreRDD for the 
`StateStoreSaveExec` operator and tries to acquire the `loadedProviders` lock 
which is held by the thread above

So basically if the maintenance thread is interleaved between the 
`restore/save` operations, there is a deadlock condition based on the 
`loadedProviders` lock and the DB instance lock.

The fix proposes to simply release the resources at the end of the 
`StateStoreRestoreExec` operator (note that `abort` for `ReadStateStore` is 
likely a misnomer - but we choose to follow the already provided API in this 
case)

Relevant Logs:
Link - 
https://github.com/anishshri-db/spark/actions/runs/7356847259/job/20027577445?pr=4
```
2023-12-27T09:59:02.6362466Z 09:59:02.635 WARN 
org.apache.spark.sql.execution.streaming.state.StateStore: Error in 
maintenanceThreadPool
2023-12-27T09:59:02.6365616Z java.io.FileNotFoundException: File 
file:/home/runner/work/spark/spark/target/tmp/spark-8ef51f34-b9de-48f2-b8df-07e14599b4c9/state/0/1
 does not exist
2023-12-27T09:59:02.6367861Zat 
org.apache.hadoop.fs.RawLocalFileSystem.listStatus(RawLocalFileSystem.java:733)
2023-12-27T09:59:02.6369383Zat 
org.apache.hadoop.fs.DelegateToFileSystem.listStatus(DelegateToFileSystem.java:177)
2023-12-27T09:59:02.6370693Zat 
org.apache.hadoop.fs.ChecksumFs.listStatus(ChecksumFs.java:571)
2023-12-27T09:59:02.6371781Zat 
org.apache.hadoop.fs.FileContext$Util$1.next(FileContext.java:1940)
2023-12-27T09:59:02.6372876Zat 
org.apache.hadoop.fs.FileContext$Util$1.next(FileContext.java:1936)
2023-12-27T09:59:02.6373967Zat 
org.apache.hadoop.fs.FSLinkResolver.resolve(FSLinkResolver.java:90)
2023-12-27T09:59:02.6375104Zat 
org.apache.hadoop.fs.FileContext$Util.listStatus(FileContext.java:1942)
2023-12-27T09:59:02.6376676Z 09:59:02.636 WARN 
org.apache.spark.sql.execution.streaming.state.StateStore: Error running 
maintenance thread
2023-12-27T09:59:02.6379079Z java.io.FileNotFoundException: File 
file:/home/runner/work/spark/spark/target/tmp/spark-8ef51f34-b9de-48f2-b8df-07e14599b4c9/state/0/1
 does not exist
2023-12-27T09:59:02.6381083Zat 
org.apache.hadoop.fs.RawLocalFileSystem.listStatus(RawLocalFileSystem.java:733)
2023-12-27T09:59:02.6382490Zat 
org.apache.hadoop.fs.DelegateToFileSystem.listStatus(DelegateToFileSystem.java:177)
2023-12-27T09:59:02.6383816Zat 
org.apache.hadoop.fs.ChecksumFs.listStatus(ChecksumFs.java:571)
2023-12-27T09:59:02.6384875Zat 
org.apache.hadoop.fs.FileContext$Util$1.next(FileContext.java:1940)
2023-12-27T09:59:02.6386294Zat 
org.apache.hadoop.fs.FileContext$Util$1.next(FileContext.java:1936)
2023-12-27T09:59:02.6387439Zat 
org.apache.hadoop.fs.FSLinkResolver.resolve(FSLinkResolver.java:90)
2023-12-27T09:59:02.6388674Zat 
org.apache.hadoop.fs.FileContext$Util.listStatus(FileContext.java:1942)
...
2023-12-27T10:01:02.4292831Z [info] - changing 
schema of state when restarting query - state format version 2 
(RocksDBStateStore) *** FAILED *** (2 minutes)
2023-12-27T10:01:02.4295311Z [info]   Timed 
out waiting for stream: The code passed to failAfter did

(spark) branch master updated: [SPARK-46547][SS] Swallow non-fatal exception in maintenance task to avoid deadlock between maintenance thread and streaming aggregation operator

2024-01-10 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new f7b0b4537917 [SPARK-46547][SS] Swallow non-fatal exception in 
maintenance task to avoid deadlock between maintenance thread and streaming 
aggregation operator
f7b0b4537917 is described below

commit f7b0b453791707b904ed0fa5508aa4b648d56bba
Author: Anish Shrigondekar 
AuthorDate: Wed Jan 10 23:18:04 2024 +0900

[SPARK-46547][SS] Swallow non-fatal exception in maintenance task to avoid 
deadlock between maintenance thread and streaming aggregation operator

### What changes were proposed in this pull request?
Swallow non-fatal exception in maintenance task to avoid deadlock between 
maintenance thread and streaming aggregation operator

### Why are the changes needed?
This change fixes a race condition that causes a deadlock between the task 
thread and the maintenance thread. This is primarily only possible with the 
streaming aggregation operator. In this case, we use 2 physical operators - 
`StateStoreRestoreExec` and `StateStoreSaveExec`. The first one opens the store 
in read-only mode and the 2nd one does the actual commit.

However, the following sequence of events creates an issue
1. Task thread runs the `StateStoreRestoreExec` and gets the store instance 
and thereby the DB instance lock
2. Maintenance thread fails with an error for some reason
3. Maintenance thread takes the `loadedProviders` lock and tries to call 
`close` on all the loaded providers
4. Task thread tries to execute the StateStoreRDD for the 
`StateStoreSaveExec` operator and tries to acquire the `loadedProviders` lock 
which is held by the thread above

So basically if the maintenance thread is interleaved between the 
`restore/save` operations, there is a deadlock condition based on the 
`loadedProviders` lock and the DB instance lock.

The fix proposes to simply release the resources at the end of the 
`StateStoreRestoreExec` operator (note that `abort` for `ReadStateStore` is 
likely a misnomer - but we choose to follow the already provided API in this 
case)

Relevant Logs:
Link - 
https://github.com/anishshri-db/spark/actions/runs/7356847259/job/20027577445?pr=4
```
2023-12-27T09:59:02.6362466Z 09:59:02.635 WARN 
org.apache.spark.sql.execution.streaming.state.StateStore: Error in 
maintenanceThreadPool
2023-12-27T09:59:02.6365616Z java.io.FileNotFoundException: File 
file:/home/runner/work/spark/spark/target/tmp/spark-8ef51f34-b9de-48f2-b8df-07e14599b4c9/state/0/1
 does not exist
2023-12-27T09:59:02.6367861Zat 
org.apache.hadoop.fs.RawLocalFileSystem.listStatus(RawLocalFileSystem.java:733)
2023-12-27T09:59:02.6369383Zat 
org.apache.hadoop.fs.DelegateToFileSystem.listStatus(DelegateToFileSystem.java:177)
2023-12-27T09:59:02.6370693Zat 
org.apache.hadoop.fs.ChecksumFs.listStatus(ChecksumFs.java:571)
2023-12-27T09:59:02.6371781Zat 
org.apache.hadoop.fs.FileContext$Util$1.next(FileContext.java:1940)
2023-12-27T09:59:02.6372876Zat 
org.apache.hadoop.fs.FileContext$Util$1.next(FileContext.java:1936)
2023-12-27T09:59:02.6373967Zat 
org.apache.hadoop.fs.FSLinkResolver.resolve(FSLinkResolver.java:90)
2023-12-27T09:59:02.6375104Zat 
org.apache.hadoop.fs.FileContext$Util.listStatus(FileContext.java:1942)
2023-12-27T09:59:02.6376676Z 09:59:02.636 WARN 
org.apache.spark.sql.execution.streaming.state.StateStore: Error running 
maintenance thread
2023-12-27T09:59:02.6379079Z java.io.FileNotFoundException: File 
file:/home/runner/work/spark/spark/target/tmp/spark-8ef51f34-b9de-48f2-b8df-07e14599b4c9/state/0/1
 does not exist
2023-12-27T09:59:02.6381083Zat 
org.apache.hadoop.fs.RawLocalFileSystem.listStatus(RawLocalFileSystem.java:733)
2023-12-27T09:59:02.6382490Zat 
org.apache.hadoop.fs.DelegateToFileSystem.listStatus(DelegateToFileSystem.java:177)
2023-12-27T09:59:02.6383816Zat 
org.apache.hadoop.fs.ChecksumFs.listStatus(ChecksumFs.java:571)
2023-12-27T09:59:02.6384875Zat 
org.apache.hadoop.fs.FileContext$Util$1.next(FileContext.java:1940)
2023-12-27T09:59:02.6386294Zat 
org.apache.hadoop.fs.FileContext$Util$1.next(FileContext.java:1936)
2023-12-27T09:59:02.6387439Zat 
org.apache.hadoop.fs.FSLinkResolver.resolve(FSLinkResolver.java:90)
2023-12-27T09:59:02.6388674Zat 
org.apache.hadoop.fs.FileContext$Util.listStatus(FileContext.java:1942)
...
2023-12-27T10:01:02.4292831Z [info] - changing 
schema of state when restarting query - state format version 2 
(RocksDBStateStore) *** FAILED *** (2 minutes)
2023-12-27T10:01:02.4295311Z [info]   Timed 
out waiting for stream: The code passed to failAfter did

(spark) branch master updated (96bf373e9002 -> d163b319e21a)

2023-12-17 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


from 96bf373e9002 [SPARK-46430][CONNECT][TESTS] Add test for 
`ProtoUtils.abbreviate`
 add d163b319e21a [SPARK-46431][SS] Convert `IllegalStateException` to 
`internalError` in session iterators

No new revisions were added by this update.

Summary of changes:
 .../aggregate/MergingSessionsIterator.scala|  5 +-
 .../aggregate/UpdatingSessionsIterator.scala   |  5 +-
 .../streaming/MergingSessionsIteratorSuite.scala   | 30 +++
 .../streaming/UpdatingSessionsIteratorSuite.scala  | 58 ++
 4 files changed, 64 insertions(+), 34 deletions(-)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



(spark) branch master updated: [SPARK-45888][SS] Apply error class framework to State (Metadata) Data Source

2023-12-06 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 59777222e72 [SPARK-45888][SS] Apply error class framework to State 
(Metadata) Data Source
59777222e72 is described below

commit 59777222e726c63cbd9077a2c76f762e06f6a5b3
Author: Jungtaek Lim 
AuthorDate: Wed Dec 6 22:38:40 2023 +0900

[SPARK-45888][SS] Apply error class framework to State (Metadata) Data 
Source

### What changes were proposed in this pull request?

This PR proposes to apply error class framework to the new data source, 
State (Metadata) Data Source.

### Why are the changes needed?

Error class framework is a standard to represent all exceptions in Spark.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Modified UT.

### Was this patch authored or co-authored using generative AI tooling?

No.

Closes #44025 from HeartSaVioR/SPARK-45888.

Authored-by: Jungtaek Lim 
Signed-off-by: Jungtaek Lim 
---
 common/utils/src/main/resources/error/README.md|   1 +
 .../src/main/resources/error/error-classes.json|  75 ++
 ...itions-stds-invalid-option-value-error-class.md |  40 ++
 docs/sql-error-conditions.md   |  60 
 .../datasources/v2/state/StateDataSource.scala |  33 +++--
 .../v2/state/StateDataSourceErrors.scala   | 160 +
 .../datasources/v2/state/StateScanBuilder.scala|   3 +-
 .../datasources/v2/state/StateTable.scala  |   9 +-
 .../StreamStreamJoinStatePartitionReader.scala |   2 +-
 .../v2/state/metadata/StateMetadataSource.scala|   4 +-
 .../v2/state/StateDataSourceReadSuite.scala|  33 +++--
 .../state/OperatorStateMetadataSuite.scala |   6 +-
 12 files changed, 389 insertions(+), 37 deletions(-)

diff --git a/common/utils/src/main/resources/error/README.md 
b/common/utils/src/main/resources/error/README.md
index 556a634e992..b062c773907 100644
--- a/common/utils/src/main/resources/error/README.md
+++ b/common/utils/src/main/resources/error/README.md
@@ -636,6 +636,7 @@ The following SQLSTATEs are collated from:
 |42613|42   |Syntax Error or Access Rule Violation |613 
|Clauses are mutually exclusive. |DB2|N 
  |DB2  
   |
 |42614|42   |Syntax Error or Access Rule Violation |614 |A 
duplicate keyword or clause is invalid.   |DB2|N
   |DB2 
|
 |42615|42   |Syntax Error or Access Rule Violation |615 
|An invalid alternative was detected.|DB2|N 
  |DB2  
   |
+|42616|42   |Syntax Error or Access Rule Violation |616 
|Invalid options specified   |DB2|N 
  |DB2  
   |
 |42617|42   |Syntax Error or Access Rule Violation |617 
|The statement string is blank or empty. |DB2|N 
  |DB2  
   |
 |42618|42   |Syntax Error or Access Rule Violation |618 |A 
variable is not allowed.  |DB2|N
   |DB2 
|
 |42620|42   |Syntax Error or Access Rule Violation |620 
|Read-only SCROLL was specified with the UPDATE clause.  |DB2|N 
  |DB2  
   |
diff --git a/common/utils/src/main/resources/error/error-classes.json 
b/common/utils/src/main/resources/error/error-classes.json
index e54d346e1bc..7a672fa5e55 100644
--- a/common/utils/src/main/resources/error/error-classes.json
+++ b/common/utils/src/main/resources/error/error-classes.json
@@ -3066,6 +3066,81 @@
 ],
 "sqlState" : "42713"
   },
+  "STDS_COMMITTED_BATCH_UNAVAILABLE" : {
+"message" : [
+  "No committed batch found, checkpoint location: . 
Ensure that the query has run and committed any microbatch before stopping."
+],
+"sqlState" : "KD006"
+  },
+  "STDS_CONFLICT_OPTIONS" : {
+"message" : [
+  "The options  cannot be specified together. Please specify the 
one."
+],
+"sqlState" : "42613"
+  },
+  "STDS_FAILED_TO_RE

(spark) branch master updated: [SPARK-46249][SS] Require instance lock for acquiring RocksDB metrics to prevent race with background operations

2023-12-04 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 1d80d80a56c4 [SPARK-46249][SS] Require instance lock for acquiring 
RocksDB metrics to prevent race with background operations
1d80d80a56c4 is described below

commit 1d80d80a56c418f841e282ad753fad6671c3baae
Author: Anish Shrigondekar 
AuthorDate: Tue Dec 5 15:00:08 2023 +0900

[SPARK-46249][SS] Require instance lock for acquiring RocksDB metrics to 
prevent race with background operations

### What changes were proposed in this pull request?
Require instance lock for acquiring RocksDB metrics to prevent race with 
background operations

### Why are the changes needed?
The changes are needed to avoid races where the statefulOperator tries to 
set storeMetrics after the commit and the DB instance has already been 
closed/aborted/reloaded.
We have seen a few query failures with the following stack trace due to 
this reason:
```
org.apache.spark.sql.streaming.StreamingQueryException: Job aborted 
due to stage failure: Task 3 in stage 531.0 failed 1 times, most recent 
failure: Lost task 3.0 in stage 531.0 (TID 1544) 
(ip-10-110-29-251.us-west-2.compute.internal executor driver): 
java.lang.NullPointerException
at 
org.apache.spark.sql.execution.streaming.state.RocksDB.getDBProperty(RocksDB.scala:838)
at 
org.apache.spark.sql.execution.streaming.state.RocksDB.metrics(RocksDB.scala:678)
at 
org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider$RocksDBStateStore.metrics(RocksDBStateStoreProvider.scala:137)
at 
org.apache.spark.sql.execution.streaming.StateStoreWriter.setStoreMetrics(statefulOperators.scala:198)
at 
org.apache.spark.sql.execution.streaming.StateStoreWriter.setStoreMetrics$(statefulOperators.scala:197)
at 
org.apache.spark.sql.execution.streaming.StateStoreSaveExec.setStoreMetrics(statefulOperators.scala:495)
at 
org.apache.spark.sql.execution.streaming.StateStoreSaveExec$$anon$2.close(statefulOperators.scala:626)
at 
org.apache.spark.util.NextIterator.closeIfNeeded(NextIterator.scala:66)
at org.apache.spark.util.NextIterator.hasNext(NextIterator.scala:75)
at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage4.hashAgg_doAggregateWithKeys_0$(Unknown
 Source)
at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage4.processNext(Unknown
 Source)
at 
org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
at 
org.apache.spark.sql.execution.WholeStageCodegenEvaluatorFactory$WholeStageCodegenPartitionEvaluator$$anon$1.hasNext(WholeStageCodegenEvaluatorFactory.scala:43)
at 
org.apache.spark.sql.execution.datasources.v2.WritingSparkTask.$anonfun$run$1(WriteToDataSourceV2Exec.scala:498)
at 
org.apache.spark.util.Utils$.tryWithSafeFinallyAndFailureCallbacks(Utils.scala:1743)
at 
org.apache.spark.sql.execution.datasources.v2.WritingSparkTask.run(WriteToDataSourceV2Exec.scala:552)
at 
org.apache.spark.sql.execution.datasources.v2.WritingSparkTask.run$(WriteToDataSourceV2Exec.scala:482)
at 
org.apache.spark.sql.execution.datasources.v2.DataWritingSparkTask$.run(WriteToDataSourceV2Exec.scala:557)
at 
org.apache.spark.sql.execution.datasources.v2.V2TableWriteExec.$anonfun$writeWithV2$2(WriteToDataSourceV2Exec.scala:445)
at 
org.apache.spark.scheduler.ResultTask.$anonfun$runTask$3(ResultTask.scala:82)
at 
com.databricks.spark.util.ExecutorFrameProfiler$.record(ExecutorFrameProfiler.scala:110)
at 
org.apache.spark.scheduler.ResultTask.$anonfun$runTask$1(ResultTask.scala:82)
at 
com.databricks.spark.util.ExecutorFrameProfiler$.record(ExecutorFrameProfiler.scala:110)
at 
org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:62)
at 
org.apache.spark.TaskContext.runTaskWithListeners(TaskContext.scala:196)
at org.apache.spark.scheduler.Task.doRunTask(Task.scala:181)
at org.apache.spark.scheduler.Task.$anonfun$run$5(Task.scala:146)
at 
com.databricks.unity.EmptyHandle$.runWithAndClose(UCSHandle.scala:125)
at org.apache.spark.scheduler.Task.$anonfun$run$1(Task.scala:146)
```

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Modified existing unit tests

```
[info] Run completed in 1 minute, 31 seconds.
[info] Total number of tests run: 150
[info] Suites: completed 1, aborted 0
[info] Tests: succeeded 150, failed 0, canceled 0, ignored 0, pending 0
[info] All

  1   2   3   4   5   6   >