You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@ignite.apache.org by Anton Vinogradov <av...@apache.org> on 2023/06/19 19:24:28 UTC

Re: IgniteTxStateImpl's non-threadsafe fields may cause crashes and/or data loss

Folks, idea to synchronize all methods unfortunately failed :(
1) TxState has 4 implementations, a lot of changes are required
2) IgniteTxEntry is not synchronized as well ...
3) And even IgniteInternalTx implementations (10000+ lines) are not
synchronized as well ...
It seems to be unreal to refactor this properly.

Also, the methods synchronization is just provides current data read
guarantee, not a thread safety.

If I understand correctly, the only proper fix is to keep everything
unsynchronized, but guarantee every tx processing only at one thread at the
same time + data visibility.
Possible fix is to process same tx at the same thread each time, but we
already found that tx can be created at the user thread, and can be, for
example, suspended or committed from the user thread again. So, seems, it's
impossible to provide such guatantee.

But, the possible solution is to wrap each tx processing with some lock or
synchronize section, like:
synchronize(tx){
    val aaa = tx.getAAA();
    tx.updateXXX();
    tx.updateYYY();
}
This will guarantee fields visibility as we as strict tx processing, step
by step.
Single lock/synchronize should not cause the perfomance problem, I think.

But, this may cause a deadlock it case some such executions will require
another at the other threads, but related to the same tx.

And the current question is:
Do we expect that Ignite is not required to process something related to
the same tx at different threads simultaneously?

On Wed, May 24, 2023 at 4:11 PM Anton Vinogradov <av...@apache.org> wrote:

> >> could you please point to this at code, it may be not needed after the
> fix and can bring the performance growth.
> BTW, found the trick.
> Still necessary to keep copying.
>
> On Wed, May 24, 2023 at 2:44 PM Anton Vinogradov <av...@apache.org> wrote:
>
>> Andrey,
>>
>> Thanks for the tip.
>> Of course, I'll benchmark the fix before the merge.
>>
>> According to the comment,
>> >>  and return entries copy form tx state in order to avoid
>> ConcurrentModificationException.
>> , could you please point to this at code, it may be not needed after the
>> fix and can bring the performance growth.
>>
>> >> I believe that mentioned invariants were broken later but ...
>> >> ... this state should be accessed mostly from one thread
>> Code was never designed to fit this statement.
>> For example, the most of cctx.tm().newTx(...) calls dated by 2014 (which
>> means "before 2014").
>> Currently, allwost all tx creations happen not at the striped pool as
>> well as tx preparations.
>> Only 1/2 of the messages now striped correctly.
>> Of course, it's theoretically possible to process tx at the same thread
>> each time, but, global refactoring with a performance drop is required in
>> this case, I think.
>>
>> My current Idea is to finish synchronization started by you.
>> I've pepared the fix [1], got the visa and going to benchmark it.
>>
>> [1] https://github.com/apache/ignite/pull/10732/files
>>
>> On Tue, May 23, 2023 at 8:54 PM Andrey Gura <ag...@apache.org> wrote:
>>
>>> Please, run benchmarks after fixing the problem. E.g. replacing HashMap
>>> to
>>> ConcurrentHashMap can significantly affect performance.
>>>
>>> See for example comments to IGNITE-2968 issue (
>>>
>>> https://issues.apache.org/jira/browse/IGNITE-2968?focusedCommentId=15415170&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15415170
>>> ).
>>>
>>> I believe that mentioned invariants were broken later but in general I
>>> agree with Alexey, this state should be accessed mostly from one thread.
>>> Exceptional cases should be synchronized or redesigned. E.g. if metrics
>>> read a transaction's state I prefer remove these metrics or ignore some
>>> inaccuracy then performance reducing.
>>>
>>>
>>>
>>>
>>> On Fri, May 19, 2023 at 7:32 PM Ivan Daschinsky <iv...@gmail.com>
>>> wrote:
>>>
>>> > >> Tx processing is supposed to be thread bound by hashing the version
>>> to a
>>> > partition
>>> > This invariant is violated in many places. The most notorious example
>>> is tx
>>> > recovery.
>>> >
>>> > Another example: I just added an assertion that checks tId of a creator
>>> > thread with tId of an accessor thread.
>>> > TxMultiCacheAsyncOpsTest fails immediately on processing of a tx
>>> prepare
>>> > request. Looks like a big issue, IMO
>>> >
>>> >
>>> > пт, 19 мая 2023 г. в 19:11, Alexei Scherbakov <
>>> > alexey.scherbakoff@gmail.com
>>> > >:
>>> >
>>> > > Tx processing is supposed to be thread bound by hashing the version
>>> to a
>>> > > partition, see methods like [1]
>>> > > If for some cases this invariant is broken, this should be fixed.
>>> > >
>>> > > [1]
>>> > >
>>> >
>>> org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareRequest#partition
>>> > >
>>> > > пт, 19 мая 2023 г. в 15:57, Anton Vinogradov <av...@apache.org>:
>>> > >
>>> > > > Igniters,
>>> > > >
>>> > > > My team was faced with node failure [1] because of non-threadsafe
>>> > > > collections usage.
>>> > > >
>>> > > > IgniteTxStateImpl's fields
>>> > > > - activeCacheIds
>>> > > > - txMap
>>> > > > are not thread safe, but are widely used from different threads
>>> without
>>> > > the
>>> > > > proper sync.
>>> > > >
>>> > > > The main question is ... why?
>>> > > >
>>> > > > According to the research, we have no guarantee that tx will be
>>> > processed
>>> > > > at the single thread.
>>> > > > It may be processed at the several! threads at the striped pool
>>> and at
>>> > > the
>>> > > > tx recovery thread as well.
>>> > > >
>>> > > > Thread at the striped pool will be selected by the message's
>>> > partition()
>>> > > > method, which can be calculated like this:
>>> > > > - return keys != null && !keys.isEmpty() ? keys.get(0).partition()
>>> :
>>> > -1;
>>> > > > - return U.safeAbs(version().hashCode());
>>> > > > - ...,
>>> > > > so, no guarantee it is processed at the same thread (proven by
>>> tests).
>>> > > >
>>> > > > Seems, we MAY lose the data.
>>> > > > For example, ignoring some or all keys from txMap at commit.
>>> > > >
>>> > > > If anyone knows why this is not a problem (I mean sync lack, not
>>> data
>>> > > loss)
>>> > > > or how to fix this properly, please give me a hint, or correct my
>>> > > > conclusions if necessary.
>>> > > >
>>> > > > [1] https://issues.apache.org/jira/browse/IGNITE-19445
>>> > > >
>>> > >
>>> > >
>>> > > --
>>> > >
>>> > > Best regards,
>>> > > Alexei Scherbakov
>>> > >
>>> >
>>> >
>>> > --
>>> > Sincerely yours, Ivan Daschinskiy
>>> >
>>>
>>

Re: IgniteTxStateImpl's non-threadsafe fields may cause crashes and/or data loss

Posted by Anton Vinogradov <av...@apache.org>.
Yes, see the root issue [1], but it [2] is happened during the recovery.

> AFAIK all required parts of txn processing are already properly linearized
I see a DCL replacements like IgniteTxAdapter#finishFuture() which call
this into question.

Anyway, this may be the correct statement, but TX processing is a complex
process and how can we be sure that there is no unexpected concurrent
executions or stale reads like we already got [1]?

> other parts are ready to be processed in parallel (like txn recovery)
Agree.
Recovery can be refactored to be executed at the striped pool, the only
problem here is the concurrent execution with a user's thread.

Ok, lets agree for now that we have this process linearized, this still
does not mean we have visibility guarantee.
Could you please help me to find how do we achive visibility when tx is
started at the user's thread and the responses are processed at the striped
pool?

[2] https://issues.apache.org/jira/browse/IGNITE-19445
[1]
https://issues.apache.org/jira/secure/attachment/13058004/reproducer-19445.patch

