anishshri-db opened a new pull request, #36090:
URL: https://github.com/apache/spark/pull/36090

   
   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: 
https://spark.apache.org/contributing.html
     2. Ensure you have added or run the appropriate tests for your PR: 
https://spark.apache.org/developer-tools.html
     3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., 
'[WIP][SPARK-XXXX] Your PR title ...'.
     4. Be sure to keep the PR description updated to reflect all changes.
     5. Please write your PR title to summarize what this PR proposes.
     6. If possible, provide a concise example to reproduce the issue for a 
faster review.
     7. If you want to add a new configuration, please read the guideline first 
for naming configurations in
        
'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
     8. If you want to add or modify an error type or message, please read the 
guideline first in
        'core/src/main/resources/error/README.md'.
   -->
   
   ### What changes were proposed in this pull request?
   <!--
   Please clarify what changes you are proposing. The purpose of this section 
is to outline the changes and how this PR fixes the issue. 
   If possible, please consider writing useful notes for better and faster 
reviews in your PR. See the examples below.
     1. If you refactor some codes with changing classes, showing the class 
hierarchy will help reviewers.
     2. If you fix some SQL features, you can provide some references of other 
DBMSes.
     3. If there is design documentation, please add the link.
     4. If there is a discussion in the mailing list, please add the link.
   -->
   In the symmetric has join state manager, we can receive entries with null 
values for a key and that caused the `removeByValue` and get iterators to fail 
and run into the NullPointerException. This is possible if the state recovered 
is written from a old spark version or its corrupted on disk. 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 impl in stream stream joins.
   
   ### Why are the changes needed?
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you propose a new API, clarify the use case for a new API.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   Without these changes, if we encounter null values for stream-stream joins, 
then the 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?
   <!--
   Note that it means *any* user-facing change including all aspects such as 
the documentation fix.
   If yes, please clarify the previous behavior and the change this PR proposes 
- provide the console output, description and/or an example to show the 
behavior difference if possible.
   If possible, please also clarify if this is a user-facing change compared to 
the released Spark versions or within the unreleased branches such as master.
   If no, write 'No'.
   -->
   No
   
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some 
test cases that check the changes thoroughly including negative and positive 
cases if possible.
   If it was tested in a way different from regular unit tests, please clarify 
how you tested step by step, ideally copy and paste-able, so that other 
reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why 
it was difficult to add.
   If benchmark tests were added, please run the benchmarks in GitHub Actions 
for the consistent environment, and the instructions could accord to: 
https://spark.apache.org/developer-tools.html#github-workflow-benchmarks.
   -->
   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.
   [info] - StreamingJoinStateManager V1 - all operations with nulls in middle 
(435 milliseconds)
   15:07:53.549 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.551 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.785 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.785 WARN 
org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManager: 
`keyWithIndexToValue` returns a null value for indices with range from 
startIndex=3 and endIndex=3.
   [info] - StreamingJoinStateManager V2 - all operations with nulls in middle 
(456 milliseconds)
   [info] - SPARK-35689: StreamingJoinStateManager V1 - printable key of 
keyWithIndexToValue (390 milliseconds)
   [info] - SPARK-35689: StreamingJoinStateManager V2 - printable key of 
keyWithIndexToValue (216 milliseconds)
   15:07:54.640 WARN 
org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManagerSuite:
   
   ===== POSSIBLE THREAD LEAK IN SUITE 
o.a.s.sql.execution.streaming.state.SymmetricHashJoinStateManagerSuite, 
threads: rpc-boss-3-1 (daemon=true), shuffle-boss-6-1 (daemon=true) =====
   [info] Run completed in 5 seconds, 714 milliseconds.
   [info] Total number of tests run: 8
   [info] Suites: completed 1, aborted 0
   [info] Tests: succeeded 8, failed 0, canceled 0, ignored 0, pending 0
   [info] All tests passed.
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to