Re: IEP-61 Transaction API desing for Ignite 3

2021-11-30 Thread Pavel Tupitsyn
Val, Alexei - no objections from my side, thanks for the explanation.

On Tue, Nov 30, 2021 at 1:18 PM Alexei Scherbakov <
alexey.scherbak...@gmail.com> wrote:

> Pavel, I agree with Val to avoid overloading due to a loss of API
> transparency.
>
> Val, moving the tx argument at the first position seems good to me.
>
> пн, 29 нояб. 2021 г. в 22:03, Valentin Kulichenko <
> valentin.kuliche...@gmail.com>:
>
> > Alexei,
> >
> > One more comment: I actually think that the transaction should be the
> first
> > argument, not the last. This way it's easier to keep the API consistent.
> > For example, if a method uses varargs as one of the parameters, you won't
> > be able to put the tx parameter at the end. There might be other cases as
> > well. What do you think?
> >
> > -Val
> >
> > On Mon, Nov 29, 2021 at 10:59 AM Valentin Kulichenko <
> > valentin.kuliche...@gmail.com> wrote:
> >
> > > I like Alexei's suggestion. This seems to be the most transparent and
> > > explicit approach. Basically, this ensures that the user is always
> aware
> > of
> > > whether an operation is enlisted in a transaction or not. Any other
> > option
> > > is either error-prone, or introduces unnecessary counter-intuitive
> > > limitations.
> > >
> > > I don't think we should keep overloads without the tx parameter,
> because
> > > that will pretty much eliminate the value of this change. One thing we
> > can
> > > do to address this is to have separate "non-tx" views, which can only
> be
> > > used to execute implicit transactions. But I would look at this after
> we
> > > more or less stabilize the primary API.
> > >
> > > -Val
> > >
> > > On Mon, Nov 29, 2021 at 5:03 AM Pavel Tupitsyn 
> > > wrote:
> > >
> > >> Alexei,
> > >>
> > >> Are we going to offer an overload without tx parameter?
> > >>
> > >> getAsync(K key);
> > >> getAsync(K key, Transaction tx);
> > >>
> > >> On Mon, Nov 29, 2021 at 3:43 PM Alexei Scherbakov <
> > >> alexey.scherbak...@gmail.com> wrote:
> > >>
> > >> > Pavel,
> > >> >
> > >> > The problem with a current approach to me is the possibility of
> > >> forgetting
> > >> > to enlist a table into a transaction, because it is not enforced.
> > >> > Having the explicit argument for this purpose seems less error-prone
> > to
> > >> me.
> > >> >
> > >> > пн, 29 нояб. 2021 г. в 15:13, Pavel Tupitsyn  >:
> > >> >
> > >> > > Taras, yes, yours is the actual syntax in main branch right now,
> > >> > > I've skipped the tx argument in my code accidentally.
> > >> > >
> > >> > > On Mon, Nov 29, 2021 at 3:03 PM Taras Ledkov <
> tled...@gridgain.com>
> > >> > wrote:
> > >> > >
> > >> > > > Hi colleagues,
> > >> > > >
> > >> > > > 2Pavel:
> > >> > > > > RecordView txView = view.withTransaction();
> > >> > > > Can we use the syntax (see below) to attach the table /
> operation
> > to
> > >> > the
> > >> > > > started transaction?
> > >> > > > RecordView  txPersonView =
> > >> > > > person.recordView().withTransaction(txView.transaction());
> > >> > > >
> > >> > > >
> > >> > > > On Mon, Nov 29, 2021 at 1:34 PM Pavel Tupitsyn <
> > >> ptupit...@apache.org>
> > >> > > > wrote:
> > >> > > >
> > >> > > > > Alexei,
> > >> > > > >
> > >> > > > > I agree that runInTransaction is confusing and error-prone.
> > >> > > > >
> > >> > > > > But we already have view.withTransaction(), which seems to be
> > the
> > >> > most
> > >> > > > > boilerplate-free approach.
> > >> > > > > The example above will look like this:
> > >> > > > >
> > >> > > > > public void testMixedPutGet() throws TransactionException {
> > >> > > > > RecordView view = accounts.recordView();
> > >> > > > >
> > >> > > > > view.upsert(makeValue(1, BALANCE_1));
> > >> > > > >
> > >> > > > > RecordView txView = view.withTransaction();
> > >> > > > >
> > >> > > > > txView.getAsync(makeKey(1)).thenCompose(r ->
> > >> > > > > txView.upsertAsync(makeValue(1, r.doubleValue("balance") +
> > DELTA),
> > >> > > > > tx)).thenCompose(txView.transaction().commitAsync()).join();
> > >> > > > >
> > >> > > > > assertEquals(BALANCE_1 + DELTA,
> > >> > > > > view.get(makeKey(1)).doubleValue("balance"));
> > >> > > > > }
> > >> > > > >
> > >> > > > > Is there any problem with this?
> > >> > > > >
> > >> > > > > On Mon, Nov 29, 2021 at 10:45 AM Alexei Scherbakov <
> > >> > > > > alexey.scherbak...@gmail.com> wrote:
> > >> > > > >
> > >> > > > > > Folks,
> > >> > > > > >
> > >> > > > > > Recently I've pushed transactions support phase 1 for Ignite
> > 3,
> > >> see
> > >> > > > [1].
> > >> > > > > > Feel free to give feedback.
> > >> > > > > > Current implementation attempts to automatically enlist a
> > table
> > >> > into
> > >> > > > > > transaction if it's started using [2] or [3] by using thread
> > >> local
> > >> > > > > context,
> > >> > > > > > similar to Ignite 2 approach, to reduce the amount of
> > >> boilerplate
> > >> > > code.
> > >> > > > > > But it turns out such an approach still has unacceptable
> > >> drawbacks
> > >> > > > from a

Re: IEP-61 Transaction API desing for Ignite 3

2021-11-30 Thread Alexei Scherbakov
Pavel, I agree with Val to avoid overloading due to a loss of API
transparency.

Val, moving the tx argument at the first position seems good to me.

пн, 29 нояб. 2021 г. в 22:03, Valentin Kulichenko <
valentin.kuliche...@gmail.com>:

