[spark] branch master updated: [SPARK-40670][SS][PYTHON] Fix NPE in applyInPandasWithState when the input schema has "non-nullable" column(s)

2022-10-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 edd6076699c [SPARK-40670][SS][PYTHON] Fix NPE in 
applyInPandasWithState when the input schema has "non-nullable" column(s)
edd6076699c is described below

commit edd6076699c36a94c1bc1b9ca853f05e55ba9f2c
Author: Jungtaek Lim 
AuthorDate: Thu Oct 6 15:17:58 2022 +0900

[SPARK-40670][SS][PYTHON] Fix NPE in applyInPandasWithState when the input 
schema has "non-nullable" column(s)

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

This PR fixes a bug which occurs NPE when the input schema of 
applyInPandasWithState has "non-nullable" column(s).
This PR also leaves a code comment explaining the fix. Quoting:

```
  // See processTimedOutState: we create a row which contains the actual 
values for grouping key,
  // but all nulls for value side by intention. This technically changes 
the schema of input to
  // be "nullable", hence the schema information and the internal 
projection of row should take
  // this into consideration. Strictly saying, it's not applied to the part 
of grouping key, but
  // it doesn't hurt much even if we apply the same for grouping key as 
well.
```

### Why are the changes needed?

There's a bug which we didn't take the non-null columns into account. This 
PR fixes the bug.

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

No.

### How was this patch tested?

New UT. The new test case failed with NPE without the fix, and succeeded 
with the fix.

Closes #38115 from HeartSaVioR/SPARK-40670.

Authored-by: Jungtaek Lim 
Signed-off-by: Jungtaek Lim 
---
 .../FlatMapGroupsInPandasWithStateExec.scala   | 16 +++-
 .../FlatMapGroupsInPandasWithStateSuite.scala  | 87 +-
 2 files changed, 99 insertions(+), 4 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapGroupsInPandasWithStateExec.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapGroupsInPandasWithStateExec.scala
index 159f805f734..09123344c2e 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapGroupsInPandasWithStateExec.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapGroupsInPandasWithStateExec.scala
@@ -83,7 +83,17 @@ case class FlatMapGroupsInPandasWithStateExec(
   private val chainedFunc = Seq(ChainedPythonFunctions(Seq(pythonFunction)))
   private lazy val (dedupAttributes, argOffsets) = resolveArgOffsets(
 groupingAttributes ++ child.output, groupingAttributes)
-  private lazy val unsafeProj = UnsafeProjection.create(dedupAttributes, 
child.output)
+
+  // See processTimedOutState: we create a row which contains the actual 
values for grouping key,
+  // but all nulls for value side by intention. This technically changes the 
schema of input to
+  // be "nullable", hence the schema information and the internal projection 
of row should take
+  // this into consideration. Strictly saying, it's not applied to the part of 
grouping key, but
+  // it doesn't hurt much even if we apply the same for grouping key as well.
+  private lazy val dedupAttributesWithNull =
+dedupAttributes.map(_.withNullability(newNullability = true))
+  private lazy val childOutputWithNull = 
child.output.map(_.withNullability(newNullability = true))
+  private lazy val unsafeProj = 
UnsafeProjection.create(dedupAttributesWithNull,
+childOutputWithNull)
 
   override def requiredChildDistribution: Seq[Distribution] =
 StatefulOperatorPartitioning.getCompatibleDistribution(
@@ -134,7 +144,7 @@ case class FlatMapGroupsInPandasWithStateExec(
   val joinedKeyRow = unsafeProj(
 new JoinedRow(
   stateData.keyRow,
-  new GenericInternalRow(Array.fill(dedupAttributes.length)(null: 
Any
+  new 
GenericInternalRow(Array.fill(dedupAttributesWithNull.length)(null: Any
 
   (stateData.keyRow, stateData, Iterator.single(joinedKeyRow))
 }
@@ -150,7 +160,7 @@ case class FlatMapGroupsInPandasWithStateExec(
 chainedFunc,
 PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF_WITH_STATE,
 Array(argOffsets),
-StructType.fromAttributes(dedupAttributes),
+StructType.fromAttributes(dedupAttributesWithNull),
 sessionLocalTimeZone,
 pythonRunnerConf,
 stateEncoder.asInstanceOf[ExpressionEncoder[Row]],
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsInPandasWithStateSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsInPandasWithStateSuite.scala

[spark] branch master updated: [SPARK-40495][SQL][TESTS] Add additional tests to StreamingSessionWindowSuite

2022-09-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 076f7e47306 [SPARK-40495][SQL][TESTS] Add additional tests to 
StreamingSessionWindowSuite
076f7e47306 is described below

commit 076f7e47306fa2a02b65ce279fb37b561abe441e
Author: Wei Liu 
AuthorDate: Thu Sep 29 17:34:48 2022 +0900

[SPARK-40495][SQL][TESTS] Add additional tests to 
StreamingSessionWindowSuite

## What changes were proposed in this pull request?
Add complex tests to `StreamingSessionWindowSuite`. Concretely, I created 
two helper functions,
- one is called `sessionWindowQueryNestedKey`, which would convert 
`sessionId` from the single word key used in `sessionWindowQuery` to a nested 
column key. For example: `"hello" -> (("hello", "hello"), "hello")`.
- The other is called `sessionWindowQueryMultiColKey`. It would convert 
`sessionId` from the single word key used in `sessionWindowQuery` to two 
columns. For example: "hello" -> col1: ("hello", "hello"), col2: "hello"

With the two new helper functions, I added more tests for the tests for 
`complete mode` and `cap gap duration` (`append` and `async state` was not 
included, because the first two is enough for testing the change I added). The 
logic of the tests are not changed at all, just the key.

For the aggregation test (`session window - with more aggregation 
functions`), I added some more functions as well as a UDAF to test, and I tried 
`first()` and `last()` function on a nested triple, created using the same 
method as the above.

## How was this patch tested?
All are tests.

Closes #37936 from WweiL/master.

Authored-by: Wei Liu 
Signed-off-by: Jungtaek Lim 
---
 .../streaming/StreamingSessionWindowSuite.scala| 356 -
 1 file changed, 354 insertions(+), 2 deletions(-)

diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingSessionWindowSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingSessionWindowSuite.scala
index cb1728a4c5a..25b7506178d 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingSessionWindowSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingSessionWindowSuite.scala
@@ -23,10 +23,11 @@ import org.scalatest.BeforeAndAfter
 import org.scalatest.matchers.must.Matchers
 
 import org.apache.spark.internal.Logging
-import org.apache.spark.sql.{AnalysisException, Column, DataFrame}
+import org.apache.spark.sql.{AnalysisException, Column, DataFrame, Encoder, 
Encoders}
 import org.apache.spark.sql.execution.streaming.MemoryStream
 import 
org.apache.spark.sql.execution.streaming.state.{HDFSBackedStateStoreProvider, 
RocksDBStateStoreProvider}
-import org.apache.spark.sql.functions.{count, session_window, sum}
+import org.apache.spark.sql.expressions.Aggregator
+import org.apache.spark.sql.functions._
 import org.apache.spark.sql.internal.SQLConf
 
 class StreamingSessionWindowSuite extends StreamTest
@@ -142,6 +143,126 @@ class StreamingSessionWindowSuite extends StreamTest
 )
   }
 
+  // Logic is the same as `complete mode - session window`
+  // just with a more complex key
+  testWithAllOptions("complete mode - session window - nested tuple key") {
+val inputData = MemoryStream[(String, Long)]
+val sessionUpdates = sessionWindowQueryNestedKey(inputData)
+
+testStream(sessionUpdates, OutputMode.Complete())(
+  AddData(
+inputData,
+("hello world spark streaming", 40L),
+("world hello structured streaming", 41L)),
+  CheckNewAnswer(
+((("hello", "hello"), "hello"), 40, 51, 11, 2),
+((("world", "world"), "world"), 40, 51, 11, 2),
+((("streaming", "streaming"), "streaming"), 40, 51, 11, 2),
+((("spark", "spark"), "spark"), 40, 50, 10, 1),
+((("structured", "structured"), "structured"), 41, 51, 10, 1)),
+  // placing new sessions "before" previous sessions
+  AddData(inputData, ("spark streaming", 25L)),
+  CheckNewAnswer(
+((("spark", "spark"), "spark"), 25, 35, 10, 1),
+((("streaming", "streaming"), "streaming"), 25, 35, 10, 1),
+((("hello", "hello"), "hello"), 40, 51, 11, 2),
+((("world", "world"), "world"), 40, 51, 11, 2),
+((("streaming", "streaming"), "streaming"), 40, 51, 11, 2),
+(

[spark] branch master updated: [SPARK-40509][SS][PYTHON] Add example for applyInPandasWithState

2022-09-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 38599e9a368 [SPARK-40509][SS][PYTHON] Add example for 
applyInPandasWithState
38599e9a368 is described below

commit 38599e9a3684d2bdb7561a9c1d6b20ac89bdcf9b
Author: Chaoqin Li 
AuthorDate: Wed Sep 28 16:37:29 2022 +0900

[SPARK-40509][SS][PYTHON] Add example for applyInPandasWithState

### What changes were proposed in this pull request?
An example for applyInPandasWithState usage. This example split lines into 
words, group by words as key and use the state per key to track session of each 
key.

### Why are the changes needed?
To demonstrate the usage of applyInPandasWithState

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

### How was this patch tested?
This is an example that can be run manually.

 To run this on your local machine, you need to first run a Netcat server
`$ nc -lk `
 and then run the example
`$ bin/spark-submit

examples/src/main/python/sql/streaming/structured_network_wordcount_session_window.py
localhost `

Closes #38013 from chaoqin-li1123/session_example.

Authored-by: Chaoqin Li 
Signed-off-by: Jungtaek Lim 
---
 .../structured_network_wordcount_session_window.py | 139 +
 1 file changed, 139 insertions(+)

diff --git 
a/examples/src/main/python/sql/streaming/structured_network_wordcount_session_window.py
 
b/examples/src/main/python/sql/streaming/structured_network_wordcount_session_window.py
new file mode 100644
index 000..77ddd0f415f
--- /dev/null
+++ 
b/examples/src/main/python/sql/streaming/structured_network_wordcount_session_window.py
@@ -0,0 +1,139 @@
+#
+# 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.
+#
+
+r"""
+ Split lines into words, group by words and use the state per key to track 
session of each key.
+
+ Usage: structured_network_wordcount_windowed.py  
+  and  describe the TCP server that Structured Streaming
+ would connect to receive data.
+
+ To run this on your local machine, you need to first run a Netcat server
+`$ nc -lk `
+ and then run the example
+`$ bin/spark-submit
+
examples/src/main/python/sql/streaming/structured_network_wordcount_session_window.py
+localhost `
+"""
+import sys
+import math
+from typing import Iterable, Any
+
+import pandas as pd
+
+from pyspark.sql import SparkSession
+from pyspark.sql.functions import explode
+from pyspark.sql.functions import split
+from pyspark.sql.types import (
+LongType,
+StringType,
+StructType,
+StructField,
+)
+from pyspark.sql.streaming.state import GroupStateTimeout, GroupState
+
+if __name__ == "__main__":
+if len(sys.argv) != 3:
+msg = "Usage: structured_network_wordcount_session_window.py 
 "
+print(msg, file=sys.stderr)
+sys.exit(-1)
+
+host = sys.argv[1]
+port = int(sys.argv[2])
+
+spark = SparkSession.builder.appName(
+"StructuredNetworkWordCountSessionWindow"
+).getOrCreate()
+
+# Create DataFrame representing the stream of input lines from connection 
to host:port
+lines = (
+spark.readStream.format("socket")
+.option("host", host)
+.option("port", port)
+.option("includeTimestamp", "true")
+.load()
+)
+
+# Split the lines into words, retaining timestamps, each word become a 
sessionId
+events = lines.select(
+explode(split(lines.value, " ")).alias("sessionId"),
+lines.timestamp.cast("long"),
+)
+
+# Type of output records.
+session_schema = StructType(
+[
+StructField("sessionId", StringType()),
+StructField("count", LongType()),
+StructField("start", LongType()),
+StructField("end", LongType()),
+]
+  

[spark] branch master updated: [SPARK-40571][SS][TESTS] Construct a new test case for applyInPandasWithState to verify fault-tolerance semantic with random python worker failures

2022-09-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 37517df68c5 [SPARK-40571][SS][TESTS] Construct a new test case for 
applyInPandasWithState to verify fault-tolerance semantic with random python 
worker failures
37517df68c5 is described below

commit 37517df68c5805a2dcff5c0c41ea273eae92ed0c
Author: Jungtaek Lim 
AuthorDate: Tue Sep 27 17:41:28 2022 +0900

[SPARK-40571][SS][TESTS] Construct a new test case for 
applyInPandasWithState to verify fault-tolerance semantic with random python 
worker failures

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

This PR proposes a new test case for applyInPandasWithState to verify 
fault-tolerance semantic is not broken despite of random python worker failure. 
If the sink provides end-to-end exactly-once, the query should respect the 
guarantee. Otherwise, the query should respect stateful exactly-once, but 
at-least-once in terms of outputs.

The test leverages file stream sink which is end-to-end exactly-once, but 
to make the verification simpler, we just verify whether the stateful 
exactly-once is guaranteed despite of python worker failures.

### Why are the changes needed?

This strengthen the test coverage, especially the fault-tolerance semantic.

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

No.

### How was this patch tested?

New test added.  Manually ran `./python/run-tests --testnames 
'pyspark.sql.tests.test_pandas_grouped_map_with_state'` 10 times and all 
succeeded.

Closes #38008 from HeartSaVioR/SPARK-40571.

Authored-by: Jungtaek Lim 
Signed-off-by: Jungtaek Lim 
---
 .../tests/test_pandas_grouped_map_with_state.py| 149 -
 1 file changed, 147 insertions(+), 2 deletions(-)

diff --git a/python/pyspark/sql/tests/test_pandas_grouped_map_with_state.py 
b/python/pyspark/sql/tests/test_pandas_grouped_map_with_state.py
index 7eb3bb92b84..8671cc8519c 100644
--- a/python/pyspark/sql/tests/test_pandas_grouped_map_with_state.py
+++ b/python/pyspark/sql/tests/test_pandas_grouped_map_with_state.py
@@ -15,9 +15,16 @@
 # limitations under the License.
 #
 
+import random
+import shutil
+import string
+import sys
+import tempfile
+
 import unittest
 from typing import cast
 
+from pyspark import SparkConf
 from pyspark.sql.streaming.state import GroupStateTimeout, GroupState
 from pyspark.sql.types import (
 LongType,
@@ -33,6 +40,7 @@ from pyspark.testing.sqlutils import (
 pandas_requirement_message,
 pyarrow_requirement_message,
 )
+from pyspark.testing.utils import eventually
 
 if have_pandas:
 import pandas as pd
@@ -46,8 +54,23 @@ if have_pyarrow:
 cast(str, pandas_requirement_message or pyarrow_requirement_message),
 )
 class GroupedMapInPandasWithStateTests(ReusedSQLTestCase):
+@classmethod
+def conf(cls):
+cfg = SparkConf()
+cfg.set("spark.sql.shuffle.partitions", "5")
+return cfg
+
 def test_apply_in_pandas_with_state_basic(self):
-df = 
self.spark.readStream.format("text").load("python/test_support/sql/streaming")
+input_path = tempfile.mkdtemp()
+
+def prepare_test_resource():
+with open(input_path + "/text-test.txt", "w") as fw:
+fw.write("hello\n")
+fw.write("this\n")
+
+prepare_test_resource()
+
+df = self.spark.readStream.format("text").load(input_path)
 
 for q in self.spark.streams.active:
 q.stop()
@@ -71,7 +94,7 @@ class GroupedMapInPandasWithStateTests(ReusedSQLTestCase):
 
 def check_results(batch_df, _):
 self.assertEqual(
-set(batch_df.collect()),
+set(batch_df.sort("key").collect()),
 {Row(key="hello", countAsString="1"), Row(key="this", 
countAsString="1")},
 )
 
@@ -90,6 +113,128 @@ class GroupedMapInPandasWithStateTests(ReusedSQLTestCase):
 self.assertTrue(q.isActive)
 q.processAllAvailable()
 
+def test_apply_in_pandas_with_state_python_worker_random_failure(self):
+input_path = tempfile.mkdtemp()
+output_path = tempfile.mkdtemp()
+checkpoint_loc = tempfile.mkdtemp()
+
+shutil.rmtree(output_path)
+shutil.rmtree(checkpoint_loc)
+
+def prepare_test_resource():
+data_range = list(string.ascii_lowercase)
+for i in range(5):
+picked_data = [
+data_range[random.randrange(0, len(data_range) - 1)] for x 
in range(100)
+]
+
+with open(input_path + "/part-%i.txt

[spark] branch master updated (c01e524c298 -> e85ee47b8da)

2022-09-25 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 c01e524c298 [SPARK-40334][PS] Implement `GroupBy.prod`
 add e85ee47b8da [SPARK-40492][SS] Do maintenance before streaming 
StateStore unload

No new revisions were added by this update.

Summary of changes:
 .../sql/execution/streaming/state/StateStore.scala |  5 +-
 .../streaming/state/StateStoreSuite.scala  | 71 ++
 2 files changed, 73 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 (c4a0360daf5 -> e5b4b32b2cd)

2022-09-22 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 c4a0360daf5 [SPARK-40473][SQL] Migrate parsing errors onto error 
classes
 add e5b4b32b2cd [SPARK-40434][SS][PYTHON][FOLLOWUP] Address review comments

No new revisions were added by this update.

Summary of changes:
 python/pyspark/sql/pandas/group_ops.py | 13 +++--
 .../org/apache/spark/sql/execution/arrow/ArrowWriter.scala | 14 --
 2 files changed, 7 insertions(+), 20 deletions(-)


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



[spark] branch master updated (32dd7534365 -> c22ddbea6b3)

2022-09-21 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 32dd7534365 [SPARK-40522][BUILD] Upgrade `kafka` to 3.2.3
 add c22ddbea6b3 [SPARK-40435][SS][PYTHON] Add test suites for 
applyInPandasWithState in PySpark

No new revisions were added by this update.

Summary of changes:
 dev/sparktestsupport/modules.py|   1 +
 .../tests/test_pandas_grouped_map_with_state.py| 103 +++
 .../apache/spark/sql/IntegratedUDFTestUtils.scala  |  73 +-
 .../org/apache/spark/sql/SQLQueryTestSuite.scala   |   8 +-
 .../sql/errors/QueryCompilationErrorsSuite.scala   |   4 +-
 .../sql/execution/python/PythonUDFSuite.scala  |   2 +-
 ...pGroupsInPandasWithStateDistributionSuite.scala | 115 
 .../FlatMapGroupsInPandasWithStateSuite.scala  | 741 +
 .../sql/streaming/continuous/ContinuousSuite.scala |   2 +-
 9 files changed, 1035 insertions(+), 14 deletions(-)
 create mode 100644 
python/pyspark/sql/tests/test_pandas_grouped_map_with_state.py
 create mode 100644 
sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsInPandasWithStateDistributionSuite.scala
 create mode 100644 
sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsInPandasWithStateSuite.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-40466][SS] Improve the error message when DSv2 is disabled while DSv1 is not avaliable

2022-09-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 8e7587a1f29 [SPARK-40466][SS] Improve the error message when DSv2 is 
disabled while DSv1 is not avaliable
8e7587a1f29 is described below

commit 8e7587a1f296e1ee1ec008a97f0823f68569b14b
Author: Huanli Wang 
AuthorDate: Tue Sep 20 08:21:18 2022 +0900

[SPARK-40466][SS] Improve the error message when DSv2 is disabled while 
DSv1 is not avaliable

…le DSv1 is not avaliable.

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

Improve the error message when DSv2 is disable while its fallback DSv1 is 
not available.

### Why are the changes needed?

Improve the user experience. When users get UnsupportOperationError for the 
disabled DSv2, they are able to know which config to modify to enable the V2 
source.

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

Yes, error message.

### How was this patch tested?

N/A, just the message change

Closes #37917 from huanliwang-db/SPARK-40466.

Authored-by: Huanli Wang 
Signed-off-by: Jungtaek Lim 
---
 .../apache/spark/sql/errors/QueryExecutionErrors.scala| 15 ---
 .../sql/execution/streaming/MicroBatchExecution.scala |  3 ++-
 2 files changed, 14 insertions(+), 4 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
index 2515d43cf82..bc778abc985 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
@@ -1599,9 +1599,18 @@ private[sql] object QueryExecutionErrors extends 
QueryErrorsBase {
   s"$commitProtocol does not support adding files with an absolute path")
   }
 
-  def microBatchUnsupportedByDataSourceError(srcName: String): Throwable = {
-new UnsupportedOperationException(
-  s"Data source $srcName does not support microbatch processing.")
+  def microBatchUnsupportedByDataSourceError(
+  srcName: String,
+  disabledSources: String,
+  table: Table): Throwable = {
+new UnsupportedOperationException(s"""
+ |Data source $srcName does not support microbatch processing.
+ |
+ |Either the data source is disabled at
+ |SQLConf.get.DISABLED_V2_STREAMING_MICROBATCH_READERS.key (The 
disabled sources
+ |are [$disabledSources]) or the table $table does not have 
MICRO_BATCH_READ
+ |capability. Meanwhile, the fallback, data source v1, is not 
available."
+   """.stripMargin)
   }
 
   def cannotExecuteStreamingRelationExecError(): Throwable = {
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala
index 12427ece236..153bc82f892 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala
@@ -106,7 +106,8 @@ class MicroBatchExecution(
 StreamingDataSourceV2Relation(output, scan, stream, catalog, 
identifier)
   })
 } else if (v1.isEmpty) {
-  throw 
QueryExecutionErrors.microBatchUnsupportedByDataSourceError(srcName)
+  throw QueryExecutionErrors.microBatchUnsupportedByDataSourceError(
+srcName, 
sparkSession.sqlContext.conf.disabledV2StreamingMicroBatchReaders, table)
 } else {
   v2ToExecutionRelationMap.getOrElseUpdate(s, {
 // Materialize source to avoid creating it in every batch


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



[spark] branch master updated (127c40366b6 -> 946a96022a5)

2022-09-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 127c40366b6 [SPARK-40482][SQL] Revert `SPARK-24544 Print actual 
failure cause when look up function failed`
 add 946a96022a5 [SPARK-40460][SS] Fix streaming metrics when selecting 
`_metadata`

No new revisions were added by this update.

Summary of changes:
 .../execution/streaming/MicroBatchExecution.scala  | 14 +---
 .../datasources/FileMetadataStructSuite.scala  | 38 --
 2 files changed, 45 insertions(+), 7 deletions(-)


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



[spark] branch master updated (9c188ce6f3e -> 50a95a456c8)

2022-09-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 9c188ce6f3e [SPARK-40463][INFRA] Update gpg's keyserver
 add 50a95a456c8 [SPARK-40467][SS] Split FlatMapGroupsWithState down to 
multiple test suites

No new revisions were added by this update.

Summary of changes:
 .../streaming/FlatMapGroupsWithStateSuite.scala| 758 +
 ...atMapGroupsWithStateWithInitialStateSuite.scala | 365 ++
 .../spark/sql/streaming/GroupStateSuite.scala  | 458 +
 3 files changed, 825 insertions(+), 756 deletions(-)
 create mode 100644 
sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateWithInitialStateSuite.scala
 create mode 100644 
sql/core/src/test/scala/org/apache/spark/sql/streaming/GroupStateSuite.scala


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



[spark] branch master updated (193b5b229c7 -> 5938e84e72b)

2022-09-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 193b5b229c7 [SPARK-40387][SQL] Improve the implementation of Spark 
Decimal
 add 5938e84e72b [SPARK-40432][SS][PYTHON] Introduce GroupStateImpl and 
GroupStateTimeout in PySpark

No new revisions were added by this update.

Summary of changes:
 .../main/scala/org/apache/spark/util/Utils.scala   |   2 +-
 python/pyspark/sql/streaming/state.py  | 192 +
 .../spark/sql/streaming/GroupStateTimeout.java |   4 +
 .../sql/execution/streaming/GroupStateImpl.scala   |  54 ++
 4 files changed, 251 insertions(+), 1 deletion(-)
 create mode 100644 python/pyspark/sql/streaming/state.py


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



[spark] branch master updated (3d14b745773 -> 5496d99241f)

2022-09-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 3d14b745773 [SPARK-40440][PS][DOCS] Fix wrong reference and content in 
PS windows related doc
 add 5496d99241f [SPARK-40433][SS][PYTHON] Add toJVMRow in PythonSQLUtils 
to convert pickled PySpark Row to JVM Row

No new revisions were added by this update.

Summary of changes:
 .../spark/sql/api/python/PythonSQLUtils.scala  | 42 ++
 1 file changed, 35 insertions(+), 7 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-40414][SQL][PYTHON] More generic type on PythonArrowInput and PythonArrowOutput

2022-09-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 a2304e45d31 [SPARK-40414][SQL][PYTHON] More generic type on 
PythonArrowInput and PythonArrowOutput
a2304e45d31 is described below

commit a2304e45d31d3d802a3976b9cd1ea0f72a2d604b
Author: Jungtaek Lim 
AuthorDate: Wed Sep 14 14:17:52 2022 +0900

[SPARK-40414][SQL][PYTHON] More generic type on PythonArrowInput and 
PythonArrowOutput

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

This PR proposes to change PythonArrowInput and PythonArrowOutput to be 
more generic to cover the complex data type on both input and output. This is a 
baseline work for #37863.

### Why are the changes needed?

The traits PythonArrowInput and PythonArrowOutput can be further 
generalized to cover complex data type on both input and output. E.g. Not all 
operators would have simple InternalRow as input data to pass to Python worker 
and vice versa for output data.

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

No.

### How was this patch tested?

Existing tests.

Closes #37864 from HeartSaVioR/SPARK-40414.

Authored-by: Jungtaek Lim 
Signed-off-by: Jungtaek Lim 
---
 .../sql/execution/python/ArrowPythonRunner.scala   |  4 +-
 .../python/CoGroupedArrowPythonRunner.scala|  2 +-
 .../sql/execution/python/PythonArrowInput.scala| 47 --
 .../sql/execution/python/PythonArrowOutput.scala   | 22 +++---
 4 files changed, 53 insertions(+), 22 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala
index 137e2fe93c7..8467feb91d1 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala
@@ -34,8 +34,8 @@ class ArrowPythonRunner(
 protected override val timeZoneId: String,
 protected override val workerConf: Map[String, String])
   extends BasePythonRunner[Iterator[InternalRow], ColumnarBatch](funcs, 
evalType, argOffsets)
-  with PythonArrowInput
-  with PythonArrowOutput {
+  with BasicPythonArrowInput
+  with BasicPythonArrowOutput {
 
   override val simplifiedTraceback: Boolean = 
SQLConf.get.pysparkSimplifiedTraceback
 
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/CoGroupedArrowPythonRunner.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/CoGroupedArrowPythonRunner.scala
index e3d8a943d8c..2661896ecec 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/CoGroupedArrowPythonRunner.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/CoGroupedArrowPythonRunner.scala
@@ -48,7 +48,7 @@ class CoGroupedArrowPythonRunner(
 conf: Map[String, String])
   extends BasePythonRunner[
 (Iterator[InternalRow], Iterator[InternalRow]), ColumnarBatch](funcs, 
evalType, argOffsets)
-  with PythonArrowOutput {
+  with BasicPythonArrowOutput {
 
   override val simplifiedTraceback: Boolean = 
SQLConf.get.pysparkSimplifiedTraceback
 
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonArrowInput.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonArrowInput.scala
index 79365080f8c..6168d0f867a 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonArrowInput.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonArrowInput.scala
@@ -32,15 +32,21 @@ import org.apache.spark.util.Utils
 
 /**
  * A trait that can be mixed-in with [[BasePythonRunner]]. It implements the 
logic from
- * JVM (an iterator of internal rows) to Python (Arrow).
+ * JVM (an iterator of internal rows + additional data if required) to Python 
(Arrow).
  */
-private[python] trait PythonArrowInput { self: 
BasePythonRunner[Iterator[InternalRow], _] =>
+private[python] trait PythonArrowInput[IN] { self: BasePythonRunner[IN, _] =>
   protected val workerConf: Map[String, String]
 
   protected val schema: StructType
 
   protected val timeZoneId: String
 
+  protected def writeIteratorToArrowStream(
+  root: VectorSchemaRoot,
+  writer: ArrowStreamWriter,
+  dataOut: DataOutputStream,
+  inputIterator: Iterator[IN]): Unit
+
   protected def handleMetadataBeforeExec(stream: DataOutputStream): Unit = {
 // Write config for the worker as a number of key -> value pairs of strings
 stream.writeInt(workerConf.size)
@@ -53,7 +59,7 @@ private[python] trait PythonArrowInput { self: 
BasePythonRunner[Iterator[Interna
   protected override def newWriterThread(
   env: SparkEnv,

[spark] branch master updated (c0785238047 -> 7e4064c02de)

2022-08-24 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 c0785238047 [SPARK-40213][SQL] Support ASCII value conversion for 
Latin-1 characters
 add 7e4064c02de [SPARK-40039][SS] Introducing a streaming checkpoint file 
manager based on Hadoop's Abortable interface

No new revisions were added by this update.

Summary of changes:
 docs/cloud-integration.md  |  12 ++-
 hadoop-cloud/README.md |  20 
 hadoop-cloud/pom.xml   |  47 +
 ...AbortableStreamBasedCheckpointFileManager.scala |  95 +
 ...ableStreamBasedCheckpointFileManagerSuite.scala |  83 +++
 .../internal/io/cloud/IntegrationTestSuite.java|   5 +-
 .../io/cloud/abortable/AbortableFileSystem.java| 114 +
 .../abortable/AbstractAbortableFileSystem.java |  45 +++-
 project/SparkBuild.scala   |   3 +-
 .../streaming/CheckpointFileManager.scala  |  57 ++-
 .../streaming/CheckpointFileManagerSuite.scala |  94 -
 11 files changed, 485 insertions(+), 90 deletions(-)
 create mode 100644 hadoop-cloud/README.md
 create mode 100644 
hadoop-cloud/src/hadoop-3/main/scala/org/apache/spark/internal/io/cloud/AbortableStreamBasedCheckpointFileManager.scala
 create mode 100644 
hadoop-cloud/src/hadoop-3/test/scala/org/apache/spark/internal/io/cloud/AbortableStreamBasedCheckpointFileManagerSuite.scala
 copy common/tags/src/test/java/org/apache/spark/tags/ExtendedLevelDBTest.java 
=> 
hadoop-cloud/src/test/java/org/apache/spark/internal/io/cloud/IntegrationTestSuite.java
 (92%)
 create mode 100644 
hadoop-cloud/src/test/java/org/apache/spark/internal/io/cloud/abortable/AbortableFileSystem.java
 copy 
common/network-yarn/src/main/java/org/apache/spark/network/yarn/util/HadoopConfigProvider.java
 => 
hadoop-cloud/src/test/java/org/apache/spark/internal/io/cloud/abortable/AbstractAbortableFileSystem.java
 (52%)


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



[spark] branch master updated (99fc389a0cd -> 83966e87348)

2022-08-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 99fc389a0cd [SPARK-39935][SQL][TESTS] Switch `validateParsingError()` 
onto `checkError()`
 add 83966e87348 [SPARK-39940][SS] Refresh catalog table on streaming query 
with DSv1 sink

No new revisions were added by this update.

Summary of changes:
 .../execution/streaming/MicroBatchExecution.scala  |  9 -
 .../streaming/test/DataStreamTableAPISuite.scala   | 40 +-
 2 files changed, 47 insertions(+), 2 deletions(-)


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



[spark] branch branch-3.2 updated: [SPARK-39839][SQL] Handle special case of null variable-length Decimal with non-zero offsetAndSize in UnsafeRow structural integrity check

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

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


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new 57f1bb74230 [SPARK-39839][SQL] Handle special case of null 
variable-length Decimal with non-zero offsetAndSize in UnsafeRow structural 
integrity check
57f1bb74230 is described below

commit 57f1bb74230f08422147e10ac981c968d0f7a5cf
Author: Kris Mok 
AuthorDate: Thu Jul 28 08:49:33 2022 +0900

[SPARK-39839][SQL] Handle special case of null variable-length Decimal with 
non-zero offsetAndSize in UnsafeRow structural integrity check

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

Update the `UnsafeRow` structural integrity check in 
`UnsafeRowUtils.validateStructuralIntegrity` to handle a special case with null 
variable-length DecimalType value.

### Why are the changes needed?

The check should follow the format that `UnsafeRowWriter` produces. In 
general, `UnsafeRowWriter` clears out a field with zero when the field is set 
to be null, c.f. `UnsafeRowWriter.setNullAt(ordinal)` and 
`UnsafeRow.setNullAt(ordinal)`.

But there's a special case for `DecimalType` values: this is the only type 
that is both:
- can be fixed-length or variable-length, depending on the precision, and
- is mutable in `UnsafeRow`.

To support a variable-length `DecimalType` to be mutable in `UnsafeRow`, 
the `UnsafeRowWriter` always leaves a 16-byte space in the variable-length 
section of the `UnsafeRow` (tail end of the row), regardless of whether the 
`Decimal` value being written is null or not. In the fixed-length part of the 
field, it would be an "OffsetAndSize", and the `offset` part always points to 
the start offset of the variable-length part of the field, while the `size` 
part will either be `0` for the n [...]
When `setNullAt(ordinal)` is called instead of passing a null value to 
`write(int, Decimal, int, int)`, however, the `offset` part gets zero'd out and 
this field stops being mutable. There's a comment on `UnsafeRow.setDecimal` 
that mentions to keep this field able to support updates, `setNullAt(ordinal)` 
cannot be called, but there's no code enforcement of that.

So we need to recognize that in the structural integrity check and allow 
variable-length `DecimalType` to have non-zero field even for null.

Note that for non-null values, the existing check does conform to the 
format from `UnsafeRowWriter`. It's only null value of variable-length 
`DecimalType` that'd trigger a bug, which can affect Structured Streaming's 
checkpoint file read where this check is applied.

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

Yes, previously the `UnsafeRow` structural integrity validation will return 
false positive for correct data, when there's a null value in a variable-length 
`DecimalType` field. The fix will no longer return false positive.
Because the Structured Streaming checkpoint file validation uses this 
check, previously a good checkpoint file may be rejected by the check, and the 
only workaround is to disable the check; with the fix, the correct checkpoint 
file will be allowed to load.

### How was this patch tested?

Added new test case in `UnsafeRowUtilsSuite`

Closes #37252 from rednaxelafx/fix-unsaferow-validation.

Authored-by: Kris Mok 
Signed-off-by: Jungtaek Lim 
(cherry picked from commit c608ae2fc6a3a50f2e67f2a3dad8d4e4be1aaf9f)
Signed-off-by: Jungtaek Lim 
---
 .../spark/sql/catalyst/util/UnsafeRowUtils.scala   | 44 ++
 .../sql/catalyst/util/UnsafeRowUtilsSuite.scala| 31 ++-
 2 files changed, 67 insertions(+), 8 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/UnsafeRowUtils.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/UnsafeRowUtils.scala
index 37a34fac663..48db0c7d971 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/UnsafeRowUtils.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/UnsafeRowUtils.scala
@@ -27,8 +27,15 @@ object UnsafeRowUtils {
* - schema.fields.length == row.numFields should always be true
* - UnsafeRow.calculateBitSetWidthInBytes(row.numFields) < 
row.getSizeInBytes should always be
*   true if the expectedSchema contains at least one field.
-   * - For variable-length fields: if null bit says it's null then don't do 
anything, else extract
-   *   offset and size:
+   * - For variable-length fields:
+   *   - if null bit says it's null, then
+   * - in general the offset-and-size should be zero
+   * - special case: variable-length DecimalType is considered mutable in 
UnsafeRow, and to
+   *   support that, the

[spark] branch branch-3.3 updated: [SPARK-39839][SQL] Handle special case of null variable-length Decimal with non-zero offsetAndSize in UnsafeRow structural integrity check

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

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


The following commit(s) were added to refs/heads/branch-3.3 by this push:
 new ee8cafbd0ff [SPARK-39839][SQL] Handle special case of null 
variable-length Decimal with non-zero offsetAndSize in UnsafeRow structural 
integrity check
ee8cafbd0ff is described below

commit ee8cafbd0ff36116a212ac99fdf65b24c486cae8
Author: Kris Mok 
AuthorDate: Thu Jul 28 08:49:33 2022 +0900

[SPARK-39839][SQL] Handle special case of null variable-length Decimal with 
non-zero offsetAndSize in UnsafeRow structural integrity check

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

Update the `UnsafeRow` structural integrity check in 
`UnsafeRowUtils.validateStructuralIntegrity` to handle a special case with null 
variable-length DecimalType value.

### Why are the changes needed?

The check should follow the format that `UnsafeRowWriter` produces. In 
general, `UnsafeRowWriter` clears out a field with zero when the field is set 
to be null, c.f. `UnsafeRowWriter.setNullAt(ordinal)` and 
`UnsafeRow.setNullAt(ordinal)`.

But there's a special case for `DecimalType` values: this is the only type 
that is both:
- can be fixed-length or variable-length, depending on the precision, and
- is mutable in `UnsafeRow`.

To support a variable-length `DecimalType` to be mutable in `UnsafeRow`, 
the `UnsafeRowWriter` always leaves a 16-byte space in the variable-length 
section of the `UnsafeRow` (tail end of the row), regardless of whether the 
`Decimal` value being written is null or not. In the fixed-length part of the 
field, it would be an "OffsetAndSize", and the `offset` part always points to 
the start offset of the variable-length part of the field, while the `size` 
part will either be `0` for the n [...]
When `setNullAt(ordinal)` is called instead of passing a null value to 
`write(int, Decimal, int, int)`, however, the `offset` part gets zero'd out and 
this field stops being mutable. There's a comment on `UnsafeRow.setDecimal` 
that mentions to keep this field able to support updates, `setNullAt(ordinal)` 
cannot be called, but there's no code enforcement of that.

So we need to recognize that in the structural integrity check and allow 
variable-length `DecimalType` to have non-zero field even for null.

Note that for non-null values, the existing check does conform to the 
format from `UnsafeRowWriter`. It's only null value of variable-length 
`DecimalType` that'd trigger a bug, which can affect Structured Streaming's 
checkpoint file read where this check is applied.

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

Yes, previously the `UnsafeRow` structural integrity validation will return 
false positive for correct data, when there's a null value in a variable-length 
`DecimalType` field. The fix will no longer return false positive.
Because the Structured Streaming checkpoint file validation uses this 
check, previously a good checkpoint file may be rejected by the check, and the 
only workaround is to disable the check; with the fix, the correct checkpoint 
file will be allowed to load.

### How was this patch tested?

Added new test case in `UnsafeRowUtilsSuite`

Closes #37252 from rednaxelafx/fix-unsaferow-validation.

Authored-by: Kris Mok 
Signed-off-by: Jungtaek Lim 
(cherry picked from commit c608ae2fc6a3a50f2e67f2a3dad8d4e4be1aaf9f)
Signed-off-by: Jungtaek Lim 
---
 .../spark/sql/catalyst/util/UnsafeRowUtils.scala   | 44 ++
 .../sql/catalyst/util/UnsafeRowUtilsSuite.scala| 31 ++-
 2 files changed, 67 insertions(+), 8 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/UnsafeRowUtils.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/UnsafeRowUtils.scala
index 37a34fac663..48db0c7d971 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/UnsafeRowUtils.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/UnsafeRowUtils.scala
@@ -27,8 +27,15 @@ object UnsafeRowUtils {
* - schema.fields.length == row.numFields should always be true
* - UnsafeRow.calculateBitSetWidthInBytes(row.numFields) < 
row.getSizeInBytes should always be
*   true if the expectedSchema contains at least one field.
-   * - For variable-length fields: if null bit says it's null then don't do 
anything, else extract
-   *   offset and size:
+   * - For variable-length fields:
+   *   - if null bit says it's null, then
+   * - in general the offset-and-size should be zero
+   * - special case: variable-length DecimalType is considered mutable in 
UnsafeRow, and to
+   *   support that, the

[spark] branch master updated (f9409ce7d49 -> c608ae2fc6a)

2022-07-27 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 f9409ce7d49 [SPARK-38493][PS] Improve the test coverage for pandas API 
on Spark
 add c608ae2fc6a [SPARK-39839][SQL] Handle special case of null 
variable-length Decimal with non-zero offsetAndSize in UnsafeRow structural 
integrity check

No new revisions were added by this update.

Summary of changes:
 .../spark/sql/catalyst/util/UnsafeRowUtils.scala   | 44 ++
 .../sql/catalyst/util/UnsafeRowUtilsSuite.scala| 31 ++-
 2 files changed, 67 insertions(+), 8 deletions(-)


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



[spark] branch master updated (869fc2198a4 -> 0e33195d1ba)

2022-07-27 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 869fc2198a4 [SPARK-39864][SQL] Lazily register ExecutionListenerBus
 add 0e33195d1ba [SPARK-39834][SQL][SS] Include the origin stats and 
constraints for LogicalRDD if it comes from DataFrame

No new revisions were added by this update.

Summary of changes:
 .../main/scala/org/apache/spark/sql/Dataset.scala  | 24 +--
 .../apache/spark/sql/execution/ExistingRDD.scala   | 80 -
 .../streaming/sources/ForeachBatchSink.scala   | 25 +--
 .../org/apache/spark/sql/DataFrameSuite.scala  | 82 +++---
 .../streaming/sources/ForeachBatchSinkSuite.scala  | 45 +---
 5 files changed, 141 insertions(+), 115 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-39805][SS] Deprecate Trigger.Once and Promote Trigger.AvailableNow

2022-07-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 66b1f79b728 [SPARK-39805][SS] Deprecate Trigger.Once and Promote 
Trigger.AvailableNow
66b1f79b728 is described below

commit 66b1f79b72855af35351ff995492f2c13872dac5
Author: Jungtaek Lim 
AuthorDate: Thu Jul 21 07:35:00 2022 +0900

[SPARK-39805][SS] Deprecate Trigger.Once and Promote Trigger.AvailableNow

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

This PR proposes to deprecate Trigger.Once and suggest Trigger.AvailableNow 
as a replacement.

This PR also tries to replace Trigger.Once to Trigger.AvailableNow in the 
test code as well, except the cases Trigger.Once is used intentionally.

### Why are the changes needed?

Trigger.Once() exposes various issues, including:

1) weak guarantee of the contract

This is the javadoc content of `Trigger.Once`:

> A trigger that processes all available data in a single batch then 
terminates the query.

Spark does not respect the contract when there is "uncommitted" batch in 
the previous run. It really works as the name represents, "just run a single 
batch", hence if there is "uncommitted" batch, Spark will execute the 
"uncommitted" batch and terminate without processing new data.

2) scalable issue on batch

This is the main rationalization we introduced Trigger.AvailableNow.

3) huge output latency for stateful operator due to the lack of no-data 
batch

Since Trigger.Once executes the single batch and terminates, the processing 
for watermark advancement is deferred to the next execution of the query, which 
tends to be multiple hours or even day(s).

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

Yes, end users will start to see the deprecation message when they use 
Trigger.Once. The deprecation message guides the end users to migrate to 
Trigger.Available, with the rationalization on migration.

### How was this patch tested?

Existing UTs

Closes #37213 from HeartSaVioR/SPARK-39805.

Authored-by: Jungtaek Lim 
Signed-off-by: Jungtaek Lim 
---
 .../sql/kafka010/KafkaMicroBatchSourceSuite.scala  |  3 +++
 docs/ss-migration-guide.md |  4 
 docs/structured-streaming-programming-guide.md | 20 
 .../org/apache/spark/sql/streaming/Trigger.java| 19 ---
 .../streaming/MicroBatchExecutionSuite.scala   |  2 +-
 .../streaming/sources/ForeachBatchSinkSuite.scala  |  4 ++--
 .../sources/RatePerMicroBatchProviderSuite.scala   |  1 +
 .../sql/streaming/EventTimeWatermarkSuite.scala| 22 ++
 .../sql/streaming/FileStreamSourceSuite.scala  |  2 ++
 .../spark/sql/streaming/StreamingQuerySuite.scala  | 17 -
 .../sources/StreamingDataSourceV2Suite.scala   | 12 
 11 files changed, 79 insertions(+), 27 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 5a8caef9e5e..af66ecd21c0 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
@@ -338,6 +338,7 @@ abstract class KafkaMicroBatchSourceSuiteBase extends 
KafkaSourceSuiteBase {
 )
 
 // When Trigger.Once() is used, the read limit should be ignored
+// NOTE: the test uses the deprecated Trigger.Once() by intention, do not 
change.
 val allData = Seq(1) ++ (10 to 20) ++ (100 to 200)
 withTempDir { dir =>
   testStream(mapped)(
@@ -435,6 +436,7 @@ abstract class KafkaMicroBatchSourceSuiteBase extends 
KafkaSourceSuiteBase {
 13, 14, 15, 16, 17, 18, 19, 2, 20, 21, 22, 23, 24, 25)
 )
 // When Trigger.Once() is used, the read limit should be ignored
+// NOTE: the test uses the deprecated Trigger.Once() by intention, do not 
change.
 val allData = Seq(1, 2) ++ (10 to 25) ++ (100 to 125)
 withTempDir { dir =>
   testStream(mapped)(
@@ -537,6 +539,7 @@ abstract class KafkaMicroBatchSourceSuiteBase extends 
KafkaSourceSuiteBase {
 )
 
 // When Trigger.Once() is used, the read limit should be ignored
+// NOTE: the test uses the deprecated Trigger.Once() by intention, do not 
change.
 val allData = Seq(1, 2) ++ (10 to 30) ++ (100 to 128)
 withTempDir { dir =>
   testStream(mapped)(
diff --git a/docs/ss-migration-guide.md b/docs/ss-migration-guide.md
index c28724576bc..0ca5b00debc 100644
--- a/docs/ss

[spark] branch master updated: [SPARK-39748][SQL][SS][FOLLOWUP] Fix a bug on column stat in LogicalRDD on mismatching exprIDs

2022-07-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 3c80ed8547d [SPARK-39748][SQL][SS][FOLLOWUP] Fix a bug on column stat 
in LogicalRDD on mismatching exprIDs
3c80ed8547d is described below

commit 3c80ed8547d550a056788267bbb395fde8b2c030
Author: Jungtaek Lim 
AuthorDate: Fri Jul 15 12:44:53 2022 +0900

[SPARK-39748][SQL][SS][FOLLOWUP] Fix a bug on column stat in LogicalRDD on 
mismatching exprIDs

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

This PR fixes a bug on #37161 (described the bug in below section) via 
making sure the output columns in LogicalRDD are always the same with output 
columns in originLogicalPlan in LogicalRDD, which is needed to inherit the 
column stats.

### Why are the changes needed?

Stats for columns in originLogicalPlan refer to the columns in 
originLogicalPlan, which could be different from the columns in output of 
LogicalRDD in terms of expression ID.

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

No.

### How was this patch tested?

New UT

Closes #37187 from HeartSaVioR/SPARK-39748-FOLLOWUP-2.

Authored-by: Jungtaek Lim 
Signed-off-by: Jungtaek Lim 
---
 .../apache/spark/sql/execution/ExistingRDD.scala   |  12 ++-
 .../streaming/sources/ForeachBatchSink.scala   |   7 +-
 .../org/apache/spark/sql/DataFrameSuite.scala  | 113 -
 3 files changed, 126 insertions(+), 6 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala
index bf9ef6991e3..149e70e56d0 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala
@@ -116,10 +116,20 @@ case class LogicalRDD(
   case e: Attribute => rewrite.getOrElse(e, e)
 }.asInstanceOf[SortOrder])
 
+val rewrittenOriginLogicalPlan = originLogicalPlan.map { plan =>
+  assert(output == plan.output, "The output columns are expected to the 
same for output " +
+s"and originLogicalPlan. output: $output / output in 
originLogicalPlan: ${plan.output}")
+
+  val projectList = output.map { attr =>
+Alias(attr, attr.name)(exprId = rewrite(attr).exprId)
+  }
+  Project(projectList, plan)
+}
+
 LogicalRDD(
   output.map(rewrite),
   rdd,
-  originLogicalPlan,
+  rewrittenOriginLogicalPlan,
   rewrittenPartitioning,
   rewrittenOrdering,
   isStreaming
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachBatchSink.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachBatchSink.scala
index 1c6bca241af..395ed056be2 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachBatchSink.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachBatchSink.scala
@@ -30,10 +30,13 @@ class ForeachBatchSink[T](batchWriter: (Dataset[T], Long) 
=> Unit, encoder: Expr
   override def addBatch(batchId: Long, data: DataFrame): Unit = {
 val rdd = data.queryExecution.toRdd
 val executedPlan = data.queryExecution.executedPlan
+val analyzedPlanWithoutMarkerNode = 
eliminateWriteMarkerNode(data.queryExecution.analyzed)
+// assertion on precondition
+assert(data.logicalPlan.output == analyzedPlanWithoutMarkerNode.output)
 val node = LogicalRDD(
-  data.schema.toAttributes,
+  data.logicalPlan.output,
   rdd,
-  Some(eliminateWriteMarkerNode(data.queryExecution.analyzed)),
+  Some(analyzedPlanWithoutMarkerNode),
   executedPlan.outputPartitioning,
   executedPlan.outputOrdering)(data.sparkSession)
 implicit val enc = encoder
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
index 41593c701a7..e802159f263 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
@@ -32,13 +32,14 @@ import org.scalatest.matchers.should.Matchers._
 import org.apache.spark.SparkException
 import org.apache.spark.scheduler.{SparkListener, SparkListenerJobEnd}
 import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
 import org.apache.spark.sql.catalyst.encoders.{ExpressionEncoder, RowEncoder}
-import org.apache.spark.sql.catalyst.expressions.Uuid
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, 
AttributeReference, Uuid}
 import org.apache.spark.sql.catalyst.

[spark] branch master updated: [SPARK-39781][SS] Add support for providing max_open_files to rocksdb state store provider

2022-07-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 c05d0fda2b3 [SPARK-39781][SS] Add support for providing max_open_files 
to rocksdb state store provider
c05d0fda2b3 is described below

commit c05d0fda2b312dbae035bb4166f4e89dd4dd0c1e
Author: Anish Shrigondekar 
AuthorDate: Fri Jul 15 12:34:31 2022 +0900

[SPARK-39781][SS] Add support for providing max_open_files to rocksdb state 
store provider

### What changes were proposed in this pull request?
For some large users of stateful queries with lot of rocksdb related files 
open, they run into IO exceptions around "too many open files".
```
Job aborted due to stage failure: ... : org.rocksdb.RocksDBException: While 
open a file for random read: ... XXX.sst: Too many open files
```
This change allows configuring the max_open_files property for the 
underlying RocksDB instance.

### Why are the changes needed?
By default, value for maxOpenFiles is -1, which means that the DB can keep 
opened files always open. However, in some cases, this will hit the OS limit 
and crash the process. As part of this change, we provide a state store config 
option for RocksDB to set this to a finite value so that number of opened files 
can be bounded per RocksDB instance.

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

### How was this patch tested?
Added tests to validate config passed through a RocksDB conf as well as 
through Spark session.

```
[info] - RocksDB confs are passed correctly from SparkSession to db 
instance (2 seconds, 377 milliseconds)
12:54:57.927 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), shuffle-boss-6-1 (daemon=true) =
[info] Run completed in 4 seconds, 24 milliseconds.
[info] Total number of tests run: 1
[info] Suites: completed 1, aborted 0
[info] Tests: succeeded 1, failed 0, canceled 0, ignored 0, pending 0
[info] All tests passed.
```

```
[info] RocksDBSuite:
12:55:56.165 WARN org.apache.hadoop.util.NativeCodeLoader: Unable to load 
native-hadoop library for your platform... using builtin-java classes where 
applicable
[info] - SPARK-39781: adding valid max_open_files=-1 config property for 
RocksDB state store instance should succeed (1 second, 553 milliseconds)
[info] - SPARK-39781: adding valid max_open_files=100 config property for 
RocksDB state store instance should succeed (664 milliseconds)
[info] - SPARK-39781: adding valid max_open_files=1000 config property for 
RocksDB state store instance should succeed (558 milliseconds)
[info] - SPARK-39781: adding invalid max_open_files=test config property 
for RocksDB state store instance should fail (9 milliseconds)
[info] - SPARK-39781: adding invalid max_open_files=true config property 
for RocksDB state store instance should fail (8 milliseconds)
[info] Run completed in 3 seconds, 815 milliseconds.
[info] Total number of tests run: 5
[info] Suites: completed 1, aborted 0
[info] Tests: succeeded 5, failed 0, canceled 0, ignored 0, pending 0
[info] All tests passed.
```

Closes #37196 from anishshri-db/task/SPARK-39781.

Authored-by: Anish Shrigondekar 
Signed-off-by: Jungtaek Lim 
---
 docs/structured-streaming-programming-guide.md |  5 +++
 .../sql/execution/streaming/state/RocksDB.scala| 17 +++-
 .../streaming/state/RocksDBStateStoreSuite.scala   |  2 +
 .../execution/streaming/state/RocksDBSuite.scala   | 49 ++
 4 files changed, 71 insertions(+), 2 deletions(-)

diff --git a/docs/structured-streaming-programming-guide.md 
b/docs/structured-streaming-programming-guide.md
index 8bb3cdf3ef3..c0f501a3d92 100644
--- a/docs/structured-streaming-programming-guide.md
+++ b/docs/structured-streaming-programming-guide.md
@@ -1958,6 +1958,11 @@ Here are the configs regarding to RocksDB instance of 
the state store provider:
 The waiting time in millisecond for acquiring lock in the load 
operation for RocksDB instance.
 6
   
+  
+spark.sql.streaming.stateStore.rocksdb.maxOpenFiles
+The number of open files that can be used by the RocksDB instance. 
Value of -1 means that files opened are always kept open. If the open file 
limit is reached, RocksDB will evict entries from the open file cache and close 
those file descriptors and remove the entries from the cache.
+-1
+  
   
 spark.sql.streaming.stateStore.rocksdb.resetStatsOnLoad
 Whether we resets all ticker and histogram stats for RocksDB on 
load.
diff --git 
a/sql/core/src/main/scala/org/apache

[spark] branch master updated: [SPARK-39748][SQL][FOLLOWUP] Add missing origin logical plan on DataFrame.checkpoint on building LogicalRDD

2022-07-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 4e374687071 [SPARK-39748][SQL][FOLLOWUP] Add missing origin logical 
plan on DataFrame.checkpoint on building LogicalRDD
4e374687071 is described below

commit 4e3746870718015ccded31e944e0afd9ad7729a3
Author: Jungtaek Lim 
AuthorDate: Wed Jul 13 11:39:36 2022 +0900

[SPARK-39748][SQL][FOLLOWUP] Add missing origin logical plan on 
DataFrame.checkpoint on building LogicalRDD

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

This PR adds missing origin logical plan on building LogicalRDD in 
DataFrame.checkpoint, via review comment 
https://github.com/apache/spark/pull/37161#discussion_r919204026.

### Why are the changes needed?

This is missing spot on previous PR and viirya helped to find out.

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

No.

### How was this patch tested?

N/A

Closes #37167 from HeartSaVioR/SPARK-39748-FOLLOWUP.

Authored-by: Jungtaek Lim 
Signed-off-by: Jungtaek Lim 
---
 sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
index f45c27d3007..7f5efd85e3b 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
@@ -705,7 +705,7 @@ class Dataset[T] private[sql](
 LogicalRDD(
   logicalPlan.output,
   internalRdd,
-  None,
+  Some(queryExecution.analyzed),
   outputPartitioning,
   physicalPlan.outputOrdering,
   isStreaming


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



[spark] branch master updated: [SPARK-39748][SQL][SS] Include the origin logical plan for LogicalRDD if it comes from DataFrame

2022-07-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 cb0b51038b0 [SPARK-39748][SQL][SS] Include the origin logical plan for 
LogicalRDD if it comes from DataFrame
cb0b51038b0 is described below

commit cb0b51038b0ae17ba2a4a38082e322f5b6087e06
Author: Jungtaek Lim 
AuthorDate: Tue Jul 12 17:59:10 2022 +0900

[SPARK-39748][SQL][SS] Include the origin logical plan for LogicalRDD if it 
comes from DataFrame

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

This PR proposes to include the origin logical plan for LogicalRDD, if the 
LogicalRDD is built from DataFrame's RDD. Once the origin logical plan is 
available, LogicalRDD produces the stats from origin logical plan rather than 
default one.

Also, this PR applies the change to ForeachBatchSink, which seems to be the 
only case as of now in current codebase.

### Why are the changes needed?

The origin logical plan can be useful for several use cases, including:

1. wants to connect the two split logical plans into one (consider the case 
of foreachBatch sink: origin logical plan represents the plan for streaming 
query, and the logical plan for new Dataset represents the plan for batch query 
in user function)
2. inherits plan stats from origin logical plan

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

No.

### How was this patch tested?

New UT.

Closes #37161 from HeartSaVioR/SPARK-39748.

Authored-by: Jungtaek Lim 
Signed-off-by: Jungtaek Lim 
---
 .../main/scala/org/apache/spark/sql/Dataset.scala  |  1 +
 .../apache/spark/sql/execution/ExistingRDD.scala   | 23 ---
 .../streaming/sources/ForeachBatchSink.scala   | 22 ++-
 .../streaming/sources/ForeachBatchSinkSuite.scala  | 45 +-
 4 files changed, 83 insertions(+), 8 deletions(-)

diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
index 39d33d80261..f45c27d3007 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
@@ -705,6 +705,7 @@ class Dataset[T] private[sql](
 LogicalRDD(
   logicalPlan.output,
   internalRdd,
+  None,
   outputPartitioning,
   physicalPlan.outputOrdering,
   isStreaming
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala
index 1ab183fe843..bf9ef6991e3 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala
@@ -83,10 +83,16 @@ case class ExternalRDDScanExec[T](
   }
 }
 
-/** Logical plan node for scanning data from an RDD of InternalRow. */
+/**
+ * Logical plan node for scanning data from an RDD of InternalRow.
+ *
+ * It is advised to set the field `originLogicalPlan` if the RDD is directly 
built from DataFrame,
+ * as the stat can be inherited from `originLogicalPlan`.
+ */
 case class LogicalRDD(
 output: Seq[Attribute],
 rdd: RDD[InternalRow],
+originLogicalPlan: Option[LogicalPlan] = None,
 outputPartitioning: Partitioning = UnknownPartitioning(0),
 override val outputOrdering: Seq[SortOrder] = Nil,
 override val isStreaming: Boolean = false)(session: SparkSession)
@@ -113,6 +119,7 @@ case class LogicalRDD(
 LogicalRDD(
   output.map(rewrite),
   rdd,
+  originLogicalPlan,
   rewrittenPartitioning,
   rewrittenOrdering,
   isStreaming
@@ -121,11 +128,15 @@ case class LogicalRDD(
 
   override protected def stringArgs: Iterator[Any] = Iterator(output, 
isStreaming)
 
-  override def computeStats(): Statistics = Statistics(
-// TODO: Instead of returning a default value here, find a way to return a 
meaningful size
-// estimate for RDDs. See PR 1238 for more discussions.
-sizeInBytes = BigInt(session.sessionState.conf.defaultSizeInBytes)
-  )
+  override def computeStats(): Statistics = {
+originLogicalPlan.map(_.stats).getOrElse {
+  Statistics(
+// TODO: Instead of returning a default value here, find a way to 
return a meaningful size
+// estimate for RDDs. See PR 1238 for more discussions.
+sizeInBytes = BigInt(session.sessionState.conf.defaultSizeInBytes)
+  )
+}
+  }
 }
 
 /** Physical plan node for scanning data from an RDD of InternalRow. */
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachBatchSink.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachBatchSink.scala
index 0893875aff5..1c6bca

[spark] branch branch-3.2 updated: [SPARK-39650][SS] Fix incorrect value schema in streaming deduplication with backward compatibility

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

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


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new 9adfc3a21fd [SPARK-39650][SS] Fix incorrect value schema in streaming 
deduplication with backward compatibility
9adfc3a21fd is described below

commit 9adfc3a21fd566f0eb37537e4137dd448af3aee1
Author: Jungtaek Lim 
AuthorDate: Sat Jul 2 22:46:03 2022 +0900

[SPARK-39650][SS] Fix incorrect value schema in streaming deduplication 
with backward compatibility

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

This PR proposes to fix the incorrect value schema in streaming 
deduplication. It stores the empty row having a single column with null (using 
NullType), but the value schema is specified as all columns, which leads 
incorrect behavior from state store schema compatibility checker.

This PR proposes to set the schema of value as 
`StructType(Array(StructField("__dummy__", NullType)))` to fit with the empty 
row. With this change, the streaming queries creating the checkpoint after this 
fix would work smoothly.

To not break the existing streaming queries having incorrect value schema, 
this PR proposes to disable the check for value schema on streaming 
deduplication. Disabling the value check was there for the format validation 
(we have two different checkers for state store), but it has been missing for 
state store schema compatibility check. To avoid adding more config, this PR 
leverages the existing config "format validation" is using.

### Why are the changes needed?

This is a bug fix. Suppose the streaming query below:

```
# df has the columns `a`, `b`, `c`
val df = spark.readStream.format("...").load()
val query = df.dropDuplicate("a").writeStream.format("...").start()
```

while the query is running, df can produce a different set of columns (e.g. 
`a`, `b`, `c`, `d`) from the same source due to schema evolution. Since we only 
deduplicate the rows with column `a`, the change of schema should not matter 
for streaming deduplication, but state store schema checker throws error saying 
"value schema is not compatible" before this fix.

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

No, this is basically a bug fix which end users wouldn't notice unless they 
encountered a bug.

### How was this patch tested?

New tests.

Closes #37041 from HeartSaVioR/SPARK-39650.

Authored-by: Jungtaek Lim 
Signed-off-by: Jungtaek Lim 
(cherry picked from commit fe536033bdd00d921b3c86af329246ca55a4f46a)
Signed-off-by: Jungtaek Lim 
---
 .../state/StateSchemaCompatibilityChecker.scala|  26 ++--
 .../sql/execution/streaming/state/StateStore.scala |   7 ++-
 .../execution/streaming/state/StateStoreConf.scala |   7 ++-
 .../execution/streaming/statefulOperators.scala|   4 +-
 .../commits/.0.crc | Bin 0 -> 12 bytes
 .../commits/.1.crc | Bin 0 -> 12 bytes
 .../commits/0  |   2 +
 .../commits/1  |   2 +
 .../metadata   |   1 +
 .../offsets/.0.crc | Bin 0 -> 16 bytes
 .../offsets/.1.crc | Bin 0 -> 16 bytes
 .../offsets/0  |   3 +
 .../offsets/1  |   3 +
 .../state/0/0/.1.delta.crc | Bin 0 -> 12 bytes
 .../state/0/0/.2.delta.crc | Bin 0 -> 12 bytes
 .../state/0/0/1.delta  | Bin 0 -> 77 bytes
 .../state/0/0/2.delta  | Bin 0 -> 46 bytes
 .../state/0/0/_metadata/.schema.crc| Bin 0 -> 12 bytes
 .../state/0/0/_metadata/schema | Bin 0 -> 254 bytes
 .../state/0/1/.1.delta.crc | Bin 0 -> 12 bytes
 .../state/0/1/.2.delta.crc | Bin 0 -> 12 bytes
 .../state/0/1/1.delta  | Bin 0 -> 46 bytes
 .../state/0/1/2.delta  | Bin 0 -> 77 bytes
 .../state/0/2/.1.delta.crc | Bin 0 -> 12 bytes
 .../state/0/2/.2.delta.crc | Bin 0 -> 12 bytes
 .../state/0/2/1.delta  | Bin 0 -> 46 bytes
 .../state/0/2/2.delta  | Bin 0 -> 46 bytes
 .../state/0/3/.1.delta.crc | Bin 0 -> 12 bytes
 .../state/0/3/.2.delta.crc | Bin 0 -> 12 bytes
 .../state/0/3/1.delta  | Bin 0 -> 46 bytes
 .../stat

[spark] branch branch-3.3 updated: [SPARK-39650][SS] Fix incorrect value schema in streaming deduplication with backward compatibility

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

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


The following commit(s) were added to refs/heads/branch-3.3 by this push:
 new 463a24d9afd [SPARK-39650][SS] Fix incorrect value schema in streaming 
deduplication with backward compatibility
463a24d9afd is described below

commit 463a24d9afdaefabfa9f1129350b38e69ccd062d
Author: Jungtaek Lim 
AuthorDate: Sat Jul 2 22:46:03 2022 +0900

[SPARK-39650][SS] Fix incorrect value schema in streaming deduplication 
with backward compatibility

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

This PR proposes to fix the incorrect value schema in streaming 
deduplication. It stores the empty row having a single column with null (using 
NullType), but the value schema is specified as all columns, which leads 
incorrect behavior from state store schema compatibility checker.

This PR proposes to set the schema of value as 
`StructType(Array(StructField("__dummy__", NullType)))` to fit with the empty 
row. With this change, the streaming queries creating the checkpoint after this 
fix would work smoothly.

To not break the existing streaming queries having incorrect value schema, 
this PR proposes to disable the check for value schema on streaming 
deduplication. Disabling the value check was there for the format validation 
(we have two different checkers for state store), but it has been missing for 
state store schema compatibility check. To avoid adding more config, this PR 
leverages the existing config "format validation" is using.

### Why are the changes needed?

This is a bug fix. Suppose the streaming query below:

```
# df has the columns `a`, `b`, `c`
val df = spark.readStream.format("...").load()
val query = df.dropDuplicate("a").writeStream.format("...").start()
```

while the query is running, df can produce a different set of columns (e.g. 
`a`, `b`, `c`, `d`) from the same source due to schema evolution. Since we only 
deduplicate the rows with column `a`, the change of schema should not matter 
for streaming deduplication, but state store schema checker throws error saying 
"value schema is not compatible" before this fix.

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

No, this is basically a bug fix which end users wouldn't notice unless they 
encountered a bug.

### How was this patch tested?

New tests.

Closes #37041 from HeartSaVioR/SPARK-39650.

Authored-by: Jungtaek Lim 
Signed-off-by: Jungtaek Lim 
(cherry picked from commit fe536033bdd00d921b3c86af329246ca55a4f46a)
Signed-off-by: Jungtaek Lim 
---
 .../state/StateSchemaCompatibilityChecker.scala|  26 ++--
 .../sql/execution/streaming/state/StateStore.scala |   7 ++-
 .../execution/streaming/state/StateStoreConf.scala |   7 ++-
 .../execution/streaming/statefulOperators.scala|   4 +-
 .../commits/.0.crc | Bin 0 -> 12 bytes
 .../commits/.1.crc | Bin 0 -> 12 bytes
 .../commits/0  |   2 +
 .../commits/1  |   2 +
 .../metadata   |   1 +
 .../offsets/.0.crc | Bin 0 -> 16 bytes
 .../offsets/.1.crc | Bin 0 -> 16 bytes
 .../offsets/0  |   3 +
 .../offsets/1  |   3 +
 .../state/0/0/.1.delta.crc | Bin 0 -> 12 bytes
 .../state/0/0/.2.delta.crc | Bin 0 -> 12 bytes
 .../state/0/0/1.delta  | Bin 0 -> 77 bytes
 .../state/0/0/2.delta  | Bin 0 -> 46 bytes
 .../state/0/0/_metadata/.schema.crc| Bin 0 -> 12 bytes
 .../state/0/0/_metadata/schema | Bin 0 -> 254 bytes
 .../state/0/1/.1.delta.crc | Bin 0 -> 12 bytes
 .../state/0/1/.2.delta.crc | Bin 0 -> 12 bytes
 .../state/0/1/1.delta  | Bin 0 -> 46 bytes
 .../state/0/1/2.delta  | Bin 0 -> 77 bytes
 .../state/0/2/.1.delta.crc | Bin 0 -> 12 bytes
 .../state/0/2/.2.delta.crc | Bin 0 -> 12 bytes
 .../state/0/2/1.delta  | Bin 0 -> 46 bytes
 .../state/0/2/2.delta  | Bin 0 -> 46 bytes
 .../state/0/3/.1.delta.crc | Bin 0 -> 12 bytes
 .../state/0/3/.2.delta.crc | Bin 0 -> 12 bytes
 .../state/0/3/1.delta  | Bin 0 -> 46 bytes
 .../stat

[spark] branch master updated: [SPARK-39650][SS] Fix incorrect value schema in streaming deduplication with backward compatibility

2022-07-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 fe536033bdd [SPARK-39650][SS] Fix incorrect value schema in streaming 
deduplication with backward compatibility
fe536033bdd is described below

commit fe536033bdd00d921b3c86af329246ca55a4f46a
Author: Jungtaek Lim 
AuthorDate: Sat Jul 2 22:46:03 2022 +0900

[SPARK-39650][SS] Fix incorrect value schema in streaming deduplication 
with backward compatibility

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

This PR proposes to fix the incorrect value schema in streaming 
deduplication. It stores the empty row having a single column with null (using 
NullType), but the value schema is specified as all columns, which leads 
incorrect behavior from state store schema compatibility checker.

This PR proposes to set the schema of value as 
`StructType(Array(StructField("__dummy__", NullType)))` to fit with the empty 
row. With this change, the streaming queries creating the checkpoint after this 
fix would work smoothly.

To not break the existing streaming queries having incorrect value schema, 
this PR proposes to disable the check for value schema on streaming 
deduplication. Disabling the value check was there for the format validation 
(we have two different checkers for state store), but it has been missing for 
state store schema compatibility check. To avoid adding more config, this PR 
leverages the existing config "format validation" is using.

### Why are the changes needed?

This is a bug fix. Suppose the streaming query below:

```
# df has the columns `a`, `b`, `c`
val df = spark.readStream.format("...").load()
val query = df.dropDuplicate("a").writeStream.format("...").start()
```

while the query is running, df can produce a different set of columns (e.g. 
`a`, `b`, `c`, `d`) from the same source due to schema evolution. Since we only 
deduplicate the rows with column `a`, the change of schema should not matter 
for streaming deduplication, but state store schema checker throws error saying 
"value schema is not compatible" before this fix.

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

No, this is basically a bug fix which end users wouldn't notice unless they 
encountered a bug.

### How was this patch tested?

New tests.

Closes #37041 from HeartSaVioR/SPARK-39650.

Authored-by: Jungtaek Lim 
Signed-off-by: Jungtaek Lim 
---
 .../state/StateSchemaCompatibilityChecker.scala|  26 ++--
 .../sql/execution/streaming/state/StateStore.scala |   7 ++-
 .../execution/streaming/state/StateStoreConf.scala |   7 ++-
 .../execution/streaming/statefulOperators.scala|   4 +-
 .../commits/.0.crc | Bin 0 -> 12 bytes
 .../commits/.1.crc | Bin 0 -> 12 bytes
 .../commits/0  |   2 +
 .../commits/1  |   2 +
 .../metadata   |   1 +
 .../offsets/.0.crc | Bin 0 -> 16 bytes
 .../offsets/.1.crc | Bin 0 -> 16 bytes
 .../offsets/0  |   3 +
 .../offsets/1  |   3 +
 .../state/0/0/.1.delta.crc | Bin 0 -> 12 bytes
 .../state/0/0/.2.delta.crc | Bin 0 -> 12 bytes
 .../state/0/0/1.delta  | Bin 0 -> 77 bytes
 .../state/0/0/2.delta  | Bin 0 -> 46 bytes
 .../state/0/0/_metadata/.schema.crc| Bin 0 -> 12 bytes
 .../state/0/0/_metadata/schema | Bin 0 -> 254 bytes
 .../state/0/1/.1.delta.crc | Bin 0 -> 12 bytes
 .../state/0/1/.2.delta.crc | Bin 0 -> 12 bytes
 .../state/0/1/1.delta  | Bin 0 -> 46 bytes
 .../state/0/1/2.delta  | Bin 0 -> 77 bytes
 .../state/0/2/.1.delta.crc | Bin 0 -> 12 bytes
 .../state/0/2/.2.delta.crc | Bin 0 -> 12 bytes
 .../state/0/2/1.delta  | Bin 0 -> 46 bytes
 .../state/0/2/2.delta  | Bin 0 -> 46 bytes
 .../state/0/3/.1.delta.crc | Bin 0 -> 12 bytes
 .../state/0/3/.2.delta.crc | Bin 0 -> 12 bytes
 .../state/0/3/1.delta  | Bin 0 -> 46 bytes
 .../state/0/3/2.delta  | Bin 0 -> 46 bytes
 .../state/0/4/.1.delta.crc

[spark] branch master updated: [SPARK-39564][SS] Expose the information of catalog table to the logical plan in streaming query

2022-06-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 c46b06c273b [SPARK-39564][SS] Expose the information of catalog table 
to the logical plan in streaming query
c46b06c273b is described below

commit c46b06c273b06df8fa06c8ca9138d81f520c8d7d
Author: Jungtaek Lim 
AuthorDate: Tue Jun 28 05:48:28 2022 +0900

[SPARK-39564][SS] Expose the information of catalog table to the logical 
plan in streaming query

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

This PR proposes to expose the information of catalog table (V1/V2) to the 
logical plan in streaming query, specifically, parsed plan and analyzed plan. 
(We may discard some information in optimized plan.)

The major change is to propagate the information of catalog table from the 
place we resolve the table to the place we execute the query. In MicroBatch 
execution, we have several transformations on the logical plan which replace 
the node with another node, hence this PR touches multiple logical nodes which 
the code path passes through.

Specifically for DSv1 sink, we don't have a specific write logical node, 
hence it's not feasible to expose the information for the destination. This PR 
introduces `WriteToMicroBatchDataSourceV1` which is DSv1 version of 
`WriteToMicroBatchDataSource` as a logical node for DSv1 sink. Worth noting 
that `WriteToMicroBatchDataSourceV1` plays as a marker - we eliminate this node 
in streaming specific optimization phase.

### Why are the changes needed?

This PR give a better UX to end users who use table API for streaming 
query. Previously it's not easy or even not feasible to check which tables are 
being read and written from the streaming query. Most likely it requires end 
users to look into their code/query.

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

Yes, in parsed/analyzed plan, we now expose the table information into the 
read/write logical node. Specifically for DSv1, we introduce a marker write 
node to expose the information for destination without majorly changing 
existing logic.

> DSv1 read and write

>> Before the patch

https://user-images.githubusercontent.com/1317309/175210731-dcc4cc4d-a70b-467d-b577-79c20600db32.png";>

>> After the patch

https://user-images.githubusercontent.com/1317309/175262327-960a23f5-6b24-4b89-bdaf-766a5c31aaf1.png";>

> DSv2 read and write

>> Before the patch

https://user-images.githubusercontent.com/1317309/175210780-4a99c670-8a42-4511-959c-cafe0c24bc00.png";>

>> After the patch

https://user-images.githubusercontent.com/1317309/175261938-fd7804a6-b98e-4202-ae23-622b512c66fa.png";>

Worth noting that the screenshot is taken with the config 
"spark.sql.ui.explainMode=extended". By default, we only show physical plan as 
formatted one, which hides the improvement being done here. Still, end users 
can run `query.explain(extended=true)` to print out plan"s" which contains 
parsed/analyzed plans.

### How was this patch tested?

New test cases. Also manually tested via running following query and 
checked the UI page:

> DSv1 read and write

```
/*
./bin/spark-shell --conf "spark.sql.ui.explainMode=extended"
*/

spark.sql("drop table if exists stream_source")

spark.sql("drop table if exists stream_target")

spark.sql("create table stream_source (col1 string, col2 int) using 
parquet")

spark.sql("create table stream_target (col1 string, col2 int) using 
parquet")

val checkpointDir = java.nio.file.Files.createTempDirectory("checkpoint-")

val q = 
spark.readStream.table("stream_source").writeStream.format("parquet").option("checkpointLocation",
 checkpointDir.toString).toTable("stream_target")

Thread.sleep(1)

spark.sql("insert into stream_source values ('a', 1)")
spark.sql("insert into stream_source values ('a', 2)")
spark.sql("insert into stream_source values ('a', 3)")

q.processAllAvailable()

spark.sql("insert into stream_source values ('b', 1)")
spark.sql("insert into stream_source values ('b', 2)")
spark.sql("insert into stream_source values ('b', 3)")

q.processAllAvailable()

spark.sql("insert into stream_source values ('c', 1)")
spark.sql("insert into stream_source values ('c', 2)&q

[spark] branch master updated (87b0a41cfb4 -> 12b7e61e16c)

2022-06-08 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 87b0a41cfb4 [SPARK-39411][BUILD] Fix release script to address type 
hint in pyspark/version.py
 add 12b7e61e16c [SPARK-39404][SS] Minor fix for querying `_metadata` in 
streaming

No new revisions were added by this update.

Summary of changes:
 .../org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala | 3 +--
 1 file changed, 1 insertion(+), 2 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-38181][SS][DOCS] Update comments in KafkaDataConsumer.scala

2022-06-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 0965fa427a7 [SPARK-38181][SS][DOCS] Update comments in 
KafkaDataConsumer.scala
0965fa427a7 is described below

commit 0965fa427a70d2855945e2008ccdb86a4989d763
Author: azheng 
AuthorDate: Tue Jun 7 21:27:07 2022 +0900

[SPARK-38181][SS][DOCS] Update comments in KafkaDataConsumer.scala

### What changes were proposed in this pull request?
Fixed some minor format issue in the code comments and rephrase some of 
them to make it more clear

### Why are the changes needed?
Minor format correction and better readability

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

### How was this patch tested?
Not needed, no real code changes

Closes #35484 from ArvinZheng/SPARK-38181.

Authored-by: azheng 
Signed-off-by: Jungtaek Lim 
---
 .../sql/kafka010/consumer/KafkaDataConsumer.scala  | 25 +++---
 1 file changed, 12 insertions(+), 13 deletions(-)

diff --git 
a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/consumer/KafkaDataConsumer.scala
 
b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/consumer/KafkaDataConsumer.scala
index 37fe38ea94e..d88e9821489 100644
--- 
a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/consumer/KafkaDataConsumer.scala
+++ 
b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/consumer/KafkaDataConsumer.scala
@@ -267,20 +267,18 @@ private[kafka010] class KafkaDataConsumer(
* within [offset, untilOffset).
*
* This method also will try its best to detect data loss. If 
`failOnDataLoss` is `true`, it will
-   * throw an exception when we detect an unavailable offset. If 
`failOnDataLoss` is `false`, this
-   * method will try to fetch next available record within [offset, 
untilOffset).
-   *
-   * When this method tries to skip offsets due to either invisible messages 
or data loss and
-   * reaches `untilOffset`, it will return `null`.
+   * throw an exception when it detects an unavailable offset. If 
`failOnDataLoss` is `false`, this
+   * method will try to fetch next available record within [offset, 
untilOffset). When this method
+   * reaches `untilOffset` and still can't find an available record, it will 
return `null`.
*
* @param offset the offset to fetch.
* @param untilOffsetthe max offset to fetch. Exclusive.
* @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
* @param failOnDataLoss When `failOnDataLoss` is `true`, this method will 
either return record at
-   *   offset if available, or throw exception.when 
`failOnDataLoss` is `false`,
-   *   this method will either return record at offset if 
available, or return
-   *   the next earliest available record less than 
untilOffset, or null. It
-   *   will not throw any exception.
+   *   offset if available, or throw an exception. When 
`failOnDataLoss` is
+   *   `false`, this method will return record at offset 
if available, or return
+   *   the record at the next earliest available offset 
that is less than
+   *   untilOffset, otherwise null.
*/
   def get(
   offset: Long,
@@ -298,9 +296,10 @@ private[kafka010] class KafkaDataConsumer(
   s"requested $offset")
 
 // The following loop is basically for `failOnDataLoss = false`. When 
`failOnDataLoss` is
-// `false`, first, we will try to fetch the record at `offset`. If no such 
record exists, then
-// we will move to the next available offset within `[offset, 
untilOffset)` and retry.
-// If `failOnDataLoss` is `true`, the loop body will be executed only once.
+// `false`, we will try to fetch the record at `offset`, if the record 
does not exist, we will
+// try to fetch next available record within [offset, untilOffset).
+// If `failOnDataLoss` is `true`, the loop body will be executed only 
once, either return the
+// record at `offset` or throw an exception when the record does not exist.
 var toFetchOffset = offset
 var fetchedRecord: FetchedRecord = null
 // We want to break out of the while loop on a successful fetch to avoid 
using "return"
@@ -452,7 +451,7 @@ private[kafka010] class KafkaDataConsumer(
   /**
* Get the fetched record for the given offset if available.
*
-   * If the record is invisible (either a  transaction message, or an aborted 
message when the
+   * If the record is invisible (either a transaction message, or an aborted 
message when the
* consumer's `isolation.level` is `read_committed`), it will 

[spark] branch master updated: [SPARK-39264][SS] Fix type check and conversion to longOffset for awaitOffset fix

2022-05-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 ea215279b0a [SPARK-39264][SS] Fix type check and conversion to 
longOffset for awaitOffset fix
ea215279b0a is described below

commit ea215279b0a4785d48723f5f24c96b8d7d9aa355
Author: Anish Shrigondekar 
AuthorDate: Tue May 24 13:28:04 2022 +0900

[SPARK-39264][SS] Fix type check and conversion to longOffset for 
awaitOffset fix

### What changes were proposed in this pull request?
Fix type check and conversion to longOffset for awaitOffset fix. Based on 
discussion with comments from alex-balikov

### Why are the changes needed?
To ensure type safety while doing comparisons and avoid type mismatch 
related bugs/issues.

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

### How was this patch tested?
Change only for making type checks explicit. Ran existing tests and 
verified that they pass. Also verified that the pattern matching change works 
by running the test for 100 iterations few times.
```
[info] RateStreamProviderSuite:
15:37:58.700 WARN org.apache.hadoop.util.NativeCodeLoader: Unable to load 
native-hadoop library for your platform... using builtin-java classes where 
applicable
[info] - RateStreamProvider in registry (438 milliseconds)
[info] - compatible with old path in registry (1 millisecond)
15:38:00.958 WARN 
org.apache.spark.sql.execution.streaming.ResolveWriteToStream: 
spark.sql.adaptive.enabled is not supported in streaming DataFrames/Datasets 
and will be disabled.
[info] - microbatch - basic (2 seconds, 346 milliseconds)
15:38:02.074 WARN 
org.apache.spark.sql.execution.streaming.ResolveWriteToStream: 
spark.sql.adaptive.enabled is not supported in streaming DataFrames/Datasets 
and will be disabled.
15:38:04.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] - microbatch - restart (4 seconds, 294 milliseconds)
15:38:06.450 WARN 
org.apache.spark.sql.execution.streaming.ResolveWriteToStream: 
spark.sql.adaptive.enabled is not supported in streaming DataFrames/Datasets 
and will be disabled.
[info] - microbatch - uniform distribution of event timestamps (547 
milliseconds)
[info] - microbatch - infer offsets (90 milliseconds)
[info] - microbatch - predetermined batch size (74 milliseconds)
[info] - microbatch - data read (73 milliseconds)
[info] - valueAtSecond (0 milliseconds)
15:38:07.243 WARN 
org.apache.spark.sql.execution.streaming.ResolveWriteToStream: 
spark.sql.adaptive.enabled is not supported in streaming DataFrames/Datasets 
and will be disabled.
[info] - rampUpTime (1 second, 633 milliseconds)
15:38:08.806 WARN 
org.apache.spark.sql.execution.streaming.ResolveWriteToStream: 
spark.sql.adaptive.enabled is not supported in streaming DataFrames/Datasets 
and will be disabled.
[info] - numPartitions (924 milliseconds)
15:38:09.702 WARN 
org.apache.spark.sql.execution.streaming.ResolveWriteToStream: 
spark.sql.adaptive.enabled is not supported in streaming DataFrames/Datasets 
and will be disabled.
[info] - overflow (241 milliseconds)
[info] - illegal option values (3 milliseconds)
[info] - user-specified schema given (8 milliseconds)
[info] - continuous data (1 second, 12 milliseconds)
15:38:11.035 WARN 
org.apache.spark.sql.execution.streaming.sources.RateStreamProviderSuite:

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

Closes #36642 from anishshri-db/task/SPARK-39264.

Authored-by: Anish Shrigondekar 
Signed-off-by: Jungtaek Lim 
---
 .../apache/spark/sql/execution/streaming/StreamExecution.scala | 10 --
 1 file changed, 4 insertions(+), 6 deletions(-)

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 ae99743baf3..88896c55455 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
@@ -449,12 +449,10 @@ abstract class StreamExecution(
 // after the stream has moved past the expected newOffset

[spark] branch master updated: [SPARK-39242][SS] Fix awaitOffset to wait for committedOffset to reach at-least expected offset for longOffset and fix RateStreamProvider test

2022-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 230c2ddd170 [SPARK-39242][SS] Fix awaitOffset to wait for 
committedOffset to reach at-least expected offset for longOffset and fix 
RateStreamProvider test
230c2ddd170 is described below

commit 230c2ddd1706aae99d166ec6df2b7c3269699630
Author: Anish Shrigondekar 
AuthorDate: Sun May 22 11:15:24 2022 +0900

[SPARK-39242][SS] Fix awaitOffset to wait for committedOffset to reach 
at-least expected offset for longOffset and fix RateStreamProvider test

### What changes were proposed in this pull request?
Fix awaitOffset to wait for committedOffset to reach atleast expected 
offset instead of exact value for long offsets. Also fixed the 
RateStreamProvider test to use only row values for requested range. Basically, 
for numeric increasing offsets, we could have called awaitOffset after the 
stream has moved past the expected newOffset or if committedOffsets changed 
after notify. In this case, its safe to exit, since at-least the given Offset 
has been reached and the equality condition migh [...]

### Why are the changes needed?
Fixing bug with awaitOffset logic and RateStreamProvider test

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

### How was this patch tested?
With the current code, we were seeing race conditions where the test would 
hang and get terminated after a 60 min timeout. With the change, ran the 
specific test 100 times multiple times and verified that we don't see the test 
failure any more. Was also able to simulate the failure by introducing 
arbitrary sleep in the code paths and ensured that the test passes with the 
above fix. Also added small unit test for longOffset conversion validation.

```
[info] RateStreamProviderSuite:
[info] Passed: Total 0, Failed 0, Errors 0, Passed 0
[info] No tests to run for avro / Test / testOnly
11:08:40.357 WARN org.apache.hadoop.util.NativeCodeLoader: Unable to load 
native-hadoop library for your platform... using builtin-java classes where 
applicable
[info] Passed: Total 0, Failed 0, Errors 0, Passed 0
[info] No tests to run for hive / Test / testOnly
[info] - RateStreamProvider in registry (531 milliseconds)
[info] - compatible with old path in registry (3 milliseconds)
[warn] multiple main classes detected: run 'show discoveredMainClasses' to 
see the list
11:08:43.608 WARN 
org.apache.spark.sql.execution.streaming.ResolveWriteToStream: 
spark.sql.adaptive.enabled is not supported in streaming DataFrames/Datasets 
and will be disabled.
[info] Passed: Total 0, Failed 0, Errors 0, Passed 0
[info] No tests to run for mllib / Test / testOnly
[info] Passed: Total 0, Failed 0, Errors 0, Passed 0
[info] No tests to run for sql-kafka-0-10 / Test / testOnly
[info] - microbatch - basic (3 seconds, 966 milliseconds)
11:08:45.807 WARN 
org.apache.spark.sql.execution.streaming.ResolveWriteToStream: 
spark.sql.adaptive.enabled is not supported in streaming DataFrames/Datasets 
and will be disabled.
[info] Passed: Total 0, Failed 0, Errors 0, Passed 0
[info] No tests to run for repl / Test / testOnly
11:08:48.493 WARN 
org.apache.spark.sql.execution.streaming.ResolveWriteToStream: 
spark.sql.adaptive.enabled is not supported in streaming DataFrames/Datasets 
and will be disabled.
[warn] multiple main classes detected: run 'show discoveredMainClasses' to 
see the list
[info] Passed: Total 0, Failed 0, Errors 0, Passed 0
[info] No tests to run for examples / Test / testOnly
[info] - microbatch - restart (4 seconds, 365 milliseconds)
11:08:50.278 WARN 
org.apache.spark.sql.execution.streaming.ResolveWriteToStream: 
spark.sql.adaptive.enabled is not supported in streaming DataFrames/Datasets 
and will be disabled.
[info] Passed: Total 0, Failed 0, Errors 0, Passed 0
[info] No tests to run for assembly / Test / testOnly
[info] - microbatch - uniform distribution of event timestamps (696 
milliseconds)
[info] - microbatch - infer offsets (98 milliseconds)
[info] - microbatch - predetermined batch size (86 milliseconds)
[info] - microbatch - data read (85 milliseconds)
[info] - valueAtSecond (0 milliseconds)
[info] - overflow (265 milliseconds)
[info] - illegal option values (4 milliseconds)
[info] - user-specified schema given (9 milliseconds)
[info] - continuous data (1 second, 6 milliseconds)
11:08:55.295 WARN 
org.apache.spark.sql.execution.streaming.sources.RateStreamProviderSuite:

= POSSIBLE THREAD LEAK IN SUITE 
o.a.s.sql.execution.streaming.sources.RateStreamProviderSuite, threads: 
rpc-boss-3-1 (daemon=true), shuffle-boss-6-1 (daemon=true), 
state-store-main

[spark] branch branch-3.1 updated: [SPARK-39219][DOC] Promote Structured Streaming over DStream

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

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


The following commit(s) were added to refs/heads/branch-3.1 by this push:
 new 3362ea0bb87 [SPARK-39219][DOC] Promote Structured Streaming over 
DStream
3362ea0bb87 is described below

commit 3362ea0bb87ff73b12905fc78b8ff412192e6024
Author: Jungtaek Lim 
AuthorDate: Thu May 19 11:50:05 2022 +0900

[SPARK-39219][DOC] Promote Structured Streaming over DStream

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

This PR proposes to add NOTE section for DStream guide doc to promote 
Structured Streaming.

Screenshot:

https://user-images.githubusercontent.com/1317309/168977732-4c32db9a-0fb1-4a82-a542-bf385e5f3683.png";>

### Why are the changes needed?

We see efforts of community are more focused on Structured Streaming (based 
on Spark SQL) than Spark Streaming (DStream). We would like to encourage end 
users to use Structured Streaming than Spark Streaming whenever possible for 
their workloads.

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

Yes, doc change.

### How was this patch tested?

N/A

Closes #36590 from HeartSaVioR/SPARK-39219.

Authored-by: Jungtaek Lim 
Signed-off-by: Jungtaek Lim 
(cherry picked from commit 7d153392db2f61104da0af1cb175f4ee7c7fbc38)
Signed-off-by: Jungtaek Lim 
---
 docs/streaming-programming-guide.md | 8 
 1 file changed, 8 insertions(+)

diff --git a/docs/streaming-programming-guide.md 
b/docs/streaming-programming-guide.md
index b13bf8dd2e7..4a104238a6d 100644
--- a/docs/streaming-programming-guide.md
+++ b/docs/streaming-programming-guide.md
@@ -23,6 +23,14 @@ license: |
 * This will become a table of contents (this text will be scraped).
 {:toc}
 
+# Note
+
+Spark Streaming is the previous generation of Spark’s streaming engine. There 
are no longer
+updates to Spark Streaming and it’s a legacy project. There is a newer and 
easier to use
+streaming engine in Spark called Structured Streaming. You should use Spark 
Structured Streaming
+for your streaming applications and pipelines. See
+[Structured Streaming Programming 
Guide](structured-streaming-programming-guide.html).
+
 # Overview
 Spark Streaming is an extension of the core Spark API that enables scalable, 
high-throughput,
 fault-tolerant stream processing of live data streams. Data can be ingested 
from many sources


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



[spark] branch branch-3.2 updated: [SPARK-39219][DOC] Promote Structured Streaming over DStream

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

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


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new 6587d294276 [SPARK-39219][DOC] Promote Structured Streaming over 
DStream
6587d294276 is described below

commit 6587d294276469cc398933ffd3bc4f93d4434a39
Author: Jungtaek Lim 
AuthorDate: Thu May 19 11:50:05 2022 +0900

[SPARK-39219][DOC] Promote Structured Streaming over DStream

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

This PR proposes to add NOTE section for DStream guide doc to promote 
Structured Streaming.

Screenshot:

https://user-images.githubusercontent.com/1317309/168977732-4c32db9a-0fb1-4a82-a542-bf385e5f3683.png";>

### Why are the changes needed?

We see efforts of community are more focused on Structured Streaming (based 
on Spark SQL) than Spark Streaming (DStream). We would like to encourage end 
users to use Structured Streaming than Spark Streaming whenever possible for 
their workloads.

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

Yes, doc change.

### How was this patch tested?

N/A

Closes #36590 from HeartSaVioR/SPARK-39219.

Authored-by: Jungtaek Lim 
Signed-off-by: Jungtaek Lim 
(cherry picked from commit 7d153392db2f61104da0af1cb175f4ee7c7fbc38)
Signed-off-by: Jungtaek Lim 
---
 docs/streaming-programming-guide.md | 8 
 1 file changed, 8 insertions(+)

diff --git a/docs/streaming-programming-guide.md 
b/docs/streaming-programming-guide.md
index b13bf8dd2e7..4a104238a6d 100644
--- a/docs/streaming-programming-guide.md
+++ b/docs/streaming-programming-guide.md
@@ -23,6 +23,14 @@ license: |
 * This will become a table of contents (this text will be scraped).
 {:toc}
 
+# Note
+
+Spark Streaming is the previous generation of Spark’s streaming engine. There 
are no longer
+updates to Spark Streaming and it’s a legacy project. There is a newer and 
easier to use
+streaming engine in Spark called Structured Streaming. You should use Spark 
Structured Streaming
+for your streaming applications and pipelines. See
+[Structured Streaming Programming 
Guide](structured-streaming-programming-guide.html).
+
 # Overview
 Spark Streaming is an extension of the core Spark API that enables scalable, 
high-throughput,
 fault-tolerant stream processing of live data streams. Data can be ingested 
from many sources


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



[spark] branch branch-3.3 updated: [SPARK-39219][DOC] Promote Structured Streaming over DStream

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

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


The following commit(s) were added to refs/heads/branch-3.3 by this push:
 new 69b7e1a1159 [SPARK-39219][DOC] Promote Structured Streaming over 
DStream
69b7e1a1159 is described below

commit 69b7e1a11597446ea4607cdb578d5404750b0cea
Author: Jungtaek Lim 
AuthorDate: Thu May 19 11:50:05 2022 +0900

[SPARK-39219][DOC] Promote Structured Streaming over DStream

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

This PR proposes to add NOTE section for DStream guide doc to promote 
Structured Streaming.

Screenshot:

https://user-images.githubusercontent.com/1317309/168977732-4c32db9a-0fb1-4a82-a542-bf385e5f3683.png";>

### Why are the changes needed?

We see efforts of community are more focused on Structured Streaming (based 
on Spark SQL) than Spark Streaming (DStream). We would like to encourage end 
users to use Structured Streaming than Spark Streaming whenever possible for 
their workloads.

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

Yes, doc change.

### How was this patch tested?

N/A

Closes #36590 from HeartSaVioR/SPARK-39219.

Authored-by: Jungtaek Lim 
Signed-off-by: Jungtaek Lim 
(cherry picked from commit 7d153392db2f61104da0af1cb175f4ee7c7fbc38)
Signed-off-by: Jungtaek Lim 
---
 docs/streaming-programming-guide.md | 8 
 1 file changed, 8 insertions(+)

diff --git a/docs/streaming-programming-guide.md 
b/docs/streaming-programming-guide.md
index b13bf8dd2e7..4a104238a6d 100644
--- a/docs/streaming-programming-guide.md
+++ b/docs/streaming-programming-guide.md
@@ -23,6 +23,14 @@ license: |
 * This will become a table of contents (this text will be scraped).
 {:toc}
 
+# Note
+
+Spark Streaming is the previous generation of Spark’s streaming engine. There 
are no longer
+updates to Spark Streaming and it’s a legacy project. There is a newer and 
easier to use
+streaming engine in Spark called Structured Streaming. You should use Spark 
Structured Streaming
+for your streaming applications and pipelines. See
+[Structured Streaming Programming 
Guide](structured-streaming-programming-guide.html).
+
 # Overview
 Spark Streaming is an extension of the core Spark API that enables scalable, 
high-throughput,
 fault-tolerant stream processing of live data streams. Data can be ingested 
from many sources


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



[spark] branch master updated: [SPARK-39219][DOC] Promote Structured Streaming over DStream

2022-05-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 7d153392db2 [SPARK-39219][DOC] Promote Structured Streaming over 
DStream
7d153392db2 is described below

commit 7d153392db2f61104da0af1cb175f4ee7c7fbc38
Author: Jungtaek Lim 
AuthorDate: Thu May 19 11:50:05 2022 +0900

[SPARK-39219][DOC] Promote Structured Streaming over DStream

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

This PR proposes to add NOTE section for DStream guide doc to promote 
Structured Streaming.

Screenshot:

https://user-images.githubusercontent.com/1317309/168977732-4c32db9a-0fb1-4a82-a542-bf385e5f3683.png";>

### Why are the changes needed?

We see efforts of community are more focused on Structured Streaming (based 
on Spark SQL) than Spark Streaming (DStream). We would like to encourage end 
users to use Structured Streaming than Spark Streaming whenever possible for 
their workloads.

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

Yes, doc change.

### How was this patch tested?

N/A

Closes #36590 from HeartSaVioR/SPARK-39219.

Authored-by: Jungtaek Lim 
Signed-off-by: Jungtaek Lim 
---
 docs/streaming-programming-guide.md | 8 
 1 file changed, 8 insertions(+)

diff --git a/docs/streaming-programming-guide.md 
b/docs/streaming-programming-guide.md
index b13bf8dd2e7..4a104238a6d 100644
--- a/docs/streaming-programming-guide.md
+++ b/docs/streaming-programming-guide.md
@@ -23,6 +23,14 @@ license: |
 * This will become a table of contents (this text will be scraped).
 {:toc}
 
+# Note
+
+Spark Streaming is the previous generation of Spark’s streaming engine. There 
are no longer
+updates to Spark Streaming and it’s a legacy project. There is a newer and 
easier to use
+streaming engine in Spark called Structured Streaming. You should use Spark 
Structured Streaming
+for your streaming applications and pipelines. See
+[Structured Streaming Programming 
Guide](structured-streaming-programming-guide.html).
+
 # Overview
 Spark Streaming is an extension of the core Spark API that enables scalable, 
high-throughput,
 fault-tolerant stream processing of live data streams. Data can be ingested 
from many sources


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



[spark] branch branch-3.2 updated: [SPARK-38931][SS] Create root dfs directory for RocksDBFileManager with unknown number of keys on 1st checkpoint

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

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


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new cc097a4d990 [SPARK-38931][SS] Create root dfs directory for 
RocksDBFileManager with unknown number of keys on 1st checkpoint
cc097a4d990 is described below

commit cc097a4d990e1f9c6ef7bf515ae966eaf35fc44c
Author: Yun Tang 
AuthorDate: Tue Apr 19 20:31:04 2022 +0900

[SPARK-38931][SS] Create root dfs directory for RocksDBFileManager with 
unknown number of keys on 1st checkpoint

### What changes were proposed in this pull request?
Create root dfs directory for RocksDBFileManager with unknown number of 
keys on 1st checkpoint.

### Why are the changes needed?
If this fix is not introduced, we might meet exception below:
~~~java
File 
/private/var/folders/rk/wyr101_562ngn8lp7tbqt7_0gp/T/spark-ce4a0607-b1d8-43b8-becd-638c6b030019/state/1/1
 does not exist
java.io.FileNotFoundException: File 
/private/var/folders/rk/wyr101_562ngn8lp7tbqt7_0gp/T/spark-ce4a0607-b1d8-43b8-becd-638c6b030019/state/1/1
 does not exist
at 
org.apache.hadoop.fs.RawLocalFileSystem.deprecatedGetFileStatus(RawLocalFileSystem.java:779)
at 
org.apache.hadoop.fs.RawLocalFileSystem.getFileLinkStatusInternal(RawLocalFileSystem.java:1100)
at 
org.apache.hadoop.fs.RawLocalFileSystem.getFileStatus(RawLocalFileSystem.java:769)
at 
org.apache.hadoop.fs.DelegateToFileSystem.getFileStatus(DelegateToFileSystem.java:128)
at 
org.apache.hadoop.fs.DelegateToFileSystem.createInternal(DelegateToFileSystem.java:93)
at 
org.apache.hadoop.fs.ChecksumFs$ChecksumFSOutputSummer.(ChecksumFs.java:353)
at 
org.apache.hadoop.fs.ChecksumFs.createInternal(ChecksumFs.java:400)
at 
org.apache.hadoop.fs.AbstractFileSystem.create(AbstractFileSystem.java:626)
at org.apache.hadoop.fs.FileContext$3.next(FileContext.java:701)
at org.apache.hadoop.fs.FileContext$3.next(FileContext.java:697)
at 
org.apache.hadoop.fs.FSLinkResolver.resolve(FSLinkResolver.java:90)
at org.apache.hadoop.fs.FileContext.create(FileContext.java:703)
at 
org.apache.spark.sql.execution.streaming.FileContextBasedCheckpointFileManager.createTempFile(CheckpointFileManager.scala:327)
at 
org.apache.spark.sql.execution.streaming.CheckpointFileManager$RenameBasedFSDataOutputStream.(CheckpointFileManager.scala:140)
at 
org.apache.spark.sql.execution.streaming.CheckpointFileManager$RenameBasedFSDataOutputStream.(CheckpointFileManager.scala:143)
at 
org.apache.spark.sql.execution.streaming.FileContextBasedCheckpointFileManager.createAtomic(CheckpointFileManager.scala:333)
at 
org.apache.spark.sql.execution.streaming.state.RocksDBFileManager.zipToDfsFile(RocksDBFileManager.scala:438)
at 
org.apache.spark.sql.execution.streaming.state.RocksDBFileManager.saveCheckpointToDfs(RocksDBFileManager.scala:174)
at 
org.apache.spark.sql.execution.streaming.state.RocksDBSuite.saveCheckpointFiles(RocksDBSuite.scala:566)
at 
org.apache.spark.sql.execution.streaming.state.RocksDBSuite.$anonfun$new$35(RocksDBSuite.scala:179)

~~~

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

### How was this patch tested?
Tested via RocksDBSuite.

Closes #36242 from Myasuka/SPARK-38931.

Authored-by: Yun Tang 
Signed-off-by: Jungtaek Lim 
(cherry picked from commit abb1df9d190e35a17b693f2b013b092af4f2528a)
Signed-off-by: Jungtaek Lim 
---
 .../streaming/state/RocksDBFileManager.scala  |  4 +++-
 .../sql/execution/streaming/state/RocksDBSuite.scala  | 19 +++
 2 files changed, 22 insertions(+), 1 deletion(-)

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 23cdbd01bc1..367062b90bc 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
@@ -161,11 +161,13 @@ class RocksDBFileManager(
 metadata.writeToFile(metadataFile)
 logInfo(s"Written metadata for version $version:\n${metadata.prettyJson}")
 
-if (version <= 1 && numKeys == 0) {
+if (version <= 1 && numKeys <= 0) {
   // If we're writing the initial version and there's no data, we have to 
explicitly initialize
   // the root directory. Normally saveImmutableFilesToDfs will do this 
initialization, but
   // when there's no data that method

[spark] branch branch-3.3 updated: [SPARK-38931][SS] Create root dfs directory for RocksDBFileManager with unknown number of keys on 1st checkpoint

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

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


The following commit(s) were added to refs/heads/branch-3.3 by this push:
 new 8811e8caaa8 [SPARK-38931][SS] Create root dfs directory for 
RocksDBFileManager with unknown number of keys on 1st checkpoint
8811e8caaa8 is described below

commit 8811e8caaa8540d1fa05fb77152043addc607b82
Author: Yun Tang 
AuthorDate: Tue Apr 19 20:31:04 2022 +0900

[SPARK-38931][SS] Create root dfs directory for RocksDBFileManager with 
unknown number of keys on 1st checkpoint

### What changes were proposed in this pull request?
Create root dfs directory for RocksDBFileManager with unknown number of 
keys on 1st checkpoint.

### Why are the changes needed?
If this fix is not introduced, we might meet exception below:
~~~java
File 
/private/var/folders/rk/wyr101_562ngn8lp7tbqt7_0gp/T/spark-ce4a0607-b1d8-43b8-becd-638c6b030019/state/1/1
 does not exist
java.io.FileNotFoundException: File 
/private/var/folders/rk/wyr101_562ngn8lp7tbqt7_0gp/T/spark-ce4a0607-b1d8-43b8-becd-638c6b030019/state/1/1
 does not exist
at 
org.apache.hadoop.fs.RawLocalFileSystem.deprecatedGetFileStatus(RawLocalFileSystem.java:779)
at 
org.apache.hadoop.fs.RawLocalFileSystem.getFileLinkStatusInternal(RawLocalFileSystem.java:1100)
at 
org.apache.hadoop.fs.RawLocalFileSystem.getFileStatus(RawLocalFileSystem.java:769)
at 
org.apache.hadoop.fs.DelegateToFileSystem.getFileStatus(DelegateToFileSystem.java:128)
at 
org.apache.hadoop.fs.DelegateToFileSystem.createInternal(DelegateToFileSystem.java:93)
at 
org.apache.hadoop.fs.ChecksumFs$ChecksumFSOutputSummer.(ChecksumFs.java:353)
at 
org.apache.hadoop.fs.ChecksumFs.createInternal(ChecksumFs.java:400)
at 
org.apache.hadoop.fs.AbstractFileSystem.create(AbstractFileSystem.java:626)
at org.apache.hadoop.fs.FileContext$3.next(FileContext.java:701)
at org.apache.hadoop.fs.FileContext$3.next(FileContext.java:697)
at 
org.apache.hadoop.fs.FSLinkResolver.resolve(FSLinkResolver.java:90)
at org.apache.hadoop.fs.FileContext.create(FileContext.java:703)
at 
org.apache.spark.sql.execution.streaming.FileContextBasedCheckpointFileManager.createTempFile(CheckpointFileManager.scala:327)
at 
org.apache.spark.sql.execution.streaming.CheckpointFileManager$RenameBasedFSDataOutputStream.(CheckpointFileManager.scala:140)
at 
org.apache.spark.sql.execution.streaming.CheckpointFileManager$RenameBasedFSDataOutputStream.(CheckpointFileManager.scala:143)
at 
org.apache.spark.sql.execution.streaming.FileContextBasedCheckpointFileManager.createAtomic(CheckpointFileManager.scala:333)
at 
org.apache.spark.sql.execution.streaming.state.RocksDBFileManager.zipToDfsFile(RocksDBFileManager.scala:438)
at 
org.apache.spark.sql.execution.streaming.state.RocksDBFileManager.saveCheckpointToDfs(RocksDBFileManager.scala:174)
at 
org.apache.spark.sql.execution.streaming.state.RocksDBSuite.saveCheckpointFiles(RocksDBSuite.scala:566)
at 
org.apache.spark.sql.execution.streaming.state.RocksDBSuite.$anonfun$new$35(RocksDBSuite.scala:179)

~~~

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

### How was this patch tested?
Tested via RocksDBSuite.

Closes #36242 from Myasuka/SPARK-38931.

Authored-by: Yun Tang 
Signed-off-by: Jungtaek Lim 
(cherry picked from commit abb1df9d190e35a17b693f2b013b092af4f2528a)
Signed-off-by: Jungtaek Lim 
---
 .../streaming/state/RocksDBFileManager.scala  |  4 +++-
 .../sql/execution/streaming/state/RocksDBSuite.scala  | 19 +++
 2 files changed, 22 insertions(+), 1 deletion(-)

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 4f2ce9b1237..26084747c32 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
@@ -161,11 +161,13 @@ class RocksDBFileManager(
 metadata.writeToFile(metadataFile)
 logInfo(s"Written metadata for version $version:\n${metadata.prettyJson}")
 
-if (version <= 1 && numKeys == 0) {
+if (version <= 1 && numKeys <= 0) {
   // If we're writing the initial version and there's no data, we have to 
explicitly initialize
   // the root directory. Normally saveImmutableFilesToDfs will do this 
initialization, but
   // when there's no data that method

[spark] branch master updated (29eea67987d -> abb1df9d190)

2022-04-19 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 29eea67987d [SPARK-38720][SQL][TESTS] Test the error class: 
CANNOT_CHANGE_DECIMAL_PRECISION
 add abb1df9d190 [SPARK-38931][SS] Create root dfs directory for 
RocksDBFileManager with unknown number of keys on 1st checkpoint

No new revisions were added by this update.

Summary of changes:
 .../streaming/state/RocksDBFileManager.scala  |  4 +++-
 .../sql/execution/streaming/state/RocksDBSuite.scala  | 19 +++
 2 files changed, 22 insertions(+), 1 deletion(-)


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



[spark] branch branch-3.2 updated: [SPARK-38809][SS] Implement option to skip null values in symmetric hash implementation of stream-stream joins

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

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


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new 99e6adb5c90 [SPARK-38809][SS] Implement option to skip null values in 
symmetric hash implementation of stream-stream joins
99e6adb5c90 is described below

commit 99e6adb5c909a2859815c4821846f0a1c2a8b989
Author: Anish Shrigondekar 
AuthorDate: Fri Apr 8 12:19:50 2022 +0900

[SPARK-38809][SS] Implement option to skip null values in symmetric hash 
implementation of stream-stream joins

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

In the symmetric has join state manager, we can receive entries with null 
values for a key and that can cause the `removeByValue` and get iterators to 
fail and run into the NullPointerException. This is possible if the state 
recovered is written from an old spark version or its corrupted on disk or due 
to issues with the iterators. Since we don't have a utility to query this 
state, we would like to provide a conf option to skip nulls for the symmetric 
hash implementation in stream str [...]

### Why are the changes needed?

Without these changes, if we encounter null values for stream-stream joins, 
the executor task will repeatedly fail with NullPointerException and will 
terminate the stage and eventually the query as well. This change allows the 
user to set a config option to continue iterating by skipping null values for 
symmetric hash based implementation of stream-stream joins.

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

No

### How was this patch tested?

Added unit tests to test the new functionality by adding nulls in between 
and forcing the iteration/get calls with nulls in the mix and tested the 
behavior with the config disabled as well as enabled.
Sample output:
```
[info] SymmetricHashJoinStateManagerSuite:
15:07:50.627 WARN org.apache.hadoop.util.NativeCodeLoader: Unable to load 
native-hadoop library for your platform... using builtin-java classes where 
applicable
[info] - StreamingJoinStateManager V1 - all operations (588 milliseconds)
[info] - StreamingJoinStateManager V2 - all operations (251 milliseconds)
15:07:52.669 WARN 
org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManager: 
`keyWithIndexToValue` returns a null value for indices with range from 
startIndex=3 and endIndex=4.
15:07:52.671 WARN 
org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManager: 
`keyWithIndexToValue` returns a null value for indices with range from 
startIndex=3 and endIndex=3.
15:07:52.672 WARN 
org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManager: 
`keyWithIndexToValue` returns a null value for indices with range from 
startIndex=1 and endIndex=3.
15:07:52.672 WARN 
org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManager: 
`keyWithIndexToValue` returns a null value for indices with range from 
startIndex=1 and endIndex=1.
[info] - StreamingJoinStateManager V1 - all operations with nulls (252 
milliseconds)
15:07:52.896 WARN 
org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManager: 
`keyWithIndexToValue` returns a null value for indices with range from 
startIndex=3 and endIndex=4.
15:07:52.897 WARN 
org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManager: 
`keyWithIndexToValue` returns a null value for indices with range from 
startIndex=3 and endIndex=3.
15:07:52.898 WARN 
org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManager: 
`keyWithIndexToValue` returns a null value for indices with range from 
startIndex=1 and endIndex=3.
15:07:52.898 WARN 
org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManager: 
`keyWithIndexToValue` returns a null value for indices with range from 
startIndex=1 and endIndex=1.
[info] - StreamingJoinStateManager V2 - all operations with nulls (221 
milliseconds)
15:07:53.114 WARN 
org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManager: 
`keyWithIndexToValue` returns a null value for indices with range from 
startIndex=5 and endIndex=6.
15:07:53.116 WARN 
org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManager: 
`keyWithIndexToValue` returns a null value for indices with range from 
startIndex=3 and endIndex=6.
15:07:53.331 WARN 
org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManager: 
`keyWithIndexToValue` returns a null value for indices with range from 
startIndex=5 and endIndex=6.
15:07:53.331 WARN 
org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManager: 
`keyWithIndexToValue` returns a null value for indices with range from 
startIndex=3 and endIndex=3.

[spark] branch branch-3.3 updated: [SPARK-38809][SS] Implement option to skip null values in symmetric hash implementation of stream-stream joins

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

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


The following commit(s) were added to refs/heads/branch-3.3 by this push:
 new 3e5407dbbfb [SPARK-38809][SS] Implement option to skip null values in 
symmetric hash implementation of stream-stream joins
3e5407dbbfb is described below

commit 3e5407dbbfb8ea955e9c44df1893ad24a9449a28
Author: Anish Shrigondekar 
AuthorDate: Fri Apr 8 12:19:50 2022 +0900

[SPARK-38809][SS] Implement option to skip null values in symmetric hash 
implementation of stream-stream joins

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

In the symmetric has join state manager, we can receive entries with null 
values for a key and that can cause the `removeByValue` and get iterators to 
fail and run into the NullPointerException. This is possible if the state 
recovered is written from an old spark version or its corrupted on disk or due 
to issues with the iterators. Since we don't have a utility to query this 
state, we would like to provide a conf option to skip nulls for the symmetric 
hash implementation in stream str [...]

### Why are the changes needed?

Without these changes, if we encounter null values for stream-stream joins, 
the executor task will repeatedly fail with NullPointerException and will 
terminate the stage and eventually the query as well. This change allows the 
user to set a config option to continue iterating by skipping null values for 
symmetric hash based implementation of stream-stream joins.

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

No

### How was this patch tested?

Added unit tests to test the new functionality by adding nulls in between 
and forcing the iteration/get calls with nulls in the mix and tested the 
behavior with the config disabled as well as enabled.
Sample output:
```
[info] SymmetricHashJoinStateManagerSuite:
15:07:50.627 WARN org.apache.hadoop.util.NativeCodeLoader: Unable to load 
native-hadoop library for your platform... using builtin-java classes where 
applicable
[info] - StreamingJoinStateManager V1 - all operations (588 milliseconds)
[info] - StreamingJoinStateManager V2 - all operations (251 milliseconds)
15:07:52.669 WARN 
org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManager: 
`keyWithIndexToValue` returns a null value for indices with range from 
startIndex=3 and endIndex=4.
15:07:52.671 WARN 
org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManager: 
`keyWithIndexToValue` returns a null value for indices with range from 
startIndex=3 and endIndex=3.
15:07:52.672 WARN 
org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManager: 
`keyWithIndexToValue` returns a null value for indices with range from 
startIndex=1 and endIndex=3.
15:07:52.672 WARN 
org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManager: 
`keyWithIndexToValue` returns a null value for indices with range from 
startIndex=1 and endIndex=1.
[info] - StreamingJoinStateManager V1 - all operations with nulls (252 
milliseconds)
15:07:52.896 WARN 
org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManager: 
`keyWithIndexToValue` returns a null value for indices with range from 
startIndex=3 and endIndex=4.
15:07:52.897 WARN 
org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManager: 
`keyWithIndexToValue` returns a null value for indices with range from 
startIndex=3 and endIndex=3.
15:07:52.898 WARN 
org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManager: 
`keyWithIndexToValue` returns a null value for indices with range from 
startIndex=1 and endIndex=3.
15:07:52.898 WARN 
org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManager: 
`keyWithIndexToValue` returns a null value for indices with range from 
startIndex=1 and endIndex=1.
[info] - StreamingJoinStateManager V2 - all operations with nulls (221 
milliseconds)
15:07:53.114 WARN 
org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManager: 
`keyWithIndexToValue` returns a null value for indices with range from 
startIndex=5 and endIndex=6.
15:07:53.116 WARN 
org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManager: 
`keyWithIndexToValue` returns a null value for indices with range from 
startIndex=3 and endIndex=6.
15:07:53.331 WARN 
org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManager: 
`keyWithIndexToValue` returns a null value for indices with range from 
startIndex=5 and endIndex=6.
15:07:53.331 WARN 
org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManager: 
`keyWithIndexToValue` returns a null value for indices with range from 
startIndex=3 and endIndex=3.

[spark] branch master updated: [SPARK-38809][SS] Implement option to skip null values in symmetric hash implementation of stream-stream joins

2022-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 61c489ea7ef [SPARK-38809][SS] Implement option to skip null values in 
symmetric hash implementation of stream-stream joins
61c489ea7ef is described below

commit 61c489ea7ef51d7d0217f770ec358ed7a7b76b42
Author: Anish Shrigondekar 
AuthorDate: Fri Apr 8 12:19:50 2022 +0900

[SPARK-38809][SS] Implement option to skip null values in symmetric hash 
implementation of stream-stream joins

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

In the symmetric has join state manager, we can receive entries with null 
values for a key and that can cause the `removeByValue` and get iterators to 
fail and run into the NullPointerException. This is possible if the state 
recovered is written from an old spark version or its corrupted on disk or due 
to issues with the iterators. Since we don't have a utility to query this 
state, we would like to provide a conf option to skip nulls for the symmetric 
hash implementation in stream str [...]

### Why are the changes needed?

Without these changes, if we encounter null values for stream-stream joins, 
the executor task will repeatedly fail with NullPointerException and will 
terminate the stage and eventually the query as well. This change allows the 
user to set a config option to continue iterating by skipping null values for 
symmetric hash based implementation of stream-stream joins.

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

No

### How was this patch tested?

Added unit tests to test the new functionality by adding nulls in between 
and forcing the iteration/get calls with nulls in the mix and tested the 
behavior with the config disabled as well as enabled.
Sample output:
```
[info] SymmetricHashJoinStateManagerSuite:
15:07:50.627 WARN org.apache.hadoop.util.NativeCodeLoader: Unable to load 
native-hadoop library for your platform... using builtin-java classes where 
applicable
[info] - StreamingJoinStateManager V1 - all operations (588 milliseconds)
[info] - StreamingJoinStateManager V2 - all operations (251 milliseconds)
15:07:52.669 WARN 
org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManager: 
`keyWithIndexToValue` returns a null value for indices with range from 
startIndex=3 and endIndex=4.
15:07:52.671 WARN 
org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManager: 
`keyWithIndexToValue` returns a null value for indices with range from 
startIndex=3 and endIndex=3.
15:07:52.672 WARN 
org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManager: 
`keyWithIndexToValue` returns a null value for indices with range from 
startIndex=1 and endIndex=3.
15:07:52.672 WARN 
org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManager: 
`keyWithIndexToValue` returns a null value for indices with range from 
startIndex=1 and endIndex=1.
[info] - StreamingJoinStateManager V1 - all operations with nulls (252 
milliseconds)
15:07:52.896 WARN 
org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManager: 
`keyWithIndexToValue` returns a null value for indices with range from 
startIndex=3 and endIndex=4.
15:07:52.897 WARN 
org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManager: 
`keyWithIndexToValue` returns a null value for indices with range from 
startIndex=3 and endIndex=3.
15:07:52.898 WARN 
org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManager: 
`keyWithIndexToValue` returns a null value for indices with range from 
startIndex=1 and endIndex=3.
15:07:52.898 WARN 
org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManager: 
`keyWithIndexToValue` returns a null value for indices with range from 
startIndex=1 and endIndex=1.
[info] - StreamingJoinStateManager V2 - all operations with nulls (221 
milliseconds)
15:07:53.114 WARN 
org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManager: 
`keyWithIndexToValue` returns a null value for indices with range from 
startIndex=5 and endIndex=6.
15:07:53.116 WARN 
org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManager: 
`keyWithIndexToValue` returns a null value for indices with range from 
startIndex=3 and endIndex=6.
15:07:53.331 WARN 
org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManager: 
`keyWithIndexToValue` returns a null value for indices with range from 
startIndex=5 and endIndex=6.
15:07:53.331 WARN 
org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManager: 
`keyWithIndexToValue` returns a null value for indices with range from 
startIndex=3 and endIndex=3.

[spark] branch branch-3.2 updated: [SPARK-38787][SS] Replace found value with non-null element in the remaining list for key and remove remaining null elements from values in keyWithIndexToValue store

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

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


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new 6f6eb3f628d [SPARK-38787][SS] Replace found value with non-null 
element in the remaining list for key and remove remaining null elements from 
values in keyWithIndexToValue store for stream-stream joins
6f6eb3f628d is described below

commit 6f6eb3f628dd3c4adaec2ef5cb0a6b052664b220
Author: Anish Shrigondekar 
AuthorDate: Thu Apr 7 05:51:57 2022 +0900

[SPARK-38787][SS] Replace found value with non-null element in the 
remaining list for key and remove remaining null elements from values in 
keyWithIndexToValue store for stream-stream joins

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

In stream-stream joins, for removing old state (watermark by value), we 
call the `removeByValue` function with a removal condition. Within the iterator 
returned, if we find null at the end for matched value at non-last index, we 
are currently not removing and swapping the matched value. With this change, we 
will find the first non-null value from end and swap current index with that 
value and remove all elements from index + 1 to the end and then drop the last 
element as before.

### Why are the changes needed?

This change fixes a bug where we were not replacing found/matching values 
for `removeByValue` when encountering nulls in the symmetric hash join code.

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

No

### How was this patch tested?

Added a unit test for this change with nulls added. Here is a sample output:
```
Executing tests from 
//sql/core:org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManagerSuite-hive-2.3__hadoop-3.2

-
2022-04-01 21:11:59,641 INFO  CodeGenerator - Code generated in 225.884757 
ms
2022-04-01 21:11:59,662 INFO  CodeGenerator - Code generated in 10.870786 ms
Run starting. Expected test count is: 4
…
= TEST OUTPUT FOR 
o.a.s.sql.execution.streaming.state.SymmetricHashJoinStateManagerSuite: 
'StreamingJoinStateManager V2 - all operations with nulls' =

2022-04-01 21:12:03,487 INFO  StateStore - State Store maintenance task 
started
2022-04-01 21:12:03,508 INFO  CheckpointFileManager - Writing atomically to 
/tmp/spark-d94b9f11-e04c-4871-aeea-1d0b5c62e292/0/0/left-keyToNumValues/_metadata/schema
 using temp file 
/tmp/spark-d94b9f11-e04c-4871-aeea-1d0b5c62e292/0/0/left-keyToNumValues/_metadata/.schema.9bcc39c9-721e-4ee0-b369-fb4f516c4fd6.tmp
2022-04-01 21:12:03,524 INFO  CheckpointFileManager - Renamed temp file 
/tmp/spark-d94b9f11-e04c-4871-aeea-1d0b5c62e292/0/0/left-keyToNumValues/_metadata/.schema.9bcc39c9-721e-4ee0-b369-fb4f516c4fd6.tmp
 to 
/tmp/spark-d94b9f11-e04c-4871-aeea-1d0b5c62e292/0/0/left-keyToNumValues/_metadata/schema
2022-04-01 21:12:03,525 INFO  StateStore - Retrieved reference to 
StateStoreCoordinator: 
org.apache.spark.sql.execution.streaming.state.StateStoreCoordinatorRef374ccb9
2022-04-01 21:12:03,525 INFO  StateStore - Reported that the loaded 
instance 
StateStoreProviderId(StateStoreId(/tmp/spark-d94b9f11-e04c-4871-aeea-1d0b5c62e292,0,0,left-keyToNumValues),47925997-9891-4025-a36a-3e18bc758b50)
 is active
2022-04-01 21:12:03,525 INFO  HDFSBackedStateStoreProvider - Retrieved 
version 0 of HDFSStateStoreProvider[id = (op=0,part=0),dir = 
/tmp/spark-d94b9f11-e04c-4871-aeea-1d0b5c62e292/0/0/left-keyToNumValues] for 
update
2022-04-01 21:12:03,525 INFO  
SymmetricHashJoinStateManager$KeyToNumValuesStore - Loaded store 
StateStoreId(/tmp/spark-d94b9f11-e04c-4871-aeea-1d0b5c62e292,0,0,left-keyToNumValues)
2022-04-01 21:12:03,541 INFO  CheckpointFileManager - Writing atomically to 
/tmp/spark-d94b9f11-e04c-4871-aeea-1d0b5c62e292/0/0/left-keyWithIndexToValue/_metadata/schema
 using temp file 
/tmp/spark-d94b9f11-e04c-4871-aeea-1d0b5c62e292/0/0/left-keyWithIndexToValue/_metadata/.schema.fcde2229-a4fa-409b-b3eb-751572f06c08.tmp
2022-04-01 21:12:03,556 INFO  CheckpointFileManager - Renamed temp file 
/tmp/spark-d94b9f11-e04c-4871-aeea-1d0b5c62e292/0/0/left-keyWithIndexToValue/_metadata/.schema.fcde2229-a4fa-409b-b3eb-751572f06c08.tmp
 to 
/tmp/spark-d94b9f11-e04c-4871-aeea-1d0b5c62e292/0/0/left-keyWithIndexToValue/_metadata/schema
2022-04-01 21:12:03,558 INFO  StateStore - Retrieved reference to 
StateStoreCoordinator: 
org.apache.spark.sql.execution.streaming.state.StateStoreCoordinatorRef1ea930eb
2022-04-01 21:12:03,559 INFO  StateStore - Reported that the loaded 
instance 
StateStoreProviderId(StateStoreId(/tmp/spark-d94b9f11-e04c-4871-aeea-1d0b5c62e292,0,0,left-keyWithIndexToValue),47925997-9891-4025-a36a-3e18bc758b50)
 is active
2022-0

[spark] branch branch-3.3 updated: [SPARK-38787][SS] Replace found value with non-null element in the remaining list for key and remove remaining null elements from values in keyWithIndexToValue store

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

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


The following commit(s) were added to refs/heads/branch-3.3 by this push:
 new 00292543ab6 [SPARK-38787][SS] Replace found value with non-null 
element in the remaining list for key and remove remaining null elements from 
values in keyWithIndexToValue store for stream-stream joins
00292543ab6 is described below

commit 00292543ab6b1f86ef920a4840dc23759e0b9e44
Author: Anish Shrigondekar 
AuthorDate: Thu Apr 7 05:51:57 2022 +0900

[SPARK-38787][SS] Replace found value with non-null element in the 
remaining list for key and remove remaining null elements from values in 
keyWithIndexToValue store for stream-stream joins

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

In stream-stream joins, for removing old state (watermark by value), we 
call the `removeByValue` function with a removal condition. Within the iterator 
returned, if we find null at the end for matched value at non-last index, we 
are currently not removing and swapping the matched value. With this change, we 
will find the first non-null value from end and swap current index with that 
value and remove all elements from index + 1 to the end and then drop the last 
element as before.

### Why are the changes needed?

This change fixes a bug where we were not replacing found/matching values 
for `removeByValue` when encountering nulls in the symmetric hash join code.

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

No

### How was this patch tested?

Added a unit test for this change with nulls added. Here is a sample output:
```
Executing tests from 
//sql/core:org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManagerSuite-hive-2.3__hadoop-3.2

-
2022-04-01 21:11:59,641 INFO  CodeGenerator - Code generated in 225.884757 
ms
2022-04-01 21:11:59,662 INFO  CodeGenerator - Code generated in 10.870786 ms
Run starting. Expected test count is: 4
…
= TEST OUTPUT FOR 
o.a.s.sql.execution.streaming.state.SymmetricHashJoinStateManagerSuite: 
'StreamingJoinStateManager V2 - all operations with nulls' =

2022-04-01 21:12:03,487 INFO  StateStore - State Store maintenance task 
started
2022-04-01 21:12:03,508 INFO  CheckpointFileManager - Writing atomically to 
/tmp/spark-d94b9f11-e04c-4871-aeea-1d0b5c62e292/0/0/left-keyToNumValues/_metadata/schema
 using temp file 
/tmp/spark-d94b9f11-e04c-4871-aeea-1d0b5c62e292/0/0/left-keyToNumValues/_metadata/.schema.9bcc39c9-721e-4ee0-b369-fb4f516c4fd6.tmp
2022-04-01 21:12:03,524 INFO  CheckpointFileManager - Renamed temp file 
/tmp/spark-d94b9f11-e04c-4871-aeea-1d0b5c62e292/0/0/left-keyToNumValues/_metadata/.schema.9bcc39c9-721e-4ee0-b369-fb4f516c4fd6.tmp
 to 
/tmp/spark-d94b9f11-e04c-4871-aeea-1d0b5c62e292/0/0/left-keyToNumValues/_metadata/schema
2022-04-01 21:12:03,525 INFO  StateStore - Retrieved reference to 
StateStoreCoordinator: 
org.apache.spark.sql.execution.streaming.state.StateStoreCoordinatorRef374ccb9
2022-04-01 21:12:03,525 INFO  StateStore - Reported that the loaded 
instance 
StateStoreProviderId(StateStoreId(/tmp/spark-d94b9f11-e04c-4871-aeea-1d0b5c62e292,0,0,left-keyToNumValues),47925997-9891-4025-a36a-3e18bc758b50)
 is active
2022-04-01 21:12:03,525 INFO  HDFSBackedStateStoreProvider - Retrieved 
version 0 of HDFSStateStoreProvider[id = (op=0,part=0),dir = 
/tmp/spark-d94b9f11-e04c-4871-aeea-1d0b5c62e292/0/0/left-keyToNumValues] for 
update
2022-04-01 21:12:03,525 INFO  
SymmetricHashJoinStateManager$KeyToNumValuesStore - Loaded store 
StateStoreId(/tmp/spark-d94b9f11-e04c-4871-aeea-1d0b5c62e292,0,0,left-keyToNumValues)
2022-04-01 21:12:03,541 INFO  CheckpointFileManager - Writing atomically to 
/tmp/spark-d94b9f11-e04c-4871-aeea-1d0b5c62e292/0/0/left-keyWithIndexToValue/_metadata/schema
 using temp file 
/tmp/spark-d94b9f11-e04c-4871-aeea-1d0b5c62e292/0/0/left-keyWithIndexToValue/_metadata/.schema.fcde2229-a4fa-409b-b3eb-751572f06c08.tmp
2022-04-01 21:12:03,556 INFO  CheckpointFileManager - Renamed temp file 
/tmp/spark-d94b9f11-e04c-4871-aeea-1d0b5c62e292/0/0/left-keyWithIndexToValue/_metadata/.schema.fcde2229-a4fa-409b-b3eb-751572f06c08.tmp
 to 
/tmp/spark-d94b9f11-e04c-4871-aeea-1d0b5c62e292/0/0/left-keyWithIndexToValue/_metadata/schema
2022-04-01 21:12:03,558 INFO  StateStore - Retrieved reference to 
StateStoreCoordinator: 
org.apache.spark.sql.execution.streaming.state.StateStoreCoordinatorRef1ea930eb
2022-04-01 21:12:03,559 INFO  StateStore - Reported that the loaded 
instance 
StateStoreProviderId(StateStoreId(/tmp/spark-d94b9f11-e04c-4871-aeea-1d0b5c62e292,0,0,left-keyWithIndexToValue),47925997-9891-4025-a36a-3e18bc758b50)
 is active
2022-0

[spark] branch master updated: [SPARK-38787][SS] Replace found value with non-null element in the remaining list for key and remove remaining null elements from values in keyWithIndexToValue store for

2022-04-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 6d9bfb675f3 [SPARK-38787][SS] Replace found value with non-null 
element in the remaining list for key and remove remaining null elements from 
values in keyWithIndexToValue store for stream-stream joins
6d9bfb675f3 is described below

commit 6d9bfb675f3e58c6e7d9facd8cf3f22069c4cc48
Author: Anish Shrigondekar 
AuthorDate: Thu Apr 7 05:51:57 2022 +0900

[SPARK-38787][SS] Replace found value with non-null element in the 
remaining list for key and remove remaining null elements from values in 
keyWithIndexToValue store for stream-stream joins

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

In stream-stream joins, for removing old state (watermark by value), we 
call the `removeByValue` function with a removal condition. Within the iterator 
returned, if we find null at the end for matched value at non-last index, we 
are currently not removing and swapping the matched value. With this change, we 
will find the first non-null value from end and swap current index with that 
value and remove all elements from index + 1 to the end and then drop the last 
element as before.

### Why are the changes needed?

This change fixes a bug where we were not replacing found/matching values 
for `removeByValue` when encountering nulls in the symmetric hash join code.

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

No

### How was this patch tested?

Added a unit test for this change with nulls added. Here is a sample output:
```
Executing tests from 
//sql/core:org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManagerSuite-hive-2.3__hadoop-3.2

-
2022-04-01 21:11:59,641 INFO  CodeGenerator - Code generated in 225.884757 
ms
2022-04-01 21:11:59,662 INFO  CodeGenerator - Code generated in 10.870786 ms
Run starting. Expected test count is: 4
…
= TEST OUTPUT FOR 
o.a.s.sql.execution.streaming.state.SymmetricHashJoinStateManagerSuite: 
'StreamingJoinStateManager V2 - all operations with nulls' =

2022-04-01 21:12:03,487 INFO  StateStore - State Store maintenance task 
started
2022-04-01 21:12:03,508 INFO  CheckpointFileManager - Writing atomically to 
/tmp/spark-d94b9f11-e04c-4871-aeea-1d0b5c62e292/0/0/left-keyToNumValues/_metadata/schema
 using temp file 
/tmp/spark-d94b9f11-e04c-4871-aeea-1d0b5c62e292/0/0/left-keyToNumValues/_metadata/.schema.9bcc39c9-721e-4ee0-b369-fb4f516c4fd6.tmp
2022-04-01 21:12:03,524 INFO  CheckpointFileManager - Renamed temp file 
/tmp/spark-d94b9f11-e04c-4871-aeea-1d0b5c62e292/0/0/left-keyToNumValues/_metadata/.schema.9bcc39c9-721e-4ee0-b369-fb4f516c4fd6.tmp
 to 
/tmp/spark-d94b9f11-e04c-4871-aeea-1d0b5c62e292/0/0/left-keyToNumValues/_metadata/schema
2022-04-01 21:12:03,525 INFO  StateStore - Retrieved reference to 
StateStoreCoordinator: 
org.apache.spark.sql.execution.streaming.state.StateStoreCoordinatorRef374ccb9
2022-04-01 21:12:03,525 INFO  StateStore - Reported that the loaded 
instance 
StateStoreProviderId(StateStoreId(/tmp/spark-d94b9f11-e04c-4871-aeea-1d0b5c62e292,0,0,left-keyToNumValues),47925997-9891-4025-a36a-3e18bc758b50)
 is active
2022-04-01 21:12:03,525 INFO  HDFSBackedStateStoreProvider - Retrieved 
version 0 of HDFSStateStoreProvider[id = (op=0,part=0),dir = 
/tmp/spark-d94b9f11-e04c-4871-aeea-1d0b5c62e292/0/0/left-keyToNumValues] for 
update
2022-04-01 21:12:03,525 INFO  
SymmetricHashJoinStateManager$KeyToNumValuesStore - Loaded store 
StateStoreId(/tmp/spark-d94b9f11-e04c-4871-aeea-1d0b5c62e292,0,0,left-keyToNumValues)
2022-04-01 21:12:03,541 INFO  CheckpointFileManager - Writing atomically to 
/tmp/spark-d94b9f11-e04c-4871-aeea-1d0b5c62e292/0/0/left-keyWithIndexToValue/_metadata/schema
 using temp file 
/tmp/spark-d94b9f11-e04c-4871-aeea-1d0b5c62e292/0/0/left-keyWithIndexToValue/_metadata/.schema.fcde2229-a4fa-409b-b3eb-751572f06c08.tmp
2022-04-01 21:12:03,556 INFO  CheckpointFileManager - Renamed temp file 
/tmp/spark-d94b9f11-e04c-4871-aeea-1d0b5c62e292/0/0/left-keyWithIndexToValue/_metadata/.schema.fcde2229-a4fa-409b-b3eb-751572f06c08.tmp
 to 
/tmp/spark-d94b9f11-e04c-4871-aeea-1d0b5c62e292/0/0/left-keyWithIndexToValue/_metadata/schema
2022-04-01 21:12:03,558 INFO  StateStore - Retrieved reference to 
StateStoreCoordinator: 
org.apache.spark.sql.execution.streaming.state.StateStoreCoordinatorRef1ea930eb
2022-04-01 21:12:03,559 INFO  StateStore - Reported that the loaded 
instance 
StateStoreProviderId(StateStoreId(/tmp/spark-d94b9f11-e04c-4871-aeea-1d0b5c62e292,0,0,left-keyWithIndexToValue),47925997-9891-4025-a36a-3e18bc758b50)
 is active
2022-04-01 21

[spark] 02/02: [SPARK-38684][SS] Fix correctness issue on stream-stream outer join with RocksDB state store provider

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

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

commit 7e542f164a2f17501b153a2d7b8c53636a5bb9b8
Author: Jungtaek Lim 
AuthorDate: Fri Apr 1 18:21:48 2022 +0900

[SPARK-38684][SS] Fix correctness issue on stream-stream outer join with 
RocksDB state store provider

(Credit to alex-balikov for the inspiration of the root cause observation, 
and anishshri-db for looking into the issue together.)

This PR fixes the correctness issue on stream-stream outer join with 
RocksDB state store provider, which can occur in certain condition, like below:

* stream-stream time interval outer join
  * left outer join has an issue on left side, right outer join has an 
issue on right side, full outer join has an issue on both sides
* At batch N, produce non-late row(s) on the problematic side
* At the same batch (batch N), some row(s) on the problematic side are 
evicted by the condition of watermark

The root cause is same as 
[SPARK-38320](https://issues.apache.org/jira/browse/SPARK-38320) - weak read 
consistency on iterator, especially with RocksDB state store provider. (Quoting 
from SPARK-38320: The problem is due to the StateStore.iterator not reflecting 
StateStore changes made after its creation.)

More specifically, if updates are performed during processing input rows 
and somehow updates the number of values for grouping key, the update is not 
seen in SymmetricHashJoinStateManager.removeByValueCondition, and the method 
does the eviction with the number of values in out of sync.

Making it more worse, if the method performs the eviction and updates the 
number of values for grouping key, it "overwrites" the number of value, 
effectively drop all rows being inserted in the same batch.

Below code blocks are references on understanding the details of the issue.


https://github.com/apache/spark/blob/ca7200b0008dc6101a252020e6c34ef7b72d81d6/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala#L327-L339


https://github.com/apache/spark/blob/ca7200b0008dc6101a252020e6c34ef7b72d81d6/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala#L619-L627


https://github.com/apache/spark/blob/ca7200b0008dc6101a252020e6c34ef7b72d81d6/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SymmetricHashJoinStateManager.scala#L195-L201


https://github.com/apache/spark/blob/ca7200b0008dc6101a252020e6c34ef7b72d81d6/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SymmetricHashJoinStateManager.scala#L208-L223

This PR fixes the outer iterators as late evaluation to ensure all updates 
on processing input rows are reflected "before" outer iterators are initialized.

The bug is described in above section.

No.

New UT added.

Closes #36002 from HeartSaVioR/SPARK-38684.

Authored-by: Jungtaek Lim 
Signed-off-by: Jungtaek Lim 
(cherry picked from commit 2f8613f22c0750c00cf1dcfb2f31c431d8dc1be7)
Signed-off-by: Jungtaek Lim 
---
 .../streaming/StreamingSymmetricHashJoinExec.scala | 81 --
 .../spark/sql/streaming/StreamingJoinSuite.scala   | 63 -
 2 files changed, 121 insertions(+), 23 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala
index 616ae08..30d87af 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala
@@ -318,17 +318,22 @@ case class StreamingSymmetricHashJoinExec(
   }
 }
 
+val initIterFn = { () =>
+  val removedRowIter = leftSideJoiner.removeOldState()
+  removedRowIter.filterNot { kv =>
+stateFormatVersion match {
+  case 1 => matchesWithRightSideState(new UnsafeRowPair(kv.key, 
kv.value))
+  case 2 => kv.matched
+  case _ => throwBadStateFormatVersionException()
+}
+  }.map(pair => joinedRow.withLeft(pair.value).withRight(nullRight))
+}
+
 // NOTE: we need to make sure `outerOutputIter` is evaluated "after" 
exhausting all of
-// elements in `innerOutputIter`, because evaluation of 
`innerOutputIter` may update
-// the match flag which the logic for outer join is relying on.
-val removedRowIter = leftSideJoiner.removeOldState()
-val outerOutputIter = removedRowIter.filterNot { kv =>
- 

[spark] 01/02: [SPARK-38333][SQL][3.2][FOLLOWUP] fix compilation error

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

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

commit 6a4b2c227553a472522a4d3f02e389141593497d
Author: Jungtaek Lim 
AuthorDate: Fri Apr 1 19:07:21 2022 +0900

[SPARK-38333][SQL][3.2][FOLLOWUP] fix compilation error
---
 .../spark/sql/catalyst/expressions/SubexpressionEliminationSuite.scala  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SubexpressionEliminationSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SubexpressionEliminationSuite.scala
index 4b353cd..b414019 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SubexpressionEliminationSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SubexpressionEliminationSuite.scala
@@ -395,7 +395,7 @@ class SubexpressionEliminationSuite extends SparkFunSuite 
with ExpressionEvalHel
   test("SPARK-38333: PlanExpression expression should skip addExprTree 
function in Executor") {
 try {
   // suppose we are in executor
-  val context1 = new TaskContextImpl(0, 0, 0, 0, 0, null, null, null, cpus 
= 0)
+  val context1 = new TaskContextImpl(0, 0, 0, 0, 0, null, null, null)
   TaskContext.setTaskContext(context1)
 
   val equivalence = new EquivalentExpressions

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



[spark] branch branch-3.2 updated (e78cca9 -> 7e542f1)

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

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


from e78cca9  [SPARK-38333][SQL] PlanExpression expression should skip 
addExprTree function in Executor
 new 6a4b2c2  [SPARK-38333][SQL][3.2][FOLLOWUP] fix compilation error
 new 7e542f1  [SPARK-38684][SS] Fix correctness issue on stream-stream 
outer join with RocksDB state store provider

The 2 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:
 .../SubexpressionEliminationSuite.scala|  2 +-
 .../streaming/StreamingSymmetricHashJoinExec.scala | 81 --
 .../spark/sql/streaming/StreamingJoinSuite.scala   | 63 -
 3 files changed, 122 insertions(+), 24 deletions(-)

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



[spark] branch branch-3.3 updated: [SPARK-38684][SS] Fix correctness issue on stream-stream outer join with RocksDB state store provider

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

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


The following commit(s) were added to refs/heads/branch-3.3 by this push:
 new 8a072ef  [SPARK-38684][SS] Fix correctness issue on stream-stream 
outer join with RocksDB state store provider
8a072ef is described below

commit 8a072ef6badad69ef5cfdd656d0c068979f6ea76
Author: Jungtaek Lim 
AuthorDate: Fri Apr 1 18:21:48 2022 +0900

[SPARK-38684][SS] Fix correctness issue on stream-stream outer join with 
RocksDB state store provider

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

(Credit to alex-balikov for the inspiration of the root cause observation, 
and anishshri-db for looking into the issue together.)

This PR fixes the correctness issue on stream-stream outer join with 
RocksDB state store provider, which can occur in certain condition, like below:

* stream-stream time interval outer join
  * left outer join has an issue on left side, right outer join has an 
issue on right side, full outer join has an issue on both sides
* At batch N, produce non-late row(s) on the problematic side
* At the same batch (batch N), some row(s) on the problematic side are 
evicted by the condition of watermark

The root cause is same as 
[SPARK-38320](https://issues.apache.org/jira/browse/SPARK-38320) - weak read 
consistency on iterator, especially with RocksDB state store provider. (Quoting 
from SPARK-38320: The problem is due to the StateStore.iterator not reflecting 
StateStore changes made after its creation.)

More specifically, if updates are performed during processing input rows 
and somehow updates the number of values for grouping key, the update is not 
seen in SymmetricHashJoinStateManager.removeByValueCondition, and the method 
does the eviction with the number of values in out of sync.

Making it more worse, if the method performs the eviction and updates the 
number of values for grouping key, it "overwrites" the number of value, 
effectively drop all rows being inserted in the same batch.

Below code blocks are references on understanding the details of the issue.


https://github.com/apache/spark/blob/ca7200b0008dc6101a252020e6c34ef7b72d81d6/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala#L327-L339


https://github.com/apache/spark/blob/ca7200b0008dc6101a252020e6c34ef7b72d81d6/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala#L619-L627


https://github.com/apache/spark/blob/ca7200b0008dc6101a252020e6c34ef7b72d81d6/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SymmetricHashJoinStateManager.scala#L195-L201


https://github.com/apache/spark/blob/ca7200b0008dc6101a252020e6c34ef7b72d81d6/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SymmetricHashJoinStateManager.scala#L208-L223

This PR fixes the outer iterators as late evaluation to ensure all updates 
on processing input rows are reflected "before" outer iterators are initialized.

### Why are the changes needed?

The bug is described in above section.

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

No.

### How was this patch tested?

New UT added.

Closes #36002 from HeartSaVioR/SPARK-38684.

Authored-by: Jungtaek Lim 
Signed-off-by: Jungtaek Lim 
(cherry picked from commit 2f8613f22c0750c00cf1dcfb2f31c431d8dc1be7)
Signed-off-by: Jungtaek Lim 
---
 .../streaming/StreamingSymmetricHashJoinExec.scala | 81 --
 .../spark/sql/streaming/StreamingJoinSuite.scala   | 63 -
 2 files changed, 121 insertions(+), 23 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala
index 81888e0..aa888c1 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala
@@ -324,17 +324,22 @@ case class StreamingSymmetricHashJoinExec(
   }
 }
 
+val initIterFn = { () =>
+  val removedRowIter = leftSideJoiner.removeOldState()
+  removedRowIter.filterNot { kv =>
+stateFormatVersion match {
+  case 1 => matchesWithRightSideState(new UnsafeRowPair(kv.key, 
kv.value))
+  case 2 => kv.matched
+  case _ => throwBadStateFormatVersionException()
+}
+  }.map(pair => joinedRow.withLeft(pair.value).withRight(nullRight))
+}
+

[spark] branch master updated: [SPARK-38684][SS] Fix correctness issue on stream-stream outer join with RocksDB state store provider

2022-04-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 2f8613f  [SPARK-38684][SS] Fix correctness issue on stream-stream 
outer join with RocksDB state store provider
2f8613f is described below

commit 2f8613f22c0750c00cf1dcfb2f31c431d8dc1be7
Author: Jungtaek Lim 
AuthorDate: Fri Apr 1 18:21:48 2022 +0900

[SPARK-38684][SS] Fix correctness issue on stream-stream outer join with 
RocksDB state store provider

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

(Credit to alex-balikov for the inspiration of the root cause observation, 
and anishshri-db for looking into the issue together.)

This PR fixes the correctness issue on stream-stream outer join with 
RocksDB state store provider, which can occur in certain condition, like below:

* stream-stream time interval outer join
  * left outer join has an issue on left side, right outer join has an 
issue on right side, full outer join has an issue on both sides
* At batch N, produce non-late row(s) on the problematic side
* At the same batch (batch N), some row(s) on the problematic side are 
evicted by the condition of watermark

The root cause is same as 
[SPARK-38320](https://issues.apache.org/jira/browse/SPARK-38320) - weak read 
consistency on iterator, especially with RocksDB state store provider. (Quoting 
from SPARK-38320: The problem is due to the StateStore.iterator not reflecting 
StateStore changes made after its creation.)

More specifically, if updates are performed during processing input rows 
and somehow updates the number of values for grouping key, the update is not 
seen in SymmetricHashJoinStateManager.removeByValueCondition, and the method 
does the eviction with the number of values in out of sync.

Making it more worse, if the method performs the eviction and updates the 
number of values for grouping key, it "overwrites" the number of value, 
effectively drop all rows being inserted in the same batch.

Below code blocks are references on understanding the details of the issue.


https://github.com/apache/spark/blob/ca7200b0008dc6101a252020e6c34ef7b72d81d6/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala#L327-L339


https://github.com/apache/spark/blob/ca7200b0008dc6101a252020e6c34ef7b72d81d6/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala#L619-L627


https://github.com/apache/spark/blob/ca7200b0008dc6101a252020e6c34ef7b72d81d6/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SymmetricHashJoinStateManager.scala#L195-L201


https://github.com/apache/spark/blob/ca7200b0008dc6101a252020e6c34ef7b72d81d6/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SymmetricHashJoinStateManager.scala#L208-L223

This PR fixes the outer iterators as late evaluation to ensure all updates 
on processing input rows are reflected "before" outer iterators are initialized.

### Why are the changes needed?

The bug is described in above section.

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

No.

### How was this patch tested?

New UT added.

Closes #36002 from HeartSaVioR/SPARK-38684.

Authored-by: Jungtaek Lim 
Signed-off-by: Jungtaek Lim 
---
 .../streaming/StreamingSymmetricHashJoinExec.scala | 81 --
 .../spark/sql/streaming/StreamingJoinSuite.scala   | 63 -
 2 files changed, 121 insertions(+), 23 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala
index 81888e0..aa888c1 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala
@@ -324,17 +324,22 @@ case class StreamingSymmetricHashJoinExec(
   }
 }
 
+val initIterFn = { () =>
+  val removedRowIter = leftSideJoiner.removeOldState()
+  removedRowIter.filterNot { kv =>
+stateFormatVersion match {
+  case 1 => matchesWithRightSideState(new UnsafeRowPair(kv.key, 
kv.value))
+  case 2 => kv.matched
+  case _ => throwBadStateFormatVersionException()
+}
+  }.map(pair => joinedRow.withLeft(pair.value).withRight(nullRight))
+}
+
 // NOTE: we need to make sure `outerOutputIter` is evaluated "after" 
exhausting all of
-// e

[spark] branch master updated: [SPARK-38349][SS] No need to filter events when sessionwindow gapDuration greater than 0

2022-03-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 a445536  [SPARK-38349][SS] No need to filter events when sessionwindow 
gapDuration greater than 0
a445536 is described below

commit a44553648f75e9243b8a7dc27185ae6901f35f94
Author: nyingping 
AuthorDate: Wed Mar 30 12:00:40 2022 +0900

[SPARK-38349][SS] No need to filter events when sessionwindow gapDuration 
greater than 0

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

Static gapDuration on session Window,No need to filter events when 
sessionwindow gapDuration greater than 0.

### Why are the changes needed?

save calculation resources and improve performance.

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

No

### How was this patch tested?

add new UT and benchmark.

a simple benchmark on 
[[9dae8a5](https://github.com/nyingping/spark/commit/9dae8a555e82a59d2160bbb4518704cec81b219e)]
 . thanks again  
[HeartSaVioRd532b6f](https://github.com/HeartSaVioR/spark/commit/d532b6f6bcdd80cdaac520b21587ebb69ff2df8f).

---
case 1
---
```
spark.range(numOfRow)
  .selectExpr("CAST(id AS timestamp) AS time")
  .select(session_window(col("time"), "10 seconds"))
  .count()
```

Result:

```
Java HotSpot(TM) 64-Bit Server VM 1.8.0_291-b10 on Windows 10 10.0
AMD64 Family 23 Model 96 Stepping 1, AuthenticAMD
session windows:  Best Time(ms)   Avg Time(ms)   
Stdev(ms)Rate(M/s)   Per Row(ns)   Relative


old logic  1688   1730  
61  5.9 168.8   1.0X
new logic21 26  
 5487.3   2.1  82.3X
```

Closes #35680 from nyingping/SPARK-38349.

Lead-authored-by: nyingping 
Co-authored-by: Nie yingping 
Signed-off-by: Jungtaek Lim 
---
 .../spark/sql/catalyst/analysis/Analyzer.scala | 16 +--
 .../spark/sql/DataFrameSessionWindowingSuite.scala | 56 +-
 2 files changed, 67 insertions(+), 5 deletions(-)

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 6d95067..f69f17d 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/catalyst/analysis/Analyzer.scala
@@ -57,7 +57,7 @@ import org.apache.spark.sql.internal.connector.V1Function
 import org.apache.spark.sql.types._
 import org.apache.spark.sql.types.DayTimeIntervalType.DAY
 import org.apache.spark.sql.util.{CaseInsensitiveStringMap, SchemaUtils}
-import org.apache.spark.unsafe.types.UTF8String
+import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
 import org.apache.spark.util.Utils
 
 /**
@@ -4058,10 +4058,20 @@ object SessionWindowing extends Rule[LogicalPlan] {
   case s: SessionWindow => sessionAttr
 }
 
+val filterByTimeRange = session.gapDuration match {
+  case Literal(interval: CalendarInterval, CalendarIntervalType) =>
+interval == null || interval.months + interval.days + 
interval.microseconds <= 0
+  case _ => true
+}
+
 // As same as tumbling window, we add a filter to filter out nulls.
 // And we also filter out events with negative or zero or invalid gap 
duration.
-val filterExpr = IsNotNull(session.timeColumn) &&
-  (sessionAttr.getField(SESSION_END) > 
sessionAttr.getField(SESSION_START))
+val filterExpr = if (filterByTimeRange) {
+  IsNotNull(session.timeColumn) &&
+(sessionAttr.getField(SESSION_END) > 
sessionAttr.getField(SESSION_START))
+} else {
+  IsNotNull(session.timeColumn)
+}
 
 replacedPlan.withNewChildren(
   Filter(filterExpr,
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSessionWindowingSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSessionWindowingSuite.scala
index a5414f3..4c2d0f5 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSessionWindowingSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSessionWindowingSuite.scala
@@ -22,8 +22,8 @@ import java.time.LocalDateTime
 import org.scalatest.BeforeAndAfterEach
 
 

[spark] branch master updated (ca7200b -> 8fab597)

2022-03-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 ca7200b  [SPARK-38633][SQL][FOLLOWUP] JDBCSQLBuilder should build cast 
to type of databases
 add 8fab597  [SPARK-38670][SS] Add offset commit time to streaming query 
listener

No new revisions were added by this update.

Summary of changes:
 .../spark/sql/execution/streaming/MicroBatchExecution.scala   | 8 +---
 .../org/apache/spark/sql/streaming/StreamingQuerySuite.scala  | 2 ++
 2 files changed, 7 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-38522][SS] Enrich the method contract of iterator in StateStore to not expect strong consistency on certain condition

2022-03-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 c309cd1  [SPARK-38522][SS] Enrich the method contract of iterator in 
StateStore to not expect strong consistency on certain condition
c309cd1 is described below

commit c309cd136a04d831872a83ad425cacc1c557aa60
Author: Jungtaek Lim 
AuthorDate: Wed Mar 23 10:09:06 2022 +0900

[SPARK-38522][SS] Enrich the method contract of iterator in StateStore to 
not expect strong consistency on certain condition

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

This PR proposes to enrich the method contract of iterator in StateStore, 
that the returned iterator is not guaranteed to reflect all updates being 
performed "after" it has been initialized.

### Why are the changes needed?

The lack of information on the method contract could mislead callers to 
expect some guarantees although not described. It would be nice to add to the 
contract to make clear. It could also help to implementations that they are not 
required to guarantee such thing.

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

No

### How was this patch tested?

N/A

Closes #35816 from HeartSaVioR/SPARK-38522.

Authored-by: Jungtaek Lim 
Signed-off-by: Jungtaek Lim 
---
 .../org/apache/spark/sql/execution/streaming/state/StateStore.scala   | 4 
 1 file changed, 4 insertions(+)

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 5020638..64c9cd1 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
@@ -127,6 +127,10 @@ trait StateStore extends ReadStateStore {
   /**
* Return an iterator containing all the key-value pairs in the StateStore. 
Implementations must
* ensure that updates (puts, removes) can be made while iterating over this 
iterator.
+   *
+   * It is not required for implementations to ensure the iterator reflects 
all updates being
+   * performed after initialization of the iterator. Callers should perform 
all updates before
+   * calling this method if all updates should be visible in the returned 
iterator.
*/
   override def iterator(): Iterator[UnsafeRowPair]
 

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



[spark] branch master updated (ee5121a -> fc5e922)

2022-03-21 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 ee5121a  [SPARK-38574][DOCS] Enrich the documentation of option 
avroSchema
 add fc5e922  [SPARK-38564][SS] Support collecting metrics from streaming 
sinks

No new revisions were added by this update.

Summary of changes:
 ...artitionOffset.java => ReportsSinkMetrics.java} |  18 +--
 .../sql/execution/streaming/ProgressReporter.scala |  13 +-
 .../org/apache/spark/sql/streaming/progress.scala  |  11 +-
 .../sql/streaming/ReportSinkMetricsSuite.scala | 153 +
 4 files changed, 181 insertions(+), 14 deletions(-)
 copy 
sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/{PartitionOffset.java
 => ReportsSinkMetrics.java} (75%)
 create mode 100644 
sql/core/src/test/scala/org/apache/spark/sql/streaming/ReportSinkMetricsSuite.scala

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



[spark] branch branch-3.3 updated: [SPARK-38593][SS] Carry over the metric of the number of dropped late events into SessionWindowStateStoreSaveExec

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

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


The following commit(s) were added to refs/heads/branch-3.3 by this push:
 new 6127878  [SPARK-38593][SS] Carry over the metric of the number of 
dropped late events into SessionWindowStateStoreSaveExec
6127878 is described below

commit 61278786adf04c5652335ec438c55bdb9c47fbf8
Author: Jungtaek Lim 
AuthorDate: Fri Mar 18 18:27:45 2022 +0900

[SPARK-38593][SS] Carry over the metric of the number of dropped late 
events into SessionWindowStateStoreSaveExec

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

This PR proposes to carry over the metric of the number of dropped late 
events from SessionWindowStateStoreRestoreExec to 
SessionWindowStateStoreSaveExec, so that state operator progress for session 
window aggregation operator will have correct number.

### Why are the changes needed?

The number of dropped late events for session window aggregation was missed 
in the mechanism of SS metrics (state operator progress).

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

Yes, the number of dropped late events for session window aggregation will 
be taken into account for state operator progress in streaming listener, as 
well as a graph on late events in the SS UI page.

### How was this patch tested?

Modified UT.

Closes #35909 from HeartSaVioR/SPARK-38593.

Authored-by: Jungtaek Lim 
Signed-off-by: Jungtaek Lim 
---
 .../execution/streaming/statefulOperators.scala| 23 ++
 .../streaming/StreamingSessionWindowSuite.scala| 17 
 2 files changed, 40 insertions(+)

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 e367637..e2a0598 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
@@ -743,6 +743,29 @@ case class SessionWindowStateStoreSaveExec(
 
   override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan =
 copy(child = newChild)
+
+  /**
+   * The class overrides this method since dropping late events are happening 
in the upstream node
+   * [[SessionWindowStateStoreRestoreExec]], and this class has responsibility 
to report the number
+   * of dropped late events as a part of StateOperatorProgress.
+   *
+   * This method should be called in the driver after this SparkPlan has been 
executed and metrics
+   * have been updated.
+   */
+  override def getProgress(): StateOperatorProgress = {
+val stateOpProgress = super.getProgress()
+
+// This should be safe, since the method is called in the driver after the 
plan has been
+// executed and metrics have been updated.
+val numRowsDroppedByWatermark = child.collectFirst {
+  case s: SessionWindowStateStoreRestoreExec =>
+s.longMetric("numRowsDroppedByWatermark").value
+}.getOrElse(0L)
+
+stateOpProgress.copy(
+  newNumRowsUpdated = stateOpProgress.numRowsUpdated,
+  newNumRowsDroppedByWatermark = numRowsDroppedByWatermark)
+  }
 }
 
 
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingSessionWindowSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingSessionWindowSuite.scala
index d0f3a87..3ed23ba 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingSessionWindowSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingSessionWindowSuite.scala
@@ -216,6 +216,7 @@ class StreamingSessionWindowSuite extends StreamTest
   // ("structured", 41, 51, 10, 1)
   CheckNewAnswer(
   ),
+  assertNumRowsDroppedByWatermark(2),
 
   // concatenating multiple previous sessions into one
   AddData(inputData, ("spark streaming", 30L)),
@@ -319,6 +320,7 @@ class StreamingSessionWindowSuite extends StreamTest
   // ("spark", 40, 50, 10, 1),
   CheckNewAnswer(
   ),
+  assertNumRowsDroppedByWatermark(2),
 
   // concatenating multiple previous sessions into one
   AddData(inputData, ("spark streaming", 30L)),
@@ -406,6 +408,21 @@ class StreamingSessionWindowSuite extends StreamTest
 }
   }
 
+  private def assertNumRowsDroppedByWatermark(
+  numRowsDroppedByWatermark: Long): AssertOnQuery = AssertOnQuery { q =>
+q.processAllAvailable()
+val progressWithData = q.recentProgress.filterNot { p =>
+  // filter out batches which are falling into one of types:
+  // 1) doesn't execute the batch run
+  // 2) empty input batch
+  p.inputRowsPerSecond == 0
+  

[spark] branch master updated (681dfee -> a9ad119)

2022-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 681dfee  [SPARK-38583][SQL] Restore the behavior of `to_timestamp` 
that allows numeric types
 add a9ad119  [SPARK-38593][SS] Carry over the metric of the number of 
dropped late events into SessionWindowStateStoreSaveExec

No new revisions were added by this update.

Summary of changes:
 .../execution/streaming/statefulOperators.scala| 23 ++
 .../streaming/StreamingSessionWindowSuite.scala| 17 
 2 files changed, 40 insertions(+)

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



[spark] branch master updated (83673c8 -> 715a06c)

2022-03-13 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 83673c8  [SPARK-38528][SQL] Eagerly iterate over aggregate sequence 
when building project list in `ExtractGenerator`
 add 715a06c  [SPARK-38532][SS][TESTS] Add test case for invalid 
gapDuration of sessionwindow

No new revisions were added by this update.

Summary of changes:
 .../spark/sql/catalyst/analysis/Analyzer.scala |  2 +-
 .../spark/sql/DataFrameSessionWindowingSuite.scala | 28 ++
 2 files changed, 29 insertions(+), 1 deletion(-)

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



[spark] branch branch-3.2 updated: [SPARK-38320][SS] Fix flatMapGroupsWithState timeout in batch with data for key

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

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


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new d1b207a  [SPARK-38320][SS] Fix flatMapGroupsWithState timeout in batch 
with data for key
d1b207a is described below

commit d1b207a2cda7fa1b6b52871796b5fdad52f45406
Author: Alex Balikov 
AuthorDate: Sun Mar 13 13:55:13 2022 +0900

[SPARK-38320][SS] Fix flatMapGroupsWithState timeout in batch with data for 
key

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

As described in 
[SPARK-38320](https://issues.apache.org/jira/browse/SPARK-38320), the bug is 
that it is possible for (flat)MapGroupsWithState to timeout a key even if that 
key received data within the same batch. This is against the documented 
(flat)MapGroupsWithState contract. The problem is due to the 
StateStore.iterator not reflecting StateStore changes made *after* its creation 
- this is illustrated in the test this PR adds to StateStoreSuite.scala.

The fix is to *late bind* the timeoutProcessorIter timeout processing 
iterator in FlatMapGroupsWithStateExec to be created *after* the input iterator 
has been exhausted and the state changes applied to the StateStore.

### Why are the changes needed?

The changes are needed to ensure the state timeout processing iterator for 
(flat)MapGroupsWithState is created *after* the input is processed and the 
changes are applied into the StateStore, otherwise it may not notice these 
changes (the change to the key timeout timestamp being updated as part of the 
input processing).

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

No. Bug fix.

### How was this patch tested?

* Added a test to StateStoreSuite.scala to illustrate the difference of 
state store iterator behavior across the different implementations of state 
stores. In particular the test illustrates the RocksDB state store iterator not 
reflecting state store changes made after its creation.
* Added test to FlatMapGroupsWithStateSuite.scala which would fail with 
unexpected state timeout if the issue was not fixed.

Closes #35810 from alex-balikov/SPARK-38320-state-iterators2.

Authored-by: Alex Balikov 
Signed-off-by: Jungtaek Lim 
(cherry picked from commit 6b64e5dc74cbdc7e2b4ae42232e9610319ad73f3)
Signed-off-by: Jungtaek Lim 
---
 .../streaming/FlatMapGroupsWithStateExec.scala | 20 +---
 .../streaming/state/StateStoreSuite.scala  | 58 ++
 .../streaming/FlatMapGroupsWithStateSuite.scala| 48 +-
 3 files changed, 118 insertions(+), 8 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala
index a00a622..4c61975 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala
@@ -167,12 +167,20 @@ case class FlatMapGroupsWithStateExec(
   timeoutProcessingStartTimeNs = System.nanoTime
 })
 
-val timeoutProcessorIter =
-  CompletionIterator[InternalRow, 
Iterator[InternalRow]](processor.processTimedOutState(), {
-// Note: `timeoutLatencyMs` also includes the time the parent operator 
took for
-// processing output returned through iterator.
-timeoutLatencyMs += NANOSECONDS.toMillis(System.nanoTime - 
timeoutProcessingStartTimeNs)
-  })
+// SPARK-38320: Late-bind the timeout processing iterator so it is created 
*after* the input is
+// processed (the input iterator is exhausted) and the state updates are 
written into the
+// state store. Otherwise the iterator may not see the updates (e.g. with 
RocksDB state store).
+val timeoutProcessorIter = new Iterator[InternalRow] {
+  private lazy val itr = getIterator()
+  override def hasNext = itr.hasNext
+  override def next() = itr.next()
+  private def getIterator(): Iterator[InternalRow] =
+CompletionIterator[InternalRow, 
Iterator[InternalRow]](processor.processTimedOutState(), {
+  // Note: `timeoutLatencyMs` also includes the time the parent 
operator took for
+  // processing output returned through iterator.
+  timeoutLatencyMs += NANOSECONDS.toMillis(System.nanoTime - 
timeoutProcessingStartTimeNs)
+})
+}
 
 // Generate a iterator that returns the rows grouped by the grouping 
function
 // Note that this code ensures that the filtering for timeout occurs only 
after
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala
 
b/sql/core/src/test/scala/org/apache

[spark] branch master updated: [SPARK-38320][SS] Fix flatMapGroupsWithState timeout in batch with data for key

2022-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 6b64e5d  [SPARK-38320][SS] Fix flatMapGroupsWithState timeout in batch 
with data for key
6b64e5d is described below

commit 6b64e5dc74cbdc7e2b4ae42232e9610319ad73f3
Author: Alex Balikov 
AuthorDate: Sun Mar 13 13:55:13 2022 +0900

[SPARK-38320][SS] Fix flatMapGroupsWithState timeout in batch with data for 
key

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

As described in 
[SPARK-38320](https://issues.apache.org/jira/browse/SPARK-38320), the bug is 
that it is possible for (flat)MapGroupsWithState to timeout a key even if that 
key received data within the same batch. This is against the documented 
(flat)MapGroupsWithState contract. The problem is due to the 
StateStore.iterator not reflecting StateStore changes made *after* its creation 
- this is illustrated in the test this PR adds to StateStoreSuite.scala.

The fix is to *late bind* the timeoutProcessorIter timeout processing 
iterator in FlatMapGroupsWithStateExec to be created *after* the input iterator 
has been exhausted and the state changes applied to the StateStore.

### Why are the changes needed?

The changes are needed to ensure the state timeout processing iterator for 
(flat)MapGroupsWithState is created *after* the input is processed and the 
changes are applied into the StateStore, otherwise it may not notice these 
changes (the change to the key timeout timestamp being updated as part of the 
input processing).

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

No. Bug fix.

### How was this patch tested?

* Added a test to StateStoreSuite.scala to illustrate the difference of 
state store iterator behavior across the different implementations of state 
stores. In particular the test illustrates the RocksDB state store iterator not 
reflecting state store changes made after its creation.
* Added test to FlatMapGroupsWithStateSuite.scala which would fail with 
unexpected state timeout if the issue was not fixed.

Closes #35810 from alex-balikov/SPARK-38320-state-iterators2.

Authored-by: Alex Balikov 
Signed-off-by: Jungtaek Lim 
---
 .../streaming/FlatMapGroupsWithStateExec.scala | 20 +---
 .../streaming/state/StateStoreSuite.scala  | 58 ++
 .../streaming/FlatMapGroupsWithStateSuite.scala| 48 +-
 3 files changed, 118 insertions(+), 8 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala
index dfcb7073..ffacfef 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala
@@ -172,12 +172,20 @@ case class FlatMapGroupsWithStateExec(
   timeoutProcessingStartTimeNs = System.nanoTime
 })
 
-val timeoutProcessorIter =
-  CompletionIterator[InternalRow, 
Iterator[InternalRow]](processor.processTimedOutState(), {
-// Note: `timeoutLatencyMs` also includes the time the parent operator 
took for
-// processing output returned through iterator.
-timeoutLatencyMs += NANOSECONDS.toMillis(System.nanoTime - 
timeoutProcessingStartTimeNs)
-  })
+// SPARK-38320: Late-bind the timeout processing iterator so it is created 
*after* the input is
+// processed (the input iterator is exhausted) and the state updates are 
written into the
+// state store. Otherwise the iterator may not see the updates (e.g. with 
RocksDB state store).
+val timeoutProcessorIter = new Iterator[InternalRow] {
+  private lazy val itr = getIterator()
+  override def hasNext = itr.hasNext
+  override def next() = itr.next()
+  private def getIterator(): Iterator[InternalRow] =
+CompletionIterator[InternalRow, 
Iterator[InternalRow]](processor.processTimedOutState(), {
+  // Note: `timeoutLatencyMs` also includes the time the parent 
operator took for
+  // processing output returned through iterator.
+  timeoutLatencyMs += NANOSECONDS.toMillis(System.nanoTime - 
timeoutProcessingStartTimeNs)
+})
+}
 
 // Generate a iterator that returns the rows grouped by the grouping 
function
 // Note that this code ensures that the filtering for timeout occurs only 
after
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala
index 601b62b..dde925b 100644
--- 
a/sql/core/src/test

[spark] branch branch-3.1 updated: [SPARK-38412][SS] Fix the swapped sequence of from and to in StateSchemaCompatibilityChecker

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

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


The following commit(s) were added to refs/heads/branch-3.1 by this push:
 new 968dd47d [SPARK-38412][SS] Fix the swapped sequence of from and to in 
StateSchemaCompatibilityChecker
968dd47d is described below

commit 968dd47d9ecb52b0627a0c9caa815b55715429ab
Author: Jungtaek Lim 
AuthorDate: Wed Mar 9 11:03:57 2022 +0900

[SPARK-38412][SS] Fix the swapped sequence of from and to in 
StateSchemaCompatibilityChecker

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

This PR fixes the StateSchemaCompatibilityChecker which mistakenly swapped 
`from` (should be provided schema) and `to` (should be existing schema).

### Why are the changes needed?

The bug mistakenly allows the case where it should not be allowed, and 
disallows the case where it should be allowed.

That allows nullable column to be stored into non-nullable column, which 
should be prohibited. This is less likely making runtime problem since state 
schema is conceptual one and row can be stored even not respecting the state 
schema.

The opposite case is worse, that disallows non-nullable column to be stored 
into nullable column, which should be allowed. Spark fails the query for this 
case.

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

Yes, after the fix, storing non-nullable column into nullable column for 
state will be allowed, which should have been allowed.

### How was this patch tested?

Modified UTs.

Closes #35731 from HeartSaVioR/SPARK-38412.

Authored-by: Jungtaek Lim 
Signed-off-by: Jungtaek Lim 
(cherry picked from commit 43c7824bba40ebfb64dcd50d8d0e84b5a4d3c8c7)
Signed-off-by: Jungtaek Lim 
---
 .../state/StateSchemaCompatibilityChecker.scala|  2 +-
 .../StateSchemaCompatibilityCheckerSuite.scala | 51 ++
 2 files changed, 44 insertions(+), 9 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityChecker.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityChecker.scala
index 20625e1..0c8cabb 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityChecker.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityChecker.scala
@@ -72,7 +72,7 @@ class StateSchemaCompatibilityChecker(
   }
 
   private def schemasCompatible(storedSchema: StructType, schema: StructType): 
Boolean =
-DataType.equalsIgnoreNameAndCompatibleNullability(storedSchema, schema)
+DataType.equalsIgnoreNameAndCompatibleNullability(schema, storedSchema)
 
   // Visible for testing
   private[sql] def readSchemaFile(): (StructType, StructType) = {
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityCheckerSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityCheckerSuite.scala
index a9cc90c..1539341 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityCheckerSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityCheckerSuite.scala
@@ -63,6 +63,8 @@ class StateSchemaCompatibilityCheckerSuite extends 
SharedSparkSession {
   private val valueSchema65535Bytes = new StructType()
 .add(StructField("v" * (65535 - 87), IntegerType, nullable = true))
 
+  // Checks on adding/removing (nested) field.
+
   test("adding field to key should fail") {
 val fieldAddedKeySchema = keySchema.add(StructField("newKey", IntegerType))
 verifyException(keySchema, valueSchema, fieldAddedKeySchema, valueSchema)
@@ -107,6 +109,8 @@ class StateSchemaCompatibilityCheckerSuite extends 
SharedSparkSession {
 verifyException(keySchema, valueSchema, keySchema, newValueSchema)
   }
 
+  // Checks on changing type of (nested) field.
+
   test("changing the type of field in key should fail") {
 val typeChangedKeySchema = StructType(keySchema.map(_.copy(dataType = 
TimestampType)))
 verifyException(keySchema, valueSchema, typeChangedKeySchema, valueSchema)
@@ -129,28 +133,59 @@ class StateSchemaCompatibilityCheckerSuite extends 
SharedSparkSession {
 verifyException(keySchema, valueSchema, keySchema, newValueSchema)
   }
 
-  test("changing the nullability of nullable to non-nullable in key should 
fail") {
+  // Checks on changing nullability of (nested) field.
+  // Note that these tests have different format of the test name compared to 
others, since it was
+  // misleading to understand the assignment as the opposite way.
+
+  test("

[spark] branch branch-3.2 updated: [SPARK-38412][SS] Fix the swapped sequence of from and to in StateSchemaCompatibilityChecker

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

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


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new f1efc95  [SPARK-38412][SS] Fix the swapped sequence of from and to in 
StateSchemaCompatibilityChecker
f1efc95 is described below

commit f1efc955940176c9fa84cdf7b2c71563c5df47d2
Author: Jungtaek Lim 
AuthorDate: Wed Mar 9 11:03:57 2022 +0900

[SPARK-38412][SS] Fix the swapped sequence of from and to in 
StateSchemaCompatibilityChecker

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

This PR fixes the StateSchemaCompatibilityChecker which mistakenly swapped 
`from` (should be provided schema) and `to` (should be existing schema).

### Why are the changes needed?

The bug mistakenly allows the case where it should not be allowed, and 
disallows the case where it should be allowed.

That allows nullable column to be stored into non-nullable column, which 
should be prohibited. This is less likely making runtime problem since state 
schema is conceptual one and row can be stored even not respecting the state 
schema.

The opposite case is worse, that disallows non-nullable column to be stored 
into nullable column, which should be allowed. Spark fails the query for this 
case.

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

Yes, after the fix, storing non-nullable column into nullable column for 
state will be allowed, which should have been allowed.

### How was this patch tested?

Modified UTs.

Closes #35731 from HeartSaVioR/SPARK-38412.

Authored-by: Jungtaek Lim 
Signed-off-by: Jungtaek Lim 
(cherry picked from commit 43c7824bba40ebfb64dcd50d8d0e84b5a4d3c8c7)
Signed-off-by: Jungtaek Lim 
---
 .../state/StateSchemaCompatibilityChecker.scala|  2 +-
 .../StateSchemaCompatibilityCheckerSuite.scala | 51 ++
 2 files changed, 44 insertions(+), 9 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityChecker.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityChecker.scala
index 20625e1..0c8cabb 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityChecker.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityChecker.scala
@@ -72,7 +72,7 @@ class StateSchemaCompatibilityChecker(
   }
 
   private def schemasCompatible(storedSchema: StructType, schema: StructType): 
Boolean =
-DataType.equalsIgnoreNameAndCompatibleNullability(storedSchema, schema)
+DataType.equalsIgnoreNameAndCompatibleNullability(schema, storedSchema)
 
   // Visible for testing
   private[sql] def readSchemaFile(): (StructType, StructType) = {
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityCheckerSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityCheckerSuite.scala
index a9cc90c..1539341 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityCheckerSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityCheckerSuite.scala
@@ -63,6 +63,8 @@ class StateSchemaCompatibilityCheckerSuite extends 
SharedSparkSession {
   private val valueSchema65535Bytes = new StructType()
 .add(StructField("v" * (65535 - 87), IntegerType, nullable = true))
 
+  // Checks on adding/removing (nested) field.
+
   test("adding field to key should fail") {
 val fieldAddedKeySchema = keySchema.add(StructField("newKey", IntegerType))
 verifyException(keySchema, valueSchema, fieldAddedKeySchema, valueSchema)
@@ -107,6 +109,8 @@ class StateSchemaCompatibilityCheckerSuite extends 
SharedSparkSession {
 verifyException(keySchema, valueSchema, keySchema, newValueSchema)
   }
 
+  // Checks on changing type of (nested) field.
+
   test("changing the type of field in key should fail") {
 val typeChangedKeySchema = StructType(keySchema.map(_.copy(dataType = 
TimestampType)))
 verifyException(keySchema, valueSchema, typeChangedKeySchema, valueSchema)
@@ -129,28 +133,59 @@ class StateSchemaCompatibilityCheckerSuite extends 
SharedSparkSession {
 verifyException(keySchema, valueSchema, keySchema, newValueSchema)
   }
 
-  test("changing the nullability of nullable to non-nullable in key should 
fail") {
+  // Checks on changing nullability of (nested) field.
+  // Note that these tests have different format of the test name compared to 
others, since it was
+  // misleading to understand the assignment as the opposite way.
+
+  test("

[spark] branch master updated (59ce0a7 -> 43c7824)

2022-03-08 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 59ce0a7  [SPARK-37865][SQL] Fix union deduplication correctness bug
 add 43c7824  [SPARK-38412][SS] Fix the swapped sequence of from and to in 
StateSchemaCompatibilityChecker

No new revisions were added by this update.

Summary of changes:
 .../state/StateSchemaCompatibilityChecker.scala|  2 +-
 .../StateSchemaCompatibilityCheckerSuite.scala | 51 ++
 2 files changed, 44 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-38033][SS] The SS processing cannot be started because the com…

2022-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 6aa83e7  [SPARK-38033][SS] The SS processing cannot be started because 
the com…
6aa83e7 is described below

commit 6aa83e7a42555982ea80e16812bed65bc16617a3
Author: qitao liu 
AuthorDate: Mon Feb 28 23:48:12 2022 +0900

[SPARK-38033][SS] The SS processing cannot be started because the com…

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

The code of method: populateStartOffsets in class: 
org.apache.spark.sql.execution.streaming.MicroBatchExecution is modified.

### Why are the changes needed?

In some unexpected cases, commit and offset are inconsistent, and offset is 
not written into HDFS continuously, as follows:

commits
/tmp/streaming_/commits/113256
/tmp/streaming_/commits/113257

offsets
/tmp/streaming_/offsets/113257
/tmp/streaming_/offsets/113259
When we start the streaming program, batch ${latestBatchId - 1} is 113258, 
but offsets 113258 doesn't exist, an exception will be thrown, resulting in the 
program cannot be started. As an improvement, Spark doesn‘t need to repair 
itself, but we could probably do some simply analysis and give better error 
message.

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

Yes.
An error message is logged if the exception is thrown.

### How was this patch tested?

I have provided a test case that can output logs correctly.
We can run test("SPARK-38033: SS cannot be") in the 
MicroBatchExecutionSuite class. In fact, I simulated a corresponding scenario 
to test the original exception.This exception validates normally and outputs a 
new error message, as follows:

11:00:26.271 WARN 
org.apache.spark.sql.execution.streaming.ResolveWriteToStream: 
spark.sql.adaptive.enabled is not supported in streaming DataFrames/Datasets 
and will be disabled.
11:00:26.675 ERROR 
org.apache.spark.sql.execution.streaming.MicroBatchExecution: The offset log 
for batch 3 doesn't exist, which is required to restart the query from the 
latest batch 4 from the offset log. Please ensure there are two subsequent 
offset logs available for the latest batch via manually deleting the offset 
file(s). Please also ensure the latest batch for commit log is equal or one 
batch earlier than the latest batch for offset log.
11:00:26.690 ERROR 
org.apache.spark.sql.execution.streaming.MicroBatchExecution: Query [id = 
d4358946-170c-49a7-823b-d8e4e9126616, runId = 
9e7f12b8-6c10-4f36-b5c5-136e1bace8de] terminated with error
java.lang.IllegalStateException: batch 3 doesn't exist
at 
org.apache.spark.sql.execution.streaming.MicroBatchExecution.$anonfun$populateStartOffsets$1(MicroBatchExecution.scala:338)
 ~[classes/:?]
at scala.Option.getOrElse(Option.scala:189) 
~[scala-library-2.12.15.jar:?]
at 
org.apache.spark.sql.execution.streaming.MicroBatchExecution.populateStartOffsets(MicroBatchExecution.scala:331)
 ~[classes/:?]
at 
org.apache.spark.sql.execution.streaming.MicroBatchExecution.$anonfun$runActivatedStream$2(MicroBatchExecution.scala:222)
 ~[classes/:?]
at 
scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23) 
~[scala-library-2.12.15.jar:?]
at 
org.apache.spark.sql.execution.streaming.ProgressReporter.reportTimeTaken(ProgressReporter.scala:375)
 ~[classes/:?]
at 
org.apache.spark.sql.execution.streaming.ProgressReporter.reportTimeTaken$(ProgressReporter.scala:373)
 ~[classes/:?]

Authored-by: LLiu liuqt1024gmail.com

Closes #35513 from LLiu/SPARK-38033-m.

Authored-by: qitao liu 
Signed-off-by: Jungtaek Lim 
---
 .../execution/streaming/MicroBatchExecution.scala  |  6 +
 .../commits/0  |  2 ++
 .../commits/1  |  2 ++
 .../commits/2  |  2 ++
 .../metadata   |  1 +
 .../offsets/0  |  3 +++
 .../offsets/1  |  3 +++
 .../offsets/2  |  3 +++
 .../offsets/4  |  3 +++
 .../streaming/MicroBatchExecutionSuite.scala   | 29 +++---
 10 files changed, 51 insertions(+), 3 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala
index b5667ee..fb434f4 100644
--- 
a/sql/core/src/main/scala/org/a

[spark] branch master updated (9758d55 -> b8b1fbc)

2022-02-24 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 9758d55  [SPARK-38303][BUILD] Upgrade `ansi-regex` from 5.0.0 to 5.0.1 
in /dev
 add b8b1fbc  [SPARK-38275][SS] Include the writeBatch's memory usage as 
the total memory usage of RocksDB state store

No new revisions were added by this update.

Summary of changes:
 .../org/apache/spark/sql/execution/streaming/state/RocksDB.scala  | 8 ++--
 .../sql/execution/streaming/state/RocksDBStateStoreProvider.scala | 2 +-
 2 files changed, 7 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-38206][SS] Ignore nullability on comparing the data type of join keys on stream-stream join

2022-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 5ebf793  [SPARK-38206][SS] Ignore nullability on comparing the data 
type of join keys on stream-stream join
5ebf793 is described below

commit 5ebf7938b6882d343a6aa9e125f24bee394bb25f
Author: Jungtaek Lim 
AuthorDate: Tue Feb 22 15:58:07 2022 +0900

[SPARK-38206][SS] Ignore nullability on comparing the data type of join 
keys on stream-stream join

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

This PR proposes to change the assertion of data type against joining keys 
on stream-stream join to ignore nullability.

### Why are the changes needed?

The existing requirement on checking data types of joining keys is too 
restricted, as it also requires the same nullability. In batch query (I checked 
with HashJoinExec), nullability is ignored when checking data types of joining 
keys.

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

Yes, end users will no longer encounter the assertion error on join keys 
with different nullability in both keys.

### How was this patch tested?

New test added.

Closes #35599 from HeartSaVioR/SPARK-38206.

Authored-by: Jungtaek Lim 
Signed-off-by: Jungtaek Lim 
---
 .../streaming/StreamingSymmetricHashJoinExec.scala |   8 +-
 .../spark/sql/streaming/StreamingJoinSuite.scala   | 158 +
 2 files changed, 165 insertions(+), 1 deletion(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala
index adb84a3..81888e0 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala
@@ -174,7 +174,13 @@ case class StreamingSymmetricHashJoinExec(
 joinType == Inner || joinType == LeftOuter || joinType == RightOuter || 
joinType == FullOuter ||
 joinType == LeftSemi,
 errorMessageForJoinType)
-  require(leftKeys.map(_.dataType) == rightKeys.map(_.dataType))
+
+  // The assertion against join keys is same as hash join for batch query.
+  require(leftKeys.length == rightKeys.length &&
+leftKeys.map(_.dataType)
+  .zip(rightKeys.map(_.dataType))
+  .forall(types => types._1.sameType(types._2)),
+"Join keys from two sides should have same length and types")
 
   private val storeConf = new StateStoreConf(conf)
   private val hadoopConfBcast = sparkContext.broadcast(
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala
index e0926ef..2fbe6c4 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala
@@ -18,6 +18,7 @@
 package org.apache.spark.sql.streaming
 
 import java.io.File
+import java.lang.{Integer => JInteger}
 import java.sql.Timestamp
 import java.util.{Locale, UUID}
 
@@ -702,6 +703,53 @@ class StreamingInnerJoinSuite extends StreamingJoinSuite {
 total = Seq(2), updated = Seq(1), droppedByWatermark = Seq(0), removed 
= Some(Seq(0)))
 )
   }
+
+  test("joining non-nullable left join key with nullable right join key") {
+val input1 = MemoryStream[Int]
+val input2 = MemoryStream[JInteger]
+
+val joined = testForJoinKeyNullability(input1.toDF(), input2.toDF())
+testStream(joined)(
+  AddData(input1, 1, 5),
+  AddData(input2, JInteger.valueOf(1), JInteger.valueOf(5), 
JInteger.valueOf(10), null),
+  CheckNewAnswer(Row(1, 1, 2, 3), Row(5, 5, 10, 15))
+)
+  }
+
+  test("joining nullable left join key with non-nullable right join key") {
+val input1 = MemoryStream[JInteger]
+val input2 = MemoryStream[Int]
+
+val joined = testForJoinKeyNullability(input1.toDF(), input2.toDF())
+testStream(joined)(
+  AddData(input1, JInteger.valueOf(1), JInteger.valueOf(5), 
JInteger.valueOf(10), null),
+  AddData(input2, 1, 5),
+  CheckNewAnswer(Row(1, 1, 2, 3), Row(5, 5, 10, 15))
+)
+  }
+
+  test("joining nullable left join key with nullable right join key") {
+val input1 = MemoryStream[JInteger]
+val input2 = MemoryStream[JInteger]
+
+val joined = testForJoinKeyNullability(input1.toDF(), input2.toDF())
+testStream(joined)(
+  AddData(input1, JInteger.valueOf(1), JInteger.valueOf(5), 
JInteger.valueOf(10), null),
+  AddData(input2, JInteger.valueOf(1), JInteger.valueOf(5), null),

[spark] branch master updated (f82cf9e -> e86c8e2)

2022-02-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 f82cf9e  [SPARK-34378][SQL][AVRO] Loosen AvroSerializer validation to 
allow extra nullable user-provided fields
 add e86c8e2  [SPARK-38214][SS] No need to filter windows when 
windowDuration is multiple of slideDuration

No new revisions were added by this update.

Summary of changes:
 .../spark/sql/catalyst/analysis/Analyzer.scala | 11 --
 .../spark/sql/DataFrameTimeWindowingSuite.scala| 39 +-
 2 files changed, 47 insertions(+), 3 deletions(-)

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



[spark] branch SPARK-38237 created (now ae1ebe0)

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

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


  at ae1ebe0  [SPARK-38237][SQL] Rename back 
StatefulOpClusteredDistribution to HashClusteredDistribution

No new revisions were added by this update.

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



[spark] branch master updated (ece34f0 -> 39166ed)

2022-02-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 ece34f0  [SPARK-38130][SQL] Remove array_sort orderable entries check
 add 39166ed  [SPARK-38124][SS][FOLLOWUP] Add test to harden assumption of 
SS partitioning requirement

No new revisions were added by this update.

Summary of changes:
 .../sql/catalyst/plans/physical/partitioning.scala |  2 +
 .../spark/sql/catalyst/DistributionSuite.scala | 78 ++
 2 files changed, 80 insertions(+)

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



[spark] branch master updated: [SPARK-38124][SS][FOLLOWUP] Document the current challenge on fixing distribution of stateful operator

2022-02-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 0be132c  [SPARK-38124][SS][FOLLOWUP] Document the current challenge on 
fixing distribution of stateful operator
0be132c is described below

commit 0be132c128e80bc9d866001a64cb3f6331c85b1e
Author: Jungtaek Lim 
AuthorDate: Tue Feb 15 11:47:42 2022 +0900

[SPARK-38124][SS][FOLLOWUP] Document the current challenge on fixing 
distribution of stateful operator

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

This PR proposes to add the context of current challenge on fixing 
distribution of stateful operator, even the distribution is a sort of "broken" 
now.

This PR addresses the review comment 
https://github.com/apache/spark/pull/35419#discussion_r801343068

### Why are the changes needed?

In SPARK-38124 we figured out the existing long-standing problem in 
stateful operator, but it is not easy to fix since the fix may break the 
existing query if the fix is not carefully designed. Anyone should also be 
pretty much careful when touching the required distribution. We want to 
document this explicitly to help others to be careful whenever someone is 
around the codebase.

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

No.

### How was this patch tested?

Code comment only changes.

Closes #35512 from HeartSaVioR/SPARK-38124-followup.

Authored-by: Jungtaek Lim 
Signed-off-by: Jungtaek Lim 
---
 .../sql/catalyst/plans/physical/partitioning.scala |  8 
 .../streaming/FlatMapGroupsWithStateExec.scala |  3 +++
 .../sql/execution/streaming/statefulOperators.scala| 18 +-
 3 files changed, 28 insertions(+), 1 deletion(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
index 4418d32..5342c8e 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
@@ -101,6 +101,14 @@ case class ClusteredDistribution(
  * Since this distribution relies on [[HashPartitioning]] on the physical 
partitioning of the
  * stateful operator, only [[HashPartitioning]] (and HashPartitioning in
  * [[PartitioningCollection]]) can satisfy this distribution.
+ *
+ * NOTE: This is applied only to stream-stream join as of now. For other 
stateful operators, we
+ * have been using ClusteredDistribution, which could construct the physical 
partitioning of the
+ * state in different way (ClusteredDistribution requires relaxed condition 
and multiple
+ * partitionings can satisfy the requirement.) We need to construct the way to 
fix this with
+ * minimizing possibility to break the existing checkpoints.
+ *
+ * TODO(SPARK-38204): address the issue explained in above note.
  */
 case class StatefulOpClusteredDistribution(
 expressions: Seq[Expression],
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala
index a00a622..93ed591 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala
@@ -93,6 +93,9 @@ case class FlatMapGroupsWithStateExec(
* to have the same grouping so that the data are co-lacated on the same 
task.
*/
   override def requiredChildDistribution: Seq[Distribution] = {
+// NOTE: Please read through the NOTE on the classdoc of 
StatefulOpClusteredDistribution
+// before making any changes.
+// TODO(SPARK-38204)
 ClusteredDistribution(groupingAttributes, stateInfo.map(_.numPartitions)) 
::
 ClusteredDistribution(initialStateGroupAttrs, 
stateInfo.map(_.numPartitions)) ::
   Nil
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 3431823..3ab2ad4 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
@@ -334,6 +334,9 @@ case class StateStoreRestoreExec(
   override def outputPartitioning: Partitioning = child.outputPartitioning
 
   override def requiredChildDistribution: Seq[Distribution] = {
+// NOTE: Please read through the NOTE on the classdoc of 
StatefulOpClusteredDistribution
+// bef

[spark] branch master updated: [SPARK-38056][WEB UI] Fix issue of Structured streaming not working in history server when using LevelDB

2022-02-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 dee294b  [SPARK-38056][WEB UI] Fix issue of Structured streaming not 
working in history server when using LevelDB
dee294b is described below

commit dee294b453b550471028fdbd9e17952963504a3a
Author: kuwii 
AuthorDate: Wed Feb 9 16:59:38 2022 +0900

[SPARK-38056][WEB UI] Fix issue of Structured streaming not working in 
history server when using LevelDB

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

Change type of `org.apache.spark.sql.streaming.ui.StreamingQueryData.runId` 
from `UUID` to `String`.

### Why are the changes needed?

In 
[SPARK-31953](https://github.com/apache/spark/commit/4f9667035886a67e6c9a4e8fad2efa390e87ca68),
 structured streaming support is added in history server. However this does not 
work when history server is using LevelDB instead of in-memory KV store.

- Level DB does not support `UUID` as key.
- If `spark.history.store.path` is set in history server to use Level DB, 
when writing info to the store during replaying events, error will occur.
- `StreamingQueryStatusListener` will throw exceptions when writing info, 
saying `java.lang.IllegalArgumentException: Type java.util.UUID not allowed as 
key.`.

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

No.

### How was this patch tested?

Added tests in `StreamingQueryStatusListenerSuite` to test whether 
`StreamingQueryData` can be successfully written to in-memory store,  LevelDB 
and RocksDB.

Closes #35356 from kuwii/hs-streaming-fix.

Authored-by: kuwii 
Signed-off-by: Jungtaek Lim 
---
 .../ui/StreamingQueryStatisticsPage.scala  |  4 +-
 .../ui/StreamingQueryStatusListener.scala  |  6 +-
 .../sql/streaming/ui/StreamingQueryPageSuite.scala |  2 +-
 .../ui/StreamingQueryStatusListenerSuite.scala | 64 +++---
 4 files changed, 62 insertions(+), 14 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
index 97691d9..e13ac4e 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.streaming.ui
 
 import java.{util => ju}
 import java.lang.{Long => JLong}
-import java.util.{Locale, UUID}
+import java.util.Locale
 import javax.servlet.http.HttpServletRequest
 
 import scala.collection.JavaConverters._
@@ -59,7 +59,7 @@ private[ui] class StreamingQueryStatisticsPage(parent: 
StreamingQueryTab)
 require(parameterId != null && parameterId.nonEmpty, "Missing id 
parameter")
 
 val query = parent.store.allQueryUIData.find { uiData =>
-  uiData.summary.runId.equals(UUID.fromString(parameterId))
+  uiData.summary.runId.equals(parameterId)
 }.getOrElse(throw new IllegalArgumentException(s"Failed to find streaming 
query $parameterId"))
 
 val resources = generateLoadResources(request)
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListener.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListener.scala
index fdd3754..b59ec04 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListener.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListener.scala
@@ -75,7 +75,7 @@ private[sql] class StreamingQueryStatusListener(
 store.write(new StreamingQueryData(
   event.name,
   event.id,
-  event.runId,
+  event.runId.toString,
   isActive = true,
   None,
   startTimestamp
@@ -100,7 +100,7 @@ private[sql] class StreamingQueryStatusListener(
 
   override def onQueryTerminated(
   event: StreamingQueryListener.QueryTerminatedEvent): Unit = {
-val querySummary = store.read(classOf[StreamingQueryData], event.runId)
+val querySummary = store.read(classOf[StreamingQueryData], 
event.runId.toString)
 val curTime = System.currentTimeMillis()
 store.write(new StreamingQueryData(
   querySummary.name,
@@ -118,7 +118,7 @@ private[sql] class StreamingQueryStatusListener(
 private[sql] class StreamingQueryData(
 val name: String,
 val id: UUID,
-@KVIndexParam val runId: UUID,
+@KVIndexParam val runId: String,
 @KVIndexParam("active") val isActive: Boolean,
 val exception: Option[String],
 @KVIndexParam("startTimestamp") val startTimestamp: Long,
diff --git 
a/

[spark] branch master updated (5b02a34 -> 43cce92)

2022-02-08 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 5b02a34  [SPARK-38142][SQL][TESTS] Move `ArrowColumnVectorSuite` to 
`org.apache.spark.sql.vectorized`
 add 43cce92  [SPARK-38124][SQL][SS] Introduce 
StatefulOpClusteredDistribution and apply to stream-stream join

No new revisions were added by this update.

Summary of changes:
 .../sql/catalyst/plans/physical/partitioning.scala | 40 ++
 .../streaming/StreamingSymmetricHashJoinExec.scala |  4 +--
 .../spark/sql/streaming/StreamingJoinSuite.scala   |  2 +-
 3 files changed, 43 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 (c69f08f8 -> 899d3bb)

2022-02-08 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 c69f08f8 [SPARK-34826][SHUFFLE] Adaptively fetch shuffle mergers for 
push based shuffle
 add 899d3bb  [SPARK-34183][SS] DataSource V2: Required distribution and 
ordering in micro-batch execution

No new revisions were added by this update.

Summary of changes:
 .../write/RequiresDistributionAndOrdering.java |  10 +
 .../spark/sql/errors/QueryCompilationErrors.scala  |   5 +
 .../sql/connector/catalog/InMemoryTable.scala  |   5 +-
 .../sql/execution/datasources/v2/V2Writes.scala|  46 +++-
 .../execution/streaming/MicroBatchExecution.scala  |  11 +-
 .../sql/execution/streaming/StreamExecution.scala  |  12 +-
 .../streaming/continuous/ContinuousExecution.scala |  33 ++-
 .../sources/WriteToMicroBatchDataSource.scala  |  20 +-
 .../WriteDistributionAndOrderingSuite.scala| 294 -
 9 files changed, 403 insertions(+), 33 deletions(-)

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



[spark] branch branch-3.2 updated: [SPARK-38046][SS][TEST] Fix KafkaSource/KafkaMicroBatch flaky test due to non-deterministic timing

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

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


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new 34c474a  [SPARK-38046][SS][TEST] Fix KafkaSource/KafkaMicroBatch flaky 
test due to non-deterministic timing
34c474a is described below

commit 34c474a86a08333556f4e05990989fc2156f3f79
Author: Jerry Peng 
AuthorDate: Mon Feb 7 17:45:26 2022 +0900

[SPARK-38046][SS][TEST] Fix KafkaSource/KafkaMicroBatch flaky test due to 
non-deterministic timing

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

Fix a flaky test in KafkaMicroBatchSourceSuite

### Why are the changes needed?

There is a test call "compositeReadLimit"


https://github.com/apache/spark/blob/master/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala#L460

that is flaky.  The problem is because the Kakfa connector is always 
getting the actual system time and not advancing it manually, thus leaving room 
for non-deterministic behaviors especially since the source determines if 
"maxTriggerDelayMs" is satisfied by comparing the last trigger time with the 
current system time.  One can simply "sleep" at points in the test to generate 
different outcomes.

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

no

### How was this patch tested?

Closes #35343 from jerrypeng/SPARK-38046.

Authored-by: Jerry Peng 
Signed-off-by: Jungtaek Lim 
(cherry picked from commit e98f13e4a4eaf4719e85ba881c894cbc8377c363)
Signed-off-by: Jungtaek Lim 
---
 .../spark/sql/kafka010/KafkaMicroBatchStream.scala | 37 +++---
 .../apache/spark/sql/kafka010/KafkaSource.scala| 15 ++---
 .../spark/sql/kafka010/KafkaSourceProvider.scala   |  2 ++
 .../sql/kafka010/KafkaMicroBatchSourceSuite.scala  | 18 ---
 4 files changed, 59 insertions(+), 13 deletions(-)

diff --git 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala
 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala
index 3b73896..a50e63a 100644
--- 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala
+++ 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala
@@ -31,8 +31,9 @@ import org.apache.spark.sql.SparkSession
 import org.apache.spark.sql.connector.read.{InputPartition, 
PartitionReaderFactory}
 import org.apache.spark.sql.connector.read.streaming._
 import org.apache.spark.sql.kafka010.KafkaSourceProvider._
+import org.apache.spark.sql.kafka010.MockedSystemClock.currentMockSystemTime
 import org.apache.spark.sql.util.CaseInsensitiveStringMap
-import org.apache.spark.util.{UninterruptibleThread, Utils}
+import org.apache.spark.util.{Clock, ManualClock, SystemClock, 
UninterruptibleThread, Utils}
 
 /**
  * A [[MicroBatchStream]] that reads data from Kafka.
@@ -73,6 +74,13 @@ private[kafka010] class KafkaMicroBatchStream(
 Utils.timeStringAsMs(Option(options.get(
   
KafkaSourceProvider.MAX_TRIGGER_DELAY)).getOrElse(DEFAULT_MAX_TRIGGER_DELAY))
 
+  // this allows us to mock system clock for testing purposes
+  private[kafka010] val clock: Clock = if 
(options.containsKey(MOCK_SYSTEM_TIME)) {
+new MockedSystemClock
+  } else {
+new SystemClock
+  }
+
   private var lastTriggerMillis = 0L
 
   private val includeHeaders = options.getBoolean(INCLUDE_HEADERS, false)
@@ -157,9 +165,9 @@ private[kafka010] class KafkaMicroBatchStream(
   currentOffsets: Map[TopicPartition, Long],
   maxTriggerDelayMs: Long): Boolean = {
 // Checking first if the maxbatchDelay time has passed
-if ((System.currentTimeMillis() - lastTriggerMillis) >= maxTriggerDelayMs) 
{
+if ((clock.getTimeMillis() - lastTriggerMillis) >= maxTriggerDelayMs) {
   logDebug("Maximum wait time is passed, triggering batch")
-  lastTriggerMillis = System.currentTimeMillis()
+  lastTriggerMillis = clock.getTimeMillis()
   false
 } else {
   val newRecords = latestOffsets.flatMap {
@@ -167,7 +175,7 @@ private[kafka010] class KafkaMicroBatchStream(
   Some(topic -> (offset - currentOffsets.getOrElse(topic, 0L)))
   }.values.sum.toDouble
   if (newRecords < minLimit) true else {
-lastTriggerMillis = System.currentTimeMillis()
+lastTriggerMillis = clock.getTimeMillis()
 false
   }
 }
@@ -333,3 +341,24 @@ object KafkaMicroBatchStream extends Logging {
 ju.Collections.emptyMap()
   }
 }
+
+/**
+ * To return a mocked system clock for testing purposes
+ */
+private[kafka010] class MockedSystemClock extends ManualClock {
+  override def getTimeMillis(): Long = {
+curre

[spark] branch master updated: [SPARK-38046][SS][TEST] Fix KafkaSource/KafkaMicroBatch flaky test due to non-deterministic timing

2022-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 e98f13e  [SPARK-38046][SS][TEST] Fix KafkaSource/KafkaMicroBatch flaky 
test due to non-deterministic timing
e98f13e is described below

commit e98f13e4a4eaf4719e85ba881c894cbc8377c363
Author: Jerry Peng 
AuthorDate: Mon Feb 7 17:45:26 2022 +0900

[SPARK-38046][SS][TEST] Fix KafkaSource/KafkaMicroBatch flaky test due to 
non-deterministic timing

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

Fix a flaky test in KafkaMicroBatchSourceSuite

### Why are the changes needed?

There is a test call "compositeReadLimit"


https://github.com/apache/spark/blob/master/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala#L460

that is flaky.  The problem is because the Kakfa connector is always 
getting the actual system time and not advancing it manually, thus leaving room 
for non-deterministic behaviors especially since the source determines if 
"maxTriggerDelayMs" is satisfied by comparing the last trigger time with the 
current system time.  One can simply "sleep" at points in the test to generate 
different outcomes.

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

no

### How was this patch tested?

Closes #35343 from jerrypeng/SPARK-38046.

Authored-by: Jerry Peng 
Signed-off-by: Jungtaek Lim 
---
 .../spark/sql/kafka010/KafkaMicroBatchStream.scala | 37 +++---
 .../apache/spark/sql/kafka010/KafkaSource.scala| 15 ++---
 .../spark/sql/kafka010/KafkaSourceProvider.scala   |  2 ++
 .../sql/kafka010/KafkaMicroBatchSourceSuite.scala  | 18 ---
 4 files changed, 59 insertions(+), 13 deletions(-)

diff --git 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala
 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala
index 829ee15..77bc658 100644
--- 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala
+++ 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala
@@ -31,8 +31,9 @@ import org.apache.spark.sql.SparkSession
 import org.apache.spark.sql.connector.read.{InputPartition, 
PartitionReaderFactory}
 import org.apache.spark.sql.connector.read.streaming._
 import org.apache.spark.sql.kafka010.KafkaSourceProvider._
+import org.apache.spark.sql.kafka010.MockedSystemClock.currentMockSystemTime
 import org.apache.spark.sql.util.CaseInsensitiveStringMap
-import org.apache.spark.util.{UninterruptibleThread, Utils}
+import org.apache.spark.util.{Clock, ManualClock, SystemClock, 
UninterruptibleThread, Utils}
 
 /**
  * A [[MicroBatchStream]] that reads data from Kafka.
@@ -73,6 +74,13 @@ private[kafka010] class KafkaMicroBatchStream(
 Utils.timeStringAsMs(Option(options.get(
   
KafkaSourceProvider.MAX_TRIGGER_DELAY)).getOrElse(DEFAULT_MAX_TRIGGER_DELAY))
 
+  // this allows us to mock system clock for testing purposes
+  private[kafka010] val clock: Clock = if 
(options.containsKey(MOCK_SYSTEM_TIME)) {
+new MockedSystemClock
+  } else {
+new SystemClock
+  }
+
   private var lastTriggerMillis = 0L
 
   private val includeHeaders = options.getBoolean(INCLUDE_HEADERS, false)
@@ -166,9 +174,9 @@ private[kafka010] class KafkaMicroBatchStream(
   currentOffsets: Map[TopicPartition, Long],
   maxTriggerDelayMs: Long): Boolean = {
 // Checking first if the maxbatchDelay time has passed
-if ((System.currentTimeMillis() - lastTriggerMillis) >= maxTriggerDelayMs) 
{
+if ((clock.getTimeMillis() - lastTriggerMillis) >= maxTriggerDelayMs) {
   logDebug("Maximum wait time is passed, triggering batch")
-  lastTriggerMillis = System.currentTimeMillis()
+  lastTriggerMillis = clock.getTimeMillis()
   false
 } else {
   val newRecords = latestOffsets.flatMap {
@@ -176,7 +184,7 @@ private[kafka010] class KafkaMicroBatchStream(
   Some(topic -> (offset - currentOffsets.getOrElse(topic, 0L)))
   }.values.sum.toDouble
   if (newRecords < minLimit) true else {
-lastTriggerMillis = System.currentTimeMillis()
+lastTriggerMillis = clock.getTimeMillis()
 false
   }
 }
@@ -347,3 +355,24 @@ object KafkaMicroBatchStream extends Logging {
 ju.Collections.emptyMap()
   }
 }
+
+/**
+ * To return a mocked system clock for testing purposes
+ */
+private[kafka010] class MockedSystemClock extends ManualClock {
+  override def getTimeMillis(): Long = {
+currentMockSystemTime
+  }
+}
+
+private[kafka010] object MockedSystemClock {
+  var currentMockSystemTime = 0L
+
+  def a

[spark] branch branch-3.2 updated: [SPARK-38045][SS][TEST][3.2] More strict validation on plan check for stream-stream join unit test

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

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


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new 89a988c  [SPARK-38045][SS][TEST][3.2] More strict validation on plan 
check for stream-stream join unit test
89a988c is described below

commit 89a988cbbbc02acdb7614e7da6d87d19f2c938ea
Author: Jungtaek Lim 
AuthorDate: Fri Jan 28 16:09:04 2022 +0900

[SPARK-38045][SS][TEST][3.2] More strict validation on plan check for 
stream-stream join unit test

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

This PR is a follow-up of SPARK-35693 to enhance the unit test on 
stream-stream join to be more strict on plan check.

### Why are the changes needed?

We would like to be more strict on plan check so that requirement of 
distribution against stream-stream join is fulfilled.

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

No, test only.

### How was this patch tested?

Modified test passed.

Closes #35347 from HeartSaVioR/SPARK-38045-branch-3.2.

Authored-by: Jungtaek Lim 
Signed-off-by: Jungtaek Lim 
---
 .../apache/spark/sql/streaming/StreamingJoinSuite.scala  | 16 +++-
 1 file changed, 15 insertions(+), 1 deletion(-)

diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala
index 8c830d3..5ec47bb 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala
@@ -28,6 +28,8 @@ import org.scalatest.BeforeAndAfter
 
 import org.apache.spark.scheduler.ExecutorCacheTaskLocation
 import org.apache.spark.sql.{DataFrame, Row, SparkSession}
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, 
Expression}
+import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
 import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec
 import org.apache.spark.sql.execution.streaming.{MemoryStream, 
StatefulOperatorStateInfo, StreamingSymmetricHashJoinExec, 
StreamingSymmetricHashJoinHelper}
 import org.apache.spark.sql.execution.streaming.state.{StateStore, 
StateStoreProviderId}
@@ -583,9 +585,21 @@ class StreamingInnerJoinSuite extends StreamingJoinSuite {
   CheckAnswer(1.to(1000): _*),
   Execute { query =>
 // Verify the query plan
+def partitionExpressionsColumns(expressions: Seq[Expression]): 
Seq[String] = {
+  expressions.flatMap {
+case ref: AttributeReference => Some(ref.name)
+  }
+}
+
+val numPartitions = 
spark.sqlContext.conf.getConf(SQLConf.SHUFFLE_PARTITIONS)
+
 assert(query.lastExecution.executedPlan.collect {
   case j @ StreamingSymmetricHashJoinExec(_, _, _, _, _, _, _, _,
-_: ShuffleExchangeExec, ShuffleExchangeExec(_, _: 
ShuffleExchangeExec, _)) => j
+ShuffleExchangeExec(opA: HashPartitioning, _, _),
+ShuffleExchangeExec(opB: HashPartitioning, _, _))
+  if partitionExpressionsColumns(opA.expressions) === Seq("a", "b")
+&& partitionExpressionsColumns(opB.expressions) === Seq("a", 
"b")
+&& opA.numPartitions == numPartitions && opB.numPartitions == 
numPartitions => j
 }.size == 1)
   })
   }

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



[spark] branch branch-3.2 updated: [SPARK-37987][SS] Fix flaky test StreamingAggregationSuite.changing schema of state when restarting query

2022-01-24 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new 988439d  [SPARK-37987][SS] Fix flaky test 
StreamingAggregationSuite.changing schema of state when restarting query
988439d is described below

commit 988439d7287482c465f7da6c8e9c14303488158f
Author: Jungtaek Lim 
AuthorDate: Mon Jan 24 17:33:24 2022 +0900

[SPARK-37987][SS] Fix flaky test StreamingAggregationSuite.changing schema 
of state when restarting query

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

This PR fixes a flaky test `StreamingAggregationSuite.changing schema of 
state when restarting query`, via adjusting the number of shuffle partition to 
1.

The flakiness was due to the optimization on schema verification - we only 
verify it in partition 0 since it is costly and redundant to verify the schema 
for all partitions. Other partitions are still possible to provide other errors 
which are considered as unexpected.

### Why are the changes needed?

This PR fixes a flaky test.

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

No.

### How was this patch tested?

Ran test suite 10 times locally.

Closes #35298 from HeartSaVioR/SPARK-37987.

Authored-by: Jungtaek Lim 
Signed-off-by: Jungtaek Lim 
(cherry picked from commit 3b540ad822a53a8cb94159dc8aa3c66d34085e3e)
Signed-off-by: Jungtaek Lim 
---
 .../spark/sql/streaming/StreamingAggregationSuite.scala  | 12 ++--
 1 file changed, 10 insertions(+), 2 deletions(-)

diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala
index 77334ad..8a7bb8b 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala
@@ -766,7 +766,11 @@ class StreamingAggregationSuite extends 
StateStoreMetricsTest with Assertions {
   }
 
   testQuietlyWithAllStateVersions("changing schema of state when restarting 
query",
-(SQLConf.STATE_STORE_FORMAT_VALIDATION_ENABLED.key, "false")) {
+(SQLConf.STATE_STORE_FORMAT_VALIDATION_ENABLED.key, "false"),
+// Since we only do the check in partition 0 and other partitions still 
may fail with
+// different errors, we change the number of shuffle partitions to 1 to 
make the test
+// result to be deterministic.
+(SQLConf.SHUFFLE_PARTITIONS.key, "1")) {
 withTempDir { tempDir =>
   val (inputData, aggregated) = 
prepareTestForChangingSchemaOfState(tempDir)
 
@@ -790,7 +794,11 @@ class StreamingAggregationSuite extends 
StateStoreMetricsTest with Assertions {
   testQuietlyWithAllStateVersions("changing schema of state when restarting 
query -" +
 " schema check off",
 (SQLConf.STATE_SCHEMA_CHECK_ENABLED.key, "false"),
-(SQLConf.STATE_STORE_FORMAT_VALIDATION_ENABLED.key, "false")) {
+(SQLConf.STATE_STORE_FORMAT_VALIDATION_ENABLED.key, "false"),
+// Since we only do the check in partition 0 and other partitions still 
may fail with
+// different errors, we change the number of shuffle partitions to 1 to 
make the test
+// result to be deterministic.
+(SQLConf.SHUFFLE_PARTITIONS.key, "1")) {
 withTempDir { tempDir =>
   val (inputData, aggregated) = 
prepareTestForChangingSchemaOfState(tempDir)
 

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



[spark] branch master updated (47276ab -> 3b540ad)

2022-01-24 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 47276ab  [SPARK-37990][SQL] Support TimestampNTZ in 
RowToColumnConverter
 add 3b540ad  [SPARK-37987][SS] Fix flaky test 
StreamingAggregationSuite.changing schema of state when restarting query

No new revisions were added by this update.

Summary of changes:
 .../spark/sql/streaming/StreamingAggregationSuite.scala  | 12 ++--
 1 file changed, 10 insertions(+), 2 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-37224][SS][FOLLOWUP] Clarify the guide doc and fix the method doc

2021-11-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 b14fc2d  [SPARK-37224][SS][FOLLOWUP] Clarify the guide doc and fix the 
method doc
b14fc2d is described below

commit b14fc2d018dc948dc48579c88748d8af34d549e2
Author: Jungtaek Lim 
AuthorDate: Fri Nov 19 11:00:35 2021 +0900

[SPARK-37224][SS][FOLLOWUP] Clarify the guide doc and fix the method doc

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

This PR is a follow-up of #34502 to address post-reviews.

This PR rewords on the explanation on performance tune on RocksDB state 
store to make it less confused, and also fix the method docs to be in sync with 
the code changes.

### Why are the changes needed?

1. The explanation on performance tune on RocksDB state store was unclear 
in a couple of spots.
2. We changed the method signature, but the change was not reflected to the 
method doc.

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

Yes, end users will get less confused from the explanation on performance 
tune on RocksDB state store.

### How was this patch tested?

N/A

Closes #34652 from HeartSaVioR/SPARK-37224-follow-up-postreview.

Authored-by: Jungtaek Lim 
Signed-off-by: Jungtaek Lim 
---
 docs/structured-streaming-programming-guide.md   | 5 +++--
 .../org/apache/spark/sql/execution/streaming/state/RocksDB.scala | 4 ++--
 2 files changed, 5 insertions(+), 4 deletions(-)

diff --git a/docs/structured-streaming-programming-guide.md 
b/docs/structured-streaming-programming-guide.md
index 9547d46..a53adde 100644
--- a/docs/structured-streaming-programming-guide.md
+++ b/docs/structured-streaming-programming-guide.md
@@ -1965,9 +1965,10 @@ Here are the configs regarding to RocksDB instance of 
the state store provider:
 
 # Performance-aspect considerations
 
-1. For write-heavy workloads, you may want to disable the track of total 
number of rows.
+1. You may want to disable the track of total number of rows to aim the better 
performance on RocksDB state store.
+
+Tracking the number of rows brings additional lookup on write operations - 
you're encouraged to try turning off the config on tuning RocksDB state store, 
especially the values of metrics for state operator are big - `numRowsUpdated`, 
`numRowsRemoved`.
 
-Tracking the number of rows brings additional lookup on write operations - for 
heavy-write workloads you're encouraged to turn off the config.
 You can change the config during restarting the query, which enables you to 
change the trade-off decision on "observability vs performance".
 If the config is disabled, the number of rows in state (`numTotalStateRows`) 
will be reported as 0.
 
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 cb31945..ea25342 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
@@ -153,7 +153,7 @@ class RocksDB(
   }
 
   /**
-   * Put the given value for the given key and return the last written value.
+   * Put the given value for the given key.
* @note This update is not committed to disk until commit() is called.
*/
   def put(key: Array[Byte], value: Array[Byte]): Unit = {
@@ -167,7 +167,7 @@ class RocksDB(
   }
 
   /**
-   * Remove the key if present, and return the previous value if it was 
present (null otherwise).
+   * Remove the key if present.
* @note This update is not committed to disk until commit() is called.
*/
   def remove(key: Array[Byte]): Unit = {

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



[spark] branch master updated (99d2d57 -> 1c26113)

2021-11-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 99d2d57  [SPARK-37368][SQL][TESTS] Explicit GC for TPC-DS query runs
 add 1c26113  [SPARK-37224][SS] Optimize write path on RocksDB state store 
provider

No new revisions were added by this update.

Summary of changes:
 docs/structured-streaming-programming-guide.md | 13 
 .../sql/execution/streaming/state/RocksDB.scala| 76 +-
 .../state/RocksDBStateStoreIntegrationSuite.scala  | 39 ++-
 .../execution/streaming/state/RocksDBSuite.scala   | 60 +
 4 files changed, 171 insertions(+), 17 deletions(-)

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



[spark] branch master updated (bb9e1d9 -> f43d8b5)

2021-11-14 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 bb9e1d9  [SPARK-37319][K8S] Support K8s image building with Java 17
 add f43d8b5  [SPARK-36533][SS][FOLLOWUP] Address Trigger.AvailableNow in 
PySpark in SS guide doc

No new revisions were added by this update.

Summary of changes:
 docs/structured-streaming-programming-guide.md | 6 ++
 1 file changed, 6 insertions(+)

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



[spark] branch master updated: [SPARK-36533][SS][FOLLOWUP] Support Trigger.AvailableNow in PySpark

2021-11-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 edbc7cf  [SPARK-36533][SS][FOLLOWUP] Support Trigger.AvailableNow in 
PySpark
edbc7cf is described below

commit edbc7cf9e00233b35c057c357bf1c6b99f2ba59b
Author: Jungtaek Lim 
AuthorDate: Mon Nov 15 08:59:07 2021 +0900

[SPARK-36533][SS][FOLLOWUP] Support Trigger.AvailableNow in PySpark

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

This PR proposes to add Trigger.AvailableNow in PySpark on top of #33763.

### Why are the changes needed?

We missed adding Trigger.AvailableNow in PySpark in #33763.

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

Yes, Trigger.AvailableNow will be available in PySpark as well.

### How was this patch tested?

Added simple validation in PySpark doc. Manually tested as below:

```
>>> 
spark.readStream.format("text").load("/WorkArea/ScalaProjects/spark-apache/dist/inputs").writeStream.format("console").trigger(once=True).start()

---
Batch: 0
---
+-+
|value|
+-+
|a|
|b|
|c|
|d|
|e|
+-+

>>> 
spark.readStream.format("text").load("/WorkArea/ScalaProjects/spark-apache/dist/inputs").writeStream.format("console").trigger(availableNow=True).start()

>>> ---
Batch: 0
---
+-+
|value|
+-+
|a|
|b|
|c|
|d|
|e|
+-+

>>> spark.readStream.format("text").option("maxfilespertrigger", 
"2").load("/WorkArea/ScalaProjects/spark-apache/dist/inputs").writeStream.format("console").trigger(availableNow=True).start()

>>> ---
Batch: 0
---
+-+
|value|
+-+
|a|
|b|
+-+

---
Batch: 1
---
+-+
|value|
+-+
|c|
|d|
+-+

---
Batch: 2
---
+-+
|value|
+-+
|e|
+-+

>>>
```

Closes #34592 from HeartSaVioR/SPARK-36533-FOLLOWUP-pyspark.

Authored-by: Jungtaek Lim 
Signed-off-by: Jungtaek Lim 
---
 python/pyspark/sql/streaming.py | 24 
 1 file changed, 20 insertions(+), 4 deletions(-)

diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py
index b2d06f2..53a098c 100644
--- a/python/pyspark/sql/streaming.py
+++ b/python/pyspark/sql/streaming.py
@@ -1005,12 +1005,17 @@ class DataStreamWriter(object):
 def trigger(self, *, continuous: str) -> "DataStreamWriter":
 ...
 
+@overload
+def trigger(self, *, availableNow: bool) -> "DataStreamWriter":
+...
+
 def trigger(
 self,
 *,
 processingTime: Optional[str] = None,
 once: Optional[bool] = None,
 continuous: Optional[str] = None,
+availableNow: Optional[bool] = None,
 ) -> "DataStreamWriter":
 """Set the trigger for the stream query. If this is not set it will 
run the query as fast
 as possible, which is equivalent to setting the trigger to 
``processingTime='0 seconds'``.
@@ -1030,6 +1035,9 @@ class DataStreamWriter(object):
 a time interval as a string, e.g. '5 seconds', '1 minute'.
 Set a trigger that runs a continuous query with a given checkpoint
 interval. Only one trigger can be set.
+availableNow : bool, optional
+if set to True, set a trigger that processes all available data in 
multiple
+batches then terminates the query. Only one trigger can be set.
 
 Notes
 -
@@ -1043,12 +1051,14 @@ class DataStreamWriter(object):
 >>> writer = sdf.writeStream.trigger(once=True)
 >>> # trigger the query for execution every 5 seconds
 >>> writer = sdf.writeStream.trigger(continuous='5 seconds')
+>>> # trigger the query for reading all available data with multiple 
batches
+>>> writer = sdf.writeStream.trigger(availableNow=True)
 """
-params = [processingTime, o

[spark] branch WIP-optimize-eviction-in-rocksdb-state-store updated: WIP still adding tests and addressing TODOs

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

kabhwan pushed a commit to branch WIP-optimize-eviction-in-rocksdb-state-store
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to 
refs/heads/WIP-optimize-eviction-in-rocksdb-state-store by this push:
 new 640eb70  WIP still adding tests and addressing TODOs
640eb70 is described below

commit 640eb70769e7be8bc5214c7d96dd334f7b3ac7e4
Author: Jungtaek Lim 
AuthorDate: Sat Nov 6 15:34:56 2021 +0900

WIP still adding tests and addressing TODOs
---
 .../streaming/state/RocksDBFileManager.scala   |  4 ---
 .../streaming/state/RocksDBStateStoreSuite.scala   |  2 ++
 .../execution/streaming/state/RocksDBSuite.scala   | 41 +++---
 3 files changed, 38 insertions(+), 9 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 567f916..bc22ee3 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
@@ -607,10 +607,6 @@ object RocksDBCheckpointMetadata {
 
   def apply(
   rocksDBFiles: Seq[RocksDBImmutableFile],
-  numKeys: Long): RocksDBCheckpointMetadata = apply(rocksDBFiles, numKeys, 
Map.empty)
-
-  def apply(
-  rocksDBFiles: Seq[RocksDBImmutableFile],
   numKeys: Long,
   customMetadata: Map[String, String]): RocksDBCheckpointMetadata = {
 val sstFiles = rocksDBFiles.collect { case file: RocksDBSstFile => file }
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 c06a9ec..ba77e29 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
@@ -128,6 +128,8 @@ class RocksDBStateStoreSuite extends 
StateStoreSuiteBase[RocksDBStateStoreProvid
 }
   }
 
+  // FIXME: test with evictOnWatermark
+
   override def newStoreProvider(): RocksDBStateStoreProvider = {
 newStoreProvider(StateStoreId(newDir(), Random.nextInt(), 0))
   }
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 31e49ae..60775d8 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
@@ -352,21 +352,21 @@ class RocksDBSuite extends SparkFunSuite {
 // scalastyle:off line.size.limit
 // should always include sstFiles and numKeys
 checkJsonRoundtrip(
-  RocksDBCheckpointMetadata(Seq.empty, 0L),
+  RocksDBCheckpointMetadata(Seq.empty, 0L, Map.empty),
   """{"sstFiles":[],"numKeys":0}"""
 )
 // shouldn't include the "logFiles" & "customMetadata" field in json when 
it's empty
 checkJsonRoundtrip(
-  RocksDBCheckpointMetadata(sstFiles, 12345678901234L),
+  RocksDBCheckpointMetadata(sstFiles, 12345678901234L, Map.empty),
   
"""{"sstFiles":[{"localFileName":"1.sst","dfsSstFileName":"1-uuid.sst","sizeBytes":12345678901234}],"numKeys":12345678901234}"""
 )
 // shouldn't include the "customMetadata" field in json when it's empty
 checkJsonRoundtrip(
   RocksDBCheckpointMetadata(sstFiles, logFiles, 12345678901234L, 
Map.empty),
   
"""{"sstFiles":[{"localFileName":"1.sst","dfsSstFileName":"1-uuid.sst","sizeBytes":12345678901234}],"logFiles":[{"localFileName":"1.log","dfsLogFileName":"1-uuid.log","sizeBytes":12345678901234}],"numKeys":12345678901234}""")
-
-// FIXME: test customMetadata here
-
+checkJsonRoundtrip(
+  RocksDBCheckpointMetadata(sstFiles, logFiles, 12345678901234L, 
Map("key1" -> "value1", "key2" -> "value2")),
+  
"""{"sstFiles":[{"localFileName":"1.sst","dfsSstFileName":"1-uuid.sst","sizeBytes":12345678901234}],"logFiles":[{"localFileName":"1.log","dfsLogFileName":"1-uuid.log&

[spark] branch WIP-optimize-eviction-in-rocksdb-state-store created (now 88187c6)

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

kabhwan pushed a change to branch WIP-optimize-eviction-in-rocksdb-state-store
in repository https://gitbox.apache.org/repos/asf/spark.git.


  at 88187c6  WIP still need to add e2e test and address FIXME/TODOs

This branch includes the following new commits:

 new 21d4f96  WIP: benchmark test code done
 new 88187c6  WIP still need to add e2e test and address FIXME/TODOs

The 2 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] 02/02: WIP still need to add e2e test and address FIXME/TODOs

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

kabhwan pushed a commit to branch WIP-optimize-eviction-in-rocksdb-state-store
in repository https://gitbox.apache.org/repos/asf/spark.git

commit 88187c64d70a5566bb8f07ce4b133a33e63ce5fc
Author: Jungtaek Lim 
AuthorDate: Fri Nov 5 16:41:24 2021 +0900

WIP still need to add e2e test and address FIXME/TODOs
---
 .../sql/execution/streaming/state/RocksDB.scala| 172 -
 .../streaming/state/RocksDBFileManager.scala   |  35 +++-
 .../streaming/state/RocksDBStateEncoder.scala  |  96 +-
 .../state/RocksDBStateStoreProvider.scala  | 213 +
 .../execution/benchmark/StateStoreBenchmark.scala  |  25 ++-
 .../execution/streaming/state/RocksDBSuite.scala   |   8 +-
 6 files changed, 227 insertions(+), 322 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 eed7827..105a446 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
@@ -18,7 +18,6 @@
 package org.apache.spark.sql.execution.streaming.state
 
 import java.io.File
-import java.util
 import java.util.Locale
 import javax.annotation.concurrent.GuardedBy
 
@@ -51,12 +50,9 @@ import org.apache.spark.util.{NextIterator, Utils}
  * @param hadoopConf   Hadoop configuration for talking to the remote file 
system
  * @param loggingIdId that will be prepended in logs for isolating 
concurrent RocksDBs
  */
-// FIXME: optionally receiving column families
 class RocksDB(
 dfsRootDir: String,
 val conf: RocksDBConf,
-// TODO: change "default" to constant
-columnFamilies: Seq[String] = Seq("default"),
 localRootDir: File = Utils.createTempDir(),
 hadoopConf: Configuration = new Configuration,
 loggingId: String = "") extends Logging {
@@ -69,10 +65,16 @@ class RocksDB(
   private val flushOptions = new FlushOptions().setWaitForFlush(true)  // wait 
for flush to complete
   private val writeBatch = new WriteBatchWithIndex(true)  // overwrite 
multiple updates to a key
 
-  private val dbOptions: DBOptions = new DBOptions() // options to open the 
RocksDB
-  dbOptions.setCreateIfMissing(true)
-  dbOptions.setCreateMissingColumnFamilies(true)
+  private val bloomFilter = new BloomFilter()
+  private val tableFormatConfig = new BlockBasedTableConfig()
+  tableFormatConfig.setBlockSize(conf.blockSizeKB * 1024)
+  tableFormatConfig.setBlockCache(new LRUCache(conf.blockCacheSizeMB * 1024 * 
1024))
+  tableFormatConfig.setFilterPolicy(bloomFilter)
+  tableFormatConfig.setFormatVersion(conf.formatVersion)
 
+  private val dbOptions = new Options() // options to open the RocksDB
+  dbOptions.setCreateIfMissing(true)
+  dbOptions.setTableFormatConfig(tableFormatConfig)
   private val dbLogger = createLogger() // for forwarding RocksDB native logs 
to log4j
   dbOptions.setStatistics(new Statistics())
   private val nativeStats = dbOptions.statistics()
@@ -85,18 +87,18 @@ class RocksDB(
   private val acquireLock = new Object
 
   @volatile private var db: NativeRocksDB = _
-  @volatile private var columnFamilyHandles: util.Map[String, 
ColumnFamilyHandle] = _
-  @volatile private var defaultColumnFamilyHandle: ColumnFamilyHandle = _
   @volatile private var loadedVersion = -1L   // -1 = nothing valid is loaded
   @volatile private var numKeysOnLoadedVersion = 0L
   @volatile private var numKeysOnWritingVersion = 0L
   @volatile private var fileManagerMetrics = 
RocksDBFileManagerMetrics.EMPTY_METRICS
+  @volatile private var customMetadataOnLoadedVersion: Map[String, String] = 
Map.empty
+  @volatile private var customMetadataOnWritingVersion: Map[String, String] = 
Map.empty
 
   @GuardedBy("acquireLock")
   @volatile private var acquiredThreadInfo: AcquiredThreadInfo = _
 
   private val prefixScanReuseIter =
-new java.util.concurrent.ConcurrentHashMap[(Long, Int), RocksIterator]()
+new java.util.concurrent.ConcurrentHashMap[Long, RocksIterator]()
 
   /**
* Load the given version of data in a native RocksDB instance.
@@ -114,6 +116,8 @@ class RocksDB(
 openDB()
 numKeysOnWritingVersion = metadata.numKeys
 numKeysOnLoadedVersion = metadata.numKeys
+customMetadataOnLoadedVersion = metadata.customMetadata
+customMetadataOnWritingVersion = metadata.customMetadata
 loadedVersion = version
 fileManagerMetrics = fileManager.latestLoadCheckpointMetrics
   }
@@ -137,28 +141,7 @@ class RocksDB(
* @note This will return the last written value even if it was uncommitted.
*/
   def get(key: Array[Byte]): Array[Byte] = {
-get(defaultColumnFamilyHandle, key)
-  }
-
-  // FIXME: method doc
-  def get(cf: String, key

[spark] 01/02: WIP: benchmark test code done

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

kabhwan pushed a commit to branch WIP-optimize-eviction-in-rocksdb-state-store
in repository https://gitbox.apache.org/repos/asf/spark.git

commit 21d4f96c6c7a56acce30d485a0747e1d62d7ad27
Author: Jungtaek Lim 
AuthorDate: Thu Sep 30 11:53:39 2021 +0900

WIP: benchmark test code done
---
 .../streaming/FlatMapGroupsWithStateExec.scala |   7 +-
 .../state/HDFSBackedStateStoreProvider.scala   |  42 +-
 .../sql/execution/streaming/state/RocksDB.scala| 158 -
 .../streaming/state/RocksDBStateEncoder.scala  | 135 -
 .../state/RocksDBStateStoreProvider.scala  | 100 +++-
 .../sql/execution/streaming/state/StateStore.scala |  40 +-
 .../execution/streaming/state/StateStoreRDD.scala  |   8 +-
 .../state/StreamingAggregationStateManager.scala   |  23 +
 .../state/SymmetricHashJoinStateManager.scala  |   3 +-
 .../sql/execution/streaming/state/package.scala|  12 +-
 .../execution/streaming/statefulOperators.scala|  62 +-
 .../sql/execution/streaming/streamingLimits.scala  |   5 +-
 .../execution/benchmark/StateStoreBenchmark.scala  | 633 +
 ...ngSortWithSessionWindowStateIteratorSuite.scala |   7 +-
 .../streaming/state/MemoryStateStore.scala |  14 +
 .../state/RocksDBStateStoreIntegrationSuite.scala  |  60 +-
 .../streaming/state/RocksDBStateStoreSuite.scala   |   6 +-
 .../streaming/state/StateStoreRDDSuite.scala   |  18 +-
 .../streaming/state/StateStoreSuite.scala  |  31 +-
 .../StreamingSessionWindowStateManagerSuite.scala  |   4 +-
 .../apache/spark/sql/streaming/StreamSuite.scala   |   4 +-
 .../sql/streaming/StreamingAggregationSuite.scala  |  34 +-
 22 files changed, 1270 insertions(+), 136 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala
index a00a622..381aeb9 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala
@@ -224,21 +224,23 @@ case class FlatMapGroupsWithStateExec(
   val stateStoreId = StateStoreId(
 stateInfo.get.checkpointLocation, stateInfo.get.operatorId, 
partitionId)
   val storeProviderId = StateStoreProviderId(stateStoreId, 
stateInfo.get.queryRunId)
+  // FIXME: would setting prefixScan / evict help?
   val store = StateStore.get(
 storeProviderId,
 groupingAttributes.toStructType,
 stateManager.stateSchema,
-numColsPrefixKey = 0,
+StatefulOperatorContext(),
 stateInfo.get.storeVersion, storeConf, 
hadoopConfBroadcast.value.value)
   val processor = new InputProcessor(store)
   processDataWithPartition(childDataIterator, store, processor, 
Some(initStateIterator))
   }
 } else {
+  // FIXME: would setting prefixScan / evict help?
   child.execute().mapPartitionsWithStateStore[InternalRow](
 getStateInfo,
 groupingAttributes.toStructType,
 stateManager.stateSchema,
-numColsPrefixKey = 0,
+StatefulOperatorContext(),
 session.sqlContext.sessionState,
 Some(session.sqlContext.streams.stateStoreCoordinator)
   ) { case (store: StateStore, singleIterator: Iterator[InternalRow]) =>
@@ -334,6 +336,7 @@ case class FlatMapGroupsWithStateExec(
 throw new IllegalStateException(
   s"Cannot filter timed out keys for $timeoutConf")
 }
+// FIXME: would setting prefixScan / evict help?
 val timingOutPairs = stateManager.getAllState(store).filter { state =>
   state.timeoutTimestamp != NO_TIMESTAMP && state.timeoutTimestamp < 
timeoutThreshold
 }
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 75b7dae..96ba2a3 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
@@ -100,8 +100,11 @@ private[sql] class HDFSBackedStateStoreProvider extends 
StateStoreProvider with
 
 /** Trait and classes representing the internal state of the store */
 trait STATE
+
 case object UPDATING extends STATE
+
 case object COMMITTED extends STATE
+
 case object ABORTED extends STATE
 
 private val newVersion = version + 1
@@ -195,6 +198,22 @@ private[sql] class HDFSBackedStateStoreProvider extends 
StateStoreProvider with
 override def toString(): String = {

[spark] branch master updated: [SPARK-37062][SS] Introduce a new data source for providing consistent set of rows per microbatch

2021-11-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 70fde44  [SPARK-37062][SS] Introduce a new data source for providing 
consistent set of rows per microbatch
70fde44 is described below

commit 70fde44e930926cbcd1fc95fa7cfb915c25cff9c
Author: Jungtaek Lim 
AuthorDate: Mon Nov 1 20:04:10 2021 +0900

[SPARK-37062][SS] Introduce a new data source for providing consistent set 
of rows per microbatch

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

This PR proposes to introduce a new data source having short name as 
"rate-micro-batch", which produces similar input rows as "rate" (increment long 
values with timestamps), but ensures that each micro-batch has a "predictable" 
set of input rows.

"rate-micro-batch" data source receives a config to specify the number of 
rows per micro-batch, which defines the set of input rows for further 
micro-batches. For example, if the number of rows per micro-batch is set to 
1000, the first batch would have 1000 rows having value range as `0~999`, the 
second batch would have 1000 rows having value range as `1000~1999`, and so on. 
This characteristic brings different use cases compared to rate data source, as 
we can't predict the input rows [...]

For generated time (timestamp column), the data source applies the same 
mechanism to make the value of column be predictable. `startTimestamp` option 
defines the starting value of generated time, and `advanceMillisPerBatch` 
option defines how much time the generated time should advance per micro-batch. 
All input rows in the same micro-batch will have same timestamp.

This source supports the following options:

* `rowsPerBatch` (e.g. 100): How many rows should be generated per 
micro-batch.
* `numPartitions` (e.g. 10, default: Spark's default parallelism): The 
partition number for the generated rows.
* `startTimestamp` (e.g. 1000, default: 0): starting value of generated time
* `advanceMillisPerBatch` (e.g. 1000, default: 1000): the amount of time 
being advanced in generated time on each micro-batch.

### Why are the changes needed?

The "rate" data source has been known to be used as a benchmark for 
streaming query.

While this helps to put the query to the limit (how many rows the query 
could process per second), the rate data source doesn't provide consistent rows 
per batch into stream, which leads two environments be hard to compare with.

For example, in many cases, you may want to compare the metrics in the 
batches between test environments (like running same streaming query with 
different options). These metrics are strongly affected if the distribution of 
input rows in batches are changing, especially a micro-batch has been lagged 
(in any reason) and rate data source produces more input rows to the next batch.

Also, when you test against streaming aggregation, you may want the data 
source produces the same set of input rows per batch (deterministic), so that 
you can plan how these input rows will be aggregated and how state rows will be 
evicted, and craft the test query based on the plan.

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

Yes, end users can leverage a new data source in micro-batch mode of 
streaming query to test/benchmark.

### How was this patch tested?

New UTs, and manually tested via below query in spark-shell:

```
spark.readStream.format("rate-micro-batch").option("rowsPerBatch", 
20).option("numPartitions", 3).load().writeStream.format("console").start()
```

Closes #34333 from HeartSaVioR/SPARK-37062.

Authored-by: Jungtaek Lim 
Signed-off-by: Jungtaek Lim 
---
 docs/structured-streaming-programming-guide.md |  13 ++
 ...org.apache.spark.sql.sources.DataSourceRegister |   1 +
 .../sources/RatePerMicroBatchProvider.scala| 127 +
 .../sources/RatePerMicroBatchStream.scala  | 175 ++
 .../sources/RatePerMicroBatchProviderSuite.scala   | 204 +
 5 files changed, 520 insertions(+)

diff --git a/docs/structured-streaming-programming-guide.md 
b/docs/structured-streaming-programming-guide.md
index b36cdc7..6237d47 100644
--- a/docs/structured-streaming-programming-guide.md
+++ b/docs/structured-streaming-programming-guide.md
@@ -517,6 +517,8 @@ There are a few built-in sources.
 
   - **Rate source (for testing)** - Generates data at the specified number of 
rows per second, each output row contains a `timestamp` and `value`. Where 
`timestamp` is a `Timestamp` type containing the time of message dispatch, and 
`value` is of `Long` type

[spark] branch branch-3.2 updated: [MINOR][SS] Remove unused config "pauseBackgroundWorkForCommit" from RocksDB

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

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


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new 67abd7a  [MINOR][SS] Remove unused config 
"pauseBackgroundWorkForCommit" from RocksDB
67abd7a is described below

commit 67abd7a49c065a933109b262b181ca84cbfc5dc6
Author: Jungtaek Lim 
AuthorDate: Sun Oct 31 07:49:48 2021 +0900

[MINOR][SS] Remove unused config "pauseBackgroundWorkForCommit" from RocksDB

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

This PR proposes to remove unused config "pauseBackgroundWorkForCommit".

### Why are the changes needed?

That is unused config which actually has to be always "true" even if it's 
being used.

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

No

### How was this patch tested?

Existing UTs

Closes #34417 from HeartSaVioR/MINOR-remove-unused-config-in-rocksdb.

Authored-by: Jungtaek Lim 
Signed-off-by: Jungtaek Lim 
(cherry picked from commit 00d9ed5dd278a10a2dff8f420fafab516f85c189)
Signed-off-by: Jungtaek Lim 
---
 .../scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala | 3 ---
 1 file changed, 3 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 6004bdb..1ff8b41 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
@@ -499,7 +499,6 @@ class ByteArrayPair(var key: Array[Byte] = null, var value: 
Array[Byte] = null)
 case class RocksDBConf(
 minVersionsToRetain: Int,
 compactOnCommit: Boolean,
-pauseBackgroundWorkForCommit: Boolean,
 blockSizeKB: Long,
 blockCacheSizeMB: Long,
 lockAcquireTimeoutMs: Long,
@@ -516,7 +515,6 @@ object RocksDBConf {
 
   // Configuration that specifies whether to compact the RocksDB data every 
time data is committed
   private val COMPACT_ON_COMMIT_CONF = ConfEntry("compactOnCommit", "false")
-  private val PAUSE_BG_WORK_FOR_COMMIT_CONF = 
ConfEntry("pauseBackgroundWorkForCommit", "true")
   private val BLOCK_SIZE_KB_CONF = ConfEntry("blockSizeKB", "4")
   private val BLOCK_CACHE_SIZE_MB_CONF = ConfEntry("blockCacheSizeMB", "8")
   private val LOCK_ACQUIRE_TIMEOUT_MS_CONF = ConfEntry("lockAcquireTimeoutMs", 
"6")
@@ -560,7 +558,6 @@ object RocksDBConf {
 RocksDBConf(
   storeConf.minVersionsToRetain,
   getBooleanConf(COMPACT_ON_COMMIT_CONF),
-  getBooleanConf(PAUSE_BG_WORK_FOR_COMMIT_CONF),
   getPositiveLongConf(BLOCK_SIZE_KB_CONF),
   getPositiveLongConf(BLOCK_CACHE_SIZE_MB_CONF),
   getPositiveLongConf(LOCK_ACQUIRE_TIMEOUT_MS_CONF),

-
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] Remove unused config "pauseBackgroundWorkForCommit" from RocksDB

2021-10-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 00d9ed5  [MINOR][SS] Remove unused config 
"pauseBackgroundWorkForCommit" from RocksDB
00d9ed5 is described below

commit 00d9ed5dd278a10a2dff8f420fafab516f85c189
Author: Jungtaek Lim 
AuthorDate: Sun Oct 31 07:49:48 2021 +0900

[MINOR][SS] Remove unused config "pauseBackgroundWorkForCommit" from RocksDB

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

This PR proposes to remove unused config "pauseBackgroundWorkForCommit".

### Why are the changes needed?

That is unused config which actually has to be always "true" even if it's 
being used.

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

No

### How was this patch tested?

Existing UTs

Closes #34417 from HeartSaVioR/MINOR-remove-unused-config-in-rocksdb.

Authored-by: Jungtaek Lim 
Signed-off-by: Jungtaek Lim 
---
 .../scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala | 3 ---
 1 file changed, 3 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 6004bdb..1ff8b41 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
@@ -499,7 +499,6 @@ class ByteArrayPair(var key: Array[Byte] = null, var value: 
Array[Byte] = null)
 case class RocksDBConf(
 minVersionsToRetain: Int,
 compactOnCommit: Boolean,
-pauseBackgroundWorkForCommit: Boolean,
 blockSizeKB: Long,
 blockCacheSizeMB: Long,
 lockAcquireTimeoutMs: Long,
@@ -516,7 +515,6 @@ object RocksDBConf {
 
   // Configuration that specifies whether to compact the RocksDB data every 
time data is committed
   private val COMPACT_ON_COMMIT_CONF = ConfEntry("compactOnCommit", "false")
-  private val PAUSE_BG_WORK_FOR_COMMIT_CONF = 
ConfEntry("pauseBackgroundWorkForCommit", "true")
   private val BLOCK_SIZE_KB_CONF = ConfEntry("blockSizeKB", "4")
   private val BLOCK_CACHE_SIZE_MB_CONF = ConfEntry("blockCacheSizeMB", "8")
   private val LOCK_ACQUIRE_TIMEOUT_MS_CONF = ConfEntry("lockAcquireTimeoutMs", 
"6")
@@ -560,7 +558,6 @@ object RocksDBConf {
 RocksDBConf(
   storeConf.minVersionsToRetain,
   getBooleanConf(COMPACT_ON_COMMIT_CONF),
-  getBooleanConf(PAUSE_BG_WORK_FOR_COMMIT_CONF),
   getPositiveLongConf(BLOCK_SIZE_KB_CONF),
   getPositiveLongConf(BLOCK_CACHE_SIZE_MB_CONF),
   getPositiveLongConf(LOCK_ACQUIRE_TIMEOUT_MS_CONF),

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



[spark] branch branch-3.2 updated: [SPARK-37147][SS] MetricsReporter producing NullPointerException when element 'triggerExecution' not present in Map[]

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

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


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new 574a43b  [SPARK-37147][SS] MetricsReporter producing 
NullPointerException when element 'triggerExecution' not present in Map[]
574a43b is described below

commit 574a43be0b4f46917467f4b7aabb2b6cb5892168
Author: Radek Busz 
AuthorDate: Sun Oct 31 07:30:14 2021 +0900

[SPARK-37147][SS] MetricsReporter producing NullPointerException when 
element 'triggerExecution' not present in Map[]

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

Bug Fix.

The problematic code is in `MetricsReporter`:
`registerGauge("latency", _.durationMs.get("triggerExecution").longValue(), 
0L)`
Instead of `.getOrDefault(...).longValue()` it uses 
`.get("triggerExecution").longValue()` which can return a null pointer 
exception if "triggerExecution" is not in the durationMs map.

Solution: use `.getOrDefault` when accessing a map.

### Why are the changes needed?

When `MetricsReporter.scala` registers a Gauge it occasionally returns a 
NPE. This breaks reporting custom metrics via Dropwizard and logs multiple 
times a stacktrace. It usually happens when using StructuredStreaming on a slow 
data source but I'm not able to reliably reproduce it every time.

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

Yes - fixes occasional failures when reporting metrics with Dropwizard

### How was this patch tested?

Added a unit-test.

Closes #34426 from gitplaneta/SPARK-37147.

Authored-by: Radek Busz 
Signed-off-by: Jungtaek Lim 
---
 .../spark/sql/execution/streaming/MetricsReporter.scala |  2 +-
 .../apache/spark/sql/streaming/StreamingQuerySuite.scala| 13 +
 2 files changed, 14 insertions(+), 1 deletion(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetricsReporter.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetricsReporter.scala
index 8709822..600b16f 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetricsReporter.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetricsReporter.scala
@@ -40,7 +40,7 @@ class MetricsReporter(
   // together in Ganglia as a single metric group
   registerGauge("inputRate-total", _.inputRowsPerSecond, 0.0)
   registerGauge("processingRate-total", _.processedRowsPerSecond, 0.0)
-  registerGauge("latency", _.durationMs.get("triggerExecution").longValue(), 
0L)
+  registerGauge("latency", _.durationMs.getOrDefault("triggerExecution", 
0L).longValue(), 0L)
 
   private val timestampFormat = new 
SimpleDateFormat("-MM-dd'T'HH:mm:ss.SSS'Z'") // ISO8601
   timestampFormat.setTimeZone(DateTimeUtils.getTimeZone("UTC"))
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
index 9c2403d..21a0b24 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
@@ -19,13 +19,16 @@ package org.apache.spark.sql.streaming
 
 import java.io.File
 import java.nio.charset.StandardCharsets.UTF_8
+import java.util.Collections
 import java.util.concurrent.CountDownLatch
 
 import scala.collection.mutable
+import scala.util.{Success, Try}
 
 import org.apache.commons.io.FileUtils
 import org.apache.commons.lang3.RandomStringUtils
 import org.apache.hadoop.fs.Path
+import org.mockito.Mockito.when
 import org.scalactic.TolerantNumerics
 import org.scalatest.BeforeAndAfter
 import org.scalatest.concurrent.PatienceConfiguration.Timeout
@@ -465,6 +468,16 @@ class StreamingQuerySuite extends StreamTest with 
BeforeAndAfter with Logging wi
 }
   }
 
+  test("SPARK-37147: MetricsReporter does not fail when durationMs is empty") {
+val stateOpProgressMock = mock[StreamingQueryProgress]
+
when(stateOpProgressMock.durationMs).thenReturn(Collections.emptyMap[String, 
java.lang.Long]())
+val streamExecMock = mock[StreamExecution]
+when(streamExecMock.lastProgress).thenReturn(stateOpProgressMock)
+
+val gauges = new MetricsReporter(streamExecMock, 
"").metricRegistry.getGauges()
+assert(Try(gauges.get("latency").getValue) == Success(0L))
+  }
+
   test("input row calculation with same V1 source used twice in self-join") {
 val streamingTriggerDF = spark.createDataset(1 to 10).toDF
 val streamingInputDF = 
createSingleTriggerStreamingDF(str

[spark] branch master updated: [SPARK-37147][SS] MetricsReporter producing NullPointerException when element 'triggerExecution' not present in Map[]

2021-10-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 547996e  [SPARK-37147][SS] MetricsReporter producing 
NullPointerException when element 'triggerExecution' not present in Map[]
547996e is described below

commit 547996efed8e4593265aea2eb26b10f8b366e141
Author: Radek Busz 
AuthorDate: Sun Oct 31 07:30:14 2021 +0900

[SPARK-37147][SS] MetricsReporter producing NullPointerException when 
element 'triggerExecution' not present in Map[]

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

Bug Fix.

The problematic code is in `MetricsReporter`:
`registerGauge("latency", _.durationMs.get("triggerExecution").longValue(), 
0L)`
Instead of `.getOrDefault(...).longValue()` it uses 
`.get("triggerExecution").longValue()` which can return a null pointer 
exception if "triggerExecution" is not in the durationMs map.

Solution: use `.getOrDefault` when accessing a map.

### Why are the changes needed?

When `MetricsReporter.scala` registers a Gauge it occasionally returns a 
NPE. This breaks reporting custom metrics via Dropwizard and logs multiple 
times a stacktrace. It usually happens when using StructuredStreaming on a slow 
data source but I'm not able to reliably reproduce it every time.

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

Yes - fixes occasional failures when reporting metrics with Dropwizard

### How was this patch tested?

Added a unit-test.

Closes #34426 from gitplaneta/SPARK-37147.

Authored-by: Radek Busz 
Signed-off-by: Jungtaek Lim 
---
 .../spark/sql/execution/streaming/MetricsReporter.scala |  2 +-
 .../apache/spark/sql/streaming/StreamingQuerySuite.scala| 13 +
 2 files changed, 14 insertions(+), 1 deletion(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetricsReporter.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetricsReporter.scala
index 8709822..600b16f 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetricsReporter.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetricsReporter.scala
@@ -40,7 +40,7 @@ class MetricsReporter(
   // together in Ganglia as a single metric group
   registerGauge("inputRate-total", _.inputRowsPerSecond, 0.0)
   registerGauge("processingRate-total", _.processedRowsPerSecond, 0.0)
-  registerGauge("latency", _.durationMs.get("triggerExecution").longValue(), 
0L)
+  registerGauge("latency", _.durationMs.getOrDefault("triggerExecution", 
0L).longValue(), 0L)
 
   private val timestampFormat = new 
SimpleDateFormat("-MM-dd'T'HH:mm:ss.SSS'Z'") // ISO8601
   timestampFormat.setTimeZone(DateTimeUtils.getTimeZone("UTC"))
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
index 9c2403d..21a0b24 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
@@ -19,13 +19,16 @@ package org.apache.spark.sql.streaming
 
 import java.io.File
 import java.nio.charset.StandardCharsets.UTF_8
+import java.util.Collections
 import java.util.concurrent.CountDownLatch
 
 import scala.collection.mutable
+import scala.util.{Success, Try}
 
 import org.apache.commons.io.FileUtils
 import org.apache.commons.lang3.RandomStringUtils
 import org.apache.hadoop.fs.Path
+import org.mockito.Mockito.when
 import org.scalactic.TolerantNumerics
 import org.scalatest.BeforeAndAfter
 import org.scalatest.concurrent.PatienceConfiguration.Timeout
@@ -465,6 +468,16 @@ class StreamingQuerySuite extends StreamTest with 
BeforeAndAfter with Logging wi
 }
   }
 
+  test("SPARK-37147: MetricsReporter does not fail when durationMs is empty") {
+val stateOpProgressMock = mock[StreamingQueryProgress]
+
when(stateOpProgressMock.durationMs).thenReturn(Collections.emptyMap[String, 
java.lang.Long]())
+val streamExecMock = mock[StreamExecution]
+when(streamExecMock.lastProgress).thenReturn(stateOpProgressMock)
+
+val gauges = new MetricsReporter(streamExecMock, 
"").metricRegistry.getGauges()
+assert(Try(gauges.get("latency").getValue) == Success(0L))
+  }
+
   test("input row calculation with same V1 source used twice in self-join") {
 val streamingTriggerDF = spark.createDataset(1 to 10).toDF
 val streamingInputDF = 
createSingleTriggerStreamingDF(str

[spark] branch branch-3.2 updated: [SPARK-36865][PYTHON][DOCS] Add PySpark API document of session_window

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

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


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new 8b2b6bb  [SPARK-36865][PYTHON][DOCS] Add PySpark API document of 
session_window
8b2b6bb is described below

commit 8b2b6bb0d3d26c5d1d121136b5916e5aeac1ade9
Author: Kousuke Saruta 
AuthorDate: Thu Sep 30 16:51:12 2021 +0900

[SPARK-36865][PYTHON][DOCS] Add PySpark API document of session_window

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

This PR adds PySpark API document of `session_window`.
The docstring of the function doesn't comply with numpydoc format so this 
PR also fix it.
Further, the API document of `window` doesn't have `Parameters` section so 
it's also added in this PR.

### Why are the changes needed?

To provide PySpark users with the API document of the newly added function.

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

No.

### How was this patch tested?

`make html` in `python/docs` and get the following docs.

[window]

![time-window-python-doc-after](https://user-images.githubusercontent.com/4736016/134963797-ce25b268-20ca-48e3-ac8d-cbcbd85ebb3e.png)

[session_window]

![session-window-python-doc-after](https://user-images.githubusercontent.com/4736016/134963853-dd9d8417-139b-41ee-9924-14544b1a91af.png)

Closes #34118 from sarutak/python-session-window-doc.

Authored-by: Kousuke Saruta 
Signed-off-by: Jungtaek Lim 
(cherry picked from commit 5a32e41e9c992c6f08a48454d783e7cd97c971fc)
Signed-off-by: Jungtaek Lim 
---
 python/docs/source/reference/pyspark.sql.rst |  1 +
 python/pyspark/sql/functions.py  | 35 
 2 files changed, 36 insertions(+)

diff --git a/python/docs/source/reference/pyspark.sql.rst 
b/python/docs/source/reference/pyspark.sql.rst
index 13c489b..ca4a95a 100644
--- a/python/docs/source/reference/pyspark.sql.rst
+++ b/python/docs/source/reference/pyspark.sql.rst
@@ -497,6 +497,7 @@ Functions
 second
 sentences
 sequence
+session_window
 sha1
 sha2
 shiftleft
diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py
index fa96ea6..c7bc581 100644
--- a/python/pyspark/sql/functions.py
+++ b/python/pyspark/sql/functions.py
@@ -2300,6 +2300,29 @@ def window(timeColumn, windowDuration, 
slideDuration=None, startTime=None):
 
 .. versionadded:: 2.0.0
 
+Parameters
+--
+timeColumn : :class:`~pyspark.sql.Column`
+The column or the expression to use as the timestamp for windowing by 
time.
+The time column must be of TimestampType.
+windowDuration : str
+A string specifying the width of the window, e.g. `10 minutes`,
+`1 second`. Check `org.apache.spark.unsafe.types.CalendarInterval` for
+valid duration identifiers. Note that the duration is a fixed length of
+time, and does not vary over time according to a calendar. For example,
+`1 day` always means 86,400,000 milliseconds, not a calendar day.
+slideDuration : str, optional
+A new window will be generated every `slideDuration`. Must be less than
+or equal to the `windowDuration`. Check
+`org.apache.spark.unsafe.types.CalendarInterval` for valid duration
+identifiers. This duration is likewise absolute, and does not vary
+according to a calendar.
+startTime : str, optional
+The offset with respect to 1970-01-01 00:00:00 UTC with which to start
+window intervals. For example, in order to have hourly tumbling 
windows that
+start 15 minutes past the hour, e.g. 12:15-13:15, 13:15-14:15... 
provide
+`startTime` as `15 minutes`.
+
 Examples
 
 >>> df = spark.createDataFrame([("2016-03-11 09:00:07", 1)]).toDF("date", 
"val")
@@ -2347,7 +2370,19 @@ def session_window(timeColumn, gapDuration):
 input row.
 The output column will be a struct called 'session_window' by default with 
the nested columns
 'start' and 'end', where 'start' and 'end' will be of 
:class:`pyspark.sql.types.TimestampType`.
+
 .. versionadded:: 3.2.0
+
+Parameters
+--
+timeColumn : :class:`~pyspark.sql.Column`
+The column or the expression to use as the timestamp for windowing by 
time.
+The time column must be of TimestampType.
+gapDuration : :class:`~pyspark.sql.Column` or str
+A column or string specifying the timeout of the session. It could be 
static value,
+e.g. `10 minutes`, `1 second`, or an expression/UDF that specifies gap
+duration dynamically based on the input row.
+
 Exa

[spark] branch master updated (17e3ca6 -> 5a32e41)

2021-09-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 17e3ca6  [SPARK-36899][R] Support ILIKE API on R
 add 5a32e41  [SPARK-36865][PYTHON][DOCS] Add PySpark API document of 
session_window

No new revisions were added by this update.

Summary of changes:
 python/docs/source/reference/pyspark.sql.rst |  1 +
 python/pyspark/sql/functions.py  | 35 
 2 files changed, 36 insertions(+)

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



[spark] branch master updated: [SPARK-36737][BUILD][CORE][SQL][SS] Upgrade Apache commons-io to 2.11.0 and revert change of SPARK-36456

2021-09-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 119ddd7  [SPARK-36737][BUILD][CORE][SQL][SS] Upgrade Apache commons-io 
to 2.11.0 and revert change of SPARK-36456
119ddd7 is described below

commit 119ddd7e9526ed899f88a944babb74af693297f5
Author: yangjie01 
AuthorDate: Tue Sep 14 21:16:58 2021 +0900

[SPARK-36737][BUILD][CORE][SQL][SS] Upgrade Apache commons-io to 2.11.0 and 
revert change of SPARK-36456

### What changes were proposed in this pull request?
SPARK-36456 change to use `JavaUtils. closeQuietly` instead of 
`IOUtils.closeQuietly`, but there is slightly different from the 2 methods in 
default behavior: swallowing IOException is same, but the former logs it as 
ERROR while the latter doesn't log by default.

`Apache commons-io` community decided to retain the `IOUtils.closeQuietly` 
method in the [new 
version](https://github.com/apache/commons-io/blob/75f20dca72656225d0dc8e7c982e40caa9277d42/src/main/java/org/apache/commons/io/IOUtils.java#L465-L467)
 and removed deprecated annotation,  the change has been released in version 
2.11.0.

So the change of this pr is to upgrade `Apache commons-io` to 2.11.0 and 
revert change of SPARK-36456 to maintain original behavior(don't print error 
log).

### Why are the changes needed?

1. Upgrade `Apache commons-io` to 2.11.0 to use non-deprecated 
`closeQuietly` API, other changes related to `Apache commons-io are detailed in 
[commons-io/changes-report](https://commons.apache.org/proper/commons-io/changes-report.html#a2.11.0)

2. Revert change of SPARK-36737 to maintain original `IOUtils.closeQuietly` 
API behavior(don't print error log).

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

### How was this patch tested?
Pass the Jenkins or GitHub Action

Closes #33977 from LuciferYang/upgrade-commons-io.

Authored-by: yangjie01 
Signed-off-by: Jungtaek Lim 
---
 core/src/main/scala/org/apache/spark/storage/BlockManager.scala | 5 +++--
 .../org/apache/spark/storage/ShuffleBlockFetcherIterator.scala  | 5 +++--
 core/src/main/scala/org/apache/spark/util/Utils.scala   | 4 ++--
 .../scala/org/apache/spark/util/logging/RollingFileAppender.scala   | 5 ++---
 core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala   | 3 +--
 core/src/test/scala/org/apache/spark/util/UtilsSuite.scala  | 6 +++---
 dev/deps/spark-deps-hadoop-3.2-hive-2.3 | 2 +-
 pom.xml | 2 +-
 .../org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala  | 4 ++--
 .../org/apache/spark/sql/execution/streaming/StreamMetadata.scala   | 4 ++--
 .../execution/streaming/state/HDFSBackedStateStoreProvider.scala| 4 ++--
 .../spark/sql/execution/streaming/state/RocksDBFileManager.scala| 5 ++---
 12 files changed, 24 insertions(+), 25 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala 
b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
index b537060..cbb4e9c 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
@@ -35,6 +35,7 @@ import scala.util.control.NonFatal
 
 import com.codahale.metrics.{MetricRegistry, MetricSet}
 import com.google.common.cache.CacheBuilder
+import org.apache.commons.io.IOUtils
 
 import org.apache.spark._
 import org.apache.spark.errors.SparkCoreErrors
@@ -51,7 +52,7 @@ import org.apache.spark.network.netty.SparkTransportConf
 import org.apache.spark.network.shuffle._
 import org.apache.spark.network.shuffle.checksum.{Cause, ShuffleChecksumHelper}
 import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo
-import org.apache.spark.network.util.{JavaUtils, TransportConf}
+import org.apache.spark.network.util.TransportConf
 import org.apache.spark.rpc.RpcEnv
 import org.apache.spark.scheduler.ExecutorCacheTaskLocation
 import org.apache.spark.serializer.{SerializerInstance, SerializerManager}
@@ -341,7 +342,7 @@ private[spark] class BlockManager(
 false
 }
   } finally {
-JavaUtils.closeQuietly(inputStream)
+IOUtils.closeQuietly(inputStream)
   }
 }
 
diff --git 
a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
 
b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
index b1713ec..eaecf65 100644
--- 
a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
+++ 
b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
@@ -29,6 +29,7 @@ import scala.collection.mutable.{ArrayBuffer, HashMap, 
HashSet, Queue}
 import scal

[spark] branch master updated (23794fb -> 375ca94)

2021-09-08 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 23794fb  [SPARK-34952][SQL][FOLLOWUP] Change column type to be 
NamedReference
 add 375ca94  [SPARK-36690][SS] Clean up deprecated api usage after upgrade 
commons-pool2 to 2.11.1

No new revisions were added by this update.

Summary of changes:
 .../spark/sql/kafka010/consumer/InternalKafkaConsumerPool.scala  | 9 +
 1 file changed, 5 insertions(+), 4 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-36533][SS] Trigger.AvailableNow for running streaming queries like Trigger.Once in multiple batches

2021-08-31 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 e33cdfb  [SPARK-36533][SS] Trigger.AvailableNow for running streaming 
queries like Trigger.Once in multiple batches
e33cdfb is described below

commit e33cdfb317498b04e077c4d6356fc3848cd78da0
Author: Bo Zhang 
AuthorDate: Wed Sep 1 15:02:21 2021 +0900

[SPARK-36533][SS] Trigger.AvailableNow for running streaming queries like 
Trigger.Once in multiple batches

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

This change creates a new type of Trigger: Trigger.AvailableNow for 
streaming queries. It is like Trigger.Once, which process all available data 
then stop the query, but with better scalability since data can be processed in 
multiple batches instead of one.

To achieve this, this change proposes a new interface 
`SupportsTriggerAvailableNow`, which is an extension of 
`SupportsAdmissionControl`. It has one method, `prepareForTriggerAvailableNow`, 
which will be called at the beginning of streaming queries with 
Trigger.AvailableNow, to let the source record the offset for the current 
latest data at the time (a.k.a. the target offset for the query). The source 
should then behave as if there is no new data coming in after the beginning of 
the q [...]

This change also updates `FileStreamSource` to be an implementation of 
`SupportsTriggerAvailableNow`.

For other sources that does not implement `SupportsTriggerAvailableNow`, 
this change also provides a new class 
`FakeLatestOffsetSupportsTriggerAvailableNow`, which wraps the sources and 
makes them support Trigger.AvailableNow, by overriding their `latestOffset` 
method to always return the latest offset at the beginning of the query.

### Why are the changes needed?

Currently streaming queries with Trigger.Once will always load all of the 
available data in a single batch. Because of this, the amount of data a query 
can process is limited, or Spark driver will run out of memory.

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

Users will be able to use Trigger.AvailableNow (to process all available 
data then stop the streaming query) with this change.

### How was this patch tested?

Added unit tests.

Closes #33763 from bozhang2820/new-trigger.

Authored-by: Bo Zhang 
Signed-off-by: Jungtaek Lim 
---
 docs/structured-streaming-programming-guide.md |  21 ++
 .../streaming/SupportsTriggerAvailableNow.java |  41 
 .../org/apache/spark/sql/streaming/Trigger.java|  21 +-
 .../streaming/AvailableNowDataStreamWrapper.scala  |  88 
 .../AvailableNowMicroBatchStreamWrapper.scala  |  39 
 .../streaming/AvailableNowSourceWrapper.scala  |  38 
 .../sql/execution/streaming/FileStreamSource.scala |  22 +-
 .../execution/streaming/MicroBatchExecution.scala  |  84 ++--
 .../sql/execution/streaming/TriggerExecutor.scala  |  12 +-
 .../spark/sql/execution/streaming/Triggers.scala   |   8 +-
 .../sql/streaming/FileStreamSourceSuite.scala  |  82 
 .../sql/streaming/TriggerAvailableNowSuite.scala   | 233 +
 12 files changed, 656 insertions(+), 33 deletions(-)

diff --git a/docs/structured-streaming-programming-guide.md 
b/docs/structured-streaming-programming-guide.md
index 18dfbec..6e98d5a 100644
--- a/docs/structured-streaming-programming-guide.md
+++ b/docs/structured-streaming-programming-guide.md
@@ -2749,6 +2749,15 @@ Here are the different kinds of triggers that are 
supported.
 
   
   
+Available-now micro-batch
+
+Similar to queries one-time micro-batch trigger, the query will 
process all the available data and then
+stop on its own. The difference is that, it will process the data in 
(possibly) multiple micro-batches
+based on the source options (e.g. maxFilesPerTrigger for 
file source), which will result
+in better query scalability.
+
+  
+  
 Continuous with fixed checkpoint 
interval(experimental)
 
 The query will be executed in the new low-latency, continuous 
processing mode. Read more
@@ -2782,6 +2791,12 @@ df.writeStream
   .trigger(Trigger.Once())
   .start()
 
+// Available-now trigger
+df.writeStream
+  .format("console")
+  .trigger(Trigger.AvailableNow())
+  .start()
+
 // Continuous trigger with one-second checkpointing interval
 df.writeStream
   .format("console")
@@ -2814,6 +2829,12 @@ df.writeStream
   .trigger(Trigger.Once())
   .start();
 
+// Available-now trigger
+df.writeStream
+  .format("console")
+  .trigger(Trigger.AvailableNow())
+  .start();
+
 // Continuous trigger with one-second checkpointing interval
 df.writeStream
   .format("console")
diff --git 
a/sql/catalyst/src/

[spark] branch master updated: [SPARK-36576][SS] Improve range split calculation for Kafka Source minPartitions option

2021-08-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 7d1be37  [SPARK-36576][SS] Improve range split calculation for Kafka 
Source minPartitions option
7d1be37 is described below

commit 7d1be3710446c23606c3871e28d211ad9776
Author: Andrew Olson 
AuthorDate: Sun Aug 29 16:38:29 2021 +0900

[SPARK-36576][SS] Improve range split calculation for Kafka Source 
minPartitions option

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

Proposing that the `KafkaOffsetRangeCalculator`'s range calculation logic 
be modified to exclude small (i.e. un-split) partitions from the overall 
proportional distribution math, in order to more reasonably divide the large 
partitions when they are accompanied by many small partitions, and to provide 
optimal behavior for cases where a `minPartitions` value is deliberately 
computed based on the volume of data being read.

### Why are the changes needed?

While the 
[documentation](https://spark.apache.org/docs/latest/structured-streaming-kafka-integration.html)
 does contain a clear disclaimer,

> Please note that this configuration is like a hint: the number of Spark 
tasks will be **approximately** `minPartitions`. It can be less or more 
depending on rounding errors or Kafka partitions that didn't receive any new 
data.

there are cases where the calculated Kafka partition range splits can 
differ greatly from expectations. For evenly distributed data and most 
`minPartitions `values this would not be a major or commonly encountered 
concern. However when the distribution of data across partitions is very 
heavily skewed, somewhat surprising range split calculations can result.

For example, given the following input data:

- 1 partition containing 10,000 messages
- 1,000 partitions each containing 1 message

Spark processing code loading from this collection of 1,001 partitions may 
decide that it would like each task to read no more than 1,000 messages. 
Consequently, it could specify a `minPartitions` value of 1,010 — expecting the 
single large partition to be split into 10 equal chunks, along with the 1,000 
small partitions each having their own task. That is far from what actually 
occurs. The `KafkaOffsetRangeCalculator` algorithm ends up splitting the large 
partition into 918 chunks of [...]

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

No

### How was this patch tested?

Existing unit tests and added new unit tests

Closes #33827 from noslowerdna/SPARK-36576.

Authored-by: Andrew Olson 
Signed-off-by: Jungtaek Lim 
---
 .../sql/kafka010/KafkaOffsetRangeCalculator.scala  | 31 ++--
 .../kafka010/KafkaOffsetRangeCalculatorSuite.scala | 58 --
 .../sql/kafka010/KafkaOffsetReaderSuite.scala  |  4 +-
 3 files changed, 84 insertions(+), 9 deletions(-)

diff --git 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeCalculator.scala
 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeCalculator.scala
index 1e9a62e..4c0620a 100644
--- 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeCalculator.scala
+++ 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeCalculator.scala
@@ -33,12 +33,13 @@ private[kafka010] class KafkaOffsetRangeCalculator(val 
minPartitions: Option[Int
* Calculate the offset ranges that we are going to process this batch. If 
`minPartitions`
* is not set or is set less than or equal the number of `topicPartitions` 
that we're going to
* consume, then we fall back to a 1-1 mapping of Spark tasks to Kafka 
partitions. If
-   * `numPartitions` is set higher than the number of our `topicPartitions`, 
then we will split up
+   * `minPartitions` is set higher than the number of our `topicPartitions`, 
then we will split up
* the read tasks of the skewed partitions to multiple Spark tasks.
-   * The number of Spark tasks will be *approximately* `numPartitions`. It can 
be less or more
+   * The number of Spark tasks will be *approximately* `minPartitions`. It can 
be less or more
* depending on rounding errors or Kafka partitions that didn't receive any 
new data.
*
-   * Empty ranges (`KafkaOffsetRange.size <= 0`) will be dropped.
+   * Empty (`KafkaOffsetRange.size == 0`) or invalid (`KafkaOffsetRange.size < 
0`) ranges  will be
+   * dropped.
*/
   def getRanges(
   ranges: Seq[KafkaOffsetRange],
@@ -56,11 +57,29 @@ private[kafka010] class KafkaOffsetRangeCalculator(val 
minPartitions: Option[Int
 
   // Splits offset ranges with relatively large amount of data to smalle

[spark] branch branch-3.2 updated: [SPARK-36595][SQL][SS][DOCS] Document window & session_window function in SQL API doc

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

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


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new 118a53d  [SPARK-36595][SQL][SS][DOCS] Document window & session_window 
function in SQL API doc
118a53d is described below

commit 118a53d87f9a809c6c47c061dfa20469405d4f69
Author: Jungtaek Lim 
AuthorDate: Fri Aug 27 12:39:09 2021 +0900

[SPARK-36595][SQL][SS][DOCS] Document window & session_window function in 
SQL API doc

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

This PR proposes to document `window` & `session_window` function in SQL 
API doc page.

Screenshot of functions:

> window

![스크린샷 2021-08-26 오후 6 34 
58](https://user-images.githubusercontent.com/1317309/130939754-0ea1b55e-39d4-4205-b79d-a9508c98921c.png)

> session_window

![스크린샷 2021-08-26 오후 6 35 
19](https://user-images.githubusercontent.com/1317309/130939773-b6cb4b98-88f8-4d57-a188-ee40ed7b2b08.png)

### Why are the changes needed?

Description is missing in both `window` / `session_window` functions for 
SQL API page.

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

Yes, the description of `window` / `session_window` functions will be 
available in SQL API page.

### How was this patch tested?

Only doc changes.

Closes #33846 from HeartSaVioR/SPARK-36595.

Authored-by: Jungtaek Lim 
Signed-off-by: Jungtaek Lim 
(cherry picked from commit bc32144a91c0f3b9f9242795dd0f777fb01d57d9)
Signed-off-by: Jungtaek Lim 
---
 .../sql/catalyst/expressions/SessionWindow.scala   | 28 
 .../sql/catalyst/expressions/TimeWindow.scala  | 37 ++
 .../sql-functions/sql-expression-schema.md |  8 ++---
 .../scala/org/apache/spark/sql/SQLQuerySuite.scala |  3 +-
 .../sql/expressions/ExpressionInfoSuite.scala  |  3 --
 5 files changed, 70 insertions(+), 9 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SessionWindow.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SessionWindow.scala
index eb46c0f..796ea27 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SessionWindow.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SessionWindow.scala
@@ -35,6 +35,34 @@ import org.apache.spark.unsafe.types.UTF8String
  *duration during the query execution. Note that the rows 
with negative or
  *zero gap duration will be filtered out from the 
aggregation.
  */
+// scalastyle:off line.size.limit line.contains.tab
+@ExpressionDescription(
+  usage = """
+_FUNC_(time_column, gap_duration) - Generates session window given a 
timestamp specifying column and gap duration.
+  See https://spark.apache.org/docs/latest/structured-streaming-programming-guide.html#types-of-time-windows";>'Types
 of time windows' in Structured Streaming guide doc for detailed 
explanation and examples.
+  """,
+  arguments = """
+Arguments:
+  * time_column - The column or the expression to use as the timestamp for 
windowing by time. The time column must be of TimestampType.
+  * gap_duration - A string specifying the timeout of the session 
represented as "interval value"
+(See https://spark.apache.org/docs/latest/sql-ref-literals.html#interval-literal";>Interval
 Literal for more details.) for the fixed gap duration, or
+an expression which is applied for each input and evaluated to the 
"interval value" for the dynamic gap duration.
+  """,
+  examples = """
+Examples:
+  > SELECT a, session_window.start, session_window.end, count(*) as cnt 
FROM VALUES ('A1', '2021-01-01 00:00:00'), ('A1', '2021-01-01 00:04:30'), 
('A1', '2021-01-01 00:10:00'), ('A2', '2021-01-01 00:01:00') AS tab(a, b) GROUP 
by a, _FUNC_(b, '5 minutes') ORDER BY a, start;
+A1 2021-01-01 00:00:00 2021-01-01 00:09:30 2
+A1 2021-01-01 00:10:00 2021-01-01 00:15:00 1
+A2 2021-01-01 00:01:00 2021-01-01 00:06:00 1
+  > SELECT a, session_window.start, session_window.end, count(*) as cnt 
FROM VALUES ('A1', '2021-01-01 00:00:00'), ('A1', '2021-01-01 00:04:30'), 
('A1', '2021-01-01 00:10:00'), ('A2', '2021-01-01 00:01:00'), ('A2', 
'2021-01-01 00:04:30') AS tab(a, b) GROUP by a, _FUNC_(b, CASE WHEN a = 'A1' 
THEN '5 minutes' WHEN a = 'A

[spark] branch master updated (dd3f0fa -> bc32144)

2021-08-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 dd3f0fa  [SPARK-35611][SS][FOLLOW-UP] Improve the user guide document
 add bc32144  [SPARK-36595][SQL][SS][DOCS] Document window & session_window 
function in SQL API doc

No new revisions were added by this update.

Summary of changes:
 .../sql/catalyst/expressions/SessionWindow.scala   | 28 
 .../sql/catalyst/expressions/TimeWindow.scala  | 37 ++
 .../sql-functions/sql-expression-schema.md |  8 ++---
 .../scala/org/apache/spark/sql/SQLQuerySuite.scala |  3 +-
 .../sql/expressions/ExpressionInfoSuite.scala  |  3 --
 5 files changed, 70 insertions(+), 9 deletions(-)

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



[spark] branch branch-3.2 updated: [SPARK-35312][SS][FOLLOW-UP] More documents and checking logic for the new options

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

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


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new 36c24a0  [SPARK-35312][SS][FOLLOW-UP] More documents and checking 
logic for the new options
36c24a0 is described below

commit 36c24a03bd6d00beb65377d37855544f406d1f18
Author: Yuanjian Li 
AuthorDate: Fri Aug 20 10:41:42 2021 +0900

[SPARK-35312][SS][FOLLOW-UP] More documents and checking logic for the new 
options

### What changes were proposed in this pull request?
Add more documents and checking logic for the new options 
`minOffsetPerTrigger` and `maxTriggerDelay`.

### Why are the changes needed?
Have a clear description of the behavior introduced in SPARK-35312

### Does this PR introduce _any_ user-facing change?
Yes.
If the user set minOffsetsPerTrigger > maxOffsetsPerTrigger, the new code 
will throw an AnalysisException. The original behavior is to ignore the 
maxOffsetsPerTrigger silenctly.

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

Closes #33792 from xuanyuanking/SPARK-35312-follow.

Authored-by: Yuanjian Li 
Signed-off-by: Jungtaek Lim 
(cherry picked from commit a0b24019edcd268968a7e0074b0a54988e408699)
Signed-off-by: Jungtaek Lim 
---
 docs/structured-streaming-kafka-integration.md| 15 +--
 .../apache/spark/sql/kafka010/KafkaSourceProvider.scala   | 13 +
 .../spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala   |  4 
 3 files changed, 26 insertions(+), 6 deletions(-)

diff --git a/docs/structured-streaming-kafka-integration.md 
b/docs/structured-streaming-kafka-integration.md
index 6926bbb..0ec359f 100644
--- a/docs/structured-streaming-kafka-integration.md
+++ b/docs/structured-streaming-kafka-integration.md
@@ -477,23 +477,26 @@ The following configurations are optional:
   maxOffsetsPerTrigger
   long
   none
-  streaming and batch
+  streaming query
   Rate limit on maximum number of offsets processed per trigger interval. 
The specified total number of offsets will be proportionally split across 
topicPartitions of different volume.
 
 
   minOffsetsPerTrigger
   long
   none
-  streaming and batch
-  Minimum number of offsets to be processed per trigger interval. The 
specified total number of offsets will
-   be proportionally split across topicPartitions of different volume.
+  streaming query
+  Minimum number of offsets to be processed per trigger interval. The 
specified total number of
+  offsets will be proportionally split across topicPartitions of different 
volume. Note, if the
+  maxTriggerDelay is exceeded, a trigger will be fired even if the number of 
available offsets
+  doesn't reach minOffsetsPerTrigger.
 
 
   maxTriggerDelay
   time with units
   15m
-  streaming and batch
-  Maximum amount of time for which trigger can be delayed between two 
triggers provided some data is available from the source.
+  streaming query
+  Maximum amount of time for which trigger can be delayed between two 
triggers provided some
+  data is available from the source. This option is only applicable if 
minOffsetsPerTrigger is set.
 
 
   minPartitions
diff --git 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala
 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala
index 38803b7..4a75ab0 100644
--- 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala
+++ 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala
@@ -322,6 +322,15 @@ private[kafka010] class KafkaSourceProvider extends 
DataSourceRegister
 s"Option 'kafka.${ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG}' must be 
specified for " +
   s"configuring Kafka consumer")
 }
+
+if (params.contains(MIN_OFFSET_PER_TRIGGER) && 
params.contains(MAX_OFFSET_PER_TRIGGER)) {
+  val minOffsets = params.get(MIN_OFFSET_PER_TRIGGER).get.toLong
+  val maxOffsets = params.get(MAX_OFFSET_PER_TRIGGER).get.toLong
+  if (minOffsets > maxOffsets) {
+throw new IllegalArgumentException(s"The value of 
minOffsetPerTrigger($minOffsets) is " +
+  s"higher than the maxOffsetsPerTrigger($maxOffsets).")
+  }
+}
   }
 
   private def validateStreamOptions(params: CaseInsensitiveMap[String]) = {
@@ -382,6 +391,10 @@ private[kafka010] class KafkaSourceProvider extends 
DataSourceRegister
 if (params.contains(MIN_OFFSET_PER_TRIGGER)) {
   logWarning("minOffsetsPerTrigger option ignored in batch queries")
 }
+
+if (params.contains(MAX_TRIGGER_DELAY)) {
+  logWarning("maxTriggerDelay option ignored in batch queries")
+}
   }
 
   clas

[spark] branch master updated: [SPARK-35312][SS][FOLLOW-UP] More documents and checking logic for the new options

2021-08-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 a0b2401  [SPARK-35312][SS][FOLLOW-UP] More documents and checking 
logic for the new options
a0b2401 is described below

commit a0b24019edcd268968a7e0074b0a54988e408699
Author: Yuanjian Li 
AuthorDate: Fri Aug 20 10:41:42 2021 +0900

[SPARK-35312][SS][FOLLOW-UP] More documents and checking logic for the new 
options

### What changes were proposed in this pull request?
Add more documents and checking logic for the new options 
`minOffsetPerTrigger` and `maxTriggerDelay`.

### Why are the changes needed?
Have a clear description of the behavior introduced in SPARK-35312

### Does this PR introduce _any_ user-facing change?
Yes.
If the user set minOffsetsPerTrigger > maxOffsetsPerTrigger, the new code 
will throw an AnalysisException. The original behavior is to ignore the 
maxOffsetsPerTrigger silenctly.

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

Closes #33792 from xuanyuanking/SPARK-35312-follow.

Authored-by: Yuanjian Li 
Signed-off-by: Jungtaek Lim 
---
 docs/structured-streaming-kafka-integration.md| 15 +--
 .../apache/spark/sql/kafka010/KafkaSourceProvider.scala   | 13 +
 .../spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala   |  4 
 3 files changed, 26 insertions(+), 6 deletions(-)

diff --git a/docs/structured-streaming-kafka-integration.md 
b/docs/structured-streaming-kafka-integration.md
index 6926bbb..0ec359f 100644
--- a/docs/structured-streaming-kafka-integration.md
+++ b/docs/structured-streaming-kafka-integration.md
@@ -477,23 +477,26 @@ The following configurations are optional:
   maxOffsetsPerTrigger
   long
   none
-  streaming and batch
+  streaming query
   Rate limit on maximum number of offsets processed per trigger interval. 
The specified total number of offsets will be proportionally split across 
topicPartitions of different volume.
 
 
   minOffsetsPerTrigger
   long
   none
-  streaming and batch
-  Minimum number of offsets to be processed per trigger interval. The 
specified total number of offsets will
-   be proportionally split across topicPartitions of different volume.
+  streaming query
+  Minimum number of offsets to be processed per trigger interval. The 
specified total number of
+  offsets will be proportionally split across topicPartitions of different 
volume. Note, if the
+  maxTriggerDelay is exceeded, a trigger will be fired even if the number of 
available offsets
+  doesn't reach minOffsetsPerTrigger.
 
 
   maxTriggerDelay
   time with units
   15m
-  streaming and batch
-  Maximum amount of time for which trigger can be delayed between two 
triggers provided some data is available from the source.
+  streaming query
+  Maximum amount of time for which trigger can be delayed between two 
triggers provided some
+  data is available from the source. This option is only applicable if 
minOffsetsPerTrigger is set.
 
 
   minPartitions
diff --git 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala
 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala
index 38803b7..4a75ab0 100644
--- 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala
+++ 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala
@@ -322,6 +322,15 @@ private[kafka010] class KafkaSourceProvider extends 
DataSourceRegister
 s"Option 'kafka.${ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG}' must be 
specified for " +
   s"configuring Kafka consumer")
 }
+
+if (params.contains(MIN_OFFSET_PER_TRIGGER) && 
params.contains(MAX_OFFSET_PER_TRIGGER)) {
+  val minOffsets = params.get(MIN_OFFSET_PER_TRIGGER).get.toLong
+  val maxOffsets = params.get(MAX_OFFSET_PER_TRIGGER).get.toLong
+  if (minOffsets > maxOffsets) {
+throw new IllegalArgumentException(s"The value of 
minOffsetPerTrigger($minOffsets) is " +
+  s"higher than the maxOffsetsPerTrigger($maxOffsets).")
+  }
+}
   }
 
   private def validateStreamOptions(params: CaseInsensitiveMap[String]) = {
@@ -382,6 +391,10 @@ private[kafka010] class KafkaSourceProvider extends 
DataSourceRegister
 if (params.contains(MIN_OFFSET_PER_TRIGGER)) {
   logWarning("minOffsetsPerTrigger option ignored in batch queries")
 }
+
+if (params.contains(MAX_TRIGGER_DELAY)) {
+  logWarning("maxTriggerDelay option ignored in batch queries")
+}
   }
 
   class KafkaTable(includeHeaders: Boolean) extends Table with SupportsRead 
with SupportsWrite {
diff --git 
a/external/kaf

<    1   2   3   4   5   6   7   >