Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[SPARK-45655][SQL][SS] Allow non-deterministic expressions inside Agg…
…regateFunctions in CollectMetrics ### What changes were proposed in this pull request? This PR allows non-deterministic expressions wrapped inside an `AggregateFunction` such as `count` inside `CollectMetrics` node. `CollectMetrics` is used to collect arbitrary metrics from the query, in certain scenarios user would like to collect metrics for filtering based on non-deterministic expressions (see query example below). Currently, Analyzer does not allow non-deterministic expressions inside a `AggregateFunction` for `CollectMetrics`. This constraint is relaxed to allow collection of such metrics. Note that the metrics are relevant for a completed batch, and can change if the batch is replayed (because non-deterministic expression can behave differently for different runs). While working on this feature, I found a issue with `checkMetric` logic to validate non-deterministic expressions inside an AggregateExpression. An expression is determined as non-deterministic if any of its children is non-deterministic, hence we need to match the case for `!e.deterministic && !seenAggregate` after we have matched if the current expression is a AggregateExpression. If the current expression is a AggregateExpression, we should validate further down in the tree recursively - otherwise we will fail for any non-deterministic expression. ``` val inputData = MemoryStream[Timestamp] inputData.toDF() .filter("value < current_date()") .observe("metrics", count(expr("value >= current_date()")).alias("dropped")) .writeStream .queryName("ts_metrics_test") .format("memory") .outputMode("append") .start() ``` ### Why are the changes needed? 1. Added a testcase to calculate dropped rows (by `CurrentBatchTimestamp`) and ensure the query is successful. As an example, the query below fails (without this change) due to observe call on the DataFrame. ``` val inputData = MemoryStream[Timestamp] inputData.toDF() .filter("value < current_date()") .observe("metrics", count(expr("value >= current_date()")).alias("dropped")) .writeStream .queryName("ts_metrics_test") .format("memory") .outputMode("append") .start() ``` 2. Added testing in AnalysisSuite for non-deterministic expressions inside a AggregateFunction. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Unit test cases added. ``` [warn] 20 warnings found WARNING: Using incubator modules: jdk.incubator.vector, jdk.incubator.foreign [info] StreamingQueryStatusAndProgressSuite: 09:14:39.684 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] - StreamingQueryProgress - prettyJson (436 milliseconds) [info] - StreamingQueryProgress - json (3 milliseconds) [info] - StreamingQueryProgress - toString (5 milliseconds) [info] - StreamingQueryProgress - jsonString and fromJson (163 milliseconds) [info] - StreamingQueryStatus - prettyJson (1 millisecond) [info] - StreamingQueryStatus - json (1 millisecond) [info] - StreamingQueryStatus - toString (2 milliseconds) 09:14:41.674 WARN org.apache.spark.sql.execution.streaming.ResolveWriteToStream: Temporary checkpoint location created which is deleted normally when the query didn't fail: /Users/bhuwan.sahni/workspace/spark/target/tmp/temporary-34d2749f-f4d0 -46d8-bc51-29da6411e1c5. If it's required to delete it under any circumstances, please set spark.sql.streaming.forceDeleteTempCheckpointLocation to true. Important to know deleting temp checkpoint folder is best effort. 09:14:41.710 WARN org.apache.spark.sql.execution.streaming.ResolveWriteToStream: spark.sql.adaptive.enabled is not supported in streaming DataFrames/Datasets and will be disabled. [info] - progress classes should be Serializable (5 seconds, 552 milliseconds) 09:14:46.345 WARN org.apache.spark.sql.execution.streaming.ResolveWriteToStream: Temporary checkpoint location created which is deleted normally when the query didn't fail: /Users/bhuwan.sahni/workspace/spark/target/tmp/temporary-3a41d397-c3c1 -490b-9cc7-d775b0c42208. If it's required to delete it under any circumstances, please set spark.sql.streaming.forceDeleteTempCheckpointLocation to true. Important to know deleting temp checkpoint folder is best effort. 09:14:46.345 WARN org.apache.spark.sql.execution.streaming.ResolveWriteToStream: spark.sql.adaptive.enabled is not supported in streaming DataFrames/Datasets and will be disabled. [info] - SPARK-19378: Continue reporting stateOp metrics even if there is no active trigger (1 second, 337 milliseconds) 09:14:47.677 WARN org.apache.spark.sql.execution.streaming.ResolveWriteToStream: spark.sql.adaptive.enabled is not supported in streaming DataFrames/Datasets and will be disabled. [info] - SPARK-29973: Make `processedRowsPerSecond` calculated more accurately and meaningfully (455 milliseconds) 09:14:48.174 WARN org.apache.spark.sql.execution.streaming.ResolveWriteToStream: Temporary checkpoint location created which is deleted normally when the query didn't fail: /Users/bhuwan.sahni/workspace/spark/target/tmp/temporary-360fc3b9-a2c5 -430c-a892-c9869f1f8339. If it's required to delete it under any circumstances, please set spark.sql.streaming.forceDeleteTempCheckpointLocation to true. Important to know deleting temp checkpoint folder is best effort. 09:14:48.174 WARN org.apache.spark.sql.execution.streaming.ResolveWriteToStream: spark.sql.adaptive.enabled is not supported in streaming DataFrames/Datasets and will be disabled. [info] - SPARK-45655: Use current batch timestamp in observe API (587 milliseconds) 09:14:48.768 WARN org.apache.spark.sql.streaming.StreamingQueryStatusAndProgressSuite: ``` ### Was this patch authored or co-authored using generative AI tooling? No Closes #43517 from sahnib/SPARK-45655. Authored-by: Bhuwan Sahni <bhuwan.sahni@databricks.com> Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
- Loading branch information