> Alexei,
>
> One more comment: I actually think that the transaction should be the first
> argument, not the last. This way it's easier to keep the API consistent.
> For example, if a method uses varargs as one of the parameters, you won't
> be able to put the tx parameter at the end. There might be other cases as
> well. What do you think?
>
> -Val
>
> On Mon, Nov 29, 2021 at 10:59 AM Valentin Kulichenko <
> valentin.kuliche...@gmail.com> wrote:
>
> > I like Alexei's suggestion. This seems to be the most transparent and
> > explicit approach. Basically, this ensures that the user is always aware
> of
> > whether an operation is enlisted in a transaction or not. Any other
> option
> > is either error-prone, or introduces unnecessary counter-intuitive
> > limitations.
> >
> > I don't think we should keep overloads without the tx parameter, because
> > that will pretty much eliminate the value of this change. One thing we
> can
> > do to address this is to have separate "non-tx" views, which can only be
> > used to execute implicit transactions. But I would look at this after we
> > more or less stabilize the primary API.
> >
> > -Val
> >
> > On Mon, Nov 29, 2021 at 5:03 AM Pavel Tupitsyn 
> > wrote:
> >
> >> Alexei,
> >>
> >> Are we going to offer an overload without tx parameter?
> >>
> >> getAsync(K key);
> >> getAsync(K key, Transaction tx);
> >>
> >> On Mon, Nov 29, 2021 at 3:43 PM Alexei Scherbakov <
> >> alexey.scherbak...@gmail.com> wrote:
> >>
> >> > Pavel,
> >> >
> >> > The problem with a current approach to me is the possibility of
> >> forgetting
> >> > to enlist a table into a transaction, because it is not enforced.
> >> > Having the explicit argument for this purpose seems less error-prone
> to
> >> me.
> >> >
> >> > пн, 29 нояб. 2021 г. в 15:13, Pavel Tupitsyn :
> >> >
> >> > > Taras, yes, yours is the actual syntax in main branch right now,
> >> > > I've skipped the tx argument in my code accidentally.
> >> > >
> >> > > On Mon, Nov 29, 2021 at 3:03 PM Taras Ledkov 
> >> > wrote:
> >> > >
> >> > > > Hi colleagues,
> >> > > >
> >> > > > 2Pavel:
> >> > > > > RecordView txView = view.withTransaction();
> >> > > > Can we use the syntax (see below) to attach the table / operation
> to
> >> > the
> >> > > > started transaction?
> >> > > > RecordView  txPersonView =
> >> > > > person.recordView().withTransaction(txView.transaction());
> >> > > >
> >> > > >
> >> > > > On Mon, Nov 29, 2021 at 1:34 PM Pavel Tupitsyn <
> >> ptupit...@apache.org>
> >> > > > wrote:
> >> > > >
> >> > > > > Alexei,
> >> > > > >
> >> > > > > I agree that runInTransaction is confusing and error-prone.
> >> > > > >
> >> > > > > But we already have view.withTransaction(), which seems to be
> the
> >> > most
> >> > > > > boilerplate-free approach.
> >> > > > > The example above will look like this:
> >> > > > >
> >> > > > > public void testMixedPutGet() throws TransactionException {
> >> > > > > RecordView view = accounts.recordView();
> >> > > > >
> >> > > > > view.upsert(makeValue(1, BALANCE_1));
> >> > > > >
> >> > > > > RecordView txView = view.withTransaction();
> >> > > > >
> >> > > > > txView.getAsync(makeKey(1)).thenCompose(r ->
> >> > > > > txView.upsertAsync(makeValue(1, r.doubleValue("balance") +
> DELTA),
> >> > > > > tx)).thenCompose(txView.transaction().commitAsync()).join();
> >> > > > >
> >> > > > > assertEquals(BALANCE_1 + DELTA,
> >> > > > > view.get(makeKey(1)).doubleValue("balance"));
> >> > > > > }
> >> > > > >
> >> > > > > Is there any problem with this?
> >> > > > >
> >> > > > > On Mon, Nov 29, 2021 at 10:45 AM Alexei Scherbakov <
> >> > > > > alexey.scherbak...@gmail.com> wrote:
> >> > > > >
> >> > > > > > Folks,
> >> > > > > >
> >> > > > > > Recently I've pushed transactions support phase 1 for Ignite
> 3,
> >> see
> >> > > > [1].
> >> > > > > > Feel free to give feedback.
> >> > > > > > Current implementation attempts to automatically enlist a
> table
> >> > into
> >> > > > > > transaction if it's started using [2] or [3] by using thread
> >> local
> >> > > > > context,
> >> > > > > > similar to Ignite 2 approach, to reduce the amount of
> >> boilerplate
> >> > > code.
> >> > > > > > But it turns out such an approach still has unacceptable
> >> drawbacks
> >> > > > from a
> >> > > > > > user experience point of view.
> >> > > > > >
> >> > > > > > Consider the example [4]:
> >> > > > > >
> >> > > > > > public void testMixedPutGet() throws TransactionException {
> >> > > > > > accounts.recordView().upsert(makeValue(1, BALANCE_1));
> >> > > > > >
> >> > > > > > igniteTransactions.runInTransaction(tx -> {
> >> > > > > > var txAcc =
> >> accounts.recordView().withTransaction(tx);
> >> > > > > >
> >> > > > 

Re: IEP-61 Transaction API desing for Ignite 3

2021-11-29 Thread Valentin Kulichenko
Alexei,

One more comment: I actually think that the transaction should be the first
argument, not the last. This way it's easier to keep the API consistent.
For example, if a method uses varargs as one of the parameters, you won't
be able to put the tx parameter at the end. There might be other cases as
well. What do you think?

-Val

On Mon, Nov 29, 2021 at 10:59 AM Valentin Kulichenko <
valentin.kuliche...@gmail.com> wrote:

> I like Alexei's suggestion. This seems to be the most transparent and
> explicit approach. Basically, this ensures that the user is always aware of
> whether an operation is enlisted in a transaction or not. Any other option
> is either error-prone, or introduces unnecessary counter-intuitive
> limitations.
>
> I don't think we should keep overloads without the tx parameter, because
> that will pretty much eliminate the value of this change. One thing we can
> do to address this is to have separate "non-tx" views, which can only be
> used to execute implicit transactions. But I would look at this after we
> more or less stabilize the primary API.
>
> -Val
>
> On Mon, Nov 29, 2021 at 5:03 AM Pavel Tupitsyn 
> wrote:
>
>> Alexei,
>>
>> Are we going to offer an overload without tx parameter?
>>
>> getAsync(K key);
>> getAsync(K key, Transaction tx);
>>
>> On Mon, Nov 29, 2021 at 3:43 PM Alexei Scherbakov <
>> alexey.scherbak...@gmail.com> wrote:
>>
>> > Pavel,
>> >
>> > The problem with a current approach to me is the possibility of
>> forgetting
>> > to enlist a table into a transaction, because it is not enforced.
>> > Having the explicit argument for this purpose seems less error-prone to
>> me.
>> >
>> > пн, 29 нояб. 2021 г. в 15:13, Pavel Tupitsyn :
>> >
>> > > Taras, yes, yours is the actual syntax in main branch right now,
>> > > I've skipped the tx argument in my code accidentally.
>> > >
>> > > On Mon, Nov 29, 2021 at 3:03 PM Taras Ledkov 
>> > wrote:
>> > >
>> > > > Hi colleagues,
>> > > >
>> > > > 2Pavel:
>> > > > > RecordView txView = view.withTransaction();
>> > > > Can we use the syntax (see below) to attach the table / operation to
>> > the
>> > > > started transaction?
>> > > > RecordView  txPersonView =
>> > > > person.recordView().withTransaction(txView.transaction());
>> > > >
>> > > >
>> > > > On Mon, Nov 29, 2021 at 1:34 PM Pavel Tupitsyn <
>> ptupit...@apache.org>
>> > > > wrote:
>> > > >
>> > > > > Alexei,
>> > > > >
>> > > > > I agree that runInTransaction is confusing and error-prone.
>> > > > >
>> > > > > But we already have view.withTransaction(), which seems to be the
>> > most
>> > > > > boilerplate-free approach.
>> > > > > The example above will look like this:
>> > > > >
>> > > > > public void testMixedPutGet() throws TransactionException {
>> > > > > RecordView view = accounts.recordView();
>> > > > >
>> > > > > view.upsert(makeValue(1, BALANCE_1));
>> > > > >
>> > > > > RecordView txView = view.withTransaction();
>> > > > >
>> > > > > txView.getAsync(makeKey(1)).thenCompose(r ->
>> > > > > txView.upsertAsync(makeValue(1, r.doubleValue("balance") + DELTA),
>> > > > > tx)).thenCompose(txView.transaction().commitAsync()).join();
>> > > > >
>> > > > > assertEquals(BALANCE_1 + DELTA,
>> > > > > view.get(makeKey(1)).doubleValue("balance"));
>> > > > > }
>> > > > >
>> > > > > Is there any problem with this?
>> > > > >
>> > > > > On Mon, Nov 29, 2021 at 10:45 AM Alexei Scherbakov <
>> > > > > alexey.scherbak...@gmail.com> wrote:
>> > > > >
>> > > > > > Folks,
>> > > > > >
>> > > > > > Recently I've pushed transactions support phase 1 for Ignite 3,
>> see
>> > > > [1].
>> > > > > > Feel free to give feedback.
>> > > > > > Current implementation attempts to automatically enlist a table
>> > into
>> > > > > > transaction if it's started using [2] or [3] by using thread
>> local
>> > > > > context,
>> > > > > > similar to Ignite 2 approach, to reduce the amount of
>> boilerplate
>> > > code.
>> > > > > > But it turns out such an approach still has unacceptable
>> drawbacks
>> > > > from a
>> > > > > > user experience point of view.
>> > > > > >
>> > > > > > Consider the example [4]:
>> > > > > >
>> > > > > > public void testMixedPutGet() throws TransactionException {
>> > > > > > accounts.recordView().upsert(makeValue(1, BALANCE_1));
>> > > > > >
>> > > > > > igniteTransactions.runInTransaction(tx -> {
>> > > > > > var txAcc =
>> accounts.recordView().withTransaction(tx);
>> > > > > >
>> > > > > > txAcc.getAsync(makeKey(1)).thenCompose(r ->
>> > > > > > txAcc.upsertAsync(makeValue(1, r.doubleValue("balance") +
>> > > > > DELTA))).join();
>> > > > > > });
>> > > > > >
>> > > > > > assertEquals(BALANCE_1 + DELTA,
>> > > > > > accounts.recordView().get(makeKey(1)).doubleValue("balance"));
>> > > > > > }
>> > > > > >
>> > > > > > Here we *have to* to manually enlist a table if it's used in
>> async
>> > > > chain
>> > > > > > call, because the caller thread will be 

Re: IEP-61 Transaction API desing for Ignite 3

2021-11-29 Thread Valentin Kulichenko
I like Alexei's suggestion. This seems to be the most transparent and
explicit approach. Basically, this ensures that the user is always aware of
whether an operation is enlisted in a transaction or not. Any other option
is either error-prone, or introduces unnecessary counter-intuitive
limitations.

I don't think we should keep overloads without the tx parameter, because
that will pretty much eliminate the value of this change. One thing we can
do to address this is to have separate "non-tx" views, which can only be
used to execute implicit transactions. But I would look at this after we
more or less stabilize the primary API.

-Val

On Mon, Nov 29, 2021 at 5:03 AM Pavel Tupitsyn  wrote:

> Alexei,
>
> Are we going to offer an overload without tx parameter?
>
> getAsync(K key);
> getAsync(K key, Transaction tx);
>
> On Mon, Nov 29, 2021 at 3:43 PM Alexei Scherbakov <
> alexey.scherbak...@gmail.com> wrote:
>
> > Pavel,
> >
> > The problem with a current approach to me is the possibility of
> forgetting
> > to enlist a table into a transaction, because it is not enforced.
> > Having the explicit argument for this purpose seems less error-prone to
> me.
> >
> > пн, 29 нояб. 2021 г. в 15:13, Pavel Tupitsyn :
> >
> > > Taras, yes, yours is the actual syntax in main branch right now,
> > > I've skipped the tx argument in my code accidentally.
> > >
> > > On Mon, Nov 29, 2021 at 3:03 PM Taras Ledkov 
> > wrote:
> > >
> > > > Hi colleagues,
> > > >
> > > > 2Pavel:
> > > > > RecordView txView = view.withTransaction();
> > > > Can we use the syntax (see below) to attach the table / operation to
> > the
> > > > started transaction?
> > > > RecordView  txPersonView =
> > > > person.recordView().withTransaction(txView.transaction());
> > > >
> > > >
> > > > On Mon, Nov 29, 2021 at 1:34 PM Pavel Tupitsyn  >
> > > > wrote:
> > > >
> > > > > Alexei,
> > > > >
> > > > > I agree that runInTransaction is confusing and error-prone.
> > > > >
> > > > > But we already have view.withTransaction(), which seems to be the
> > most
> > > > > boilerplate-free approach.
> > > > > The example above will look like this:
> > > > >
> > > > > public void testMixedPutGet() throws TransactionException {
> > > > > RecordView view = accounts.recordView();
> > > > >
> > > > > view.upsert(makeValue(1, BALANCE_1));
> > > > >
> > > > > RecordView txView = view.withTransaction();
> > > > >
> > > > > txView.getAsync(makeKey(1)).thenCompose(r ->
> > > > > txView.upsertAsync(makeValue(1, r.doubleValue("balance") + DELTA),
> > > > > tx)).thenCompose(txView.transaction().commitAsync()).join();
> > > > >
> > > > > assertEquals(BALANCE_1 + DELTA,
> > > > > view.get(makeKey(1)).doubleValue("balance"));
> > > > > }
> > > > >
> > > > > Is there any problem with this?
> > > > >
> > > > > On Mon, Nov 29, 2021 at 10:45 AM Alexei Scherbakov <
> > > > > alexey.scherbak...@gmail.com> wrote:
> > > > >
> > > > > > Folks,
> > > > > >
> > > > > > Recently I've pushed transactions support phase 1 for Ignite 3,
> see
> > > > [1].
> > > > > > Feel free to give feedback.
> > > > > > Current implementation attempts to automatically enlist a table
> > into
> > > > > > transaction if it's started using [2] or [3] by using thread
> local
> > > > > context,
> > > > > > similar to Ignite 2 approach, to reduce the amount of boilerplate
> > > code.
> > > > > > But it turns out such an approach still has unacceptable
> drawbacks
> > > > from a
> > > > > > user experience point of view.
> > > > > >
> > > > > > Consider the example [4]:
> > > > > >
> > > > > > public void testMixedPutGet() throws TransactionException {
> > > > > > accounts.recordView().upsert(makeValue(1, BALANCE_1));
> > > > > >
> > > > > > igniteTransactions.runInTransaction(tx -> {
> > > > > > var txAcc =
> accounts.recordView().withTransaction(tx);
> > > > > >
> > > > > > txAcc.getAsync(makeKey(1)).thenCompose(r ->
> > > > > > txAcc.upsertAsync(makeValue(1, r.doubleValue("balance") +
> > > > > DELTA))).join();
> > > > > > });
> > > > > >
> > > > > > assertEquals(BALANCE_1 + DELTA,
> > > > > > accounts.recordView().get(makeKey(1)).doubleValue("balance"));
> > > > > > }
> > > > > >
> > > > > > Here we *have to* to manually enlist a table if it's used in
> async
> > > > chain
> > > > > > call, because the caller thread will be different and the chained
> > > > > operation
> > > > > > will be executed in separate tx.
> > > > > > This works similarly in Ignite 2 and is very confusing.
> > > > > >
> > > > > > To avoid this, I propose to add an explicit Transaction argument
> to
> > > > each
> > > > > > table API method. Null value means to start the implicit
> > transaction
> > > > > > (autocommit mode). For example:
> > > > > >
> > > > > > /**
> > > > > >  * Asynchronously inserts a record into the table if it
> doesn't
> > > > exist
> > > > > > or replaces the existed one.
> > > > > >  *
> > > > > >  * 

Re: IEP-61 Transaction API desing for Ignite 3

2021-11-29 Thread Pavel Tupitsyn
Alexei,

Are we going to offer an overload without tx parameter?

getAsync(K key);
getAsync(K key, Transaction tx);

On Mon, Nov 29, 2021 at 3:43 PM Alexei Scherbakov <
alexey.scherbak...@gmail.com> wrote:

> Pavel,
>
> The problem with a current approach to me is the possibility of forgetting
> to enlist a table into a transaction, because it is not enforced.
> Having the explicit argument for this purpose seems less error-prone to me.
>
> пн, 29 нояб. 2021 г. в 15:13, Pavel Tupitsyn :
>
> > Taras, yes, yours is the actual syntax in main branch right now,
> > I've skipped the tx argument in my code accidentally.
> >
> > On Mon, Nov 29, 2021 at 3:03 PM Taras Ledkov 
> wrote:
> >
> > > Hi colleagues,
> > >
> > > 2Pavel:
> > > > RecordView txView = view.withTransaction();
> > > Can we use the syntax (see below) to attach the table / operation to
> the
> > > started transaction?
> > > RecordView  txPersonView =
> > > person.recordView().withTransaction(txView.transaction());
> > >
> > >
> > > On Mon, Nov 29, 2021 at 1:34 PM Pavel Tupitsyn 
> > > wrote:
> > >
> > > > Alexei,
> > > >
> > > > I agree that runInTransaction is confusing and error-prone.
> > > >
> > > > But we already have view.withTransaction(), which seems to be the
> most
> > > > boilerplate-free approach.
> > > > The example above will look like this:
> > > >
> > > > public void testMixedPutGet() throws TransactionException {
> > > > RecordView view = accounts.recordView();
> > > >
> > > > view.upsert(makeValue(1, BALANCE_1));
> > > >
> > > > RecordView txView = view.withTransaction();
> > > >
> > > > txView.getAsync(makeKey(1)).thenCompose(r ->
> > > > txView.upsertAsync(makeValue(1, r.doubleValue("balance") + DELTA),
> > > > tx)).thenCompose(txView.transaction().commitAsync()).join();
> > > >
> > > > assertEquals(BALANCE_1 + DELTA,
> > > > view.get(makeKey(1)).doubleValue("balance"));
> > > > }
> > > >
> > > > Is there any problem with this?
> > > >
> > > > On Mon, Nov 29, 2021 at 10:45 AM Alexei Scherbakov <
> > > > alexey.scherbak...@gmail.com> wrote:
> > > >
> > > > > Folks,
> > > > >
> > > > > Recently I've pushed transactions support phase 1 for Ignite 3, see
> > > [1].
> > > > > Feel free to give feedback.
> > > > > Current implementation attempts to automatically enlist a table
> into
> > > > > transaction if it's started using [2] or [3] by using thread local
> > > > context,
> > > > > similar to Ignite 2 approach, to reduce the amount of boilerplate
> > code.
> > > > > But it turns out such an approach still has unacceptable drawbacks
> > > from a
> > > > > user experience point of view.
> > > > >
> > > > > Consider the example [4]:
> > > > >
> > > > > public void testMixedPutGet() throws TransactionException {
> > > > > accounts.recordView().upsert(makeValue(1, BALANCE_1));
> > > > >
> > > > > igniteTransactions.runInTransaction(tx -> {
> > > > > var txAcc = accounts.recordView().withTransaction(tx);
> > > > >
> > > > > txAcc.getAsync(makeKey(1)).thenCompose(r ->
> > > > > txAcc.upsertAsync(makeValue(1, r.doubleValue("balance") +
> > > > DELTA))).join();
> > > > > });
> > > > >
> > > > > assertEquals(BALANCE_1 + DELTA,
> > > > > accounts.recordView().get(makeKey(1)).doubleValue("balance"));
> > > > > }
> > > > >
> > > > > Here we *have to* to manually enlist a table if it's used in async
> > > chain
> > > > > call, because the caller thread will be different and the chained
> > > > operation
> > > > > will be executed in separate tx.
> > > > > This works similarly in Ignite 2 and is very confusing.
> > > > >
> > > > > To avoid this, I propose to add an explicit Transaction argument to
> > > each
> > > > > table API method. Null value means to start the implicit
> transaction
> > > > > (autocommit mode). For example:
> > > > >
> > > > > /**
> > > > >  * Asynchronously inserts a record into the table if it doesn't
> > > exist
> > > > > or replaces the existed one.
> > > > >  *
> > > > >  * @param rec A record to insert into the table. The record
> > cannot
> > > be
> > > > > {@code null}.
> > > > >  * @param tx The transaction or {@code null} to auto commit.
> > > > >  * @return Future representing pending completion of the
> > operation.
> > > > >  */
> > > > > @NotNull CompletableFuture upsertAsync(@NotNull R rec,
> > > > @Nullable
> > > > > Transaction tx);
> > > > >
> > > > > The example [4] turns to
> > > > >
> > > > > public void testMixedPutGet() throws TransactionException {
> > > > > RecordView view = accounts.recordView();
> > > > >
> > > > > view.upsert(makeValue(1, BALANCE_1));
> > > > >
> > > > > igniteTransactions.runInTransaction(tx -> {
> > > > > view.getAsync(makeKey(1), tx).thenCompose(r ->
> > > > > view.upsertAsync(makeValue(1, r.doubleValue("balance") + DELTA),
> > > > > tx)).join();
> > > > > });
> > > > >
> > > > > 

Re: IEP-61 Transaction API desing for Ignite 3

2021-11-29 Thread Alexei Scherbakov
Pavel,

The problem with a current approach to me is the possibility of forgetting
to enlist a table into a transaction, because it is not enforced.
Having the explicit argument for this purpose seems less error-prone to me.

пн, 29 нояб. 2021 г. в 15:13, Pavel Tupitsyn :

> Taras, yes, yours is the actual syntax in main branch right now,
> I've skipped the tx argument in my code accidentally.
>
> On Mon, Nov 29, 2021 at 3:03 PM Taras Ledkov  wrote:
>
> > Hi colleagues,
> >
> > 2Pavel:
> > > RecordView txView = view.withTransaction();
> > Can we use the syntax (see below) to attach the table / operation to the
> > started transaction?
> > RecordView  txPersonView =
> > person.recordView().withTransaction(txView.transaction());
> >
> >
> > On Mon, Nov 29, 2021 at 1:34 PM Pavel Tupitsyn 
> > wrote:
> >
> > > Alexei,
> > >
> > > I agree that runInTransaction is confusing and error-prone.
> > >
> > > But we already have view.withTransaction(), which seems to be the most
> > > boilerplate-free approach.
> > > The example above will look like this:
> > >
> > > public void testMixedPutGet() throws TransactionException {
> > > RecordView view = accounts.recordView();
> > >
> > > view.upsert(makeValue(1, BALANCE_1));
> > >
> > > RecordView txView = view.withTransaction();
> > >
> > > txView.getAsync(makeKey(1)).thenCompose(r ->
> > > txView.upsertAsync(makeValue(1, r.doubleValue("balance") + DELTA),
> > > tx)).thenCompose(txView.transaction().commitAsync()).join();
> > >
> > > assertEquals(BALANCE_1 + DELTA,
> > > view.get(makeKey(1)).doubleValue("balance"));
> > > }
> > >
> > > Is there any problem with this?
> > >
> > > On Mon, Nov 29, 2021 at 10:45 AM Alexei Scherbakov <
> > > alexey.scherbak...@gmail.com> wrote:
> > >
> > > > Folks,
> > > >
> > > > Recently I've pushed transactions support phase 1 for Ignite 3, see
> > [1].
> > > > Feel free to give feedback.
> > > > Current implementation attempts to automatically enlist a table into
> > > > transaction if it's started using [2] or [3] by using thread local
> > > context,
> > > > similar to Ignite 2 approach, to reduce the amount of boilerplate
> code.
> > > > But it turns out such an approach still has unacceptable drawbacks
> > from a
> > > > user experience point of view.
> > > >
> > > > Consider the example [4]:
> > > >
> > > > public void testMixedPutGet() throws TransactionException {
> > > > accounts.recordView().upsert(makeValue(1, BALANCE_1));
> > > >
> > > > igniteTransactions.runInTransaction(tx -> {
> > > > var txAcc = accounts.recordView().withTransaction(tx);
> > > >
> > > > txAcc.getAsync(makeKey(1)).thenCompose(r ->
> > > > txAcc.upsertAsync(makeValue(1, r.doubleValue("balance") +
> > > DELTA))).join();
> > > > });
> > > >
> > > > assertEquals(BALANCE_1 + DELTA,
> > > > accounts.recordView().get(makeKey(1)).doubleValue("balance"));
> > > > }
> > > >
> > > > Here we *have to* to manually enlist a table if it's used in async
> > chain
> > > > call, because the caller thread will be different and the chained
> > > operation
> > > > will be executed in separate tx.
> > > > This works similarly in Ignite 2 and is very confusing.
> > > >
> > > > To avoid this, I propose to add an explicit Transaction argument to
> > each
> > > > table API method. Null value means to start the implicit transaction
> > > > (autocommit mode). For example:
> > > >
> > > > /**
> > > >  * Asynchronously inserts a record into the table if it doesn't
> > exist
> > > > or replaces the existed one.
> > > >  *
> > > >  * @param rec A record to insert into the table. The record
> cannot
> > be
> > > > {@code null}.
> > > >  * @param tx The transaction or {@code null} to auto commit.
> > > >  * @return Future representing pending completion of the
> operation.
> > > >  */
> > > > @NotNull CompletableFuture upsertAsync(@NotNull R rec,
> > > @Nullable
> > > > Transaction tx);
> > > >
> > > > The example [4] turns to
> > > >
> > > > public void testMixedPutGet() throws TransactionException {
> > > > RecordView view = accounts.recordView();
> > > >
> > > > view.upsert(makeValue(1, BALANCE_1));
> > > >
> > > > igniteTransactions.runInTransaction(tx -> {
> > > > view.getAsync(makeKey(1), tx).thenCompose(r ->
> > > > view.upsertAsync(makeValue(1, r.doubleValue("balance") + DELTA),
> > > > tx)).join();
> > > > });
> > > >
> > > > assertEquals(BALANCE_1 + DELTA,
> > > > view.get(makeKey(1)).doubleValue("balance"));
> > > > }
> > > >
> > > > Share your thoughts.
> > > >
> > > > [1] https://issues.apache.org/jira/browse/IGNITE-15085
> > > > [2]
> > > >
> > >
> >
> org.apache.ignite.tx.IgniteTransactions#runInTransaction(java.util.function.Consumer)
> > > > [3]
> > > >
> > >
> >
> org.apache.ignite.tx.IgniteTransactions#runInTransaction(java.util.function.Function)
> > > > [4] 

Re: IEP-61 Transaction API desing for Ignite 3

2021-11-29 Thread Pavel Tupitsyn
Taras, yes, yours is the actual syntax in main branch right now,
I've skipped the tx argument in my code accidentally.

On Mon, Nov 29, 2021 at 3:03 PM Taras Ledkov  wrote:

> Hi colleagues,
>
> 2Pavel:
> > RecordView txView = view.withTransaction();
> Can we use the syntax (see below) to attach the table / operation to the
> started transaction?
> RecordView  txPersonView =
> person.recordView().withTransaction(txView.transaction());
>
>
> On Mon, Nov 29, 2021 at 1:34 PM Pavel Tupitsyn 
> wrote:
>
> > Alexei,
> >
> > I agree that runInTransaction is confusing and error-prone.
> >
> > But we already have view.withTransaction(), which seems to be the most
> > boilerplate-free approach.
> > The example above will look like this:
> >
> > public void testMixedPutGet() throws TransactionException {
> > RecordView view = accounts.recordView();
> >
> > view.upsert(makeValue(1, BALANCE_1));
> >
> > RecordView txView = view.withTransaction();
> >
> > txView.getAsync(makeKey(1)).thenCompose(r ->
> > txView.upsertAsync(makeValue(1, r.doubleValue("balance") + DELTA),
> > tx)).thenCompose(txView.transaction().commitAsync()).join();
> >
> > assertEquals(BALANCE_1 + DELTA,
> > view.get(makeKey(1)).doubleValue("balance"));
> > }
> >
> > Is there any problem with this?
> >
> > On Mon, Nov 29, 2021 at 10:45 AM Alexei Scherbakov <
> > alexey.scherbak...@gmail.com> wrote:
> >
> > > Folks,
> > >
> > > Recently I've pushed transactions support phase 1 for Ignite 3, see
> [1].
> > > Feel free to give feedback.
> > > Current implementation attempts to automatically enlist a table into
> > > transaction if it's started using [2] or [3] by using thread local
> > context,
> > > similar to Ignite 2 approach, to reduce the amount of boilerplate code.
> > > But it turns out such an approach still has unacceptable drawbacks
> from a
> > > user experience point of view.
> > >
> > > Consider the example [4]:
> > >
> > > public void testMixedPutGet() throws TransactionException {
> > > accounts.recordView().upsert(makeValue(1, BALANCE_1));
> > >
> > > igniteTransactions.runInTransaction(tx -> {
> > > var txAcc = accounts.recordView().withTransaction(tx);
> > >
> > > txAcc.getAsync(makeKey(1)).thenCompose(r ->
> > > txAcc.upsertAsync(makeValue(1, r.doubleValue("balance") +
> > DELTA))).join();
> > > });
> > >
> > > assertEquals(BALANCE_1 + DELTA,
> > > accounts.recordView().get(makeKey(1)).doubleValue("balance"));
> > > }
> > >
> > > Here we *have to* to manually enlist a table if it's used in async
> chain
> > > call, because the caller thread will be different and the chained
> > operation
> > > will be executed in separate tx.
> > > This works similarly in Ignite 2 and is very confusing.
> > >
> > > To avoid this, I propose to add an explicit Transaction argument to
> each
> > > table API method. Null value means to start the implicit transaction
> > > (autocommit mode). For example:
> > >
> > > /**
> > >  * Asynchronously inserts a record into the table if it doesn't
> exist
> > > or replaces the existed one.
> > >  *
> > >  * @param rec A record to insert into the table. The record cannot
> be
> > > {@code null}.
> > >  * @param tx The transaction or {@code null} to auto commit.
> > >  * @return Future representing pending completion of the operation.
> > >  */
> > > @NotNull CompletableFuture upsertAsync(@NotNull R rec,
> > @Nullable
> > > Transaction tx);
> > >
> > > The example [4] turns to
> > >
> > > public void testMixedPutGet() throws TransactionException {
> > > RecordView view = accounts.recordView();
> > >
> > > view.upsert(makeValue(1, BALANCE_1));
> > >
> > > igniteTransactions.runInTransaction(tx -> {
> > > view.getAsync(makeKey(1), tx).thenCompose(r ->
> > > view.upsertAsync(makeValue(1, r.doubleValue("balance") + DELTA),
> > > tx)).join();
> > > });
> > >
> > > assertEquals(BALANCE_1 + DELTA,
> > > view.get(makeKey(1)).doubleValue("balance"));
> > > }
> > >
> > > Share your thoughts.
> > >
> > > [1] https://issues.apache.org/jira/browse/IGNITE-15085
> > > [2]
> > >
> >
> org.apache.ignite.tx.IgniteTransactions#runInTransaction(java.util.function.Consumer)
> > > [3]
> > >
> >
> org.apache.ignite.tx.IgniteTransactions#runInTransaction(java.util.function.Function)
> > > [4] org.apache.ignite.internal.table.TxAbstractTest#testMixedPutGet
> > >
> > > ср, 14 июл. 2021 г. в 14:12, Alexei Scherbakov <
> > > alexey.scherbak...@gmail.com
> > > >:
> > >
> > > > Andrey,
> > > >
> > > > 1) "As a user, I'd expect runInTransaction(closure) will create Tx
> for
> > > me,
> > > > commit Tx after a successful closure call, and rollback Tx in case of
> > > > error."
> > > > - I'm ok with this behavior, and will alter javadoc.
> > > >
> > > > 2) "Transaction tx = beginTx()" - there is no such method "beginTx"
> in
> > > the
> > > > proposed API, and I'm not 

Re: IEP-61 Transaction API desing for Ignite 3

2021-11-29 Thread Taras Ledkov
Hi colleagues,

2Pavel:
> RecordView txView = view.withTransaction();
Can we use the syntax (see below) to attach the table / operation to the
started transaction?
RecordView  txPersonView =
person.recordView().withTransaction(txView.transaction());


On Mon, Nov 29, 2021 at 1:34 PM Pavel Tupitsyn  wrote:

> Alexei,
>
> I agree that runInTransaction is confusing and error-prone.
>
> But we already have view.withTransaction(), which seems to be the most
> boilerplate-free approach.
> The example above will look like this:
>
> public void testMixedPutGet() throws TransactionException {
> RecordView view = accounts.recordView();
>
> view.upsert(makeValue(1, BALANCE_1));
>
> RecordView txView = view.withTransaction();
>
> txView.getAsync(makeKey(1)).thenCompose(r ->
> txView.upsertAsync(makeValue(1, r.doubleValue("balance") + DELTA),
> tx)).thenCompose(txView.transaction().commitAsync()).join();
>
> assertEquals(BALANCE_1 + DELTA,
> view.get(makeKey(1)).doubleValue("balance"));
> }
>
> Is there any problem with this?
>
> On Mon, Nov 29, 2021 at 10:45 AM Alexei Scherbakov <
> alexey.scherbak...@gmail.com> wrote:
>
> > Folks,
> >
> > Recently I've pushed transactions support phase 1 for Ignite 3, see [1].
> > Feel free to give feedback.
> > Current implementation attempts to automatically enlist a table into
> > transaction if it's started using [2] or [3] by using thread local
> context,
> > similar to Ignite 2 approach, to reduce the amount of boilerplate code.
> > But it turns out such an approach still has unacceptable drawbacks from a
> > user experience point of view.
> >
> > Consider the example [4]:
> >
> > public void testMixedPutGet() throws TransactionException {
> > accounts.recordView().upsert(makeValue(1, BALANCE_1));
> >
> > igniteTransactions.runInTransaction(tx -> {
> > var txAcc = accounts.recordView().withTransaction(tx);
> >
> > txAcc.getAsync(makeKey(1)).thenCompose(r ->
> > txAcc.upsertAsync(makeValue(1, r.doubleValue("balance") +
> DELTA))).join();
> > });
> >
> > assertEquals(BALANCE_1 + DELTA,
> > accounts.recordView().get(makeKey(1)).doubleValue("balance"));
> > }
> >
> > Here we *have to* to manually enlist a table if it's used in async chain
> > call, because the caller thread will be different and the chained
> operation
> > will be executed in separate tx.
> > This works similarly in Ignite 2 and is very confusing.
> >
> > To avoid this, I propose to add an explicit Transaction argument to each
> > table API method. Null value means to start the implicit transaction
> > (autocommit mode). For example:
> >
> > /**
> >  * Asynchronously inserts a record into the table if it doesn't exist
> > or replaces the existed one.
> >  *
> >  * @param rec A record to insert into the table. The record cannot be
> > {@code null}.
> >  * @param tx The transaction or {@code null} to auto commit.
> >  * @return Future representing pending completion of the operation.
> >  */
> > @NotNull CompletableFuture upsertAsync(@NotNull R rec,
> @Nullable
> > Transaction tx);
> >
> > The example [4] turns to
> >
> > public void testMixedPutGet() throws TransactionException {
> > RecordView view = accounts.recordView();
> >
> > view.upsert(makeValue(1, BALANCE_1));
> >
> > igniteTransactions.runInTransaction(tx -> {
> > view.getAsync(makeKey(1), tx).thenCompose(r ->
> > view.upsertAsync(makeValue(1, r.doubleValue("balance") + DELTA),
> > tx)).join();
> > });
> >
> > assertEquals(BALANCE_1 + DELTA,
> > view.get(makeKey(1)).doubleValue("balance"));
> > }
> >
> > Share your thoughts.
> >
> > [1] https://issues.apache.org/jira/browse/IGNITE-15085
> > [2]
> >
> org.apache.ignite.tx.IgniteTransactions#runInTransaction(java.util.function.Consumer)
> > [3]
> >
> org.apache.ignite.tx.IgniteTransactions#runInTransaction(java.util.function.Function)
> > [4] org.apache.ignite.internal.table.TxAbstractTest#testMixedPutGet
> >
> > ср, 14 июл. 2021 г. в 14:12, Alexei Scherbakov <
> > alexey.scherbak...@gmail.com
> > >:
> >
> > > Andrey,
> > >
> > > 1) "As a user, I'd expect runInTransaction(closure) will create Tx for
> > me,
> > > commit Tx after a successful closure call, and rollback Tx in case of
> > > error."
> > > - I'm ok with this behavior, and will alter javadoc.
> > >
> > > 2) "Transaction tx = beginTx()" - there is no such method "beginTx" in
> > the
> > > proposed API, and I'm not intending to add it.
> > > For the synchronous case I suggest to use "runInTransaction", which
> > > eliminates the need in AutoClosable.
> > >
> > >
> > >
> > > ср, 14 июл. 2021 г. в 13:21, Ivan Daschinsky :
> > >
> > >> > yes, it is stated in the javadoc in the PR.
> > >> Ah, I see.
> > >>
> > >> ср, 14 июл. 2021 г. в 12:16, Alexei Scherbakov <
> > >> alexey.scherbak...@gmail.com
> > >> >:
> > >>
> > >> > Ivan,
> > >> >
> > >> > And what if I have already 