On Wed, Jun 21, 2023 at 5:58 PM Alexei Scherbakov <
alexey.scherbakoff@gmail.com> wrote:

> Do we have a real reproducer for thread unsafe behavior, which causes data
> inconsistency ?
> AFAIK all required parts of txn processing are already properly linearized,
> and other parts are ready to be processed in parallel (like txn recovery)
>
> пн, 19 июн. 2023 г. в 22:25, Anton Vinogradov <av...@apache.org>:
>
> > Folks, idea to synchronize all methods unfortunately failed :(
> > 1) TxState has 4 implementations, a lot of changes are required
> > 2) IgniteTxEntry is not synchronized as well ...
> > 3) And even IgniteInternalTx implementations (10000+ lines) are not
> > synchronized as well ...
> > It seems to be unreal to refactor this properly.
> >
> > Also, the methods synchronization is just provides current data read
> > guarantee, not a thread safety.
> >
> > If I understand correctly, the only proper fix is to keep everything
> > unsynchronized, but guarantee every tx processing only at one thread at
> the
> > same time + data visibility.
> > Possible fix is to process same tx at the same thread each time, but we
> > already found that tx can be created at the user thread, and can be, for
> > example, suspended or committed from the user thread again. So, seems,
> it's
> > impossible to provide such guatantee.
> >
> > But, the possible solution is to wrap each tx processing with some lock
> or
> > synchronize section, like:
> > synchronize(tx){
> >     val aaa = tx.getAAA();
> >     tx.updateXXX();
> >     tx.updateYYY();
> > }
> > This will guarantee fields visibility as we as strict tx processing, step
> > by step.
> > Single lock/synchronize should not cause the perfomance problem, I think.
> >
> > But, this may cause a deadlock it case some such executions will require
> > another at the other threads, but related to the same tx.
> >
> > And the current question is:
> > Do we expect that Ignite is not required to process something related to
> > the same tx at different threads simultaneously?
> >
> > On Wed, May 24, 2023 at 4:11 PM Anton Vinogradov <av...@apache.org> wrote:
> >
> > > >> could you please point to this at code, it may be not needed after
> the
> > > fix and can bring the performance growth.
> > > BTW, found the trick.
> > > Still necessary to keep copying.
> > >
> > > On Wed, May 24, 2023 at 2:44 PM Anton Vinogradov <av...@apache.org>
> wrote:
> > >
> > >> Andrey,
> > >>
> > >> Thanks for the tip.
> > >> Of course, I'll benchmark the fix before the merge.
> > >>
> > >> According to the comment,
> > >> >>  and return entries copy form tx state in order to avoid
> > >> ConcurrentModificationException.
> > >> , could you please point to this at code, it may be not needed after
> the
> > >> fix and can bring the performance growth.
> > >>
> > >> >> I believe that mentioned invariants were broken later but ...
> > >> >> ... this state should be accessed mostly from one thread
> > >> Code was never designed to fit this statement.
> > >> For example, the most of cctx.tm().newTx(...) calls dated by 2014
> > (which
> > >> means "before 2014").
> > >> Currently, allwost all tx creations happen not at the striped pool as
> > >> well as tx preparations.
> > >> Only 1/2 of the messages now striped correctly.
> > >> Of course, it's theoretically possible to process tx at the same
> thread
> > >> each time, but, global refactoring with a performance drop is required
> > in
> > >> this case, I think.
> > >>
> > >> My current Idea is to finish synchronization started by you.
> > >> I've pepared the fix [1], got the visa and going to benchmark it.
> > >>
> > >> [1] https://github.com/apache/ignite/pull/10732/files
> > >>
> > >> On Tue, May 23, 2023 at 8:54 PM Andrey Gura <ag...@apache.org> wrote:
> > >>
> > >>> Please, run benchmarks after fixing the problem. E.g. replacing
> HashMap
> > >>> to
> > >>> ConcurrentHashMap can significantly affect performance.
> > >>>
> > >>> See for example comments to IGNITE-2968 issue (
> > >>>
> > >>>
> >
> https://issues.apache.org/jira/browse/IGNITE-2968?focusedCommentId=15415170&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15415170
> > >>> ).
> > >>>
> > >>> I believe that mentioned invariants were broken later but in general
> I
> > >>> agree with Alexey, this state should be accessed mostly from one
> > thread.
> > >>> Exceptional cases should be synchronized or redesigned. E.g. if
> metrics
> > >>> read a transaction's state I prefer remove these metrics or ignore
> some
> > >>> inaccuracy then performance reducing.
> > >>>
> > >>>
> > >>>
> > >>>
> > >>> On Fri, May 19, 2023 at 7:32 PM Ivan Daschinsky <ivandasch@gmail.com
> >
> > >>> wrote:
> > >>>
> > >>> > >> Tx processing is supposed to be thread bound by hashing the
> > version
> > >>> to a
> > >>> > partition
> > >>> > This invariant is violated in many places. The most notorious
> example
> > >>> is tx
> > >>> > recovery.
> > >>> >
> > >>> > Another example: I just added an assertion that checks tId of a
> > creator
> > >>> > thread with tId of an accessor thread.
> > >>> > TxMultiCacheAsyncOpsTest fails immediately on processing of a tx
> > >>> prepare
> > >>> > request. Looks like a big issue, IMO
> > >>> >
> > >>> >
> > >>> > пт, 19 мая 2023 г. в 19:11, Alexei Scherbakov <
> > >>> > alexey.scherbakoff@gmail.com
> > >>> > >:
> > >>> >
> > >>> > > Tx processing is supposed to be thread bound by hashing the
> version
> > >>> to a
> > >>> > > partition, see methods like [1]
> > >>> > > If for some cases this invariant is broken, this should be fixed.
> > >>> > >
> > >>> > > [1]
> > >>> > >
> > >>> >
> > >>>
> >
> org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareRequest#partition
> > >>> > >
> > >>> > > пт, 19 мая 2023 г. в 15:57, Anton Vinogradov <av...@apache.org>:
> > >>> > >
> > >>> > > > Igniters,
> > >>> > > >
> > >>> > > > My team was faced with node failure [1] because of
> non-threadsafe
> > >>> > > > collections usage.
> > >>> > > >
> > >>> > > > IgniteTxStateImpl's fields
> > >>> > > > - activeCacheIds
> > >>> > > > - txMap
> > >>> > > > are not thread safe, but are widely used from different threads
> > >>> without
> > >>> > > the
> > >>> > > > proper sync.
> > >>> > > >
> > >>> > > > The main question is ... why?
> > >>> > > >
> > >>> > > > According to the research, we have no guarantee that tx will be
> > >>> > processed
> > >>> > > > at the single thread.
> > >>> > > > It may be processed at the several! threads at the striped pool
> > >>> and at
> > >>> > > the
> > >>> > > > tx recovery thread as well.
> > >>> > > >
> > >>> > > > Thread at the striped pool will be selected by the message's
> > >>> > partition()
> > >>> > > > method, which can be calculated like this:
> > >>> > > > - return keys != null && !keys.isEmpty() ?
> > keys.get(0).partition()
> > >>> :
> > >>> > -1;
> > >>> > > > - return U.safeAbs(version().hashCode());
> > >>> > > > - ...,
> > >>> > > > so, no guarantee it is processed at the same thread (proven by
> > >>> tests).
> > >>> > > >
> > >>> > > > Seems, we MAY lose the data.
> > >>> > > > For example, ignoring some or all keys from txMap at commit.
> > >>> > > >
> > >>> > > > If anyone knows why this is not a problem (I mean sync lack,
> not
> > >>> data
> > >>> > > loss)
> > >>> > > > or how to fix this properly, please give me a hint, or correct
> my
> > >>> > > > conclusions if necessary.
> > >>> > > >
> > >>> > > > [1] https://issues.apache.org/jira/browse/IGNITE-19445
> > >>> > > >
> > >>> > >
> > >>> > >
> > >>> > > --
> > >>> > >
> > >>> > > Best regards,
> > >>> > > Alexei Scherbakov
> > >>> > >
> > >>> >
> > >>> >
> > >>> > --
> > >>> > Sincerely yours, Ivan Daschinskiy
> > >>> >
> > >>>
> > >>
> >
>
>
> --
>
> Best regards,
> Alexei Scherbakov
>

