Already +1 in the PR. It would be great to mention the new config in the SS
migration guide.

Ryan Blue <rb...@netflix.com.invalid> 于2020年11月11日周三 上午7:48写道:

> +1, I agree with Tom.
>
> On Tue, Nov 10, 2020 at 3:00 PM Dongjoon Hyun <dongjoon.h...@gmail.com>
> wrote:
>
>> +1 for Apache Spark 3.1.0.
>>
>> Bests,
>> Dongjoon.
>>
>> On Tue, Nov 10, 2020 at 6:17 AM Tom Graves <tgraves...@yahoo.com.invalid>
>> wrote:
>>
>>> +1 since its a correctness issue, I think its ok to change the behavior
>>> to make sure the user is aware of it and let them decide.
>>>
>>> Tom
>>>
>>> On Saturday, November 7, 2020, 01:00:11 AM CST, Liang-Chi Hsieh <
>>> vii...@gmail.com> wrote:
>>>
>>>
>>> Hi devs,
>>>
>>> In Spark structured streaming, chained stateful operators possibly
>>> produces
>>> incorrect results under the global watermark. SPARK-33259
>>> (https://issues.apache.org/jira/browse/SPARK-33259) has an example
>>> demostrating what the correctness issue could be.
>>>
>>> Currently we don't prevent users running such queries. Because the
>>> possible
>>> correctness in chained stateful operators in streaming query is not
>>> straightforward for users. From users perspective, it will possibly be
>>> considered as a Spark bug like SPARK-33259. It is also possible the worse
>>> case, users are not aware of the correctness issue and use wrong results.
>>>
>>> IMO, it is better to disable such queries and let users choose to run the
>>> query if they understand there is such risk, instead of implicitly
>>> running
>>> the query and let users to find out correctness issue by themselves.
>>>
>>> I would like to propose to disable the streaming query with possible
>>> correctness issue in chained stateful operators. The behavior can be
>>> controlled by a SQL config, so if users understand the risk and still
>>> want
>>> to run the query, they can disable the check.
>>>
>>> In the PR (https://github.com/apache/spark/pull/30210), the concern I
>>> got
>>> for now is, this changes current behavior and by default it will break
>>> some
>>> existing streaming queries. But I think it is pretty easy to disable the
>>> check with the new config. In the PR currently there is no objection but
>>> suggestion to hear more voices. Please let me know if you have some
>>> thoughts.
>>>
>>> Thanks.
>>> Liang-Chi Hsieh
>>>
>>>
>>>
>>> --
>>> Sent from: http://apache-spark-developers-list.1001551.n3.nabble.com/
>>>
>>> ---------------------------------------------------------------------
>>> To unsubscribe e-mail: dev-unsubscr...@spark.apache.org
>>>
>>>
>
> --
> Ryan Blue
> Software Engineer
> Netflix
>

Reply via email to