Re: IEP-61 Transaction API desing for Ignite 3

2021-11-29 Thread Pavel Tupitsyn
Alexei,

I agree that runInTransaction is confusing and error-prone.

But we already have view.withTransaction(), which seems to be the most
boilerplate-free approach.
The example above will look like this:

public void testMixedPutGet() throws TransactionException {
RecordView view = accounts.recordView();

view.upsert(makeValue(1, BALANCE_1));

RecordView txView = view.withTransaction();

txView.getAsync(makeKey(1)).thenCompose(r ->
txView.upsertAsync(makeValue(1, r.doubleValue("balance") + DELTA),
tx)).thenCompose(txView.transaction().commitAsync()).join();

assertEquals(BALANCE_1 + DELTA,
view.get(makeKey(1)).doubleValue("balance"));
}

Is there any problem with this?

On Mon, Nov 29, 2021 at 10:45 AM Alexei Scherbakov <
alexey.scherbak...@gmail.com> wrote:

> Folks,
>
> Recently I've pushed transactions support phase 1 for Ignite 3, see [1].
> Feel free to give feedback.
> Current implementation attempts to automatically enlist a table into
> transaction if it's started using [2] or [3] by using thread local context,
> similar to Ignite 2 approach, to reduce the amount of boilerplate code.
> But it turns out such an approach still has unacceptable drawbacks from a
> user experience point of view.
>
> Consider the example [4]:
>
> public void testMixedPutGet() throws TransactionException {
> accounts.recordView().upsert(makeValue(1, BALANCE_1));
>
> igniteTransactions.runInTransaction(tx -> {
> var txAcc = accounts.recordView().withTransaction(tx);
>
> txAcc.getAsync(makeKey(1)).thenCompose(r ->
> txAcc.upsertAsync(makeValue(1, r.doubleValue("balance") + DELTA))).join();
> });
>
> assertEquals(BALANCE_1 + DELTA,
> accounts.recordView().get(makeKey(1)).doubleValue("balance"));
> }
>
> Here we *have to* to manually enlist a table if it's used in async chain
> call, because the caller thread will be different and the chained operation
> will be executed in separate tx.
> This works similarly in Ignite 2 and is very confusing.
>
> To avoid this, I propose to add an explicit Transaction argument to each
> table API method. Null value means to start the implicit transaction
> (autocommit mode). For example:
>
> /**
>  * Asynchronously inserts a record into the table if it doesn't exist
> or replaces the existed one.
>  *
>  * @param rec A record to insert into the table. The record cannot be
> {@code null}.
>  * @param tx The transaction or {@code null} to auto commit.
>  * @return Future representing pending completion of the operation.
>  */
> @NotNull CompletableFuture upsertAsync(@NotNull R rec, @Nullable
> Transaction tx);
>
> The example [4] turns to
>
> public void testMixedPutGet() throws TransactionException {
> RecordView view = accounts.recordView();
>
> view.upsert(makeValue(1, BALANCE_1));
>
> igniteTransactions.runInTransaction(tx -> {
> view.getAsync(makeKey(1), tx).thenCompose(r ->
> view.upsertAsync(makeValue(1, r.doubleValue("balance") + DELTA),
> tx)).join();
> });
>
> assertEquals(BALANCE_1 + DELTA,
> view.get(makeKey(1)).doubleValue("balance"));
> }
>
> Share your thoughts.
>
> [1] https://issues.apache.org/jira/browse/IGNITE-15085
> [2]
> org.apache.ignite.tx.IgniteTransactions#runInTransaction(java.util.function.Consumer)
> [3]
> org.apache.ignite.tx.IgniteTransactions#runInTransaction(java.util.function.Function)
> [4] org.apache.ignite.internal.table.TxAbstractTest#testMixedPutGet
>
> ср, 14 июл. 2021 г. в 14:12, Alexei Scherbakov <
> alexey.scherbak...@gmail.com
> >:
>
> > Andrey,
> >
> > 1) "As a user, I'd expect runInTransaction(closure) will create Tx for
> me,
> > commit Tx after a successful closure call, and rollback Tx in case of
> > error."
> > - I'm ok with this behavior, and will alter javadoc.
> >
> > 2) "Transaction tx = beginTx()" - there is no such method "beginTx" in
> the
> > proposed API, and I'm not intending to add it.
> > For the synchronous case I suggest to use "runInTransaction", which
> > eliminates the need in AutoClosable.
> >
> >
> >
> > ср, 14 июл. 2021 г. в 13:21, Ivan Daschinsky :
> >
> >> > yes, it is stated in the javadoc in the PR.
> >> Ah, I see.
> >>
> >> ср, 14 июл. 2021 г. в 12:16, Alexei Scherbakov <
> >> alexey.scherbak...@gmail.com
> >> >:
> >>
> >> > Ivan,
> >> >
> >> > And what if I have already committed transaction? Is it safe rollback
> >> > already committed transaction? Rollback will silently return and do
> >> > nothing? - yes, it is stated in the javadoc in the PR.
> >> >
> >> > Andrey,
> >> >
> >> > Then using "runInTransaction", lack of commit will cause a transaction
> >> to
> >> > rollback automatically.
> >> >
> >> > There is no need for a "close" method, it just adds confusion.
> >> >
> >> >
> >> > ср, 14 июл. 2021 г. в 11:37, Andrey Mashenkov <
> >> andrey.mashen...@gmail.com
> >> > >:
> >> >
> >> > > Agree with Ivan.
> >> > >
> >> > > Method 