Re: IgniteTxStateImpl's non-threadsafe fields may cause crashes and/or data loss

Posted by Alexei Scherbakov <al...@gmail.com>.
Do we have a real reproducer for thread unsafe behavior, which causes data
inconsistency ?
AFAIK all required parts of txn processing are already properly linearized,
and other parts are ready to be processed in parallel (like txn recovery)

пн, 19 июн. 2023 г. в 22:25, Anton Vinogradov <av...@apache.org>:

> Folks, idea to synchronize all methods unfortunately failed :(
> 1) TxState has 4 implementations, a lot of changes are required
> 2) IgniteTxEntry is not synchronized as well ...
> 3) And even IgniteInternalTx implementations (10000+ lines) are not
> synchronized as well ...
> It seems to be unreal to refactor this properly.
>
> Also, the methods synchronization is just provides current data read
> guarantee, not a thread safety.
>
> If I understand correctly, the only proper fix is to keep everything
> unsynchronized, but guarantee every tx processing only at one thread at the
> same time + data visibility.
> Possible fix is to process same tx at the same thread each time, but we
> already found that tx can be created at the user thread, and can be, for
> example, suspended or committed from the user thread again. So, seems, it's
> impossible to provide such guatantee.
>
> But, the possible solution is to wrap each tx processing with some lock or
> synchronize section, like:
> synchronize(tx){
>     val aaa = tx.getAAA();
>     tx.updateXXX();
>     tx.updateYYY();
> }
> This will guarantee fields visibility as we as strict tx processing, step
> by step.
> Single lock/synchronize should not cause the perfomance problem, I think.
>
> But, this may cause a deadlock it case some such executions will require
> another at the other threads, but related to the same tx.
>
> And the current question is:
> Do we expect that Ignite is not required to process something related to
> the same tx at different threads simultaneously?
>
> On Wed, May 24, 2023 at 4:11 PM Anton Vinogradov <av...@apache.org> wrote:
>
> > >> could you please point to this at code, it may be not needed after the
> > fix and can bring the performance growth.
> > BTW, found the trick.
> > Still necessary to keep copying.
> >
> > On Wed, May 24, 2023 at 2:44 PM Anton Vinogradov <av...@apache.org> wrote:
> >
> >> Andrey,
> >>
> >> Thanks for the tip.
> >> Of course, I'll benchmark the fix before the merge.
> >>
> >> According to the comment,
> >> >>  and return entries copy form tx state in order to avoid
> >> ConcurrentModificationException.
> >> , could you please point to this at code, it may be not needed after the
> >> fix and can bring the performance growth.
> >>
> >> >> I believe that mentioned invariants were broken later but ...
> >> >> ... this state should be accessed mostly from one thread
> >> Code was never designed to fit this statement.
> >> For example, the most of cctx.tm().newTx(...) calls dated by 2014
> (which
> >> means "before 2014").
> >> Currently, allwost all tx creations happen not at the striped pool as
> >> well as tx preparations.
> >> Only 1/2 of the messages now striped correctly.
> >> Of course, it's theoretically possible to process tx at the same thread
> >> each time, but, global refactoring with a performance drop is required
> in
> >> this case, I think.
> >>
> >> My current Idea is to finish synchronization started by you.
> >> I've pepared the fix [1], got the visa and going to benchmark it.
> >>
> >> [1] https://github.com/apache/ignite/pull/10732/files
> >>
> >> On Tue, May 23, 2023 at 8:54 PM Andrey Gura <ag...@apache.org> wrote:
> >>
> >>> Please, run benchmarks after fixing the problem. E.g. replacing HashMap
> >>> to
> >>> ConcurrentHashMap can significantly affect performance.
> >>>
> >>> See for example comments to IGNITE-2968 issue (
> >>>
> >>>
> https://issues.apache.org/jira/browse/IGNITE-2968?focusedCommentId=15415170&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15415170
> >>> ).
> >>>
> >>> I believe that mentioned invariants were broken later but in general I
> >>> agree with Alexey, this state should be accessed mostly from one
> thread.
> >>> Exceptional cases should be synchronized or redesigned. E.g. if metrics
> >>> read a transaction's state I prefer remove these metrics or ignore some
> >>> inaccuracy then performance reducing.
> >>>
> >>>
> >>>
> >>>
> >>> On Fri, May 19, 2023 at 7:32 PM Ivan Daschinsky <iv...@gmail.com>
> >>> wrote:
> >>>
> >>> > >> Tx processing is supposed to be thread bound by hashing the
> version
> >>> to a
> >>> > partition
> >>> > This invariant is violated in many places. The most notorious example
> >>> is tx
> >>> > recovery.
> >>> >
> >>> > Another example: I just added an assertion that checks tId of a
> creator
> >>> > thread with tId of an accessor thread.
> >>> > TxMultiCacheAsyncOpsTest fails immediately on processing of a tx
> >>> prepare
> >>> > request. Looks like a big issue, IMO
> >>> >
> >>> >
> >>> > пт, 19 мая 2023 г. в 19:11, Alexei Scherbakov <
> >>> > alexey.scherbakoff@gmail.com
> >>> > >:
> >>> >
> >>> > > Tx processing is supposed to be thread bound by hashing the version
> >>> to a
> >>> > > partition, see methods like [1]
> >>> > > If for some cases this invariant is broken, this should be fixed.
> >>> > >
> >>> > > [1]
> >>> > >
> >>> >
> >>>
> org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareRequest#partition
> >>> > >
> >>> > > пт, 19 мая 2023 г. в 15:57, Anton Vinogradov <av...@apache.org>:
> >>> > >
> >>> > > > Igniters,
> >>> > > >
> >>> > > > My team was faced with node failure [1] because of non-threadsafe
> >>> > > > collections usage.
> >>> > > >
> >>> > > > IgniteTxStateImpl's fields
> >>> > > > - activeCacheIds
> >>> > > > - txMap
> >>> > > > are not thread safe, but are widely used from different threads
> >>> without
> >>> > > the
> >>> > > > proper sync.
> >>> > > >
> >>> > > > The main question is ... why?
> >>> > > >
> >>> > > > According to the research, we have no guarantee that tx will be
> >>> > processed
> >>> > > > at the single thread.
> >>> > > > It may be processed at the several! threads at the striped pool
> >>> and at
> >>> > > the
> >>> > > > tx recovery thread as well.
> >>> > > >
> >>> > > > Thread at the striped pool will be selected by the message's
> >>> > partition()
> >>> > > > method, which can be calculated like this:
> >>> > > > - return keys != null && !keys.isEmpty() ?
> keys.get(0).partition()
> >>> :
> >>> > -1;
> >>> > > > - return U.safeAbs(version().hashCode());
> >>> > > > - ...,
> >>> > > > so, no guarantee it is processed at the same thread (proven by
> >>> tests).
> >>> > > >
> >>> > > > Seems, we MAY lose the data.
> >>> > > > For example, ignoring some or all keys from txMap at commit.
> >>> > > >
> >>> > > > If anyone knows why this is not a problem (I mean sync lack, not
> >>> data
> >>> > > loss)
> >>> > > > or how to fix this properly, please give me a hint, or correct my
> >>> > > > conclusions if necessary.
> >>> > > >
> >>> > > > [1] https://issues.apache.org/jira/browse/IGNITE-19445
> >>> > > >
> >>> > >
> >>> > >
> >>> > > --
> >>> > >
> >>> > > Best regards,
> >>> > > Alexei Scherbakov
> >>> > >
> >>> >
> >>> >
> >>> > --
> >>> > Sincerely yours, Ivan Daschinskiy
> >>> >
> >>>
> >>
>


-- 

Best regards,
Alexei Scherbakov