Re: IEP-61 Transaction API desing for Ignite 3

2021-11-28 Thread Alexei Scherbakov
Folks,

Recently I've pushed transactions support phase 1 for Ignite 3, see [1].
Feel free to give feedback.
Current implementation attempts to automatically enlist a table into
transaction if it's started using [2] or [3] by using thread local context,
similar to Ignite 2 approach, to reduce the amount of boilerplate code.
But it turns out such an approach still has unacceptable drawbacks from a
user experience point of view.

Consider the example [4]:

public void testMixedPutGet() throws TransactionException {
accounts.recordView().upsert(makeValue(1, BALANCE_1));

igniteTransactions.runInTransaction(tx -> {
var txAcc = accounts.recordView().withTransaction(tx);

txAcc.getAsync(makeKey(1)).thenCompose(r ->
txAcc.upsertAsync(makeValue(1, r.doubleValue("balance") + DELTA))).join();
});

assertEquals(BALANCE_1 + DELTA,
accounts.recordView().get(makeKey(1)).doubleValue("balance"));
}

Here we *have to* to manually enlist a table if it's used in async chain
call, because the caller thread will be different and the chained operation
will be executed in separate tx.
This works similarly in Ignite 2 and is very confusing.

To avoid this, I propose to add an explicit Transaction argument to each
table API method. Null value means to start the implicit transaction
(autocommit mode). For example:

/**
 * Asynchronously inserts a record into the table if it doesn't exist
or replaces the existed one.
 *
 * @param rec A record to insert into the table. The record cannot be
{@code null}.
 * @param tx The transaction or {@code null} to auto commit.
 * @return Future representing pending completion of the operation.
 */
@NotNull CompletableFuture upsertAsync(@NotNull R rec, @Nullable
Transaction tx);

The example [4] turns to

public void testMixedPutGet() throws TransactionException {
RecordView view = accounts.recordView();

view.upsert(makeValue(1, BALANCE_1));

igniteTransactions.runInTransaction(tx -> {
view.getAsync(makeKey(1), tx).thenCompose(r ->
view.upsertAsync(makeValue(1, r.doubleValue("balance") + DELTA),
tx)).join();
});

assertEquals(BALANCE_1 + DELTA,
view.get(makeKey(1)).doubleValue("balance"));
}

Share your thoughts.

[1] https://issues.apache.org/jira/browse/IGNITE-15085
[2] 
org.apache.ignite.tx.IgniteTransactions#runInTransaction(java.util.function.Consumer)
[3] 
org.apache.ignite.tx.IgniteTransactions#runInTransaction(java.util.function.Function)
[4] org.apache.ignite.internal.table.TxAbstractTest#testMixedPutGet

ср, 14 июл. 2021 г. в 14:12, Alexei Scherbakov :

> Andrey,
>
> 1) "As a user, I'd expect runInTransaction(closure) will create Tx for me,
> commit Tx after a successful closure call, and rollback Tx in case of
> error."
> - I'm ok with this behavior, and will alter javadoc.
>
> 2) "Transaction tx = beginTx()" - there is no such method "beginTx" in the
> proposed API, and I'm not intending to add it.
> For the synchronous case I suggest to use "runInTransaction", which
> eliminates the need in AutoClosable.
>
>
>
> ср, 14 июл. 2021 г. в 13:21, Ivan Daschinsky :
>
>> > yes, it is stated in the javadoc in the PR.
>> Ah, I see.
>>
>> ср, 14 июл. 2021 г. в 12:16, Alexei Scherbakov <
>> alexey.scherbak...@gmail.com
>> >:
>>
>> > Ivan,
>> >
>> > And what if I have already committed transaction? Is it safe rollback
>> > already committed transaction? Rollback will silently return and do
>> > nothing? - yes, it is stated in the javadoc in the PR.
>> >
>> > Andrey,
>> >
>> > Then using "runInTransaction", lack of commit will cause a transaction
>> to
>> > rollback automatically.
>> >
>> > There is no need for a "close" method, it just adds confusion.
>> >
>> >
>> > ср, 14 июл. 2021 г. в 11:37, Andrey Mashenkov <
>> andrey.mashen...@gmail.com
>> > >:
>> >
>> > > Agree with Ivan.
>> > >
>> > > Method runInTransaction() should try to finish the transaction if the
>> > user
>> > > forgot to commit one.
>> > > I guess it might be a common mistake among new users.
>> > >
>> > > Also, I suggest to extent all table projections for better UX.
>> > > Let's allow
>> > > table.kvView().withTx(tx)
>> > > to user may cache kvVew instance and do
>> > > kvView.withTx(tx)
>> > > rather than
>> > > table.withTx(tx).kvVew()
>> > >
>> > >
>> > >
>> > > On Wed, Jul 14, 2021 at 10:13 AM Ivan Daschinsky > >
>> > > wrote:
>> > >
>> > > > Alexey, and is there any analogue to close() of transaction? When
>> you
>> > > start
>> > > > transaction, you should somehow to close it, if you don't catch
>> > exception
>> > > > or forget to commit.
>> > > >
>> > > > I suggest to add method closeAsync() to Transaction, so user can
>> call
>> > it
>> > > in
>> > > > handle or whenComplete, i.e.
>> > > >
>> > > > So code will looks like
>> > > >
>> > > > CacheApi cache = CacheApi.getCache("testCache");
>> > > >
>> > > > Transactions
>> > > > .beginTransaction()
>> > > > 

Re: IEP-61 Transaction API desing for Ignite 3

2021-07-14 Thread Alexei Scherbakov
Andrey,

1) "As a user, I'd expect runInTransaction(closure) will create Tx for me,
commit Tx after a successful closure call, and rollback Tx in case of
error."
- I'm ok with this behavior, and will alter javadoc.

2) "Transaction tx = beginTx()" - there is no such method "beginTx" in the
proposed API, and I'm not intending to add it.
For the synchronous case I suggest to use "runInTransaction", which
eliminates the need in AutoClosable.



ср, 14 июл. 2021 г. в 13:21, Ivan Daschinsky :

> > yes, it is stated in the javadoc in the PR.
> Ah, I see.
>
> ср, 14 июл. 2021 г. в 12:16, Alexei Scherbakov <
> alexey.scherbak...@gmail.com
> >:
>
> > Ivan,
> >
> > And what if I have already committed transaction? Is it safe rollback
> > already committed transaction? Rollback will silently return and do
> > nothing? - yes, it is stated in the javadoc in the PR.
> >
> > Andrey,
> >
> > Then using "runInTransaction", lack of commit will cause a transaction to
> > rollback automatically.
> >
> > There is no need for a "close" method, it just adds confusion.
> >
> >
> > ср, 14 июл. 2021 г. в 11:37, Andrey Mashenkov <
> andrey.mashen...@gmail.com
> > >:
> >
> > > Agree with Ivan.
> > >
> > > Method runInTransaction() should try to finish the transaction if the
> > user
> > > forgot to commit one.
> > > I guess it might be a common mistake among new users.
> > >
> > > Also, I suggest to extent all table projections for better UX.
> > > Let's allow
> > > table.kvView().withTx(tx)
> > > to user may cache kvVew instance and do
> > > kvView.withTx(tx)
> > > rather than
> > > table.withTx(tx).kvVew()
> > >
> > >
> > >
> > > On Wed, Jul 14, 2021 at 10:13 AM Ivan Daschinsky 
> > > wrote:
> > >
> > > > Alexey, and is there any analogue to close() of transaction? When you
> > > start
> > > > transaction, you should somehow to close it, if you don't catch
> > exception
> > > > or forget to commit.
> > > >
> > > > I suggest to add method closeAsync() to Transaction, so user can call
> > it
> > > in
> > > > handle or whenComplete, i.e.
> > > >
> > > > So code will looks like
> > > >
> > > > CacheApi cache = CacheApi.getCache("testCache");
> > > >
> > > > Transactions
> > > > .beginTransaction()
> > > > .thenCompose(tx -> {
> > > > CacheApi txCache = cache.withTx(tx);
> > > > CompletableFuture result = txCache.getAsync("key")
> > > > .thenCompose(val -> {
> > > > if (val == "test") {
> > > > return txCache.putAsync("key", "test1");
> > > > }
> > > > else
> > > > return CompletableFuture.completedFuture(null);
> > > > })
> > > > .thenCompose(v -> tx.commitAsync())
> > > > .handle((v, ex) -> null);
> > > > return result.thenCompose(v -> tx.closeAsync());
> > > > });
> > > >
> > > > I also suggests to add method something like this
> > > >
> > > > static CompletableFuture inTxAsync(Function > > > CompletableFuture> action) {
> > > > return Transactions
> > > > .beginTransaction()
> > > > .thenCompose(tx -> {
> > > > CompletableFuture result = action.apply(tx)
> > > > .handle((v, ex) -> null);
> > > > return result.thenCompose(v -> tx.closeAsync());
> > > > });
> > > > }
> > > >
> > > > Async api is not very readable, but this method can help user write
> > code,
> > > > this is rewritten first example:
> > > >
> > > > Transactions.inTxAsync(tx -> {
> > > > CacheApi txCache = cache.withTx(tx);
> > > > return txCache.getAsync("key")
> > > > .thenCompose(val -> {
> > > > if (val == "test") {
> > > > return txCache.putAsync("key", "test1");
> > > > }
> > > > else
> > > > return CompletableFuture.completedFuture(null);
> > > > })
> > > > .thenCompose(v -> tx.commitAsync());
> > > > });
> > > >
> > > > ср, 14 июл. 2021 г. в 10:03, Alexei Scherbakov <
> > > > alexey.scherbak...@gmail.com
> > > > >:
> > > >
> > > > > Andrey,
> > > > >
> > > > > I suggest you look at the PR [1], if you haven't.
> > > > >
> > > > > A transaction [2]
> > > > > Transactions facade [3]
> > > > > Examples [4]
> > > > >
> > > > > [1] https://github.com/apache/ignite-3/pull/214/files
> > > > > [2]
> > > > >
> > > > >
> > > >
> > >
> >
> https://github.com/apache/ignite-3/blob/d2122ce8c15de020e121f53509bd5a097aac9cf2/modules/api/src/main/java/org/apache/ignite/tx/Transaction.java
> > > > > [3]
> > > > >
> > > > >
> > > >
> > >
> >
> https://github.com/apache/ignite-3/blob/d2122ce8c15de020e121f53509bd5a097aac9cf2/modules/api/src/main/java/org/apache/ignite/tx/IgniteTransactions.java
> > > > > [4]
> > > > >
> > > > >
> > > >
> > >
> >
> https://github.com/apache/ignite-3/blob/d2122ce8c15de020e121f53509bd5a097aac9cf2/modules/table/src/test/java/org/apache/ignite/internal/table/TxTest.java
> > > > >
> > > > >
> > > > > вт, 13 

Re: IEP-61 Transaction API desing for Ignite 3

2021-07-14 Thread Ivan Daschinsky
> yes, it is stated in the javadoc in the PR.
Ah, I see.

ср, 14 июл. 2021 г. в 12:16, Alexei Scherbakov :

> Ivan,
>
> And what if I have already committed transaction? Is it safe rollback
> already committed transaction? Rollback will silently return and do
> nothing? - yes, it is stated in the javadoc in the PR.
>
> Andrey,
>
> Then using "runInTransaction", lack of commit will cause a transaction to
> rollback automatically.
>
> There is no need for a "close" method, it just adds confusion.
>
>
> ср, 14 июл. 2021 г. в 11:37, Andrey Mashenkov  >:
>
> > Agree with Ivan.
> >
> > Method runInTransaction() should try to finish the transaction if the
> user
> > forgot to commit one.
> > I guess it might be a common mistake among new users.
> >
> > Also, I suggest to extent all table projections for better UX.
> > Let's allow
> > table.kvView().withTx(tx)
> > to user may cache kvVew instance and do
> > kvView.withTx(tx)
> > rather than
> > table.withTx(tx).kvVew()
> >
> >
> >
> > On Wed, Jul 14, 2021 at 10:13 AM Ivan Daschinsky 
> > wrote:
> >
> > > Alexey, and is there any analogue to close() of transaction? When you
> > start
> > > transaction, you should somehow to close it, if you don't catch
> exception
> > > or forget to commit.
> > >
> > > I suggest to add method closeAsync() to Transaction, so user can call
> it
> > in
> > > handle or whenComplete, i.e.
> > >
> > > So code will looks like
> > >
> > > CacheApi cache = CacheApi.getCache("testCache");
> > >
> > > Transactions
> > > .beginTransaction()
> > > .thenCompose(tx -> {
> > > CacheApi txCache = cache.withTx(tx);
> > > CompletableFuture result = txCache.getAsync("key")
> > > .thenCompose(val -> {
> > > if (val == "test") {
> > > return txCache.putAsync("key", "test1");
> > > }
> > > else
> > > return CompletableFuture.completedFuture(null);
> > > })
> > > .thenCompose(v -> tx.commitAsync())
> > > .handle((v, ex) -> null);
> > > return result.thenCompose(v -> tx.closeAsync());
> > > });
> > >
> > > I also suggests to add method something like this
> > >
> > > static CompletableFuture inTxAsync(Function > > CompletableFuture> action) {
> > > return Transactions
> > > .beginTransaction()
> > > .thenCompose(tx -> {
> > > CompletableFuture result = action.apply(tx)
> > > .handle((v, ex) -> null);
> > > return result.thenCompose(v -> tx.closeAsync());
> > > });
> > > }
> > >
> > > Async api is not very readable, but this method can help user write
> code,
> > > this is rewritten first example:
> > >
> > > Transactions.inTxAsync(tx -> {
> > > CacheApi txCache = cache.withTx(tx);
> > > return txCache.getAsync("key")
> > > .thenCompose(val -> {
> > > if (val == "test") {
> > > return txCache.putAsync("key", "test1");
> > > }
> > > else
> > > return CompletableFuture.completedFuture(null);
> > > })
> > > .thenCompose(v -> tx.commitAsync());
> > > });
> > >
> > > ср, 14 июл. 2021 г. в 10:03, Alexei Scherbakov <
> > > alexey.scherbak...@gmail.com
> > > >:
> > >
> > > > Andrey,
> > > >
> > > > I suggest you look at the PR [1], if you haven't.
> > > >
> > > > A transaction [2]
> > > > Transactions facade [3]
> > > > Examples [4]
> > > >
> > > > [1] https://github.com/apache/ignite-3/pull/214/files
> > > > [2]
> > > >
> > > >
> > >
> >
> https://github.com/apache/ignite-3/blob/d2122ce8c15de020e121f53509bd5a097aac9cf2/modules/api/src/main/java/org/apache/ignite/tx/Transaction.java
> > > > [3]
> > > >
> > > >
> > >
> >
> https://github.com/apache/ignite-3/blob/d2122ce8c15de020e121f53509bd5a097aac9cf2/modules/api/src/main/java/org/apache/ignite/tx/IgniteTransactions.java
> > > > [4]
> > > >
> > > >
> > >
> >
> https://github.com/apache/ignite-3/blob/d2122ce8c15de020e121f53509bd5a097aac9cf2/modules/table/src/test/java/org/apache/ignite/internal/table/TxTest.java
> > > >
> > > >
> > > > вт, 13 июл. 2021 г. в 19:41, Andrey Gura :
> > > >
> > > > > Alexey,
> > > > >
> > > > > could you please describe Transaction interface?
> > > > >
> > > > > Also it would be great to have a couple examples of using the
> > proposed
> > > > API.
> > > > >
> > > > > On Tue, Jul 13, 2021 at 4:43 PM Alexei Scherbakov
> > > > >  wrote:
> > > > > >
> > > > > > Folks,
> > > > > >
> > > > > > I've prepared a PR implementing my vision of public transactions
> > API.
> > > > > >
> > > > > > API is very simple and similar to Ignite 2, but has some
> > differences.
> > > > > >
> > > > > > More details can be found here [1]
> > > > > >
> > > > > > Share your thoughts.
> > > > > >
> > > > > > [1] https://issues.apache.org/jira/browse/IGNITE-15086
> > > > > >
> > > > > > --
> > > > > >
> > > > > > Best regards,
> > > > > > Alexei Scherbakov
> > > > >
> 

Re: IEP-61 Transaction API desing for Ignite 3

2021-07-14 Thread Andrey Mashenkov
Alex,

I'm not sure CloseAsync() is useful.
As a user, I'd expect runInTransaction(closure) will create Tx for me,
commit Tx after a successful closure call, and rollback Tx in case of error.

However, I insist Transaction should extend AutoClosable, that may be
useful for synchronous case.
try (Transaction tx = beginTx()) {
   ...
   tx.commit();
}
Block code above has the same semantic and shorter, rather than
Transaction tx = beginTx();
try {
   ...
   tx.commit();
}
catch (Throwable th) {
tx.rollback();

throw tx;
}




On Wed, Jul 14, 2021 at 12:16 PM Alexei Scherbakov <
alexey.scherbak...@gmail.com> wrote:

> Ivan,
>
> And what if I have already committed transaction? Is it safe rollback
> already committed transaction? Rollback will silently return and do
> nothing? - yes, it is stated in the javadoc in the PR.
>
> Andrey,
>
> Then using "runInTransaction", lack of commit will cause a transaction to
> rollback automatically.
>
> There is no need for a "close" method, it just adds confusion.
>
>
> ср, 14 июл. 2021 г. в 11:37, Andrey Mashenkov  >:
>
> > Agree with Ivan.
> >
> > Method runInTransaction() should try to finish the transaction if the
> user
> > forgot to commit one.
> > I guess it might be a common mistake among new users.
> >
> > Also, I suggest to extent all table projections for better UX.
> > Let's allow
> > table.kvView().withTx(tx)
> > to user may cache kvVew instance and do
> > kvView.withTx(tx)
> > rather than
> > table.withTx(tx).kvVew()
> >
> >
> >
> > On Wed, Jul 14, 2021 at 10:13 AM Ivan Daschinsky 
> > wrote:
> >
> > > Alexey, and is there any analogue to close() of transaction? When you
> > start
> > > transaction, you should somehow to close it, if you don't catch
> exception
> > > or forget to commit.
> > >
> > > I suggest to add method closeAsync() to Transaction, so user can call
> it
> > in
> > > handle or whenComplete, i.e.
> > >
> > > So code will looks like
> > >
> > > CacheApi cache = CacheApi.getCache("testCache");
> > >
> > > Transactions
> > > .beginTransaction()
> > > .thenCompose(tx -> {
> > > CacheApi txCache = cache.withTx(tx);
> > > CompletableFuture result = txCache.getAsync("key")
> > > .thenCompose(val -> {
> > > if (val == "test") {
> > > return txCache.putAsync("key", "test1");
> > > }
> > > else
> > > return CompletableFuture.completedFuture(null);
> > > })
> > > .thenCompose(v -> tx.commitAsync())
> > > .handle((v, ex) -> null);
> > > return result.thenCompose(v -> tx.closeAsync());
> > > });
> > >
> > > I also suggests to add method something like this
> > >
> > > static CompletableFuture inTxAsync(Function > > CompletableFuture> action) {
> > > return Transactions
> > > .beginTransaction()
> > > .thenCompose(tx -> {
> > > CompletableFuture result = action.apply(tx)
> > > .handle((v, ex) -> null);
> > > return result.thenCompose(v -> tx.closeAsync());
> > > });
> > > }
> > >
> > > Async api is not very readable, but this method can help user write
> code,
> > > this is rewritten first example:
> > >
> > > Transactions.inTxAsync(tx -> {
> > > CacheApi txCache = cache.withTx(tx);
> > > return txCache.getAsync("key")
> > > .thenCompose(val -> {
> > > if (val == "test") {
> > > return txCache.putAsync("key", "test1");
> > > }
> > > else
> > > return CompletableFuture.completedFuture(null);
> > > })
> > > .thenCompose(v -> tx.commitAsync());
> > > });
> > >
> > > ср, 14 июл. 2021 г. в 10:03, Alexei Scherbakov <
> > > alexey.scherbak...@gmail.com
> > > >:
> > >
> > > > Andrey,
> > > >
> > > > I suggest you look at the PR [1], if you haven't.
> > > >
> > > > A transaction [2]
> > > > Transactions facade [3]
> > > > Examples [4]
> > > >
> > > > [1] https://github.com/apache/ignite-3/pull/214/files
> > > > [2]
> > > >
> > > >
> > >
> >
> https://github.com/apache/ignite-3/blob/d2122ce8c15de020e121f53509bd5a097aac9cf2/modules/api/src/main/java/org/apache/ignite/tx/Transaction.java
> > > > [3]
> > > >
> > > >
> > >
> >
> https://github.com/apache/ignite-3/blob/d2122ce8c15de020e121f53509bd5a097aac9cf2/modules/api/src/main/java/org/apache/ignite/tx/IgniteTransactions.java
> > > > [4]
> > > >
> > > >
> > >
> >
> https://github.com/apache/ignite-3/blob/d2122ce8c15de020e121f53509bd5a097aac9cf2/modules/table/src/test/java/org/apache/ignite/internal/table/TxTest.java
> > > >
> > > >
> > > > вт, 13 июл. 2021 г. в 19:41, Andrey Gura :
> > > >
> > > > > Alexey,
> > > > >
> > > > > could you please describe Transaction interface?
> > > > >
> > > > > Also it would be great to have a couple examples of using the
> > proposed
> > > > API.
> > > > >
> > > > > On Tue, Jul 

Re: IEP-61 Transaction API desing for Ignite 3

2021-07-14 Thread Alexei Scherbakov
Adding table.kvView().withTx(tx) seems fine to me.

ср, 14 июл. 2021 г. в 12:15, Alexei Scherbakov :

> Ivan,
>
> And what if I have already committed transaction? Is it safe rollback
> already committed transaction? Rollback will silently return and do
> nothing? - yes, it is stated in the javadoc in the PR.
>
> Andrey,
>
> Then using "runInTransaction", lack of commit will cause a transaction to
> rollback automatically.
>
> There is no need for a "close" method, it just adds confusion.
>
>
> ср, 14 июл. 2021 г. в 11:37, Andrey Mashenkov  >:
>
>> Agree with Ivan.
>>
>> Method runInTransaction() should try to finish the transaction if the user
>> forgot to commit one.
>> I guess it might be a common mistake among new users.
>>
>> Also, I suggest to extent all table projections for better UX.
>> Let's allow
>> table.kvView().withTx(tx)
>> to user may cache kvVew instance and do
>> kvView.withTx(tx)
>> rather than
>> table.withTx(tx).kvVew()
>>
>>
>>
>> On Wed, Jul 14, 2021 at 10:13 AM Ivan Daschinsky 
>> wrote:
>>
>> > Alexey, and is there any analogue to close() of transaction? When you
>> start
>> > transaction, you should somehow to close it, if you don't catch
>> exception
>> > or forget to commit.
>> >
>> > I suggest to add method closeAsync() to Transaction, so user can call
>> it in
>> > handle or whenComplete, i.e.
>> >
>> > So code will looks like
>> >
>> > CacheApi cache = CacheApi.getCache("testCache");
>> >
>> > Transactions
>> > .beginTransaction()
>> > .thenCompose(tx -> {
>> > CacheApi txCache = cache.withTx(tx);
>> > CompletableFuture result = txCache.getAsync("key")
>> > .thenCompose(val -> {
>> > if (val == "test") {
>> > return txCache.putAsync("key", "test1");
>> > }
>> > else
>> > return CompletableFuture.completedFuture(null);
>> > })
>> > .thenCompose(v -> tx.commitAsync())
>> > .handle((v, ex) -> null);
>> > return result.thenCompose(v -> tx.closeAsync());
>> > });
>> >
>> > I also suggests to add method something like this
>> >
>> > static CompletableFuture inTxAsync(Function> > CompletableFuture> action) {
>> > return Transactions
>> > .beginTransaction()
>> > .thenCompose(tx -> {
>> > CompletableFuture result = action.apply(tx)
>> > .handle((v, ex) -> null);
>> > return result.thenCompose(v -> tx.closeAsync());
>> > });
>> > }
>> >
>> > Async api is not very readable, but this method can help user write
>> code,
>> > this is rewritten first example:
>> >
>> > Transactions.inTxAsync(tx -> {
>> > CacheApi txCache = cache.withTx(tx);
>> > return txCache.getAsync("key")
>> > .thenCompose(val -> {
>> > if (val == "test") {
>> > return txCache.putAsync("key", "test1");
>> > }
>> > else
>> > return CompletableFuture.completedFuture(null);
>> > })
>> > .thenCompose(v -> tx.commitAsync());
>> > });
>> >
>> > ср, 14 июл. 2021 г. в 10:03, Alexei Scherbakov <
>> > alexey.scherbak...@gmail.com
>> > >:
>> >
>> > > Andrey,
>> > >
>> > > I suggest you look at the PR [1], if you haven't.
>> > >
>> > > A transaction [2]
>> > > Transactions facade [3]
>> > > Examples [4]
>> > >
>> > > [1] https://github.com/apache/ignite-3/pull/214/files
>> > > [2]
>> > >
>> > >
>> >
>> https://github.com/apache/ignite-3/blob/d2122ce8c15de020e121f53509bd5a097aac9cf2/modules/api/src/main/java/org/apache/ignite/tx/Transaction.java
>> > > [3]
>> > >
>> > >
>> >
>> https://github.com/apache/ignite-3/blob/d2122ce8c15de020e121f53509bd5a097aac9cf2/modules/api/src/main/java/org/apache/ignite/tx/IgniteTransactions.java
>> > > [4]
>> > >
>> > >
>> >
>> https://github.com/apache/ignite-3/blob/d2122ce8c15de020e121f53509bd5a097aac9cf2/modules/table/src/test/java/org/apache/ignite/internal/table/TxTest.java
>> > >
>> > >
>> > > вт, 13 июл. 2021 г. в 19:41, Andrey Gura :
>> > >
>> > > > Alexey,
>> > > >
>> > > > could you please describe Transaction interface?
>> > > >
>> > > > Also it would be great to have a couple examples of using the
>> proposed
>> > > API.
>> > > >
>> > > > On Tue, Jul 13, 2021 at 4:43 PM Alexei Scherbakov
>> > > >  wrote:
>> > > > >
>> > > > > Folks,
>> > > > >
>> > > > > I've prepared a PR implementing my vision of public transactions
>> API.
>> > > > >
>> > > > > API is very simple and similar to Ignite 2, but has some
>> differences.
>> > > > >
>> > > > > More details can be found here [1]
>> > > > >
>> > > > > Share your thoughts.
>> > > > >
>> > > > > [1] https://issues.apache.org/jira/browse/IGNITE-15086
>> > > > >
>> > > > > --
>> > > > >
>> > > > > Best regards,
>> > > > > Alexei Scherbakov
>> > > >
>> > >
>> > >
>> > > --
>> > >
>> > > Best regards,
>> > > Alexei Scherbakov
>> > >
>> >
>> >
>> > --
>> > Sincerely yours, Ivan Daschinskiy
>> >
>>
>>
>> 

Re: IEP-61 Transaction API desing for Ignite 3

2021-07-14 Thread Alexei Scherbakov
Ivan,

And what if I have already committed transaction? Is it safe rollback
already committed transaction? Rollback will silently return and do
nothing? - yes, it is stated in the javadoc in the PR.

Andrey,

Then using "runInTransaction", lack of commit will cause a transaction to
rollback automatically.

There is no need for a "close" method, it just adds confusion.


ср, 14 июл. 2021 г. в 11:37, Andrey Mashenkov :

> Agree with Ivan.
>
> Method runInTransaction() should try to finish the transaction if the user
> forgot to commit one.
> I guess it might be a common mistake among new users.
>
> Also, I suggest to extent all table projections for better UX.
> Let's allow
> table.kvView().withTx(tx)
> to user may cache kvVew instance and do
> kvView.withTx(tx)
> rather than
> table.withTx(tx).kvVew()
>
>
>
> On Wed, Jul 14, 2021 at 10:13 AM Ivan Daschinsky 
> wrote:
>
> > Alexey, and is there any analogue to close() of transaction? When you
> start
> > transaction, you should somehow to close it, if you don't catch exception
> > or forget to commit.
> >
> > I suggest to add method closeAsync() to Transaction, so user can call it
> in
> > handle or whenComplete, i.e.
> >
> > So code will looks like
> >
> > CacheApi cache = CacheApi.getCache("testCache");
> >
> > Transactions
> > .beginTransaction()
> > .thenCompose(tx -> {
> > CacheApi txCache = cache.withTx(tx);
> > CompletableFuture result = txCache.getAsync("key")
> > .thenCompose(val -> {
> > if (val == "test") {
> > return txCache.putAsync("key", "test1");
> > }
> > else
> > return CompletableFuture.completedFuture(null);
> > })
> > .thenCompose(v -> tx.commitAsync())
> > .handle((v, ex) -> null);
> > return result.thenCompose(v -> tx.closeAsync());
> > });
> >
> > I also suggests to add method something like this
> >
> > static CompletableFuture inTxAsync(Function > CompletableFuture> action) {
> > return Transactions
> > .beginTransaction()
> > .thenCompose(tx -> {
> > CompletableFuture result = action.apply(tx)
> > .handle((v, ex) -> null);
> > return result.thenCompose(v -> tx.closeAsync());
> > });
> > }
> >
> > Async api is not very readable, but this method can help user write code,
> > this is rewritten first example:
> >
> > Transactions.inTxAsync(tx -> {
> > CacheApi txCache = cache.withTx(tx);
> > return txCache.getAsync("key")
> > .thenCompose(val -> {
> > if (val == "test") {
> > return txCache.putAsync("key", "test1");
> > }
> > else
> > return CompletableFuture.completedFuture(null);
> > })
> > .thenCompose(v -> tx.commitAsync());
> > });
> >
> > ср, 14 июл. 2021 г. в 10:03, Alexei Scherbakov <
> > alexey.scherbak...@gmail.com
> > >:
> >
> > > Andrey,
> > >
> > > I suggest you look at the PR [1], if you haven't.
> > >
> > > A transaction [2]
> > > Transactions facade [3]
> > > Examples [4]
> > >
> > > [1] https://github.com/apache/ignite-3/pull/214/files
> > > [2]
> > >
> > >
> >
> https://github.com/apache/ignite-3/blob/d2122ce8c15de020e121f53509bd5a097aac9cf2/modules/api/src/main/java/org/apache/ignite/tx/Transaction.java
> > > [3]
> > >
> > >
> >
> https://github.com/apache/ignite-3/blob/d2122ce8c15de020e121f53509bd5a097aac9cf2/modules/api/src/main/java/org/apache/ignite/tx/IgniteTransactions.java
> > > [4]
> > >
> > >
> >
> https://github.com/apache/ignite-3/blob/d2122ce8c15de020e121f53509bd5a097aac9cf2/modules/table/src/test/java/org/apache/ignite/internal/table/TxTest.java
> > >
> > >
> > > вт, 13 июл. 2021 г. в 19:41, Andrey Gura :
> > >
> > > > Alexey,
> > > >
> > > > could you please describe Transaction interface?
> > > >
> > > > Also it would be great to have a couple examples of using the
> proposed
> > > API.
> > > >
> > > > On Tue, Jul 13, 2021 at 4:43 PM Alexei Scherbakov
> > > >  wrote:
> > > > >
> > > > > Folks,
> > > > >
> > > > > I've prepared a PR implementing my vision of public transactions
> API.
> > > > >
> > > > > API is very simple and similar to Ignite 2, but has some
> differences.
> > > > >
> > > > > More details can be found here [1]
> > > > >
> > > > > Share your thoughts.
> > > > >
> > > > > [1] https://issues.apache.org/jira/browse/IGNITE-15086
> > > > >
> > > > > --
> > > > >
> > > > > Best regards,
> > > > > Alexei Scherbakov
> > > >
> > >
> > >
> > > --
> > >
> > > Best regards,
> > > Alexei Scherbakov
> > >
> >
> >
> > --
> > Sincerely yours, Ivan Daschinskiy
> >
>
>
> --
> Best regards,
> Andrey V. Mashenkov
>


-- 

Best regards,
Alexei Scherbakov


Re: IEP-61 Transaction API desing for Ignite 3

2021-07-14 Thread Andrey Mashenkov
Agree with Ivan.

Method runInTransaction() should try to finish the transaction if the user
forgot to commit one.
I guess it might be a common mistake among new users.

Also, I suggest to extent all table projections for better UX.
Let's allow
table.kvView().withTx(tx)
to user may cache kvVew instance and do
kvView.withTx(tx)
rather than
table.withTx(tx).kvVew()



On Wed, Jul 14, 2021 at 10:13 AM Ivan Daschinsky 
wrote:

> Alexey, and is there any analogue to close() of transaction? When you start
> transaction, you should somehow to close it, if you don't catch exception
> or forget to commit.
>
> I suggest to add method closeAsync() to Transaction, so user can call it in
> handle or whenComplete, i.e.
>
> So code will looks like
>
> CacheApi cache = CacheApi.getCache("testCache");
>
> Transactions
> .beginTransaction()
> .thenCompose(tx -> {
> CacheApi txCache = cache.withTx(tx);
> CompletableFuture result = txCache.getAsync("key")
> .thenCompose(val -> {
> if (val == "test") {
> return txCache.putAsync("key", "test1");
> }
> else
> return CompletableFuture.completedFuture(null);
> })
> .thenCompose(v -> tx.commitAsync())
> .handle((v, ex) -> null);
> return result.thenCompose(v -> tx.closeAsync());
> });
>
> I also suggests to add method something like this
>
> static CompletableFuture inTxAsync(Function CompletableFuture> action) {
> return Transactions
> .beginTransaction()
> .thenCompose(tx -> {
> CompletableFuture result = action.apply(tx)
> .handle((v, ex) -> null);
> return result.thenCompose(v -> tx.closeAsync());
> });
> }
>
> Async api is not very readable, but this method can help user write code,
> this is rewritten first example:
>
> Transactions.inTxAsync(tx -> {
> CacheApi txCache = cache.withTx(tx);
> return txCache.getAsync("key")
> .thenCompose(val -> {
> if (val == "test") {
> return txCache.putAsync("key", "test1");
> }
> else
> return CompletableFuture.completedFuture(null);
> })
> .thenCompose(v -> tx.commitAsync());
> });
>
> ср, 14 июл. 2021 г. в 10:03, Alexei Scherbakov <
> alexey.scherbak...@gmail.com
> >:
>
> > Andrey,
> >
> > I suggest you look at the PR [1], if you haven't.
> >
> > A transaction [2]
> > Transactions facade [3]
> > Examples [4]
> >
> > [1] https://github.com/apache/ignite-3/pull/214/files
> > [2]
> >
> >
> https://github.com/apache/ignite-3/blob/d2122ce8c15de020e121f53509bd5a097aac9cf2/modules/api/src/main/java/org/apache/ignite/tx/Transaction.java
> > [3]
> >
> >
> https://github.com/apache/ignite-3/blob/d2122ce8c15de020e121f53509bd5a097aac9cf2/modules/api/src/main/java/org/apache/ignite/tx/IgniteTransactions.java
> > [4]
> >
> >
> https://github.com/apache/ignite-3/blob/d2122ce8c15de020e121f53509bd5a097aac9cf2/modules/table/src/test/java/org/apache/ignite/internal/table/TxTest.java
> >
> >
> > вт, 13 июл. 2021 г. в 19:41, Andrey Gura :
> >
> > > Alexey,
> > >
> > > could you please describe Transaction interface?
> > >
> > > Also it would be great to have a couple examples of using the proposed
> > API.
> > >
> > > On Tue, Jul 13, 2021 at 4:43 PM Alexei Scherbakov
> > >  wrote:
> > > >
> > > > Folks,
> > > >
> > > > I've prepared a PR implementing my vision of public transactions API.
> > > >
> > > > API is very simple and similar to Ignite 2, but has some differences.
> > > >
> > > > More details can be found here [1]
> > > >
> > > > Share your thoughts.
> > > >
> > > > [1] https://issues.apache.org/jira/browse/IGNITE-15086
> > > >
> > > > --
> > > >
> > > > Best regards,
> > > > Alexei Scherbakov
> > >
> >
> >
> > --
> >
> > Best regards,
> > Alexei Scherbakov
> >
>
>
> --
> Sincerely yours, Ivan Daschinskiy
>


-- 
Best regards,
Andrey V. Mashenkov


Re: IEP-61 Transaction API desing for Ignite 3

2021-07-14 Thread Ivan Daschinsky
>>  Semantically "close" is the same as "rollback".
And what if I have already committed transaction? Is it safe rollback
already committed transaction? Rollback will silently return and do nothing?

ср, 14 июл. 2021 г. в 11:25, Alexei Scherbakov :

> Ivan,
>
> We don't need the "close" method in the proposed approach, because it's
> enough having "commit(Async)" and "rollback(Async)" to finish a
> transaction. Semantically "close" is the same as "rollback".
>
> If you are using "runInTransaction" API, you can't forget to call "close",
> it will be done automatically (rollback will be called at the end of a
> transaction closure).
>
> For async API it's required semantically to call commit or rollback.
>
> As for utility methods, I would keep tx API as small as possible for now.
> Anyway, these methods can be added later, if the need arises.
>
>
>
>
>
> ср, 14 июл. 2021 г. в 10:13, Ivan Daschinsky :
>
> > Alexey, and is there any analogue to close() of transaction? When you
> start
> > transaction, you should somehow to close it, if you don't catch exception
> > or forget to commit.
> >
> > I suggest to add method closeAsync() to Transaction, so user can call it
> in
> > handle or whenComplete, i.e.
> >
> > So code will looks like
> >
> > CacheApi cache = CacheApi.getCache("testCache");
> >
> > Transactions
> > .beginTransaction()
> > .thenCompose(tx -> {
> > CacheApi txCache = cache.withTx(tx);
> > CompletableFuture result = txCache.getAsync("key")
> > .thenCompose(val -> {
> > if (val == "test") {
> > return txCache.putAsync("key", "test1");
> > }
> > else
> > return CompletableFuture.completedFuture(null);
> > })
> > .thenCompose(v -> tx.commitAsync())
> > .handle((v, ex) -> null);
> > return result.thenCompose(v -> tx.closeAsync());
> > });
> >
> > I also suggests to add method something like this
> >
> > static CompletableFuture inTxAsync(Function > CompletableFuture> action) {
> > return Transactions
> > .beginTransaction()
> > .thenCompose(tx -> {
> > CompletableFuture result = action.apply(tx)
> > .handle((v, ex) -> null);
> > return result.thenCompose(v -> tx.closeAsync());
> > });
> > }
> >
> > Async api is not very readable, but this method can help user write code,
> > this is rewritten first example:
> >
> > Transactions.inTxAsync(tx -> {
> > CacheApi txCache = cache.withTx(tx);
> > return txCache.getAsync("key")
> > .thenCompose(val -> {
> > if (val == "test") {
> > return txCache.putAsync("key", "test1");
> > }
> > else
> > return CompletableFuture.completedFuture(null);
> > })
> > .thenCompose(v -> tx.commitAsync());
> > });
> >
> > ср, 14 июл. 2021 г. в 10:03, Alexei Scherbakov <
> > alexey.scherbak...@gmail.com
> > >:
> >
> > > Andrey,
> > >
> > > I suggest you look at the PR [1], if you haven't.
> > >
> > > A transaction [2]
> > > Transactions facade [3]
> > > Examples [4]
> > >
> > > [1] https://github.com/apache/ignite-3/pull/214/files
> > > [2]
> > >
> > >
> >
> https://github.com/apache/ignite-3/blob/d2122ce8c15de020e121f53509bd5a097aac9cf2/modules/api/src/main/java/org/apache/ignite/tx/Transaction.java
> > > [3]
> > >
> > >
> >
> https://github.com/apache/ignite-3/blob/d2122ce8c15de020e121f53509bd5a097aac9cf2/modules/api/src/main/java/org/apache/ignite/tx/IgniteTransactions.java
> > > [4]
> > >
> > >
> >
> https://github.com/apache/ignite-3/blob/d2122ce8c15de020e121f53509bd5a097aac9cf2/modules/table/src/test/java/org/apache/ignite/internal/table/TxTest.java
> > >
> > >
> > > вт, 13 июл. 2021 г. в 19:41, Andrey Gura :
> > >
> > > > Alexey,
> > > >
> > > > could you please describe Transaction interface?
> > > >
> > > > Also it would be great to have a couple examples of using the
> proposed
> > > API.
> > > >
> > > > On Tue, Jul 13, 2021 at 4:43 PM Alexei Scherbakov
> > > >  wrote:
> > > > >
> > > > > Folks,
> > > > >
> > > > > I've prepared a PR implementing my vision of public transactions
> API.
> > > > >
> > > > > API is very simple and similar to Ignite 2, but has some
> differences.
> > > > >
> > > > > More details can be found here [1]
> > > > >
> > > > > Share your thoughts.
> > > > >
> > > > > [1] https://issues.apache.org/jira/browse/IGNITE-15086
> > > > >
> > > > > --
> > > > >
> > > > > Best regards,
> > > > > Alexei Scherbakov
> > > >
> > >
> > >
> > > --
> > >
> > > Best regards,
> > > Alexei Scherbakov
> > >
> >
> >
> > --
> > Sincerely yours, Ivan Daschinskiy
> >
>
>
> --
>
> Best regards,
> Alexei Scherbakov
>


-- 
Sincerely yours, Ivan Daschinskiy


Re: IEP-61 Transaction API desing for Ignite 3

2021-07-14 Thread Alexei Scherbakov
Ivan,

We don't need the "close" method in the proposed approach, because it's
enough having "commit(Async)" and "rollback(Async)" to finish a
transaction. Semantically "close" is the same as "rollback".

If you are using "runInTransaction" API, you can't forget to call "close",
it will be done automatically (rollback will be called at the end of a
transaction closure).

For async API it's required semantically to call commit or rollback.

As for utility methods, I would keep tx API as small as possible for now.
Anyway, these methods can be added later, if the need arises.





ср, 14 июл. 2021 г. в 10:13, Ivan Daschinsky :

> Alexey, and is there any analogue to close() of transaction? When you start
> transaction, you should somehow to close it, if you don't catch exception
> or forget to commit.
>
> I suggest to add method closeAsync() to Transaction, so user can call it in
> handle or whenComplete, i.e.
>
> So code will looks like
>
> CacheApi cache = CacheApi.getCache("testCache");
>
> Transactions
> .beginTransaction()
> .thenCompose(tx -> {
> CacheApi txCache = cache.withTx(tx);
> CompletableFuture result = txCache.getAsync("key")
> .thenCompose(val -> {
> if (val == "test") {
> return txCache.putAsync("key", "test1");
> }
> else
> return CompletableFuture.completedFuture(null);
> })
> .thenCompose(v -> tx.commitAsync())
> .handle((v, ex) -> null);
> return result.thenCompose(v -> tx.closeAsync());
> });
>
> I also suggests to add method something like this
>
> static CompletableFuture inTxAsync(Function CompletableFuture> action) {
> return Transactions
> .beginTransaction()
> .thenCompose(tx -> {
> CompletableFuture result = action.apply(tx)
> .handle((v, ex) -> null);
> return result.thenCompose(v -> tx.closeAsync());
> });
> }
>
> Async api is not very readable, but this method can help user write code,
> this is rewritten first example:
>
> Transactions.inTxAsync(tx -> {
> CacheApi txCache = cache.withTx(tx);
> return txCache.getAsync("key")
> .thenCompose(val -> {
> if (val == "test") {
> return txCache.putAsync("key", "test1");
> }
> else
> return CompletableFuture.completedFuture(null);
> })
> .thenCompose(v -> tx.commitAsync());
> });
>
> ср, 14 июл. 2021 г. в 10:03, Alexei Scherbakov <
> alexey.scherbak...@gmail.com
> >:
>
> > Andrey,
> >
> > I suggest you look at the PR [1], if you haven't.
> >
> > A transaction [2]
> > Transactions facade [3]
> > Examples [4]
> >
> > [1] https://github.com/apache/ignite-3/pull/214/files
> > [2]
> >
> >
> https://github.com/apache/ignite-3/blob/d2122ce8c15de020e121f53509bd5a097aac9cf2/modules/api/src/main/java/org/apache/ignite/tx/Transaction.java
> > [3]
> >
> >
> https://github.com/apache/ignite-3/blob/d2122ce8c15de020e121f53509bd5a097aac9cf2/modules/api/src/main/java/org/apache/ignite/tx/IgniteTransactions.java
> > [4]
> >
> >
> https://github.com/apache/ignite-3/blob/d2122ce8c15de020e121f53509bd5a097aac9cf2/modules/table/src/test/java/org/apache/ignite/internal/table/TxTest.java
> >
> >
> > вт, 13 июл. 2021 г. в 19:41, Andrey Gura :
> >
> > > Alexey,
> > >
> > > could you please describe Transaction interface?
> > >
> > > Also it would be great to have a couple examples of using the proposed
> > API.
> > >
> > > On Tue, Jul 13, 2021 at 4:43 PM Alexei Scherbakov
> > >  wrote:
> > > >
> > > > Folks,
> > > >
> > > > I've prepared a PR implementing my vision of public transactions API.
> > > >
> > > > API is very simple and similar to Ignite 2, but has some differences.
> > > >
> > > > More details can be found here [1]
> > > >
> > > > Share your thoughts.
> > > >
> > > > [1] https://issues.apache.org/jira/browse/IGNITE-15086
> > > >
> > > > --
> > > >
> > > > Best regards,
> > > > Alexei Scherbakov
> > >
> >
> >
> > --
> >
> > Best regards,
> > Alexei Scherbakov
> >
>
>
> --
> Sincerely yours, Ivan Daschinskiy
>


-- 

Best regards,
Alexei Scherbakov


Re: IEP-61 Transaction API desing for Ignite 3

2021-07-14 Thread Ivan Daschinsky
Alexey, and is there any analogue to close() of transaction? When you start
transaction, you should somehow to close it, if you don't catch exception
or forget to commit.

I suggest to add method closeAsync() to Transaction, so user can call it in
handle or whenComplete, i.e.

So code will looks like

CacheApi cache = CacheApi.getCache("testCache");

Transactions
.beginTransaction()
.thenCompose(tx -> {
CacheApi txCache = cache.withTx(tx);
CompletableFuture result = txCache.getAsync("key")
.thenCompose(val -> {
if (val == "test") {
return txCache.putAsync("key", "test1");
}
else
return CompletableFuture.completedFuture(null);
})
.thenCompose(v -> tx.commitAsync())
.handle((v, ex) -> null);
return result.thenCompose(v -> tx.closeAsync());
});

I also suggests to add method something like this

static CompletableFuture inTxAsync(Function> action) {
return Transactions
.beginTransaction()
.thenCompose(tx -> {
CompletableFuture result = action.apply(tx)
.handle((v, ex) -> null);
return result.thenCompose(v -> tx.closeAsync());
});
}

Async api is not very readable, but this method can help user write code,
this is rewritten first example:

Transactions.inTxAsync(tx -> {
CacheApi txCache = cache.withTx(tx);
return txCache.getAsync("key")
.thenCompose(val -> {
if (val == "test") {
return txCache.putAsync("key", "test1");
}
else
return CompletableFuture.completedFuture(null);
})
.thenCompose(v -> tx.commitAsync());
});

ср, 14 июл. 2021 г. в 10:03, Alexei Scherbakov :

> Andrey,
>
> I suggest you look at the PR [1], if you haven't.
>
> A transaction [2]
> Transactions facade [3]
> Examples [4]
>
> [1] https://github.com/apache/ignite-3/pull/214/files
> [2]
>
> https://github.com/apache/ignite-3/blob/d2122ce8c15de020e121f53509bd5a097aac9cf2/modules/api/src/main/java/org/apache/ignite/tx/Transaction.java
> [3]
>
> https://github.com/apache/ignite-3/blob/d2122ce8c15de020e121f53509bd5a097aac9cf2/modules/api/src/main/java/org/apache/ignite/tx/IgniteTransactions.java
> [4]
>
> https://github.com/apache/ignite-3/blob/d2122ce8c15de020e121f53509bd5a097aac9cf2/modules/table/src/test/java/org/apache/ignite/internal/table/TxTest.java
>
>
> вт, 13 июл. 2021 г. в 19:41, Andrey Gura :
>
> > Alexey,
> >
> > could you please describe Transaction interface?
> >
> > Also it would be great to have a couple examples of using the proposed
> API.
> >
> > On Tue, Jul 13, 2021 at 4:43 PM Alexei Scherbakov
> >  wrote:
> > >
> > > Folks,
> > >
> > > I've prepared a PR implementing my vision of public transactions API.
> > >
> > > API is very simple and similar to Ignite 2, but has some differences.
> > >
> > > More details can be found here [1]
> > >
> > > Share your thoughts.
> > >
> > > [1] https://issues.apache.org/jira/browse/IGNITE-15086
> > >
> > > --
> > >
> > > Best regards,
> > > Alexei Scherbakov
> >
>
>
> --
>
> Best regards,
> Alexei Scherbakov
>


-- 
Sincerely yours, Ivan Daschinskiy


Re: IEP-61 Transaction API desing for Ignite 3

2021-07-14 Thread Alexei Scherbakov
Andrey,

I suggest you look at the PR [1], if you haven't.

A transaction [2]
Transactions facade [3]
Examples [4]

[1] https://github.com/apache/ignite-3/pull/214/files
[2]
https://github.com/apache/ignite-3/blob/d2122ce8c15de020e121f53509bd5a097aac9cf2/modules/api/src/main/java/org/apache/ignite/tx/Transaction.java
[3]
https://github.com/apache/ignite-3/blob/d2122ce8c15de020e121f53509bd5a097aac9cf2/modules/api/src/main/java/org/apache/ignite/tx/IgniteTransactions.java
[4]
https://github.com/apache/ignite-3/blob/d2122ce8c15de020e121f53509bd5a097aac9cf2/modules/table/src/test/java/org/apache/ignite/internal/table/TxTest.java


вт, 13 июл. 2021 г. в 19:41, Andrey Gura :

> Alexey,
>
> could you please describe Transaction interface?
>
> Also it would be great to have a couple examples of using the proposed API.
>
> On Tue, Jul 13, 2021 at 4:43 PM Alexei Scherbakov
>  wrote:
> >
> > Folks,
> >
> > I've prepared a PR implementing my vision of public transactions API.
> >
> > API is very simple and similar to Ignite 2, but has some differences.
> >
> > More details can be found here [1]
> >
> > Share your thoughts.
> >
> > [1] https://issues.apache.org/jira/browse/IGNITE-15086
> >
> > --
> >
> > Best regards,
> > Alexei Scherbakov
>


-- 

Best regards,
Alexei Scherbakov


Re: IEP-61 Transaction API desing for Ignite 3

2021-07-13 Thread Andrey Gura
Alexey,

could you please describe Transaction interface?

Also it would be great to have a couple examples of using the proposed API.

On Tue, Jul 13, 2021 at 4:43 PM Alexei Scherbakov
 wrote:
>
> Folks,
>
> I've prepared a PR implementing my vision of public transactions API.
>
> API is very simple and similar to Ignite 2, but has some differences.
>
> More details can be found here [1]
>
> Share your thoughts.
>
> [1] https://issues.apache.org/jira/browse/IGNITE-15086
>
> --
>
> Best regards,
> Alexei Scherbakov


Re: IEP-61 Transaction API desing for Ignite 3

2021-07-13 Thread Pavel Tupitsyn
> avoid any thread based control of transactions...
> single thread will be able to work with any amount of
> transactions at the same time

Yes, and one TX can be used by many threads, which is demonstrated
by testTxAsync.
I completely agree with this approach.

Ok, let's see what others say about "runInTransaction".






On Tue, Jul 13, 2021 at 6:51 PM Alexei Scherbakov <
alexey.scherbak...@gmail.com> wrote:

> Pavel,
>
> "runInTransaction" is supposed to provide an "old-fashioned" way to write a
> transaction for easier migration.
>
> Manual enlisting of tables is required, because I strive to avoid any
> thread based control of transactions in Ignite 3.
>
> Actually, a single thread will be able to work with any amount of
> transactions at the same time.
>
> I would keep it for convenience, but let's see other opinions.
>
>
>
>
>
>
> вт, 13 июл. 2021 г. в 18:22, Pavel Tupitsyn :
>
> > Alexei,
> >
> > The API looks good to me, except "runInTransaction", which I find
> > confusing.
> >
> > It looks like every operation performed by the passed Consumer will be
> > automatically enlisted in a transaction,
> > but, looking at tests, "withTx" call is still required inside the
> Consumer.
> >
> > I don't think we need this method at all, it barely provides any
> > convenience but may confuse some users.
> >
> > On Tue, Jul 13, 2021 at 4:43 PM Alexei Scherbakov <
> > alexey.scherbak...@gmail.com> wrote:
> >
> > > Folks,
> > >
> > > I've prepared a PR implementing my vision of public transactions API.
> > >
> > > API is very simple and similar to Ignite 2, but has some differences.
> > >
> > > More details can be found here [1]
> > >
> > > Share your thoughts.
> > >
> > > [1] https://issues.apache.org/jira/browse/IGNITE-15086
> > >
> > > --
> > >
> > > Best regards,
> > > Alexei Scherbakov
> > >
> >
>
>
> --
>
> Best regards,
> Alexei Scherbakov
>


Re: IEP-61 Transaction API desing for Ignite 3

2021-07-13 Thread Alexei Scherbakov
Pavel,

"runInTransaction" is supposed to provide an "old-fashioned" way to write a
transaction for easier migration.

Manual enlisting of tables is required, because I strive to avoid any
thread based control of transactions in Ignite 3.

Actually, a single thread will be able to work with any amount of
transactions at the same time.

I would keep it for convenience, but let's see other opinions.






вт, 13 июл. 2021 г. в 18:22, Pavel Tupitsyn :

> Alexei,
>
> The API looks good to me, except "runInTransaction", which I find
> confusing.
>
> It looks like every operation performed by the passed Consumer will be
> automatically enlisted in a transaction,
> but, looking at tests, "withTx" call is still required inside the Consumer.
>
> I don't think we need this method at all, it barely provides any
> convenience but may confuse some users.
>
> On Tue, Jul 13, 2021 at 4:43 PM Alexei Scherbakov <
> alexey.scherbak...@gmail.com> wrote:
>
> > Folks,
> >
> > I've prepared a PR implementing my vision of public transactions API.
> >
> > API is very simple and similar to Ignite 2, but has some differences.
> >
> > More details can be found here [1]
> >
> > Share your thoughts.
> >
> > [1] https://issues.apache.org/jira/browse/IGNITE-15086
> >
> > --
> >
> > Best regards,
> > Alexei Scherbakov
> >
>


-- 

Best regards,
Alexei Scherbakov


Re: IEP-61 Transaction API desing for Ignite 3

2021-07-13 Thread Pavel Tupitsyn
Alexei,

The API looks good to me, except "runInTransaction", which I find confusing.

It looks like every operation performed by the passed Consumer will be
automatically enlisted in a transaction,
but, looking at tests, "withTx" call is still required inside the Consumer.

I don't think we need this method at all, it barely provides any
convenience but may confuse some users.

On Tue, Jul 13, 2021 at 4:43 PM Alexei Scherbakov <
alexey.scherbak...@gmail.com> wrote:

> Folks,
>
> I've prepared a PR implementing my vision of public transactions API.
>
> API is very simple and similar to Ignite 2, but has some differences.
>
> More details can be found here [1]
>
> Share your thoughts.
>
> [1] https://issues.apache.org/jira/browse/IGNITE-15086
>
> --
>
> Best regards,
> Alexei Scherbakov
>


IEP-61 Transaction API desing for Ignite 3

2021-07-13 Thread Alexei Scherbakov
Folks,

I've prepared a PR implementing my vision of public transactions API.

API is very simple and similar to Ignite 2, but has some differences.

More details can be found here [1]

Share your thoughts.

[1] https://issues.apache.org/jira/browse/IGNITE-15086

-- 

Best regards,
Alexei Scherbakov