Re: [HACKERS] logical decoding of two-phase transactions

2022-01-04 Thread Amit Kapila
On Tue, Jan 4, 2022 at 9:00 AM Masahiko Sawada  wrote:
>
> According to the doc, the two_phase field has:
>
> True if the slot is enabled for decoding prepared transactions. Always
> false for physical slots.
>
> It's unnatural a bit to me that replication slots have such a property
> since the replication slots have been used to protect WAL and tuples
> that are required for logical decoding, physical replication, and
> backup, etc from removal. Also, it seems that even if a replication
> slot is created with two_phase = off, it's overwritten to on if the
> plugin enables two-phase option. Is there any reason why we can turn
> on and off this value on the replication slot side and is there any
> use case where the replication slot’s two_phase is false and the
> plugin’s two-phase option is on and vice versa?
>

We enable two_phase only when we start streaming from the
subscriber-side. This is required because we can't enable it till the
initial sync is complete, otherwise, it could lead to loss of data.
See comments atop worker.c (description under the title: TWO_PHASE
TRANSACTIONS).

> I think that we can
> have replication slots always have two_phase_at value and remove the
> two_phase field from the view.
>

I am not sure how that will work because we can allow streaming of
prepared transactions when the same is enabled at the CREATE
SUBSCRIPTION time, the default for which is false.

-- 
With Regards,
Amit Kapila.




Re: [HACKERS] logical decoding of two-phase transactions

2022-01-03 Thread Masahiko Sawada
Hi,

On Mon, Aug 9, 2021 at 12:00 PM Amit Kapila  wrote:
>
> On Wed, Aug 4, 2021 at 4:14 PM Amit Kapila  wrote:
> >
> > I have pushed this last patch in the series.
> >
>
> I have closed this CF entry. Thanks to everyone involved in this work!
>

I have a questoin about two_phase column of pg_replication_slots view:
with this feature, pg_replication_slots has a new column two_phase:

 View "pg_catalog.pg_replication_slots"
   Column|  Type   | Collation | Nullable | Default
-+-+---+--+-
 slot_name   | name|   |  |
 plugin  | name|   |  |
 slot_type   | text|   |  |
 datoid  | oid |   |  |
 database| name|   |  |
 temporary   | boolean |   |  |
 active  | boolean |   |  |
 active_pid  | integer |   |  |
 xmin| xid |   |  |
 catalog_xmin| xid |   |  |
 restart_lsn | pg_lsn  |   |  |
 confirmed_flush_lsn | pg_lsn  |   |  |
 wal_status  | text|   |  |
 safe_wal_size   | bigint  |   |  |
 two_phase   | boolean |   |  |

According to the doc, the two_phase field has:

True if the slot is enabled for decoding prepared transactions. Always
false for physical slots.

It's unnatural a bit to me that replication slots have such a property
since the replication slots have been used to protect WAL and tuples
that are required for logical decoding, physical replication, and
backup, etc from removal. Also, it seems that even if a replication
slot is created with two_phase = off, it's overwritten to on if the
plugin enables two-phase option. Is there any reason why we can turn
on and off this value on the replication slot side and is there any
use case where the replication slot’s two_phase is false and the
plugin’s two-phase option is on and vice versa? I think that we can
have replication slots always have two_phase_at value and remove the
two_phase field from the view.

Regards,

-- 
Masahiko Sawada
EDB:  https://www.enterprisedb.com/




Re: [HACKERS] logical decoding of two-phase transactions

2021-08-08 Thread Amit Kapila
On Wed, Aug 4, 2021 at 4:14 PM Amit Kapila  wrote:
>
> I have pushed this last patch in the series.
>

I have closed this CF entry. Thanks to everyone involved in this work!

-- 
With Regards,
Amit Kapila.




Re: [HACKERS] logical decoding of two-phase transactions

2021-08-04 Thread Amit Kapila
On Wed, Aug 4, 2021 at 6:51 AM tanghy.f...@fujitsu.com
 wrote:
>
> On Tuesday, August 3, 2021 6:03 PM vignesh C wrote:
> >
> > On Tue, Aug 3, 2021 at 12:32 PM Amit Kapila  wrote:
> > >
> > > On Tue, Aug 3, 2021 at 6:17 AM Peter Smith  wrote:
> > > >
> > > > Please find attached the latest patch set v102*
> > > >
> > >
> > > I have made minor modifications in the comments and docs, please see
> > > attached. Can you please check whether the names of contributors in
> > > the commit message are correct or do we need to change it?
> >
> > The patch applies neatly, the tests pass and documentation built with
> > the updates provided. I could not find any comments. The patch looks
> > good to me.
> >
>
> I did some stress tests on the patch and found no issues.
> It also works well when using synchronized replication.
> So the patch LGTM.
>

I have pushed this last patch in the series.

-- 
With Regards,
Amit Kapila.




RE: [HACKERS] logical decoding of two-phase transactions

2021-08-03 Thread tanghy.f...@fujitsu.com
On Tuesday, August 3, 2021 6:03 PM vignesh C wrote:
> 
> On Tue, Aug 3, 2021 at 12:32 PM Amit Kapila  wrote:
> >
> > On Tue, Aug 3, 2021 at 6:17 AM Peter Smith  wrote:
> > >
> > > Please find attached the latest patch set v102*
> > >
> >
> > I have made minor modifications in the comments and docs, please see
> > attached. Can you please check whether the names of contributors in
> > the commit message are correct or do we need to change it?
> 
> The patch applies neatly, the tests pass and documentation built with
> the updates provided. I could not find any comments. The patch looks
> good to me.
> 

I did some stress tests on the patch and found no issues. 
It also works well when using synchronized replication.
So the patch LGTM.

Regards
Tang


Re: [HACKERS] logical decoding of two-phase transactions

2021-08-03 Thread vignesh C
On Tue, Aug 3, 2021 at 12:32 PM Amit Kapila  wrote:
>
> On Tue, Aug 3, 2021 at 6:17 AM Peter Smith  wrote:
> >
> > Please find attached the latest patch set v102*
> >
>
> I have made minor modifications in the comments and docs, please see
> attached. Can you please check whether the names of contributors in
> the commit message are correct or do we need to change it?

The patch applies neatly, the tests pass and documentation built with
the updates provided. I could not find any comments. The patch looks
good to me.

Regards,
Vignesh




Re: [HACKERS] logical decoding of two-phase transactions

2021-08-03 Thread Peter Smith
On Tue, Aug 3, 2021 at 5:02 PM Amit Kapila  wrote:
>
> On Tue, Aug 3, 2021 at 6:17 AM Peter Smith  wrote:
> >
> > Please find attached the latest patch set v102*
> >
>
> I have made minor modifications in the comments and docs, please see
> attached. Can you please check whether the names of contributors in
> the commit message are correct or do we need to change it?
>

I checked the differences between v102 and v103 and have no review
comments about the latest changes.

The commit message looks ok.

I applied the v103 to the current HEAD; no errors.
The build is ok.
The make check is ok.
The TAP subscription tests are ok.

I also rebuilt the PG docs and verified rendering of the updated pages looks ok.

The patch v103 LGTM.

--
Kind Regards,
Peter Smith.
Fujitsu Australia




Re: [HACKERS] logical decoding of two-phase transactions

2021-08-03 Thread Amit Kapila
On Tue, Aug 3, 2021 at 6:17 AM Peter Smith  wrote:
>
> Please find attached the latest patch set v102*
>

I have made minor modifications in the comments and docs, please see
attached. Can you please check whether the names of contributors in
the commit message are correct or do we need to change it?

-- 
With Regards,
Amit Kapila.


v103-0001-Add-prepare-API-support-for-streaming-transacti.patch
Description: Binary data


Re: [HACKERS] logical decoding of two-phase transactions

2021-08-02 Thread Peter Smith
On Mon, May 10, 2021 at 1:31 PM vignesh C  wrote:
>
...
>
> 2) I felt we can change lsn data type from Int64 to XLogRecPtr
> +
> +Int64
> +
> +The LSN of the prepare.
> +
> +
> +
> +
> +Int64
> +
> +The end LSN of the transaction.
> +
> +
>
> 3) I felt we can change lsn data type from Int32 to TransactionId
> +
> +Int32
> +
> +Xid of the subtransaction (will be same as xid of the
> transaction for top-level
> +transactions).
> +
> +
>
...
>
> Similar problems related to comments 2 and 3 are being discussed at
> [1], we can change it accordingly based on the conclusion in the other
> thread.
> [1] - 
> https://www.postgresql.org/message-id/flat/CAHut%2BPs2JsSd_OpBR9kXt1Rt4bwyXAjh875gUpFw6T210ttO7Q%40mail.gmail.com#cf2a85d0623dcadfbb1204a196681313
>

Earlier today the other documentation patch mentioned above was
committed by Tom Lane.

The 2PC patch v102 now fixes your review comments 2 and 3 by matching
the same datatype annotation style of that commit.

--
Kind Regards,
Peter Smith
Fujitsu Australia




Re: [HACKERS] logical decoding of two-phase transactions

2021-08-02 Thread Peter Smith
Please find attached the latest patch set v102*

Differences:

* Rebased to HEAD @ today.

* This is a documentation change only. A recent commit [1] has changed
the documentation style for the message formats slightly to annotate
the data types. For consistency, the same style change needs to be
adopted for the newly added message of this patch.  This same change
also finally addresses some old review comments [2] from Vignesh.


[1] 
https://github.com/postgres/postgres/commit/a5cb4f9829fbfd68655543d2d371a18a8eb43b84
[2] 
https://www.postgresql.org/message-id/CALDaNm3U4fGxTnQfaT1TqUkgX5c0CSDvmW12Bfksis8zB_XinA%40mail.gmail.com

Kind Regards,
Peter Smith.
Fujitsu Australia


v102-0001-Add-prepare-API-support-for-streaming-transacti.patch
Description: Binary data


Re: [HACKERS] logical decoding of two-phase transactions

2021-08-02 Thread Amit Kapila
On Sun, Aug 1, 2021 at 3:51 PM Peter Smith  wrote:
>
> On Sun, Aug 1, 2021 at 3:05 AM vignesh C  wrote:
> >
> > On Sat, Jul 31, 2021 at 11:12 AM Ajin Cherian  wrote:
> > >
> > > On Sat, Jul 31, 2021 at 2:39 PM Amit Kapila  
> > > wrote:
> > >
> > > > Here, the test is expecting 2 prepared transactions corresponding to
> > > > two subscriptions but it waits for just one subscription via
> > > > appname_copy. It should wait for the second subscription using
> > > > $appname as well.
> > > >
> > > > What do you think?
> > >
> > > I agree with this analysis. The test needs to wait for both
> > > subscriptions to catch up.
> > > Attached is a patch that addresses this issue.
> >
> > The changes look good to me.
> >
>
> The patch to the test code posted by Ajin LGTM also.
>

Pushed.

-- 
With Regards,
Amit Kapila.




Re: [HACKERS] logical decoding of two-phase transactions

2021-08-02 Thread Peter Smith
On Fri, Jul 30, 2021 at 4:33 PM Greg Nancarrow  wrote:
>
> On Fri, Jul 30, 2021 at 2:02 PM Peter Smith  wrote:
> >
> > Please find attached the latest patch set v100*
> >
> > v99-0002 --> v100-0001
> >
>
> A few minor comments:
>
> (1) doc/src/sgml/protocol.sgml
>
> In the following description, is the word "large" really needed? Also
> "the message ... for a ... message" sounds a bit odd, as does
> "two-phase prepare".
>
> What about the following:
>
> BEFORE:
> +Identifies the message as a two-phase prepare for a
> large in-progress transaction message.
> AFTER:
> +Identifies the message as a prepare for an
> in-progress two-phase transaction.
>

Updated in v101.

The other nearby messages are referring refer to a “streamed
transaction” so I’ve changed this to say “Identifies the message as a
two-phase prepare for a streamed transaction message.” (e.g. compare
this text with the existing similar text for ‘P’).

BTW, I agree with you that "the message ... for a ... message" seems
odd; it was written in this way only to be consistent with existing
documentation, which all uses the same odd phrasing.

> (2) src/backend/replication/logical/worker.c
>
> Similar format comment, but one uses a full-stop and the other
> doesn't, looks a bit odd, since the lines are near each other.
>
> * 1. Replay all the spooled operations - Similar code as for
>
> * 2. Mark the transaction as prepared. - Similar code as for
>

Updated in v101 to make the comments consistent.

> (3) src/test/subscription/t/023_twophase_stream.pl
>
> Shouldn't the following comment mention, for example, "with streaming"
> or something to that effect?
>
> # logical replication of 2PC test
>

Fixed as suggested in v101.

--
Kind Regards,
Peter Smith.
Fujitsu Australia.




Re: [HACKERS] logical decoding of two-phase transactions

2021-08-02 Thread Peter Smith
On Fri, Jul 30, 2021 at 3:18 PM vignesh C  wrote:
>
> On Fri, Jul 30, 2021 at 9:32 AM Peter Smith  wrote:
> >
> > Please find attached the latest patch set v100*
> >
> > v99-0002 --> v100-0001
> >
> > Differences:
> >
> > * Rebased to HEAD @ today (needed because some recent commits [1][2] broke 
> > v99)
> >
>
> The patch applies neatly, tests passes and documentation looks good.
> A Few minor comments.
> 1) This blank line is not required:
> +-- two_phase and streaming are compatible.
> +CREATE SUBSCRIPTION regress_testsub CONNECTION
> 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect =
> false, streaming = true, two_phase = true);
> +
>

Fixed in v101.

> 2) Few points have punctuation mark and few don't have, we can make it
> consistent:
> +###
> +# Test 2PC PREPARE / ROLLBACK PREPARED.
> +# 1. Table is deleted back to 2 rows which are replicated on subscriber.
> +# 2. Data is streamed using 2PC
> +# 3. Do rollback prepared.
> +#
> +# Expect data rolls back leaving only the original 2 rows.
> +###
>

Fixed in v101.

> 3) similarly here too:
> +###
> +# Do INSERT after the PREPARE but before ROLLBACK PREPARED.
> +# 1. Table is deleted back to 2 rows which are replicated on subscriber.
> +# 2. Data is streamed using 2PC.
> +# 3. A single row INSERT is done which is after the PREPARE
> +# 4. Then do a ROLLBACK PREPARED.
> +#
> +# Expect the 2PC data rolls back leaving only 3 rows on the subscriber.
> +# (the original 2 + inserted 1)
> +###
>

Fixed in v101.

--
Kind Regards,
Peter Smith.
Fujitsu Australia




Re: [HACKERS] logical decoding of two-phase transactions

2021-08-02 Thread Peter Smith
On Fri, Jul 30, 2021 at 6:25 PM tanghy.f...@fujitsu.com
 wrote:
>
> On Friday, July 30, 2021 12:02 PM Peter Smith wrote:
> >
> > Please find attached the latest patch set v100*
> >
> > v99-0002 --> v100-0001
> >
>
> Thanks for your patch. A few comments on the test file:
>
> 1. src/test/subscription/t/022_twophase_cascade.pl
>
> 1.1
> I saw your test cases for "PREPARE / COMMIT PREPARED" and "PREPARE with a 
> nested ROLLBACK TO SAVEPOINT", but didn't see cases for "PREPARE / ROLLBACK 
> PREPARED". Is it needless or just missing?
>

Yes, that test used to exist but it was removed in response to a
previous review (see [1] comment #10,  Amit said there were too many
tests).

> 1.2
> +# check inserts are visible at subscriber(s).
> +# All the streamed data (prior to the SAVEPOINT) should be rolled back.
> +# (3, 'foobar') should be committed.
>
> I think it should be (, 'foobar') here.
>

Good catch. Fixed in v101.

> 1.3
> +$result = $node_B->safe_psql('postgres', "SELECT count(*) FROM test_tab 
> where b = 'foobar';");
> +is($result, qq(1), 'Rows committed are present on subscriber B');
> +$result = $node_B->safe_psql('postgres', "SELECT count(*) FROM test_tab;");
> +
>
> It seems the test is not finished yet. We didn't check the value of 'result'. 
> Besides, maybe we should also check node_C, right?
>

Oops. Thanks for finding this! Fixed in v101 by adding the missing tests.

> 1.4
> +$node_B->append_conf('postgresql.conf',qq(max_prepared_transactions 
> = 10));
> +$node_B->append_conf('postgresql.conf', qq(logical_decoding_work_mem = 
> 64kB));
>
> You see, the first line uses a TAB but the second line uses a space.
> Also, we could use only one statement to append these two settings to run 
> tests a bit faster. Thoughts?
> Something like:
>
> $node_B->append_conf(
> 'postgresql.conf', qq(
> max_prepared_transactions = 10
> logical_decoding_work_mem = 64kB
> ));
>

OK. In v101 I changed the config as you suggested for both the 022 and
023 TAP tests.

--
[1] 
https://www.postgresql.org/message-id/CAHut%2BPts_bWx_RrXu%2BYwbiJva33nTROoQQP5H4pVrF%2BNcCMkRA%40mail.gmail.com

Kind Regards,
Peter Smith.
Fujitsu Australia.




Re: [HACKERS] logical decoding of two-phase transactions

2021-08-02 Thread Peter Smith
On Sat, Jul 31, 2021 at 9:36 PM Amit Kapila  wrote:
>
> On Fri, Jul 30, 2021 at 9:32 AM Peter Smith  wrote:
> >
> > Please find attached the latest patch set v100*
> >
>
> Few minor comments:
> 1.
> CREATE SUBSCRIPTION regress_testsub CONNECTION
> 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect =
> false, two_phase = true);
>
>  \dRs+
> +
>  --fail - alter of two_phase option not supported.
>  ALTER SUBSCRIPTION regress_testsub SET (two_phase = false);
>
> Spurious line addition.
>

OK. Fixed in v101.

> 2.
> +TransactionId
> +logicalrep_read_stream_prepare(StringInfo in,
> LogicalRepPreparedTxnData *prepare_data)
> +{
> + logicalrep_read_prepare_common(in, "stream prepare", prepare_data);
> +
> + return prepare_data->xid;
> +}
>
> There is no need to return TransactionId separately. The caller can
> use from prepare_data, if required.
>

OK. Modified in v101

> 3.
>  extern void logicalrep_read_stream_abort(StringInfo in, TransactionId *xid,
>   TransactionId *subxid);
>
> +extern void logicalrep_write_stream_prepare(StringInfo out,
> ReorderBufferTXN *txn,
> + XLogRecPtr prepare_lsn);
> +extern TransactionId logicalrep_read_stream_prepare(StringInfo in,
> + LogicalRepPreparedTxnData *prepare_data);
> +
> +
>
> Keep the order of declarations the same as its definitions in proto.c
> which means move these after logicalrep_read_rollback_prepared() and
> be careful about extra blank lines.
>

OK. Reordered in v101.

--
Kind Regards,
Peter Smith.
Fujitsu Australia




Re: [HACKERS] logical decoding of two-phase transactions

2021-08-02 Thread Peter Smith
Please find attached the latest patch v101

Differences:

* Rebased to HEAD @ today.

* Addresses all v100 review comments from Vignesh [1], Greg [2], Tang
[3], and Amit [2].


[1] 
https://www.postgresql.org/message-id/CALDaNm2N3qgSv3XyHW%2Bop_SJcLmz1s%3D0jJc-taxUmeEBXW5EPw%40mail.gmail.com
[2] 
https://www.postgresql.org/message-id/CAJcOf-eGCg8s%2BtT_Mo5xKksAhA%3D%3D1QAH_Sj7SqBotHQhwapdEw%40mail.gmail.com
[3] 
https://www.postgresql.org/message-id/OS0PR01MB6113B6F3C88C3C11A2F62CFFFBEC9%40OS0PR01MB6113.jpnprd01.prod.outlook.com
[4] 
https://www.postgresql.org/message-id/CAA4eK1%2BVcNDUYSZVm3xNg4YLzaMqcZHqxznfbAYvJWoVzvLqFQ%40mail.gmail.com

Kind Regards,
Peter Smith.
Fujitsu Australia


v101-0001-Add-prepare-API-support-for-streaming-transacti.patch
Description: Binary data


Re: [HACKERS] logical decoding of two-phase transactions

2021-08-01 Thread Peter Smith
On Sun, Aug 1, 2021 at 3:05 AM vignesh C  wrote:
>
> On Sat, Jul 31, 2021 at 11:12 AM Ajin Cherian  wrote:
> >
> > On Sat, Jul 31, 2021 at 2:39 PM Amit Kapila  wrote:
> >
> > > Here, the test is expecting 2 prepared transactions corresponding to
> > > two subscriptions but it waits for just one subscription via
> > > appname_copy. It should wait for the second subscription using
> > > $appname as well.
> > >
> > > What do you think?
> >
> > I agree with this analysis. The test needs to wait for both
> > subscriptions to catch up.
> > Attached is a patch that addresses this issue.
>
> The changes look good to me.
>

The patch to the test code posted by Ajin LGTM also.

I applied the patch and re-ran the TAP subscription tests. All OK.

--
Kind Regards,
Peter Smith.
Fujitsu Australia




Re: [HACKERS] logical decoding of two-phase transactions

2021-07-31 Thread vignesh C
On Sat, Jul 31, 2021 at 11:12 AM Ajin Cherian  wrote:
>
> On Sat, Jul 31, 2021 at 2:39 PM Amit Kapila  wrote:
>
> > Here, the test is expecting 2 prepared transactions corresponding to
> > two subscriptions but it waits for just one subscription via
> > appname_copy. It should wait for the second subscription using
> > $appname as well.
> >
> > What do you think?
>
> I agree with this analysis. The test needs to wait for both
> subscriptions to catch up.
> Attached is a patch that addresses this issue.

The changes look good to me.

Regards,
Vignesh




Re: [HACKERS] logical decoding of two-phase transactions

2021-07-31 Thread Amit Kapila
On Fri, Jul 30, 2021 at 9:32 AM Peter Smith  wrote:
>
> Please find attached the latest patch set v100*
>

Few minor comments:
1.
CREATE SUBSCRIPTION regress_testsub CONNECTION
'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect =
false, two_phase = true);

 \dRs+
+
 --fail - alter of two_phase option not supported.
 ALTER SUBSCRIPTION regress_testsub SET (two_phase = false);

Spurious line addition.

2.
+TransactionId
+logicalrep_read_stream_prepare(StringInfo in,
LogicalRepPreparedTxnData *prepare_data)
+{
+ logicalrep_read_prepare_common(in, "stream prepare", prepare_data);
+
+ return prepare_data->xid;
+}

There is no need to return TransactionId separately. The caller can
use from prepare_data, if required.

3.
 extern void logicalrep_read_stream_abort(StringInfo in, TransactionId *xid,
  TransactionId *subxid);

+extern void logicalrep_write_stream_prepare(StringInfo out,
ReorderBufferTXN *txn,
+ XLogRecPtr prepare_lsn);
+extern TransactionId logicalrep_read_stream_prepare(StringInfo in,
+ LogicalRepPreparedTxnData *prepare_data);
+
+

Keep the order of declarations the same as its definitions in proto.c
which means move these after logicalrep_read_rollback_prepared() and
be careful about extra blank lines.

-- 
With Regards,
Amit Kapila.




Re: [HACKERS] logical decoding of two-phase transactions

2021-07-31 Thread Amit Kapila
On Sat, Jul 31, 2021 at 11:12 AM Ajin Cherian  wrote:
>
> On Sat, Jul 31, 2021 at 2:39 PM Amit Kapila  wrote:
>
> > Here, the test is expecting 2 prepared transactions corresponding to
> > two subscriptions but it waits for just one subscription via
> > appname_copy. It should wait for the second subscription using
> > $appname as well.
> >
> > What do you think?
>
> I agree with this analysis. The test needs to wait for both
> subscriptions to catch up.
> Attached is a patch that addresses this issue.
>

LGTM, unless Peter Smith has any comments or thinks otherwise, I'll
push this on Monday.

-- 
With Regards,
Amit Kapila.




Re: [HACKERS] logical decoding of two-phase transactions

2021-07-30 Thread Ajin Cherian
On Sat, Jul 31, 2021 at 2:39 PM Amit Kapila  wrote:

> Here, the test is expecting 2 prepared transactions corresponding to
> two subscriptions but it waits for just one subscription via
> appname_copy. It should wait for the second subscription using
> $appname as well.
>
> What do you think?

I agree with this analysis. The test needs to wait for both
subscriptions to catch up.
Attached is a patch that addresses this issue.

regards,
Ajin Cherian
Fujitsu Australia


v1-0001-Fix-possible-failure-in-021_twophase-tap-test.patch
Description: Binary data


Re: [HACKERS] logical decoding of two-phase transactions

2021-07-30 Thread Amit Kapila
On Wed, Jul 14, 2021 at 11:52 AM Amit Kapila  wrote:
>
> On Mon, Jul 12, 2021 at 9:14 AM Peter Smith  wrote:
>
> Pushed.
>

As reported by Michael [1], there is one test failure related to this
commit. The failure is as below:

#   Failed test 'transaction is prepared on subscriber'
#   at t/021_twophase.pl line 324.
#  got: '1'
# expected: '2'
# Looks like you failed 1 test of 24.
[12:14:02] t/021_twophase.pl ..
Dubious, test returned 1 (wstat 256, 0x100)
Failed 1/24 subtests
[12:14:12] t/022_twophase_cascade.pl .. ok10542 ms ( 0.00
usr  0.00 sys +  2.03 cusr  0.61 csys =  2.64 CPU)
[12:14:31] t/100_bugs.pl .. ok18550 ms ( 0.00
usr  0.00 sys +  3.85 cusr  1.36 csys =  5.21 CPU)
[12:14:31]

I think I know what's going wrong here. The corresponding test is:

# Now do a prepare on publisher and check that it IS replicated
$node_publisher->safe_psql('postgres', "
BEGIN;
INSERT INTO tab_copy VALUES (99);
PREPARE TRANSACTION 'mygid';");

$node_publisher->wait_for_catchup($appname_copy);

# Check that the transaction has been prepared on the subscriber,
there will be 2
# prepared transactions for the 2 subscriptions.
$result = $node_subscriber->safe_psql('postgres', "SELECT count(*)
FROM pg_prepared_xacts;");
is($result, qq(2), 'transaction is prepared on subscriber');

Here, the test is expecting 2 prepared transactions corresponding to
two subscriptions but it waits for just one subscription via
appname_copy. It should wait for the second subscription using
$appname as well.

What do you think?

[1] - https://www.postgresql.org/message-id/YQP02%2B5yLCIgmdJY%40paquier.xyz

-- 
With Regards,
Amit Kapila.




RE: [HACKERS] logical decoding of two-phase transactions

2021-07-30 Thread tanghy.f...@fujitsu.com
On Friday, July 30, 2021 12:02 PM Peter Smith wrote:
> 
> Please find attached the latest patch set v100*
> 
> v99-0002 --> v100-0001
> 

Thanks for your patch. A few comments on the test file:

1. src/test/subscription/t/022_twophase_cascade.pl

1.1
I saw your test cases for "PREPARE / COMMIT PREPARED" and "PREPARE with a 
nested ROLLBACK TO SAVEPOINT", but didn't see cases for "PREPARE / ROLLBACK 
PREPARED". Is it needless or just missing?

1.2
+# check inserts are visible at subscriber(s).
+# All the streamed data (prior to the SAVEPOINT) should be rolled back.
+# (3, 'foobar') should be committed.

I think it should be (, 'foobar') here.

1.3
+$result = $node_B->safe_psql('postgres', "SELECT count(*) FROM test_tab where 
b = 'foobar';");
+is($result, qq(1), 'Rows committed are present on subscriber B');
+$result = $node_B->safe_psql('postgres', "SELECT count(*) FROM test_tab;");
+

It seems the test is not finished yet. We didn't check the value of 'result'. 
Besides, maybe we should also check node_C, right?

1.4
+$node_B->append_conf('postgresql.conf',qq(max_prepared_transactions = 
10));
+$node_B->append_conf('postgresql.conf', qq(logical_decoding_work_mem = 64kB));

You see, the first line uses a TAB but the second line uses a space.
Also, we could use only one statement to append these two settings to run tests 
a bit faster. Thoughts?
Something like:

$node_B->append_conf(
'postgresql.conf', qq(
max_prepared_transactions = 10
logical_decoding_work_mem = 64kB
));

Regards
Tang


Re: [HACKERS] logical decoding of two-phase transactions

2021-07-30 Thread Greg Nancarrow
On Fri, Jul 30, 2021 at 2:02 PM Peter Smith  wrote:
>
> Please find attached the latest patch set v100*
>
> v99-0002 --> v100-0001
>

A few minor comments:

(1) doc/src/sgml/protocol.sgml

In the following description, is the word "large" really needed? Also
"the message ... for a ... message" sounds a bit odd, as does
"two-phase prepare".

What about the following:

BEFORE:
+Identifies the message as a two-phase prepare for a
large in-progress transaction message.
AFTER:
+Identifies the message as a prepare for an
in-progress two-phase transaction.

(2) src/backend/replication/logical/worker.c

Similar format comment, but one uses a full-stop and the other
doesn't, looks a bit odd, since the lines are near each other.

* 1. Replay all the spooled operations - Similar code as for

* 2. Mark the transaction as prepared. - Similar code as for

(3) src/test/subscription/t/023_twophase_stream.pl

Shouldn't the following comment mention, for example, "with streaming"
or something to that effect?

# logical replication of 2PC test


Regards,
Greg Nancarrow
Fujitsu Australia




Re: [HACKERS] logical decoding of two-phase transactions

2021-07-29 Thread vignesh C
On Fri, Jul 30, 2021 at 9:32 AM Peter Smith  wrote:
>
> Please find attached the latest patch set v100*
>
> v99-0002 --> v100-0001
>
> Differences:
>
> * Rebased to HEAD @ today (needed because some recent commits [1][2] broke 
> v99)
>

The patch applies neatly, tests passes and documentation looks good.
A Few minor comments.
1) This blank line is not required:
+-- two_phase and streaming are compatible.
+CREATE SUBSCRIPTION regress_testsub CONNECTION
'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect =
false, streaming = true, two_phase = true);
+

2) Few points have punctuation mark and few don't have, we can make it
consistent:
+###
+# Test 2PC PREPARE / ROLLBACK PREPARED.
+# 1. Table is deleted back to 2 rows which are replicated on subscriber.
+# 2. Data is streamed using 2PC
+# 3. Do rollback prepared.
+#
+# Expect data rolls back leaving only the original 2 rows.
+###

3) similarly here too:
+###
+# Do INSERT after the PREPARE but before ROLLBACK PREPARED.
+# 1. Table is deleted back to 2 rows which are replicated on subscriber.
+# 2. Data is streamed using 2PC.
+# 3. A single row INSERT is done which is after the PREPARE
+# 4. Then do a ROLLBACK PREPARED.
+#
+# Expect the 2PC data rolls back leaving only 3 rows on the subscriber.
+# (the original 2 + inserted 1)
+###

Regards,
Vignesh




Re: [HACKERS] logical decoding of two-phase transactions

2021-07-29 Thread Peter Smith
On Thu, Jul 29, 2021 at 9:56 PM Amit Kapila  wrote:
>
> On Tue, Jul 27, 2021 at 11:41 AM Peter Smith  wrote:
> >
> > Please find attached the latest patch set v99*
> >
> > v98-0001 --> split into v99-0001 + v99-0002
> >
>
> Pushed the first refactoring patch after making few modifications as below.
> 1.
> - /* open the spool file for the committed transaction */
> + /* Open the spool file for the committed/prepared transaction */
>   changes_filename(path, MyLogicalRepWorker->subid, xid);
>
> In the above comment, we don't need to say prepared. It can be done as
> part of the second patch.
>
Updated comment in v100.

> 2.
> +apply_handle_prepare_internal(LogicalRepPreparedTxnData
> *prepare_data, char *gid)
>
> I don't think there is any need for this function to take gid as
> input. It can compute by itself instead of callers doing it.
>
OK.

> 3.
> +static TransactionId+logicalrep_read_prepare_common(StringInfo in,
> char *msgtype,
> +   LogicalRepPreparedTxnData *prepare_data)
>
> I don't think the above function needs to return xid because it is
> already present as part of prepare_data. Even, if it is required due
> to some reason for the second patch then let's do it as part of if but
> I don't think it is required for the second patch.
>
OK.

> 4.
>  /*
> - * Write PREPARE to the output stream.
> + * Common code for logicalrep_write_prepare and
> logicalrep_write_stream_prepare.
>   */
>
> Here and at a similar another place, we don't need to refer to
> logicalrep_write_stream_prepare as that is part of the second patch.
>
Updated comment in v100

> Few comments on 0002 patch:
> ==
> 1.
> +# -
> +# 2PC + STREAMING TESTS
> +# -
> +
> +# Setup logical replication (streaming = on)
> +
> +$node_B->safe_psql('postgres', "
> + ALTER SUBSCRIPTION tap_sub_B
> + SET (streaming = on);");
> +
> +$node_C->safe_psql('postgres', "
> + ALTER SUBSCRIPTION tap_sub_C
> + SET (streaming = on)");
> +
> +# Wait for subscribers to finish initialization
> +$node_A->wait_for_catchup($appname_B);
> +$node_B->wait_for_catchup($appname_C);
>
> This is not the right way to determine if the new streaming option is
> enabled on the publisher. Even if there is no restart of apply workers
> (and walsender) after you have enabled the option, the above wait will
> succeed. You need to do something like below as we are doing in
> 001_rep_changes.pl:
>
> $oldpid = $node_publisher->safe_psql('postgres',
> "SELECT pid FROM pg_stat_replication WHERE application_name = 'tap_sub';"
> );
> $node_subscriber->safe_psql('postgres',
> "ALTER SUBSCRIPTION tap_sub SET PUBLICATION tap_pub_ins_only WITH
> (copy_data = false)"
> );
> $node_publisher->poll_query_until('postgres',
> "SELECT pid != $oldpid FROM pg_stat_replication WHERE application_name
> = 'tap_sub';"
> ) or die "Timed out while waiting for apply to restart";
>
Fixed in v100 as suggested.

> 2.
> +# Create some pre-existing content on publisher (uses same DDL as
> 015_stream test)
>
> Here, in the comments, I don't see the need to same uses same DDL ...
>
Fixed in v100. Comment removed.

--
Kind Regards,
Peter Smith.
Fujitsu Australia




Re: [HACKERS] logical decoding of two-phase transactions

2021-07-29 Thread Peter Smith
Please find attached the latest patch set v100*

v99-0002 --> v100-0001

Differences:

* Rebased to HEAD @ today (needed because some recent commits [1][2] broke v99)

* Addresses v99 review comments from Amit [1].


[1] 
https://github.com/postgres/postgres/commit/201a76183e2056c2217129e12d68c25ec9c559c8
[2] 
https://github.com/postgres/postgres/commit/91f9861242cd7dcf28fae216b1d8b47551c9159d
[1] 
https://www.postgresql.org/message-id/CAA4eK1%2BNzcz%3DhzZJO16ZcqA7hZRa4RzGRwL_XXM%2Bdin8ehROaQ%40mail.gmail.com

Kind Regards,
Peter Smith.
Fujitsu Australia


v100-0001-Add-prepare-API-support-for-streaming-transacti.patch
Description: Binary data


Re: [HACKERS] logical decoding of two-phase transactions

2021-07-29 Thread Amit Kapila
On Tue, Jul 27, 2021 at 11:41 AM Peter Smith  wrote:
>
> Please find attached the latest patch set v99*
>
> v98-0001 --> split into v99-0001 + v99-0002
>

Pushed the first refactoring patch after making few modifications as below.
1.
- /* open the spool file for the committed transaction */
+ /* Open the spool file for the committed/prepared transaction */
  changes_filename(path, MyLogicalRepWorker->subid, xid);

In the above comment, we don't need to say prepared. It can be done as
part of the second patch.

2.
+apply_handle_prepare_internal(LogicalRepPreparedTxnData
*prepare_data, char *gid)

I don't think there is any need for this function to take gid as
input. It can compute by itself instead of callers doing it.

3.
+static TransactionId+logicalrep_read_prepare_common(StringInfo in,
char *msgtype,
+   LogicalRepPreparedTxnData *prepare_data)

I don't think the above function needs to return xid because it is
already present as part of prepare_data. Even, if it is required due
to some reason for the second patch then let's do it as part of if but
I don't think it is required for the second patch.

4.
 /*
- * Write PREPARE to the output stream.
+ * Common code for logicalrep_write_prepare and
logicalrep_write_stream_prepare.
  */

Here and at a similar another place, we don't need to refer to
logicalrep_write_stream_prepare as that is part of the second patch.

Few comments on 0002 patch:
==
1.
+# -
+# 2PC + STREAMING TESTS
+# -
+
+# Setup logical replication (streaming = on)
+
+$node_B->safe_psql('postgres', "
+ ALTER SUBSCRIPTION tap_sub_B
+ SET (streaming = on);");
+
+$node_C->safe_psql('postgres', "
+ ALTER SUBSCRIPTION tap_sub_C
+ SET (streaming = on)");
+
+# Wait for subscribers to finish initialization
+$node_A->wait_for_catchup($appname_B);
+$node_B->wait_for_catchup($appname_C);

This is not the right way to determine if the new streaming option is
enabled on the publisher. Even if there is no restart of apply workers
(and walsender) after you have enabled the option, the above wait will
succeed. You need to do something like below as we are doing in
001_rep_changes.pl:

$oldpid = $node_publisher->safe_psql('postgres',
"SELECT pid FROM pg_stat_replication WHERE application_name = 'tap_sub';"
);
$node_subscriber->safe_psql('postgres',
"ALTER SUBSCRIPTION tap_sub SET PUBLICATION tap_pub_ins_only WITH
(copy_data = false)"
);
$node_publisher->poll_query_until('postgres',
"SELECT pid != $oldpid FROM pg_stat_replication WHERE application_name
= 'tap_sub';"
) or die "Timed out while waiting for apply to restart";

2.
+# Create some pre-existing content on publisher (uses same DDL as
015_stream test)

Here, in the comments, I don't see the need to same uses same DDL ...

-- 
With Regards,
Amit Kapila.




Re: [HACKERS] logical decoding of two-phase transactions

2021-07-27 Thread Peter Smith
Please find attached the latest patch set v99*

v98-0001 --> split into v99-0001 + v99-0002

Differences:

* Rebased to HEAD @ yesterday.

* Addresses review comments from Amit [1] and split the v98 patch as requested.


[1] 
https://www.postgresql.org/message-id/CAA4eK1%2BizpAybqpEFp8%2BRx%3DC1Z1H_XLcRod_WYjBRv2Rn%2BDO2w%40mail.gmail.com

Kind Regards,
Peter Smith.
Fujitsu Australia


v99-0001-Refactor-to-make-common-functions.patch
Description: Binary data


v99-0002-Add-prepare-API-support-for-streaming-transactio.patch
Description: Binary data


Re: [HACKERS] logical decoding of two-phase transactions

2021-07-27 Thread Peter Smith
On Fri, Jul 23, 2021 at 8:08 PM Amit Kapila  wrote:
>
> On Tue, Jul 20, 2021 at 9:24 AM Peter Smith  wrote:
> >
> > Please find attached the latest patch set v98*
> >
>
> Review comments:
> 

All the following review comments are addressed in v99* patch set.

> 1.
> /*
> - * Handle STREAM COMMIT message.
> + * Common spoolfile processing.
> + * Returns how many changes were applied.
>   */
> -static void
> -apply_handle_stream_commit(StringInfo s)
> +static int
> +apply_spooled_messages(TransactionId xid, XLogRecPtr lsn)
>
> Let's extract this common functionality (common to current code and
> the patch) as a separate patch? I think we can commit this as a
> separate patch.
>

Done. Split patches as requested.

> 2.
> apply_spooled_messages()
> {
> ..
>   elog(DEBUG1, "replayed %d (all) changes from file \"%s\"",
>   nchanges, path);
> ..
> }
>
> You have this DEBUG1 message in apply_spooled_messages and its
> callers. You can remove it from callers as the patch already has
> another debug message to indicate whether it is stream prepare or
> stream commit. Also, if this is the only reason to return nchanges
> from apply_spooled_messages() then we can get rid of that as well.
>

Done.

> 3.
> + /*
> + * 2. Mark the transaction as prepared. - Similar code as for
> + * apply_handle_prepare (i.e. two-phase non-streamed prepare)
> + */
> +
> + /*
> + * BeginTransactionBlock is necessary to balance the EndTransactionBlock
> + * called within the PrepareTransactionBlock below.
> + */
> + BeginTransactionBlock();
> + CommitTransactionCommand(); /* Completes the preceding Begin command. */
> +
> + /*
> + * Update origin state so we can restart streaming from correct position
> + * in case of crash.
> + */
> + replorigin_session_origin_lsn = prepare_data.end_lsn;
> + replorigin_session_origin_timestamp = prepare_data.prepare_time;
> +
> + PrepareTransactionBlock(gid);
>
> I think you can move this part into a common function
> apply_handle_prepare_internal. If that is possible then you might want
> to move this part into a common functionality patch as mentioned in
> point-1.
>

Done. (The common function is included in patch 0001)

> 4.
> + xid = logicalrep_read_stream_prepare(s, _data);
> + elog(DEBUG1, "received prepare for streamed transaction %u", xid);
>
> It is better to have an empty line between the above code lines for
> the sake of clarity.
>

Done.

> 5.
> +/* Commit (and abort) information */
>  typedef struct LogicalRepCommitData
>
> How is this structure related to abort? Even if it is, why this
> comment belongs to this patch?
>

OK. Removed this from the patch.

> 6. Most of the code in logicalrep_write_stream_prepare() and
> logicalrep_write_prepare() is same except for message. I think if we
> want we can handle both of them with a single message by setting some
> flag for stream case but probably there will be some additional
> checking required on the worker-side. What do you think? I think if we
> want to keep them separate then at least we should keep the common
> functionality in logicalrep_write_*/logicalrep_read_* in separate
> functions. This way we will avoid minor inconsistencies in-stream and
> non-stream functions.
>

Done. (The common functions are included in patch 0001).

> 7.
> +++ b/doc/src/sgml/protocol.sgml
> @@ -2881,7 +2881,7 @@ The commands accepted in replication mode are:
> Begin Prepare and Prepare messages belong to the same transaction.
> It also sends changes of large in-progress transactions between a pair of
> Stream Start and Stream Stop messages. The last stream of such a 
> transaction
> -   contains a Stream Commit or Stream Abort message.
> +   contains a Stream Prepare, Stream Commit or Stream Abort message.
>
> I am not sure if it is correct to mention Stream Prepare here because
> after that we will send commit prepared as well for such a
> transaction. So, I think we should remove this change.
>

Done.

> 8.
> -ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
> -
>  \dRs+
>
> +ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
>
> Is there a reason for this change in the tests?
>

Yes, the setting of slot_name = NONE really belongs with the DROP
SUBSCRIPTION. Similarly, the \dRs+ is done to test the effect of the
setting of the streaming option (not the slot_name = NONE). Since I
needed to add a new DROP SUBSCRIPTION (because now the streaming
option works) so I also refactored this exiting test to make all the
test formats consistent.

> 9.
> I think this contains a lot of streaming tests in 023_twophase_stream.
> Let's keep just one test for crash-restart scenario (+# Check that 2PC
> COMMIT PREPARED is decoded properly on crash restart.) where both
> publisher and subscriber get restarted. I think others are covered in
> one or another way by other existing tests. Apart from that, I also
> don't see the need for the below tests:
> # Do DELETE after PREPARE but before COMMIT PREPARED.
> This is mostly the same 

Re: [HACKERS] logical decoding of two-phase transactions

2021-07-26 Thread Peter Smith
On Fri, Jul 23, 2021 at 8:08 PM Amit Kapila  wrote:
>
> On Tue, Jul 20, 2021 at 9:24 AM Peter Smith  wrote:
> >
> > Please find attached the latest patch set v98*
> >
>
> Review comments:
> 
[...]

> With Regards,
> Amit Kapila.

Thanks for your review comments.

I having been working through them today and hope to post the v99*
patches tomorrow.

--
Kind Regards,
Peter Smith.
Fujitsu Australia




Re: [HACKERS] logical decoding of two-phase transactions

2021-07-23 Thread Amit Kapila
On Tue, Jul 20, 2021 at 9:24 AM Peter Smith  wrote:
>
> Please find attached the latest patch set v98*
>

Review comments:

1.
/*
- * Handle STREAM COMMIT message.
+ * Common spoolfile processing.
+ * Returns how many changes were applied.
  */
-static void
-apply_handle_stream_commit(StringInfo s)
+static int
+apply_spooled_messages(TransactionId xid, XLogRecPtr lsn)

Let's extract this common functionality (common to current code and
the patch) as a separate patch? I think we can commit this as a
separate patch.

2.
apply_spooled_messages()
{
..
  elog(DEBUG1, "replayed %d (all) changes from file \"%s\"",
  nchanges, path);
..
}

You have this DEBUG1 message in apply_spooled_messages and its
callers. You can remove it from callers as the patch already has
another debug message to indicate whether it is stream prepare or
stream commit. Also, if this is the only reason to return nchanges
from apply_spooled_messages() then we can get rid of that as well.

3.
+ /*
+ * 2. Mark the transaction as prepared. - Similar code as for
+ * apply_handle_prepare (i.e. two-phase non-streamed prepare)
+ */
+
+ /*
+ * BeginTransactionBlock is necessary to balance the EndTransactionBlock
+ * called within the PrepareTransactionBlock below.
+ */
+ BeginTransactionBlock();
+ CommitTransactionCommand(); /* Completes the preceding Begin command. */
+
+ /*
+ * Update origin state so we can restart streaming from correct position
+ * in case of crash.
+ */
+ replorigin_session_origin_lsn = prepare_data.end_lsn;
+ replorigin_session_origin_timestamp = prepare_data.prepare_time;
+
+ PrepareTransactionBlock(gid);

I think you can move this part into a common function
apply_handle_prepare_internal. If that is possible then you might want
to move this part into a common functionality patch as mentioned in
point-1.

4.
+ xid = logicalrep_read_stream_prepare(s, _data);
+ elog(DEBUG1, "received prepare for streamed transaction %u", xid);

It is better to have an empty line between the above code lines for
the sake of clarity.

5.
+/* Commit (and abort) information */
 typedef struct LogicalRepCommitData

How is this structure related to abort? Even if it is, why this
comment belongs to this patch?

6. Most of the code in logicalrep_write_stream_prepare() and
logicalrep_write_prepare() is same except for message. I think if we
want we can handle both of them with a single message by setting some
flag for stream case but probably there will be some additional
checking required on the worker-side. What do you think? I think if we
want to keep them separate then at least we should keep the common
functionality in logicalrep_write_*/logicalrep_read_* in separate
functions. This way we will avoid minor inconsistencies in-stream and
non-stream functions.

7.
+++ b/doc/src/sgml/protocol.sgml
@@ -2881,7 +2881,7 @@ The commands accepted in replication mode are:
Begin Prepare and Prepare messages belong to the same transaction.
It also sends changes of large in-progress transactions between a pair of
Stream Start and Stream Stop messages. The last stream of such a transaction
-   contains a Stream Commit or Stream Abort message.
+   contains a Stream Prepare, Stream Commit or Stream Abort message.

I am not sure if it is correct to mention Stream Prepare here because
after that we will send commit prepared as well for such a
transaction. So, I think we should remove this change.

8.
-ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
-
 \dRs+

+ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);

Is there a reason for this change in the tests?

9.
I think this contains a lot of streaming tests in 023_twophase_stream.
Let's keep just one test for crash-restart scenario (+# Check that 2PC
COMMIT PREPARED is decoded properly on crash restart.) where both
publisher and subscriber get restarted. I think others are covered in
one or another way by other existing tests. Apart from that, I also
don't see the need for the below tests:
# Do DELETE after PREPARE but before COMMIT PREPARED.
This is mostly the same as the previous test where the patch is testing Insert
# Try 2PC transaction works using an empty GID literal
This is covered in 021_twophase.

10.
+++ b/src/test/subscription/t/024_twophase_cascade_stream.pl
@@ -0,0 +1,271 @@
+
+# Copyright (c) 2021, PostgreSQL Global Development Group
+
+# Test cascading logical replication of 2PC.

In the above comment, you might want to say something about streaming.
In general, I am not sure if it is really adding value to have these
many streaming tests for cascaded setup and doing the whole setup
again after we have done in tests 022_twophase_cascade. I think it is
sufficient to do just one or two streaming tests by enhancing
022_twophase_cascade, you can alter subscription to enable streaming
after doing non-streaming tests.

11. Have you verified that all these tests went through the streaming
code path? If not, you can once enable DEBUG message in

Re: [HACKERS] logical decoding of two-phase transactions

2021-07-19 Thread Peter Smith
On Fri, Jul 16, 2021 at 4:08 PM vignesh C  wrote:
>
[...]
> Thanks for the updated patch, the patch applies cleanly and test passes:
> I had couple of comments:
> 1) Should we include "stream_prepare_cb" here  in
> logicaldecoding-streaming section of logicaldecoding.sgml
> documentation:
> To reduce the apply lag caused by large transactions, an output plugin
> may provide additional callback to support incremental streaming of
> in-progress transactions. There are multiple required streaming
> callbacks (stream_start_cb, stream_stop_cb, stream_abort_cb,
> stream_commit_cb and stream_change_cb) and two optional callbacks
> (stream_message_cb and stream_truncate_cb).
>

Modified in v98. The information about 'stream_prepare_cb' and friends
is given in detail in section 49.10 so I added a link to that page.


> 2) Should we add an example for stream_prepare_cb here  in
> logicaldecoding-streaming section of logicaldecoding.sgml
> documentation:
> One example sequence of streaming callback calls for one transaction
> may look like this:
>
> stream_start_cb(...);   <-- start of first block of changes
>   stream_change_cb(...);
>   stream_change_cb(...);
>   stream_message_cb(...);
>   stream_change_cb(...);
>   ...
>   stream_change_cb(...);
> stream_stop_cb(...);<-- end of first block of changes
>
> stream_start_cb(...);   <-- start of second block of changes
>   stream_change_cb(...);
>   stream_change_cb(...);
>   stream_change_cb(...);
>   ...
>   stream_message_cb(...);
>   stream_change_cb(...);
> stream_stop_cb(...);<-- end of second block of changes
>
> stream_commit_cb(...);  <-- commit of the streamed transaction
>

Modified in v98. I felt it would be too verbose to add another full
example since it would be 90% the same as the current example. So I
have combined the information.

--
Kind Regards,
Peter Smith.
Fujitsu Australia




Re: [HACKERS] logical decoding of two-phase transactions

2021-07-19 Thread Peter Smith
On Mon, Jul 19, 2021 at 3:28 PM Greg Nancarrow  wrote:
>
> On Wed, Jul 14, 2021 at 6:33 PM Peter Smith  wrote:
> >
> > Please find attached the latest patch set v97*
> >
>
> I couldn't spot spot any significant issues in the v97-0001 patch, but
> do have the following trivial feedback comments:
>
> (1) doc/src/sgml/protocol.sgml
> Suggestion:
>
> BEFORE:
> +   contains a Stream Prepare or Stream Commit or Stream Abort message.
> AFTER:
> +   contains a Stream Prepare, Stream Commit or Stream Abort message.
>
>
> (2) src/backend/replication/logical/worker.c
> It seems a bit weird to add a forward declaration here, without a
> comment, like for the one immediately above it
>
> /* Compute GID for two_phase transactions */
> static void TwoPhaseTransactionGid(Oid subid, TransactionId xid, char
> *gid, int szgid);
> -
> +static int apply_spooled_messages(TransactionId xid, XLogRecPtr lsn);
>
>
> (3) src/backend/replication/logical/worker.c
> Other DEBUG1 messages don't end with "."
>
> + elog(DEBUG1, "apply_handle_stream_prepare: replayed %d
> (all) changes.", nchanges);
>

Thanks for the feedback. All these are fixed as suggested in v98.

--
Kind Regards,
Peter Smith.
Fujitsu Australia




Re: [HACKERS] logical decoding of two-phase transactions

2021-07-19 Thread Peter Smith
Please find attached the latest patch set v98*

Patches:

v97-0001 --> v98-0001

Differences:

* Rebased to HEAD @ yesterday.

* Code/Docs changes:

1. Fixed the same strcpy problem as reported by Tom Lane [1] for the
previous 2PC patch.

2. Addressed all feedback suggestions given by Greg [2].

3. Added some more documentation as suggested by Vignesh [3].


[1] https://www.postgresql.org/message-id/161029.1626639923%40sss.pgh.pa.us
[2] 
https://www.postgresql.org/message-id/CAJcOf-ckGONzyAj0Y70ju_tfLWF819JYb%3Ddv9p5AnoZxm50j0g%40mail.gmail.com
[3] 
https://www.postgresql.org/message-id/CALDaNm0LVY5A98xrgaodynnj6c%3DWQ5%3DZMpauC44aRio7-jWBYQ%40mail.gmail.com

Kind Regards,
Peter Smith.
Fujitsu Australia


v98-0001-Add-prepare-API-support-for-streaming-transactio.patch
Description: Binary data


Re: [HACKERS] logical decoding of two-phase transactions

2021-07-19 Thread Amit Kapila
On Mon, Jul 19, 2021 at 1:00 PM Peter Smith  wrote:
>
> On Mon, Jul 19, 2021 at 4:41 PM Amit Kapila  wrote:
> >
> > >
> > > OK. I have implemented this reported [1] potential buffer overrun
> > > using the constraining strlcpy, because the GID limitation of 200
> > > bytes is already mentioned in the documentation [2].
> > >
> >
> > This will work but I think it is better to use sizeof gid buffer as we
> > are using in ParseCommitRecord() and ParseAbortRecord(). Tomorrow, if
> > due to some unforeseen reason if we change the size of gid buffer to
> > be different than the GIDSIZE then it will work seamlessly.
> >
>
> Modified as requested. PSA patch v2.
>

LGTM. I'll push this tomorrow unless Tom or someone else has any comments.

-- 
With Regards,
Amit Kapila.




Re: [HACKERS] logical decoding of two-phase transactions

2021-07-19 Thread Peter Smith
On Mon, Jul 19, 2021 at 4:41 PM Amit Kapila  wrote:
>
> On Mon, Jul 19, 2021 at 9:19 AM Peter Smith  wrote:
> >
> > On Mon, Jul 19, 2021 at 12:43 PM Amit Kapila  
> > wrote:
> > >
> > > On Mon, Jul 19, 2021 at 1:55 AM Tom Lane  wrote:
> > > >
> > > > Amit Kapila  writes:
> > > > > Pushed.
> > > >
> > > > I think you'd be way better off making the gid fields be "char *"
> > > > and pstrdup'ing the result of pq_getmsgstring.  Another possibility
> > > > perhaps is to use strlcpy, but I'd only go that way if it's important
> > > > to constrain the received strings to 200 bytes.
> > > >
> > >
> > > I think it is important to constrain length to 200 bytes for this case
> > > as here we receive a prepared transaction identifier which according
> > > to docs [1] has a max length of 200 bytes. Also, in
> > > ParseCommitRecord() and ParseAbortRecord(), we are using strlcpy with
> > > 200 as max length to copy prepare transaction identifier. So, I think
> > > it is better to use strlcpy here unless you or Peter feels otherwise.
> > >
> >
> > OK. I have implemented this reported [1] potential buffer overrun
> > using the constraining strlcpy, because the GID limitation of 200
> > bytes is already mentioned in the documentation [2].
> >
>
> This will work but I think it is better to use sizeof gid buffer as we
> are using in ParseCommitRecord() and ParseAbortRecord(). Tomorrow, if
> due to some unforeseen reason if we change the size of gid buffer to
> be different than the GIDSIZE then it will work seamlessly.
>

Modified as requested. PSA patch v2.

--
Kind Regards,
Peter Smith.
Fujitsu Australia


v2-0001-Fix-potential-buffer-overruns.patch
Description: Binary data


Re: [HACKERS] logical decoding of two-phase transactions

2021-07-19 Thread Amit Kapila
On Mon, Jul 19, 2021 at 9:19 AM Peter Smith  wrote:
>
> On Mon, Jul 19, 2021 at 12:43 PM Amit Kapila  wrote:
> >
> > On Mon, Jul 19, 2021 at 1:55 AM Tom Lane  wrote:
> > >
> > > Amit Kapila  writes:
> > > > Pushed.
> > >
> > > I think you'd be way better off making the gid fields be "char *"
> > > and pstrdup'ing the result of pq_getmsgstring.  Another possibility
> > > perhaps is to use strlcpy, but I'd only go that way if it's important
> > > to constrain the received strings to 200 bytes.
> > >
> >
> > I think it is important to constrain length to 200 bytes for this case
> > as here we receive a prepared transaction identifier which according
> > to docs [1] has a max length of 200 bytes. Also, in
> > ParseCommitRecord() and ParseAbortRecord(), we are using strlcpy with
> > 200 as max length to copy prepare transaction identifier. So, I think
> > it is better to use strlcpy here unless you or Peter feels otherwise.
> >
>
> OK. I have implemented this reported [1] potential buffer overrun
> using the constraining strlcpy, because the GID limitation of 200
> bytes is already mentioned in the documentation [2].
>

This will work but I think it is better to use sizeof gid buffer as we
are using in ParseCommitRecord() and ParseAbortRecord(). Tomorrow, if
due to some unforeseen reason if we change the size of gid buffer to
be different than the GIDSIZE then it will work seamlessly.

-- 
With Regards,
Amit Kapila.




Re: [HACKERS] logical decoding of two-phase transactions

2021-07-18 Thread Greg Nancarrow
On Wed, Jul 14, 2021 at 6:33 PM Peter Smith  wrote:
>
> Please find attached the latest patch set v97*
>

I couldn't spot spot any significant issues in the v97-0001 patch, but
do have the following trivial feedback comments:

(1) doc/src/sgml/protocol.sgml
Suggestion:

BEFORE:
+   contains a Stream Prepare or Stream Commit or Stream Abort message.
AFTER:
+   contains a Stream Prepare, Stream Commit or Stream Abort message.


(2) src/backend/replication/logical/worker.c
It seems a bit weird to add a forward declaration here, without a
comment, like for the one immediately above it

/* Compute GID for two_phase transactions */
static void TwoPhaseTransactionGid(Oid subid, TransactionId xid, char
*gid, int szgid);
-
+static int apply_spooled_messages(TransactionId xid, XLogRecPtr lsn);


(3) src/backend/replication/logical/worker.c
Other DEBUG1 messages don't end with "."

+ elog(DEBUG1, "apply_handle_stream_prepare: replayed %d
(all) changes.", nchanges);


Regards,
Greg Nancarrow
Fujitsu Australia




Re: [HACKERS] logical decoding of two-phase transactions

2021-07-18 Thread Peter Smith
On Mon, Jul 19, 2021 at 12:43 PM Amit Kapila  wrote:
>
> On Mon, Jul 19, 2021 at 1:55 AM Tom Lane  wrote:
> >
> > Amit Kapila  writes:
> > > Pushed.
> >
> > I think you'd be way better off making the gid fields be "char *"
> > and pstrdup'ing the result of pq_getmsgstring.  Another possibility
> > perhaps is to use strlcpy, but I'd only go that way if it's important
> > to constrain the received strings to 200 bytes.
> >
>
> I think it is important to constrain length to 200 bytes for this case
> as here we receive a prepared transaction identifier which according
> to docs [1] has a max length of 200 bytes. Also, in
> ParseCommitRecord() and ParseAbortRecord(), we are using strlcpy with
> 200 as max length to copy prepare transaction identifier. So, I think
> it is better to use strlcpy here unless you or Peter feels otherwise.
>

OK. I have implemented this reported [1] potential buffer overrun
using the constraining strlcpy, because the GID limitation of 200
bytes is already mentioned in the documentation [2].

PSA.

--
[1] https://www.postgresql.org/message-id/161029.1626639923%40sss.pgh.pa.us
[2] https://www.postgresql.org/docs/devel/sql-prepare-transaction.html

Kind Regards,
Peter Smith.
Fujitsu Australia


v1-0001-Fix-potential-buffer-overruns.patch
Description: Binary data


Re: [HACKERS] logical decoding of two-phase transactions

2021-07-18 Thread Amit Kapila
On Mon, Jul 19, 2021 at 1:55 AM Tom Lane  wrote:
>
> Amit Kapila  writes:
> > Pushed.
>
> I think you'd be way better off making the gid fields be "char *"
> and pstrdup'ing the result of pq_getmsgstring.  Another possibility
> perhaps is to use strlcpy, but I'd only go that way if it's important
> to constrain the received strings to 200 bytes.
>

I think it is important to constrain length to 200 bytes for this case
as here we receive a prepared transaction identifier which according
to docs [1] has a max length of 200 bytes. Also, in
ParseCommitRecord() and ParseAbortRecord(), we are using strlcpy with
200 as max length to copy prepare transaction identifier. So, I think
it is better to use strlcpy here unless you or Peter feels otherwise.

[1] - https://www.postgresql.org/docs/devel/sql-prepare-transaction.html

-- 
With Regards,
Amit Kapila.




Re: [HACKERS] logical decoding of two-phase transactions

2021-07-18 Thread Tom Lane
Amit Kapila  writes:
> Pushed.

Coverity thinks this has security issues, and I agree.

/srv/coverity/git/pgsql-git/postgresql/src/backend/replication/logical/proto.c: 
144 in logicalrep_read_begin_prepare()
143 /* read gid (copy it into a pre-allocated buffer) */
>>> CID 1487517:  Security best practices violations  (STRING_OVERFLOW)
>>> You might overrun the 200-character fixed-size string "begin_data->gid" 
>>> by copying the return value of "pq_getmsgstring" without checking the 
>>> length.
144 strcpy(begin_data->gid, pq_getmsgstring(in));

200 /* read gid (copy it into a pre-allocated buffer) */
>>> CID 1487515:  Security best practices violations  (STRING_OVERFLOW)
>>> You might overrun the 200-character fixed-size string 
>>> "prepare_data->gid" by copying the return value of "pq_getmsgstring" 
>>> without checking the length.
201 strcpy(prepare_data->gid, pq_getmsgstring(in));

256 /* read gid (copy it into a pre-allocated buffer) */
>>> CID 1487516:  Security best practices violations  (STRING_OVERFLOW)
>>> You might overrun the 200-character fixed-size string 
>>> "prepare_data->gid" by copying the return value of "pq_getmsgstring" 
>>> without checking the length.
257 strcpy(prepare_data->gid, pq_getmsgstring(in));

316 /* read gid (copy it into a pre-allocated buffer) */
>>> CID 1487519:  Security best practices violations  (STRING_OVERFLOW)
>>> You might overrun the 200-character fixed-size string 
>>> "rollback_data->gid" by copying the return value of "pq_getmsgstring" 
>>> without checking the length.
317 strcpy(rollback_data->gid, pq_getmsgstring(in));

I think you'd be way better off making the gid fields be "char *"
and pstrdup'ing the result of pq_getmsgstring.  Another possibility
perhaps is to use strlcpy, but I'd only go that way if it's important
to constrain the received strings to 200 bytes.

regards, tom lane




Re: [HACKERS] logical decoding of two-phase transactions

2021-07-16 Thread vignesh C
On Wed, Jul 14, 2021 at 2:03 PM Peter Smith  wrote:
>
> On Wed, Jul 14, 2021 at 4:23 PM Amit Kapila  wrote:
> >
> > On Mon, Jul 12, 2021 at 9:14 AM Peter Smith  wrote:
> > >
> > > On Sun, Jul 11, 2021 at 8:20 PM Amit Kapila  
> > > wrote:
> > > >
> > > > On Fri, Jul 9, 2021 at 4:43 AM Peter Smith  
> > > > wrote:
> > > > >
> > > > > > The patch looks good to me, I don't have any comments.
> > > > >
> > > > > I tried the v95-0001 patch.
> > > > >
> > > > > - The patch applied cleanly and all build / testing was OK.
> > > > > - The documentation also builds OK.
> > > > > - I checked all v95-0001 / v93-0001 differences and found no problems.
> > > > > - Furthermore, I noted that v95-0001 patch is passing the cfbot [1].
> > > > >
> > > > > So this patch LGTM.
> > > > >
> > > >
> > > > Thanks, I took another pass over it and made a few changes in docs and
> > > > comments. I am planning to push this next week sometime (by 14th July)
> > > > unless there are more comments from you or someone else. Just to
> > > > summarize, this patch will add support for prepared transactions to
> > > > built-in logical replication. To add support for streaming
> > > > transactions at prepare time into the
> > > > built-in logical replication, we need to do the following things: (a)
> > > > Modify the output plugin (pgoutput) to implement the new two-phase API
> > > > callbacks, by leveraging the extended replication protocol. (b) Modify
> > > > the replication apply worker, to properly handle two-phase
> > > > transactions by replaying them on prepare. (c) Add a new SUBSCRIPTION
> > > > option "two_phase" to allow users to enable
> > > > two-phase transactions. We enable the two_phase once the initial data
> > > > sync is over. Refer to comments atop worker.c in the patch and commit
> > > > message to see further details about this patch. After this patch,
> > > > there is a follow-up patch to allow streaming and two-phase options
> > > > together which I feel needs some more review and can be committed
> > > > separately.
> > > >
> > >
> > > FYI - I repeated the same verification of the v96-0001 patch as I did
> > > previously for v95-0001
> > >
> > > - The v96 patch applied cleanly and all build / testing was OK.
> > > - The documentation also builds OK.
> > > - I checked the v95-0001 / v96-0001 differences and found no problems.
> > > - Furthermore, I noted that v96-0001 patch is passing the cfbot.
> > >
> > > LGTM.
> > >
> >
> > Pushed.
> >
> > Feel free to submit the remaining patches after rebase. Is it possible
> > to post patches related to skipping empty transactions in the other
> > thread [1] where that topic is being discussed?
> >
> > [1] - 
> > https://www.postgresql.org/message-id/CAMkU%3D1yohp9-dv48FLoSPrMqYEyyS5ZWkaZGD41RJr10xiNo_Q%40mail.gmail.com
> >
>
>
> Please find attached the latest patch set v97*
>
> * Rebased v94* to HEAD @ today.

Thanks for the updated patch, the patch applies cleanly and test passes:
I had couple of comments:
1) Should we include "stream_prepare_cb" here  in
logicaldecoding-streaming section of logicaldecoding.sgml
documentation:
To reduce the apply lag caused by large transactions, an output plugin
may provide additional callback to support incremental streaming of
in-progress transactions. There are multiple required streaming
callbacks (stream_start_cb, stream_stop_cb, stream_abort_cb,
stream_commit_cb and stream_change_cb) and two optional callbacks
(stream_message_cb and stream_truncate_cb).

2) Should we add an example for stream_prepare_cb here  in
logicaldecoding-streaming section of logicaldecoding.sgml
documentation:
One example sequence of streaming callback calls for one transaction
may look like this:

stream_start_cb(...);   <-- start of first block of changes
  stream_change_cb(...);
  stream_change_cb(...);
  stream_message_cb(...);
  stream_change_cb(...);
  ...
  stream_change_cb(...);
stream_stop_cb(...);<-- end of first block of changes

stream_start_cb(...);   <-- start of second block of changes
  stream_change_cb(...);
  stream_change_cb(...);
  stream_change_cb(...);
  ...
  stream_message_cb(...);
  stream_change_cb(...);
stream_stop_cb(...);<-- end of second block of changes

stream_commit_cb(...);  <-- commit of the streamed transaction

Regards,
Vignesh




Re: [HACKERS] logical decoding of two-phase transactions

2021-07-14 Thread Peter Smith
On Wed, Jul 14, 2021 at 4:23 PM Amit Kapila  wrote:
>
> On Mon, Jul 12, 2021 at 9:14 AM Peter Smith  wrote:
> >
> > On Sun, Jul 11, 2021 at 8:20 PM Amit Kapila  wrote:
> > >
> > > On Fri, Jul 9, 2021 at 4:43 AM Peter Smith  wrote:
> > > >
> > > > > The patch looks good to me, I don't have any comments.
> > > >
> > > > I tried the v95-0001 patch.
> > > >
> > > > - The patch applied cleanly and all build / testing was OK.
> > > > - The documentation also builds OK.
> > > > - I checked all v95-0001 / v93-0001 differences and found no problems.
> > > > - Furthermore, I noted that v95-0001 patch is passing the cfbot [1].
> > > >
> > > > So this patch LGTM.
> > > >
> > >
> > > Thanks, I took another pass over it and made a few changes in docs and
> > > comments. I am planning to push this next week sometime (by 14th July)
> > > unless there are more comments from you or someone else. Just to
> > > summarize, this patch will add support for prepared transactions to
> > > built-in logical replication. To add support for streaming
> > > transactions at prepare time into the
> > > built-in logical replication, we need to do the following things: (a)
> > > Modify the output plugin (pgoutput) to implement the new two-phase API
> > > callbacks, by leveraging the extended replication protocol. (b) Modify
> > > the replication apply worker, to properly handle two-phase
> > > transactions by replaying them on prepare. (c) Add a new SUBSCRIPTION
> > > option "two_phase" to allow users to enable
> > > two-phase transactions. We enable the two_phase once the initial data
> > > sync is over. Refer to comments atop worker.c in the patch and commit
> > > message to see further details about this patch. After this patch,
> > > there is a follow-up patch to allow streaming and two-phase options
> > > together which I feel needs some more review and can be committed
> > > separately.
> > >
> >
> > FYI - I repeated the same verification of the v96-0001 patch as I did
> > previously for v95-0001
> >
> > - The v96 patch applied cleanly and all build / testing was OK.
> > - The documentation also builds OK.
> > - I checked the v95-0001 / v96-0001 differences and found no problems.
> > - Furthermore, I noted that v96-0001 patch is passing the cfbot.
> >
> > LGTM.
> >
>
> Pushed.
>
> Feel free to submit the remaining patches after rebase. Is it possible
> to post patches related to skipping empty transactions in the other
> thread [1] where that topic is being discussed?
>
> [1] - 
> https://www.postgresql.org/message-id/CAMkU%3D1yohp9-dv48FLoSPrMqYEyyS5ZWkaZGD41RJr10xiNo_Q%40mail.gmail.com
>


Please find attached the latest patch set v97*

* Rebased v94* to HEAD @ today.

This rebase was made necessary by the recent push of the first patch
from this set.

v94-0001 ==> already pushed [1]
v94-0002 ==> v97-0001
v94-0003 ==> will be relocated to other thread [2]
v94-0004 ==> this is omitted for now


[1] 
https://github.com/postgres/postgres/commit/a8fd13cab0ba815e9925dc9676e6309f699b5f72
[2] 
https://www.postgresql.org/message-id/CAMkU%3D1yohp9-dv48FLoSPrMqYEyyS5ZWkaZGD41RJr10xiNo_Q%40mail.gmail.com

Kind Regards,
Peter Smith.
Fujitsu Australia


v97-0001-Add-prepare-API-support-for-streaming-transactio.patch
Description: Binary data


Re: [HACKERS] logical decoding of two-phase transactions

2021-07-14 Thread Amit Kapila
On Mon, Jul 12, 2021 at 9:14 AM Peter Smith  wrote:
>
> On Sun, Jul 11, 2021 at 8:20 PM Amit Kapila  wrote:
> >
> > On Fri, Jul 9, 2021 at 4:43 AM Peter Smith  wrote:
> > >
> > > > The patch looks good to me, I don't have any comments.
> > >
> > > I tried the v95-0001 patch.
> > >
> > > - The patch applied cleanly and all build / testing was OK.
> > > - The documentation also builds OK.
> > > - I checked all v95-0001 / v93-0001 differences and found no problems.
> > > - Furthermore, I noted that v95-0001 patch is passing the cfbot [1].
> > >
> > > So this patch LGTM.
> > >
> >
> > Thanks, I took another pass over it and made a few changes in docs and
> > comments. I am planning to push this next week sometime (by 14th July)
> > unless there are more comments from you or someone else. Just to
> > summarize, this patch will add support for prepared transactions to
> > built-in logical replication. To add support for streaming
> > transactions at prepare time into the
> > built-in logical replication, we need to do the following things: (a)
> > Modify the output plugin (pgoutput) to implement the new two-phase API
> > callbacks, by leveraging the extended replication protocol. (b) Modify
> > the replication apply worker, to properly handle two-phase
> > transactions by replaying them on prepare. (c) Add a new SUBSCRIPTION
> > option "two_phase" to allow users to enable
> > two-phase transactions. We enable the two_phase once the initial data
> > sync is over. Refer to comments atop worker.c in the patch and commit
> > message to see further details about this patch. After this patch,
> > there is a follow-up patch to allow streaming and two-phase options
> > together which I feel needs some more review and can be committed
> > separately.
> >
>
> FYI - I repeated the same verification of the v96-0001 patch as I did
> previously for v95-0001
>
> - The v96 patch applied cleanly and all build / testing was OK.
> - The documentation also builds OK.
> - I checked the v95-0001 / v96-0001 differences and found no problems.
> - Furthermore, I noted that v96-0001 patch is passing the cfbot.
>
> LGTM.
>

Pushed.

Feel free to submit the remaining patches after rebase. Is it possible
to post patches related to skipping empty transactions in the other
thread [1] where that topic is being discussed?

[1] - 
https://www.postgresql.org/message-id/CAMkU%3D1yohp9-dv48FLoSPrMqYEyyS5ZWkaZGD41RJr10xiNo_Q%40mail.gmail.com

-- 
With Regards,
Amit Kapila.




Re: [HACKERS] logical decoding of two-phase transactions

2021-07-11 Thread Peter Smith
On Sun, Jul 11, 2021 at 8:20 PM Amit Kapila  wrote:
>
> On Fri, Jul 9, 2021 at 4:43 AM Peter Smith  wrote:
> >
> > > The patch looks good to me, I don't have any comments.
> >
> > I tried the v95-0001 patch.
> >
> > - The patch applied cleanly and all build / testing was OK.
> > - The documentation also builds OK.
> > - I checked all v95-0001 / v93-0001 differences and found no problems.
> > - Furthermore, I noted that v95-0001 patch is passing the cfbot [1].
> >
> > So this patch LGTM.
> >
>
> Thanks, I took another pass over it and made a few changes in docs and
> comments. I am planning to push this next week sometime (by 14th July)
> unless there are more comments from you or someone else. Just to
> summarize, this patch will add support for prepared transactions to
> built-in logical replication. To add support for streaming
> transactions at prepare time into the
> built-in logical replication, we need to do the following things: (a)
> Modify the output plugin (pgoutput) to implement the new two-phase API
> callbacks, by leveraging the extended replication protocol. (b) Modify
> the replication apply worker, to properly handle two-phase
> transactions by replaying them on prepare. (c) Add a new SUBSCRIPTION
> option "two_phase" to allow users to enable
> two-phase transactions. We enable the two_phase once the initial data
> sync is over. Refer to comments atop worker.c in the patch and commit
> message to see further details about this patch. After this patch,
> there is a follow-up patch to allow streaming and two-phase options
> together which I feel needs some more review and can be committed
> separately.
>

FYI - I repeated the same verification of the v96-0001 patch as I did
previously for v95-0001

- The v96 patch applied cleanly and all build / testing was OK.
- The documentation also builds OK.
- I checked the v95-0001 / v96-0001 differences and found no problems.
- Furthermore, I noted that v96-0001 patch is passing the cfbot.

LGTM.

--
Kind Regards,
Peter Smith.
Fujitsu Australia




RE: [HACKERS] logical decoding of two-phase transactions

2021-07-09 Thread tanghy.f...@fujitsu.com
On Friday, July 9, 2021 2:56 PM Ajin Cherian wrote:
> 
> On Fri, Jul 9, 2021 at 9:13 AM Peter Smith  wrote:
> 
> > I tried the v95-0001 patch.
> >
> > - The patch applied cleanly and all build / testing was OK.
> > - The documentation also builds OK.
> > - I checked all v95-0001 / v93-0001 differences and found no problems.
> > - Furthermore, I noted that v95-0001 patch is passing the cfbot [1].
> >
> > So this patch LGTM.
> >
> 
> Applied, reviewed and tested the patch.
> Also ran a 5 level cascaded standby setup running a modified pgbench
> that does two phase commits and it ran fine.
> Did some testing using empty transactions and no issues found
> The patch looks good to me.

I did some cross version tests on patch v95 (publisher is PG14 and subscriber 
is PG15, or publisher is PG15 and subscriber is PG14; set two_phase option to 
on or off/default). It worked as expected, data could be replicated correctly.

Besides, I tested some scenarios using synchronized replication, it worked fine 
in my cases.

So this patch LGTM.

Regards
Tang


Re: [HACKERS] logical decoding of two-phase transactions

2021-07-09 Thread Ajin Cherian
On Fri, Jul 9, 2021 at 9:13 AM Peter Smith  wrote:

> I tried the v95-0001 patch.
>
> - The patch applied cleanly and all build / testing was OK.
> - The documentation also builds OK.
> - I checked all v95-0001 / v93-0001 differences and found no problems.
> - Furthermore, I noted that v95-0001 patch is passing the cfbot [1].
>
> So this patch LGTM.
>

Applied, reviewed and tested the patch.
Also ran a 5 level cascaded standby setup running a modified pgbench
that does two phase commits and it ran fine.
Did some testing using empty transactions and no issues found
The patch looks good to me.

regards,
Ajin Cherian




Re: [HACKERS] logical decoding of two-phase transactions

2021-07-08 Thread Peter Smith
On Thu, Jul 8, 2021 at 10:08 PM vignesh C  wrote:
>
> On Thu, Jul 8, 2021 at 11:37 AM Amit Kapila  wrote:
> >
> > On Tue, Jul 6, 2021 at 9:58 AM Peter Smith  wrote:
> > >
> > > Please find attached the latest patch set v93*
> > >
> >
> > Thanks, I have gone through the 0001 patch and made a number of
> > changes. (a) Removed some of the code which was leftover from previous
> > versions, (b) Removed the Assert in apply_handle_begin_prepare() as I
> > don't think that makes sense, (c) added/changed comments and made a
> > few other cosmetic changes, (d) ran pgindent.
> >
> > Let me know what you think of the attached?
>
> The patch looks good to me, I don't have any comments.

I tried the v95-0001 patch.

- The patch applied cleanly and all build / testing was OK.
- The documentation also builds OK.
- I checked all v95-0001 / v93-0001 differences and found no problems.
- Furthermore, I noted that v95-0001 patch is passing the cfbot [1].

So this patch LGTM.

--
[1] http://cfbot.cputube.org/patch_33_2914.log

Kind Regards,
Peter Smith.
Fujitsu Australia




Re: [HACKERS] logical decoding of two-phase transactions

2021-07-08 Thread vignesh C
On Thu, Jul 8, 2021 at 11:37 AM Amit Kapila  wrote:
>
> On Tue, Jul 6, 2021 at 9:58 AM Peter Smith  wrote:
> >
> > Please find attached the latest patch set v93*
> >
>
> Thanks, I have gone through the 0001 patch and made a number of
> changes. (a) Removed some of the code which was leftover from previous
> versions, (b) Removed the Assert in apply_handle_begin_prepare() as I
> don't think that makes sense, (c) added/changed comments and made a
> few other cosmetic changes, (d) ran pgindent.
>
> Let me know what you think of the attached?

The patch looks good to me, I don't have any comments.

Regards,
Vignesh




RE: [HACKERS] logical decoding of two-phase transactions

2021-07-06 Thread tanghy.f...@fujitsu.com
On Tuesday, July 6, 2021 7:18 PM Ajin Cherian 
>
> thanks for the test!
> I hadn't updated the case where sending schema across was the first
> change of the transaction as part of the decoding of the
> truncate command. In this test case, the schema was sent across
> without the stream start, hence the error on the apply worker.
> I have updated with a fix. Please do a test and confirm.
> 

Thanks for your patch.
I have tested and confirmed that the issue was fixed.

Regards
Tang


RE: [HACKERS] logical decoding of two-phase transactions

2021-07-02 Thread tanghy.f...@fujitsu.com
On Thursday, July 1, 2021 11:48 AM Ajin Cherian 
> 
> Adding a new patch (0004) to this patch-set that handles skipping of
> empty streamed transactions. patch-0003 did not
> handle empty streamed transactions. To support this, added a new flag
> "sent_stream_start" to PGOutputTxnData.
> Also transactions which do not have any data will not be stream
> committed or stream prepared or stream aborted.
> Do review and let me know if you have any comments.
> 

Thanks for your patch. I met an issue while using it. When a transaction 
contains TRUNCATE, the subscriber reported an error: " ERROR:  no data left in 
message" and the data couldn't be replicated. 

Steps to reproduce the issue:

(set logical_decoding_work_mem to 64kB at publisher so that streaming could 
work. )

--publisher--
create table test (a int primary key, b varchar);
create publication pub for table test;

--subscriber--
create table test (a int primary key, b varchar);
create subscription sub connection 'dbname=postgres' publication pub 
with(two_phase=on, streaming=on);

--publisher--
BEGIN;
TRUNCATE test;
INSERT INTO test SELECT i, md5(i::text) FROM generate_series(1001, 6000) s(i);
UPDATE test SET b = md5(b) WHERE mod(a,2) = 0;
DELETE FROM test WHERE mod(a,3) = 0;
COMMIT;

The above case worked ok when remove 0004 patch, so I think it’s a problem of 
0004 patch. Please have a look.

Regards
Tang


Re: [HACKERS] logical decoding of two-phase transactions

2021-06-30 Thread Amit Kapila
On Tue, Jun 29, 2021 at 5:31 PM vignesh C  wrote:
>
> On Tue, Jun 29, 2021 at 12:26 PM Amit Kapila  wrote:
> >
> > On Wed, Jun 23, 2021 at 4:10 PM Ajin Cherian  wrote:
> > >
> >
> > The first two patches look mostly good to me. I have combined them
> > into one and made some minor changes. (a) Removed opt_two_phase and
> > related code from repl_gram.y as that is not required for this version
> > of the patch. (b) made some changes in docs. Kindly check the attached
> > and let me know if you have any comments? I am planning to push this
> > first patch in the series tomorrow unless you or others have any
> > comments.
>
> Thanks for the updated patch, patch applies neatly and tests passed.
> If you are ok, One of the documentation changes could be slightly
> changed while committing:
>

Pushed the patch after taking care of your suggestion. Now, the next
step is to rebase the remaining patches and adapt some of the checks
to PG-15.

-- 
With Regards,
Amit Kapila.




Re: [HACKERS] logical decoding of two-phase transactions

2021-06-29 Thread vignesh C
On Tue, Jun 29, 2021 at 12:26 PM Amit Kapila  wrote:
>
> On Wed, Jun 23, 2021 at 4:10 PM Ajin Cherian  wrote:
> >
>
> The first two patches look mostly good to me. I have combined them
> into one and made some minor changes. (a) Removed opt_two_phase and
> related code from repl_gram.y as that is not required for this version
> of the patch. (b) made some changes in docs. Kindly check the attached
> and let me know if you have any comments? I am planning to push this
> first patch in the series tomorrow unless you or others have any
> comments.

Thanks for the updated patch, patch applies neatly and tests passed.
If you are ok, One of the documentation changes could be slightly
changed while committing:
+   
+Enables two-phase decoding. This option should only be used with
+--create-slot
+   
to:
+   
+Enables two-phase decoding. This option should only be specified with
+--create-slot option.
+   

Regards,
Vignesh




Re: [HACKERS] logical decoding of two-phase transactions

2021-06-29 Thread Ajin Cherian
On Tue, Jun 29, 2021 at 4:56 PM Amit Kapila  wrote:
>
> On Wed, Jun 23, 2021 at 4:10 PM Ajin Cherian  wrote:
> >
>
> The first two patches look mostly good to me. I have combined them
> into one and made some minor changes. (a) Removed opt_two_phase and
> related code from repl_gram.y as that is not required for this version
> of the patch. (b) made some changes in docs. Kindly check the attached
> and let me know if you have any comments? I am planning to push this
> first patch in the series tomorrow unless you or others have any
> comments.

The patch applies cleanly, tests pass. I reviewed the patch and have
no comments, it looks good.

regards,
Ajin Cherian
Fujitsu Australia




Re: [HACKERS] logical decoding of two-phase transactions

2021-06-29 Thread Amit Kapila
On Wed, Jun 23, 2021 at 4:10 PM Ajin Cherian  wrote:
>

The first two patches look mostly good to me. I have combined them
into one and made some minor changes. (a) Removed opt_two_phase and
related code from repl_gram.y as that is not required for this version
of the patch. (b) made some changes in docs. Kindly check the attached
and let me know if you have any comments? I am planning to push this
first patch in the series tomorrow unless you or others have any
comments.

-- 
With Regards,
Amit Kapila.


0001-Allow-enabling-two-phase-option-via-replication-prot.patch
Description: Binary data


Re: [HACKERS] logical decoding of two-phase transactions

2021-06-22 Thread vignesh C
On Wed, Jun 23, 2021 at 9:10 AM Ajin Cherian  wrote:
>
> On Tue, Jun 22, 2021 at 3:36 PM Greg Nancarrow  wrote:
>
> > Some minor comments:
> >
> > (1)
> > v88-0002
> >
> > doc/src/sgml/logicaldecoding.sgml
> >
> > "examples shows" is not correct.
> > I think there is only ONE example being referred to.
> >
> > BEFORE:
> > +The following examples shows how logical decoding is controlled over 
> > the
> > AFTER:
> > +The following example shows how logical decoding is controlled over the
> >
> >
> fixed.
>
> > (2)
> > v88 - 0003
> >
> > doc/src/sgml/ref/create_subscription.sgml
> >
> > (i)
> >
> > BEFORE:
> > +  to the subscriber on the PREPARE TRANSACTION. By default,
> > the transaction
> > +  prepared on publisher is decoded as a normal transaction at 
> > commit.
> > AFTER:
> > +  to the subscriber on the PREPARE TRANSACTION. By default,
> > the transaction
> > +  prepared on the publisher is decoded as a normal
> > transaction at commit time.
> >
>
> fixed.
>
> > (ii)
> >
> > src/backend/access/transam/twophase.c
> >
> > The double-bracketing is unnecessary:
> >
> > BEFORE:
> > + if ((gxact->valid && strcmp(gxact->gid, gid) == 0))
> > AFTER:
> > + if (gxact->valid && strcmp(gxact->gid, gid) == 0)
> >
>
> fixed.
>
> > (iii)
> >
> > src/backend/replication/logical/snapbuild.c
> >
> > Need to add some commas to make the following easier to read, and
> > change "needs" to "need":
> >
> > BEFORE:
> > + * The prepared transactions that were skipped because previously
> > + * two-phase was not enabled or are not covered by initial snapshot needs
> > + * to be sent later along with commit prepared and they must be before
> > + * this point.
> > AFTER:
> > + * The prepared transactions, that were skipped because previously
> > + * two-phase was not enabled or are not covered by initial snapshot, need
> > + * to be sent later along with commit prepared and they must be before
> > + * this point.
> >
>
> fixed.
>
> > (iv)
> >
> > src/backend/replication/logical/tablesync.c
> >
> > I think the convention used in Postgres code is to check for empty
> > Lists using "== NIL" and non-empty Lists using "!= NIL".
> >
> > BEFORE:
> > + if (table_states_not_ready && !last_start_times)
> > AFTER:
> > + if (table_states_not_ready != NIL && !last_start_times)
> >
> >
> > BEFORE:
> > + else if (!table_states_not_ready && last_start_times)
> > AFTER:
> > + else if (table_states_not_ready == NIL && last_start_times)
> >
>
> fixed.
>
> Also fixed comments from Vignesh:
>
> 1) This content is present in
> v87-0001-Add-option-to-set-two-phase-in-CREATE_REPLICATIO.patch and
> v87-0003-Add-support-for-prepared-transactions-to-built-i.patch, it
> can be removed from one of them
>
> +   TWO_PHASE
> +   
> +
> + Specify that this logical replication slot supports decoding
> of two-phase
> + transactions. With this option, two-phase commands like
> + PREPARE TRANSACTION, COMMIT
> PREPARED
> + and ROLLBACK PREPARED are decoded and 
> transmitted.
> + The transaction will be decoded and transmitted at
> + PREPARE TRANSACTION time.
> +
> +   
> +  
> +
> +  
>
> I don't see this duplicate content.

Thanks for the updated patch.
The patch v89-0001-Add-option-to-set-two-phase-in-CREATE_REPLICATIO.patch
has the following:
+   TWO_PHASE
+   
+
+ Specify that this logical replication slot supports decoding
of two-phase
+ transactions. With this option, two-phase commands like
+ PREPARE TRANSACTION, COMMIT
PREPARED
+ and ROLLBACK PREPARED are decoded and transmitted.
+ The transaction will be decoded and transmitted at
+ PREPARE TRANSACTION time.
+
+   
+  

The patch v89-0003-Add-support-for-prepared-transactions-to-built-i.patch
has the following:
+   TWO_PHASE
+   
+
+ Specify that this replication slot supports decode of two-phase
+ transactions. With this option, two-phase commands like
+ PREPARE TRANSACTION, COMMIT
PREPARED
+ and ROLLBACK PREPARED are decoded and transmitted.
+ The transaction will be decoded and transmitted at
+ PREPARE TRANSACTION time.
+
+   
+  

We can remove one of them.

Regards,
Vignesh




Re: [HACKERS] logical decoding of two-phase transactions

2021-06-21 Thread Greg Nancarrow
On Mon, Jun 21, 2021 at 4:37 PM Peter Smith  wrote:
>
> Please find attached the latest patch set v88*
>

Some minor comments:

(1)
v88-0002

doc/src/sgml/logicaldecoding.sgml

"examples shows" is not correct.
I think there is only ONE example being referred to.

BEFORE:
+The following examples shows how logical decoding is controlled over the
AFTER:
+The following example shows how logical decoding is controlled over the


(2)
v88 - 0003

doc/src/sgml/ref/create_subscription.sgml

(i)

BEFORE:
+  to the subscriber on the PREPARE TRANSACTION. By default,
the transaction
+  prepared on publisher is decoded as a normal transaction at commit.
AFTER:
+  to the subscriber on the PREPARE TRANSACTION. By default,
the transaction
+  prepared on the publisher is decoded as a normal
transaction at commit time.

(ii)

src/backend/access/transam/twophase.c

The double-bracketing is unnecessary:

BEFORE:
+ if ((gxact->valid && strcmp(gxact->gid, gid) == 0))
AFTER:
+ if (gxact->valid && strcmp(gxact->gid, gid) == 0)

(iii)

src/backend/replication/logical/snapbuild.c

Need to add some commas to make the following easier to read, and
change "needs" to "need":

BEFORE:
+ * The prepared transactions that were skipped because previously
+ * two-phase was not enabled or are not covered by initial snapshot needs
+ * to be sent later along with commit prepared and they must be before
+ * this point.
AFTER:
+ * The prepared transactions, that were skipped because previously
+ * two-phase was not enabled or are not covered by initial snapshot, need
+ * to be sent later along with commit prepared and they must be before
+ * this point.

(iv)

src/backend/replication/logical/tablesync.c

I think the convention used in Postgres code is to check for empty
Lists using "== NIL" and non-empty Lists using "!= NIL".

BEFORE:
+ if (table_states_not_ready && !last_start_times)
AFTER:
+ if (table_states_not_ready != NIL && !last_start_times)


BEFORE:
+ else if (!table_states_not_ready && last_start_times)
AFTER:
+ else if (table_states_not_ready == NIL && last_start_times)


Regards,
Greg Nancarrow
Fujitsu Australia




Re: [HACKERS] logical decoding of two-phase transactions

2021-06-17 Thread Amit Kapila
On Fri, Jun 18, 2021 at 7:43 AM Peter Smith  wrote:
>
> On Thu, Jun 17, 2021 at 6:22 PM Greg Nancarrow  wrote:
> >
> > For example, in the v86-0001 patch:
> >
> > +/*
> > + * Handle PREPARE message.
> > + */
> > +static void
> > +apply_handle_prepare(StringInfo s)
> > +{
> > + LogicalRepPreparedTxnData prepare_data;
> > + char gid[GIDSIZE];
> > +
> > + logicalrep_read_prepare(s, _data);
> > +
> > + Assert(prepare_data.prepare_lsn == remote_final_lsn);
> >
> > The above Assert() should be changed to something like:
> >
> > +if (prepare_data.prepare_lsn != remote_final_lsn)
> > +ereport(ERROR,
> > +(errcode(ERRCODE_PROTOCOL_VIOLATION),
> > + errmsg_internal("incorrect prepare LSN %X/%X in
> > prepare message (expected %X/%X)",
> > + LSN_FORMAT_ARGS(prepare_data.prepare_lsn),
> > + LSN_FORMAT_ARGS(remote_final_lsn;
> >
> > Without being more familiar with this code, it's difficult for me to
> > judge exactly how many of such cases are in these patches.
>
> Thanks for the above example. I will fix this one later, after
> receiving some more reviews and reports of other Assert cases just
> like this one.
>

I think on similar lines below asserts also need to be changed.

1.
+static void
+apply_handle_begin_prepare(StringInfo s)
+{
+ LogicalRepPreparedTxnData begin_data;
+ char gid[GIDSIZE];
+
+ /* Tablesync should never receive prepare. */
+ Assert(!am_tablesync_worker());

2.
+static void
+TwoPhaseTransactionGid(Oid subid, TransactionId xid, char *gid, int szgid)
+{
..
+ Assert(TransactionIdIsValid(xid));

3.
+static void
+apply_handle_stream_prepare(StringInfo s)
+{
+ int nchanges = 0;
+ LogicalRepPreparedTxnData prepare_data;
+ TransactionId xid;
+ char gid[GIDSIZE];
+
..
..
+
+ /* Tablesync should never receive prepare. */
+ Assert(!am_tablesync_worker());


-- 
With Regards,
Amit Kapila.




Re: [HACKERS] logical decoding of two-phase transactions

2021-06-17 Thread vignesh C
On Thu, Jun 17, 2021 at 7:40 PM Ajin Cherian  wrote:
>
> On Wed, Jun 16, 2021 at 9:08 AM Peter Smith  wrote:
> >
> > On Fri, Jun 11, 2021 at 6:34 PM Peter Smith  wrote:
> >
> > > KNOWN ISSUES: This v85 patch was built and tested using yesterday's
> > > master, but due to lots of recent activity in the replication area I
> > > expect it will be broken for HEAD very soon (if not already). I'll
> > > rebase it again ASAP to try to keep it in working order.
> > >
> >
> > Please find attached the latest patch set v86*
>
>
> I've modified the patchset based on comments received on thread [1]
> for the CREATE_REPLICATION_SLOT
> changes. Based on the request from that thread, I've taken out those
> changes as two new patches (patch-1 and patch-2)
> and made this into 5 patches. I've also changed the logic to align
> with the changes in the command syntax.

Few comments:
1) This content is present in
v87-0001-Add-option-to-set-two-phase-in-CREATE_REPLICATIO.patch and
v87-0003-Add-support-for-prepared-transactions-to-built-i.patch, it
can be removed from one of them
   
+   TWO_PHASE
+   
+
+ Specify that this logical replication slot supports decoding
of two-phase
+ transactions. With this option, two-phase commands like
+ PREPARE TRANSACTION, COMMIT
PREPARED
+ and ROLLBACK PREPARED are decoded and transmitted.
+ The transaction will be decoded and transmitted at
+ PREPARE TRANSACTION time.
+
+   
+  
+
+  

2) This change is not required, it can be removed:
   
Logical Decoding Examples
-

 The following example demonstrates controlling logical decoding using the
 SQL interface.

3) We could add comment mentioning example 1 at the beginning of
example 1 and example 2 for the newly added example with description,
that will clearly mark the examples.
COMMIT 693
 ControlC
 $ pg_recvlogical -d postgres --slot=test --drop-slot
+
+$ pg_recvlogical -d postgres --slot=test --create-slot --two-phase
+$ pg_recvlogical -d postgres --slot=test --start -f -

4) You could mention "Before you use two-phase commit commands, you
must set max_prepared_transactions to at least 1" for example 2.
 $ pg_recvlogical -d postgres --slot=test --drop-slot
+
+$ pg_recvlogical -d postgres --slot=test --create-slot --two-phase
+$ pg_recvlogical -d postgres --slot=test --start -f -

5) This should be before verbose, the options are documented alphabetically
+ 
+   -t
+   --two-phase
+   
+   
+Enables two-phase decoding. This option should only be used with
+--create-slot
+   
+   
+ 

6)  This should be before verbose, the options are printed alphabetically
printf(_("  -v, --verbose  output verbose messages\n"));
+   printf(_("  -t, --two-phaseenable two-phase decoding
when creating a slot\n"));
printf(_("  -V, --version  output version information,
then exit\n"));

Regards,
Vignesh




Re: [HACKERS] logical decoding of two-phase transactions

2021-06-17 Thread Peter Smith
On Thu, Jun 17, 2021 at 6:22 PM Greg Nancarrow  wrote:
>
> On Wed, Jun 16, 2021 at 9:08 AM Peter Smith  wrote:
> >
> >
> > Please find attached the latest patch set v86*
> >
>
> A couple of comments:
>
> (1)  I think one of my suggested changes was missed (or was that 
> intentional?):
>
> BEFORE:
> +The LSN of the commit prepared.
> AFTER:
> +The LSN of the commit prepared transaction.
>

No, not missed. I already dismissed that one and wrote about it when I
posted v85 [1].

>
> (2)  In light of Tom Lane's recent changes in:
>
> fe6a20ce54cbbb6fcfe9f6675d563af836ae799a (Don't use Asserts to check
> for violations of replication protocol)
>
> there appear to be some instances of such code in these patches.

Yes, I already noted [2] there are likely to be such cases which need
to be fixed.

>
> For example, in the v86-0001 patch:
>
> +/*
> + * Handle PREPARE message.
> + */
> +static void
> +apply_handle_prepare(StringInfo s)
> +{
> + LogicalRepPreparedTxnData prepare_data;
> + char gid[GIDSIZE];
> +
> + logicalrep_read_prepare(s, _data);
> +
> + Assert(prepare_data.prepare_lsn == remote_final_lsn);
>
> The above Assert() should be changed to something like:
>
> +if (prepare_data.prepare_lsn != remote_final_lsn)
> +ereport(ERROR,
> +(errcode(ERRCODE_PROTOCOL_VIOLATION),
> + errmsg_internal("incorrect prepare LSN %X/%X in
> prepare message (expected %X/%X)",
> + LSN_FORMAT_ARGS(prepare_data.prepare_lsn),
> + LSN_FORMAT_ARGS(remote_final_lsn;
>
> Without being more familiar with this code, it's difficult for me to
> judge exactly how many of such cases are in these patches.

Thanks for the above example. I will fix this one later, after
receiving some more reviews and reports of other Assert cases just
like this one.

--
[1] 
https://www.postgresql.org/message-id/CAHut%2BPvOVkiVBf4P5chdVSoVs5%3Da%3DF_GtTSHHoXDb4LiOM_8Qw%40mail.gmail.com
[2] 
https://www.postgresql.org/message-id/CAHut%2BPvdio4%3DOE6cz5pr8VcJNcAgt5uGBPdKf-tnGEMa1mANGg%40mail.gmail.com

Kind Regards,
Peter Smith.
Fujitsu Australia




Re: [HACKERS] logical decoding of two-phase transactions

2021-06-17 Thread Greg Nancarrow
On Wed, Jun 16, 2021 at 9:08 AM Peter Smith  wrote:
>
>
> Please find attached the latest patch set v86*
>

A couple of comments:

(1)  I think one of my suggested changes was missed (or was that intentional?):

BEFORE:
+The LSN of the commit prepared.
AFTER:
+The LSN of the commit prepared transaction.


(2)  In light of Tom Lane's recent changes in:

fe6a20ce54cbbb6fcfe9f6675d563af836ae799a (Don't use Asserts to check
for violations of replication protocol)

there appear to be some instances of such code in these patches.

For example, in the v86-0001 patch:

+/*
+ * Handle PREPARE message.
+ */
+static void
+apply_handle_prepare(StringInfo s)
+{
+ LogicalRepPreparedTxnData prepare_data;
+ char gid[GIDSIZE];
+
+ logicalrep_read_prepare(s, _data);
+
+ Assert(prepare_data.prepare_lsn == remote_final_lsn);

The above Assert() should be changed to something like:

+if (prepare_data.prepare_lsn != remote_final_lsn)
+ereport(ERROR,
+(errcode(ERRCODE_PROTOCOL_VIOLATION),
+ errmsg_internal("incorrect prepare LSN %X/%X in
prepare message (expected %X/%X)",
+ LSN_FORMAT_ARGS(prepare_data.prepare_lsn),
+ LSN_FORMAT_ARGS(remote_final_lsn;

Without being more familiar with this code, it's difficult for me to
judge exactly how many of such cases are in these patches.

Regards,
Greg Nancarrow
Fujitsu Australia




Re: [HACKERS] logical decoding of two-phase transactions

2021-06-08 Thread Amit Kapila
On Wed, Jun 9, 2021 at 9:58 AM Greg Nancarrow  wrote:
>
> (5) src/backend/access/transam/twophase.c
>
> Question:
>
> Is:
>
> + * do this optimization if we encounter many collisions in GID
>
> meant to be:
>
> + * do this optimization if we encounter any collisions in GID
>

No, it should be fine if there are very few collisions.

-- 
With Regards,
Amit Kapila.




Re: [HACKERS] logical decoding of two-phase transactions

2021-06-08 Thread Amit Kapila
On Wed, Jun 9, 2021 at 10:34 AM Ajin Cherian  wrote:
>
> On Tue, Jun 8, 2021 at 4:19 PM Peter Smith  wrote:
>
> > > 3.
> > > @@ -432,10 +432,19 @@ CreateInitDecodingContext(const char *plugin,
> > >   MemoryContextSwitchTo(old_context);
> > >
> > >   /*
> > > - * We allow decoding of prepared transactions iff the two_phase option is
> > > - * enabled at the time of slot creation.
> > > + * We allow decoding of prepared transactions when the two_phase is
> > > + * enabled at the time of slot creation, or when the two_phase option is
> > > + * given at the streaming start.
> > >   */
> > > - ctx->twophase &= MyReplicationSlot->data.two_phase;
> > > + ctx->twophase &= (ctx->twophase_opt_given || slot->data.two_phase);
> > > +
> > > + /* Mark slot to allow two_phase decoding if not already marked */
> > > + if (ctx->twophase && !slot->data.two_phase)
> > > + {
> > > + slot->data.two_phase = true;
> > > + ReplicationSlotMarkDirty();
> > > + ReplicationSlotSave();
> > > + }
> > >
> > > Why do we need to change this during CreateInitDecodingContext which
> > > is called at create_slot time? At that time, we don't need to consider
> > > any options and there is no need to toggle slot's two_phase value.
> > >
> > >
> >
> > TODO
>
> As part of the recent changes, we do turn on two_phase at create_slot time 
> when
> the subscription is created with (copy_data = false, two_phase = on).
> So, this code is required.
>

But in that case, won't we deal it with the value passed in
CreateReplicationSlotCmd. It should be enabled after we call
ReplicationSlotCreate.

-- 
With Regards,
Amit Kapila.




Re: [HACKERS] logical decoding of two-phase transactions

2021-06-08 Thread Greg Nancarrow
On Tue, Jun 8, 2021 at 4:12 PM Peter Smith  wrote:
>
> Please find attached the latest patch set v83*
>

Some feedback for the v83 patch set:

v83-0001:

(1) doc/src/sgml/protocol.sgml

(i) Remove extra space:

BEFORE:
+ The transaction will be  decoded and transmitted at
AFTER:
+ The transaction will be decoded and transmitted at

(ii)
BEFORE:
+   contains Stream Commit or Stream Abort message.
AFTER:
+   contains a Stream Commit or Stream Abort message.

(iii)
BEFORE:
+The LSN of the commit prepared.
AFTER:
+The LSN of the commit prepared transaction.

(iv) Should documentation say "prepared transaction" as opposed to
"prepare transaction" ???

BEFORE:
+The end LSN of the prepare transaction.
AFTER:
+The end LSN of the prepared transaction.


(2) doc/src/sgml/ref/create_subscription.sgml

(i)
BEFORE:
+  The streaming option cannot be used along with
+  two_phase option.
AFTER:
+  The streaming option cannot be used with the
+  two_phase option.


(3) doc/src/sgml/ref/create_subscription.sgml

(i)
BEFORE:
+  prepared on publisher is decoded as normal transaction at commit.
AFTER:
+  prepared on the publisher is decoded as a normal
transaction at commit.

(ii)
BEFORE:
+  The two_phase option cannot be used along with
+  streaming option.
AFTER:
+  The two_phase option cannot be used with the
+  streaming option.


(4) src/backend/access/transam/twophase.c

(i)
BEFORE:
+ * Check if the prepared transaction with the given GID, lsn and timestamp
+ * is around.
AFTER:
+ * Check if the prepared transaction with the given GID, lsn and timestamp
+ * exists.


(5) src/backend/access/transam/twophase.c

Question:

Is:

+ * do this optimization if we encounter many collisions in GID

meant to be:

+ * do this optimization if we encounter any collisions in GID

???

(6) src/backend/replication/logical/decode.c

Grammar:

BEFORE:
+ * distributed 2PC. This can be avoided by disallowing to
+ * prepare transactions that have locked [user] catalog tables
+ * exclusively but as of now we ask users not to do such
+ * operation.
AFTER:
+ * distributed 2PC. This can be avoided by disallowing
+ * prepared transactions that have locked [user] catalog tables
+ * exclusively but as of now we ask users not to do such an
+ * operation.


(7) src/backend/replication/logical/logical.c

>From the comment above it, it's not clear if the "&=" in the following
line is intentional:

+ ctx->twophase &= (ctx->twophase_opt_given || slot->data.two_phase);

Also, the boolean conditions tested are in the reverse order of what
is mentioned in that comment.
Based on the comment, I would expect the following code:

+ ctx->twophase = (slot->data.two_phase || ctx->twophase_opt_given);

Please check it, and maybe update the comment if "&=" is really intended.

There are TWO places where this same code is used.

(8) src/backend/replication/logical/tablesync.c

In the following code, "has_subrels" should be a bool, not an int.

+static bool
+FetchTableStates(bool *started_tx)
+{
+ static int has_subrels = false;


(9) src/backend/replication/logical/worker.c

Mixed current/past tense:

BEFORE:
+ * was still busy (see the condition of should_apply_changes_for_rel). The
AFTER:
+ * is still busy (see the condition of should_apply_changes_for_rel). The

(10)

2 places:

BEFORE:
+ /* there is no transaction when COMMIT PREPARED is called */
AFTER:
+ /* There is no transaction when COMMIT PREPARED is called */


v83-0002:

1) doc/src/sgml/protocol.sgml

BEFORE:
+   contains Stream Prepare or Stream Commit or Stream Abort message.
AFTER:
+   contains a Stream Prepare or Stream Commit or Stream Abort message.


v83-0003:

1) src/backend/replication/pgoutput/pgoutput.c

i) In pgoutput_commit_txn(), the following code that pfree()s a
pointer in a struct, without then NULLing it out, seems dangerous to
me (because what is to stop other code, either now or in the future,
from subsequently referencing that freed data or perhaps trying to
pfree() again?):

+ PGOutputTxnData *data = (PGOutputTxnData *) txn->output_plugin_private;
+ boolskip;
+
+ Assert(data);
+ skip = !data->sent_begin_txn;
+ pfree(data);


I suggest adding the following line of code after the pfree():
+ txn->output_plugin_private = NULL;


ii) In pgoutput_commit_prepared_txn(), there's the same type of code:

+ if (data)
+ {
+ bool skip = !data->sent_begin_txn;
+ pfree(data);
+ if (skip)
+ return;
+ }

I suggest adding the following line after the pfree() above:

+ txn->output_plugin_private = NULL;


iii) Again, same thing in pgoutput_rollback_prepared_txn():

I suggest adding the following line after the pfree() above:

+ txn->output_plugin_private = NULL;


Regards,
Greg Nancarrow
Fujitsu Australia




Re: [HACKERS] logical decoding of two-phase transactions

2021-06-08 Thread Peter Smith
On Thu, Jun 3, 2021 at 7:56 PM Amit Kapila  wrote:
>
> On Wed, Jun 2, 2021 at 4:34 AM Peter Smith  wrote:
> >
> > Please find attached the latest patch set v82*
> >
>
> Few comments on 0001:
> 
> 1.
> + /*
> + * BeginTransactionBlock is necessary to balance the EndTransactionBlock
> + * called within the PrepareTransactionBlock below.
> + */
> + BeginTransactionBlock();
> + CommitTransactionCommand();
> +
> + /*
> + * Update origin state so we can restart streaming from correct position
> + * in case of crash.
> + */
> + replorigin_session_origin_lsn = prepare_data.end_lsn;
> + replorigin_session_origin_timestamp = prepare_data.prepare_time;
> +
> + PrepareTransactionBlock(gid);
> + CommitTransactionCommand();
>
> Here, the call to CommitTransactionCommand() twice looks a bit odd.
> Before the first call, can we write a comment like "This is to
> complete the Begin command started by the previous call"?
>

Fixed in v83-0001 and v83-0002

> 2.
> @@ -85,11 +85,16 @@ typedef struct LogicalDecodingContext
>   bool streaming;
>
>   /*
> - * Does the output plugin support two-phase decoding, and is it enabled?
> + * Does the output plugin support two-phase decoding.
>   */
>   bool twophase;
>
>   /*
> + * Is two-phase option given by output plugin?
> + */
> + bool twophase_opt_given;
> +
> + /*
>   * State for writing output.
>
> I think we can write few comments as to why we need a separate
> twophase parameter here? The description of twophase_opt_given can be
> changed to: "Is two-phase option given by output plugin? This is to
> allow output plugins to enable two_phase at the start of streaming. We
> can't rely on twophase parameter that tells whether the plugin
> provides all the necessary two_phase APIs for this purpose." Feel free
> to add more to it.
>

TODO

> 3.
> @@ -432,10 +432,19 @@ CreateInitDecodingContext(const char *plugin,
>   MemoryContextSwitchTo(old_context);
>
>   /*
> - * We allow decoding of prepared transactions iff the two_phase option is
> - * enabled at the time of slot creation.
> + * We allow decoding of prepared transactions when the two_phase is
> + * enabled at the time of slot creation, or when the two_phase option is
> + * given at the streaming start.
>   */
> - ctx->twophase &= MyReplicationSlot->data.two_phase;
> + ctx->twophase &= (ctx->twophase_opt_given || slot->data.two_phase);
> +
> + /* Mark slot to allow two_phase decoding if not already marked */
> + if (ctx->twophase && !slot->data.two_phase)
> + {
> + slot->data.two_phase = true;
> + ReplicationSlotMarkDirty();
> + ReplicationSlotSave();
> + }
>
> Why do we need to change this during CreateInitDecodingContext which
> is called at create_slot time? At that time, we don't need to consider
> any options and there is no need to toggle slot's two_phase value.
>
>

TODO

> 4.
> - /* Binary mode and streaming are only supported in v14 and higher */
> + /*
> + * Binary, streaming, and two_phase are only supported in v14 and
> + * higher
> + */
>
> We can say v15 for two_phase.
>

Fixed in v83-0001

> 5.
> -#define LOGICALREP_PROTO_MAX_VERSION_NUM LOGICALREP_PROTO_STREAM_VERSION_NUM
> +#define LOGICALREP_PROTO_TWOPHASE_VERSION_NUM 3
> +#define LOGICALREP_PROTO_MAX_VERSION_NUM 3
>
> Isn't it better to define LOGICALREP_PROTO_MAX_VERSION_NUM as
> LOGICALREP_PROTO_TWOPHASE_VERSION_NUM instead of specifying directly
> the number?
>

Fixed in v83-0001

> 6.
> +/* Commit (and abort) information */
>  typedef struct LogicalRepCommitData
>  {
>   XLogRecPtr commit_lsn;
> @@ -122,6 +132,48 @@ typedef struct LogicalRepCommitData
>   TimestampTz committime;
>  } LogicalRepCommitData;
>
> Is there a reason for the above comment addition? If so, how is it
> related to this patch?
>

The LogicalRepCommitData is used by the 0002 patch and during
implementation it was not clear what was this struct, so I added the
missing comment (all other nearby typedefs except this one were
commented). But it is not strictly related to anything in patch 0001
so I have moved this change into the v83-0002 patch.


> 7.
> +++ b/src/test/subscription/t/021_twophase.pl
> @@ -0,0 +1,299 @@
> +# logical replication of 2PC test
> +use strict;
> +use warnings;
> +use PostgresNode;
> +use TestLib;
>
> In the nearby test files, we have Copyright notice like "# Copyright
> (c) 2021, PostgreSQL Global Development Group". We should add one to
> the new test files in this patch as well.
>

Fixed in v83-0001 and v83-0002

> 8.
> +# Also wait for two-phase to be enabled
> +my $twophase_query =
> + "SELECT count(1) = 0 FROM pg_subscription WHERE subtwophasestate NOT
> IN ('e');";
> +$node_subscriber->poll_query_until('postgres', $twophase_query)
> +  or die "Timed out while waiting for subscriber to enable twophase";
>
> Isn't it better to write this query as: "SELECT count(1) = 1 FROM
> pg_subscription WHERE subtwophasestate ='e';"? It looks a bit odd to
> use the NOT IN operator here. Similarly, change the same query used at
> another place in the 

Re: [HACKERS] logical decoding of two-phase transactions

2021-06-07 Thread Greg Nancarrow
On Wed, Jun 2, 2021 at 9:04 AM Peter Smith  wrote:
>
> Please find attached the latest patch set v82*
>

Some suggested changes to the 0001 patch comments (and note also the
typo "doumentation"):
diff of before and after follows:


8c8
< built-in logical replication, we need to do the below things:
---
> built-in logical replication, we need to do the following things:
16,17c16,17
< * Add a new SUBSCRIPTION option "two_phase" to allow users to enable it.
< We enable the two_phase once the initial data sync is over.
---
> * Add a new SUBSCRIPTION option "two_phase" to allow users to enable two-phase
> transactions. We enable the two_phase once the initial data sync is over.
23c23
< * Adds new subscription TAP tests, and new subscription.sql regression tests.
---
> * Add new subscription TAP tests, and new subscription.sql regression tests.
25c25
< * Updates PG doumentation.
---
> * Update PG documentation.
33c33
< * Prepare API for in-progress transactions is not supported.
---
> * Prepare API for in-progress transactions.


Regards,
Greg Nancarrow
Fujitsu Australia




Re: [HACKERS] logical decoding of two-phase transactions

2021-06-03 Thread Amit Kapila
On Wed, Jun 2, 2021 at 4:34 AM Peter Smith  wrote:
>
> Please find attached the latest patch set v82*
>

Few comments on 0001:

1.
+ /*
+ * BeginTransactionBlock is necessary to balance the EndTransactionBlock
+ * called within the PrepareTransactionBlock below.
+ */
+ BeginTransactionBlock();
+ CommitTransactionCommand();
+
+ /*
+ * Update origin state so we can restart streaming from correct position
+ * in case of crash.
+ */
+ replorigin_session_origin_lsn = prepare_data.end_lsn;
+ replorigin_session_origin_timestamp = prepare_data.prepare_time;
+
+ PrepareTransactionBlock(gid);
+ CommitTransactionCommand();

Here, the call to CommitTransactionCommand() twice looks a bit odd.
Before the first call, can we write a comment like "This is to
complete the Begin command started by the previous call"?

2.
@@ -85,11 +85,16 @@ typedef struct LogicalDecodingContext
  bool streaming;

  /*
- * Does the output plugin support two-phase decoding, and is it enabled?
+ * Does the output plugin support two-phase decoding.
  */
  bool twophase;

  /*
+ * Is two-phase option given by output plugin?
+ */
+ bool twophase_opt_given;
+
+ /*
  * State for writing output.

I think we can write few comments as to why we need a separate
twophase parameter here? The description of twophase_opt_given can be
changed to: "Is two-phase option given by output plugin? This is to
allow output plugins to enable two_phase at the start of streaming. We
can't rely on twophase parameter that tells whether the plugin
provides all the necessary two_phase APIs for this purpose." Feel free
to add more to it.

3.
@@ -432,10 +432,19 @@ CreateInitDecodingContext(const char *plugin,
  MemoryContextSwitchTo(old_context);

  /*
- * We allow decoding of prepared transactions iff the two_phase option is
- * enabled at the time of slot creation.
+ * We allow decoding of prepared transactions when the two_phase is
+ * enabled at the time of slot creation, or when the two_phase option is
+ * given at the streaming start.
  */
- ctx->twophase &= MyReplicationSlot->data.two_phase;
+ ctx->twophase &= (ctx->twophase_opt_given || slot->data.two_phase);
+
+ /* Mark slot to allow two_phase decoding if not already marked */
+ if (ctx->twophase && !slot->data.two_phase)
+ {
+ slot->data.two_phase = true;
+ ReplicationSlotMarkDirty();
+ ReplicationSlotSave();
+ }

Why do we need to change this during CreateInitDecodingContext which
is called at create_slot time? At that time, we don't need to consider
any options and there is no need to toggle slot's two_phase value.


4.
- /* Binary mode and streaming are only supported in v14 and higher */
+ /*
+ * Binary, streaming, and two_phase are only supported in v14 and
+ * higher
+ */

We can say v15 for two_phase.

5.
-#define LOGICALREP_PROTO_MAX_VERSION_NUM LOGICALREP_PROTO_STREAM_VERSION_NUM
+#define LOGICALREP_PROTO_TWOPHASE_VERSION_NUM 3
+#define LOGICALREP_PROTO_MAX_VERSION_NUM 3

Isn't it better to define LOGICALREP_PROTO_MAX_VERSION_NUM as
LOGICALREP_PROTO_TWOPHASE_VERSION_NUM instead of specifying directly
the number?

6.
+/* Commit (and abort) information */
 typedef struct LogicalRepCommitData
 {
  XLogRecPtr commit_lsn;
@@ -122,6 +132,48 @@ typedef struct LogicalRepCommitData
  TimestampTz committime;
 } LogicalRepCommitData;

Is there a reason for the above comment addition? If so, how is it
related to this patch?

7.
+++ b/src/test/subscription/t/021_twophase.pl
@@ -0,0 +1,299 @@
+# logical replication of 2PC test
+use strict;
+use warnings;
+use PostgresNode;
+use TestLib;

In the nearby test files, we have Copyright notice like "# Copyright
(c) 2021, PostgreSQL Global Development Group". We should add one to
the new test files in this patch as well.

8.
+# Also wait for two-phase to be enabled
+my $twophase_query =
+ "SELECT count(1) = 0 FROM pg_subscription WHERE subtwophasestate NOT
IN ('e');";
+$node_subscriber->poll_query_until('postgres', $twophase_query)
+  or die "Timed out while waiting for subscriber to enable twophase";

Isn't it better to write this query as: "SELECT count(1) = 1 FROM
pg_subscription WHERE subtwophasestate ='e';"? It looks a bit odd to
use the NOT IN operator here. Similarly, change the same query used at
another place in the patch.

9.
+# check that transaction is in prepared state on subscriber
+my $result = $node_subscriber->safe_psql('postgres', "SELECT count(*)
FROM pg_prepared_xacts;");
+is($result, qq(1), 'transaction is prepared on subscriber');
+
+# Wait for the statistics to be updated
+$node_publisher->poll_query_until(
+ 'postgres', qq[
+ SELECT count(slot_name) >= 1 FROM pg_stat_replication_slots
+ WHERE slot_name = 'tap_sub'
+ AND total_txns > 0 AND total_bytes > 0;
+]) or die "Timed out while waiting for statistics to be updated";

I don't see the need to check for stats in this test. If we really
want to test stats then we can add a separate test in
contrib\test_decoding\sql\stats but I suggest leaving it. Please do
the same for other stats 

Re: [HACKERS] logical decoding of two-phase transactions

2021-06-01 Thread Peter Smith
On Mon, May 31, 2021 at 9:16 PM Amit Kapila  wrote:
>
> On Fri, May 28, 2021 at 11:55 AM Amit Kapila  wrote:
> >
> > One minor comment for 0001.
> > * Special case: if when tables were specified but copy_data is
> > + * false then it is safe to enable two_phase up-front because
> > + * those tables are already initially READY state. Note, if
> > + * the subscription has no tables then enablement cannot be
> > + * done here - we must leave the twophase state as PENDING, to
> > + * allow ALTER SUBSCRIPTION ... REFRESH PUBLICATION to work.
> >
> > Can we slightly modify this comment as: "Note that if tables were
> > specified but copy_data is false then it is safe to enable two_phase
> > up-front because those tables are already initially READY state. When
> > the subscription has no tables, we leave the twophase state as
> > PENDING, to allow ALTER SUBSCRIPTION ... REFRESH PUBLICATION to work."
> >
> > Also, I don't see any test after you enable this special case. Is it
> > covered by existing tests, if not then let's try to add a test for
> > this?
> >
>
> I see that Ajin's latest patch has addressed the other comments except
> for the above test case suggestion.

Yes, this is a known pending task.

> I have again reviewed the first
> patch and have some comments.
>
> Comments on v81-0001-Add-support-for-prepared-transactions-to-built-i
> 
> 1.
> 
> The logical replication solution that builds distributed two
> phase commit
> using this feature can deadlock if the prepared transaction has locked
> -   [user] catalog tables exclusively. They need to inform users to not 
> have
> -   locks on catalog tables (via explicit LOCK
> command) in
> -   such transactions.
> +   [user] catalog tables exclusively. To avoid this users must refrain 
> from
> +   having locks on catalog tables (via explicit
> LOCK command)
> +   in such transactions.
>
>
> This change doesn't belong to this patch. I see the proposed text
> could be considered as an improvement but still we can do this
> separately. We are already trying to improve things in this regard in
> the thread [1], so you can propose this change there.
>

OK. This change has been removed in v82, and a patch posted to other
thread here [1]

> 2.
> +
> +Byte1('K')
> +
> +Identifies the message as the commit of a two-phase
> transaction message.
> +
> +
> +
> +
> +Int8
> +
> +Flags; currently unused (must be 0).
> +
> +
> +
> +
> +Int64
> +
> +The LSN of the commit.
> +
> +
> +
> +
> +Int64
> +
> +The end LSN of the commit transaction.
> +
> +
>
> Can we change the description of LSN's as "The LSN of the commit
> prepared." and "The end LSN of the commit prepared transaction."
> respectively? This will make their description different from regular
> commit and I think that defines them better.
>
> 3.
> +
> +Int64
> +
> +The end LSN of the rollback transaction.
> +
> +
>
> Similar to above, can we change the description here as: "The end LSN
> of the rollback prepared transaction."?
>
> 4.
> + * The exception to this restriction is when copy_data =
> + * false, because when copy_data is false the tablesync will
> + * start already in READY state and will exit directly without
> + * doing anything which could interfere with the apply
> + * worker's message handling.
> + *
> + * For more details see comments atop worker.c.
> + */
> + if (sub->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED && copy_data)
> + ereport(ERROR,
> + (errcode(ERRCODE_SYNTAX_ERROR),
> + errmsg("ALTER SUBSCRIPTION ... REFRESH with copy_data is not allowed
> when two_phase is enabled"),
> + errhint("Use ALTER SUBSCRIPTION ... REFRESH with copy_data = false"
> + ", or use DROP/CREATE SUBSCRIPTION.")));
>
> The above comment is a bit unclear because it seems you are saying
> there is some problem even when copy_data is false. Are you missing
> 'not' after 'could' in the comment?
>
> 5.
>  XXX Now, this can even lead to a deadlock if the prepare
>   * transaction is waiting to get it logically replicated for
> - * distributed 2PC. Currently, we don't have an in-core
> - * implementation of prepares for distributed 2PC but some
> - * out-of-core logical replication solution can have such an
> - * implementation. They need to inform users to not have locks
> - * on catalog tables in such transactions.
> + * distributed 2PC. This can be avoided by disallowing to
> + * prepare transactions that have locked [user] catalog tables
> + * exclusively.
>
> Can we slightly modify this part of the comment as: "This can be
> avoided by disallowing to prepare transactions that have locked [user]
> catalog tables exclusively but as of now we ask users not to do such
> operation"?
>
> 6.
> +AllTablesyncsReady(void)
> +{
> + bool found_busy = false;
> + bool started_tx = false;
> + bool has_subrels = false;
> +
> 

Re: [HACKERS] logical decoding of two-phase transactions

2021-05-31 Thread Amit Kapila
On Fri, May 28, 2021 at 11:55 AM Amit Kapila  wrote:
>
> One minor comment for 0001.
> * Special case: if when tables were specified but copy_data is
> + * false then it is safe to enable two_phase up-front because
> + * those tables are already initially READY state. Note, if
> + * the subscription has no tables then enablement cannot be
> + * done here - we must leave the twophase state as PENDING, to
> + * allow ALTER SUBSCRIPTION ... REFRESH PUBLICATION to work.
>
> Can we slightly modify this comment as: "Note that if tables were
> specified but copy_data is false then it is safe to enable two_phase
> up-front because those tables are already initially READY state. When
> the subscription has no tables, we leave the twophase state as
> PENDING, to allow ALTER SUBSCRIPTION ... REFRESH PUBLICATION to work."
>
> Also, I don't see any test after you enable this special case. Is it
> covered by existing tests, if not then let's try to add a test for
> this?
>

I see that Ajin's latest patch has addressed the other comments except
for the above test case suggestion. I have again reviewed the first
patch and have some comments.

Comments on v81-0001-Add-support-for-prepared-transactions-to-built-i

1.

The logical replication solution that builds distributed two
phase commit
using this feature can deadlock if the prepared transaction has locked
-   [user] catalog tables exclusively. They need to inform users to not have
-   locks on catalog tables (via explicit LOCK
command) in
-   such transactions.
+   [user] catalog tables exclusively. To avoid this users must refrain from
+   having locks on catalog tables (via explicit
LOCK command)
+   in such transactions.
   

This change doesn't belong to this patch. I see the proposed text
could be considered as an improvement but still we can do this
separately. We are already trying to improve things in this regard in
the thread [1], so you can propose this change there.

2.
+
+Byte1('K')
+
+Identifies the message as the commit of a two-phase
transaction message.
+
+
+
+
+Int8
+
+Flags; currently unused (must be 0).
+
+
+
+
+Int64
+
+The LSN of the commit.
+
+
+
+
+Int64
+
+The end LSN of the commit transaction.
+
+

Can we change the description of LSN's as "The LSN of the commit
prepared." and "The end LSN of the commit prepared transaction."
respectively? This will make their description different from regular
commit and I think that defines them better.

3.
+
+Int64
+
+The end LSN of the rollback transaction.
+
+

Similar to above, can we change the description here as: "The end LSN
of the rollback prepared transaction."?

4.
+ * The exception to this restriction is when copy_data =
+ * false, because when copy_data is false the tablesync will
+ * start already in READY state and will exit directly without
+ * doing anything which could interfere with the apply
+ * worker's message handling.
+ *
+ * For more details see comments atop worker.c.
+ */
+ if (sub->twophasestate == LOGICALREP_TWOPHASE_STATE_ENABLED && copy_data)
+ ereport(ERROR,
+ (errcode(ERRCODE_SYNTAX_ERROR),
+ errmsg("ALTER SUBSCRIPTION ... REFRESH with copy_data is not allowed
when two_phase is enabled"),
+ errhint("Use ALTER SUBSCRIPTION ... REFRESH with copy_data = false"
+ ", or use DROP/CREATE SUBSCRIPTION.")));

The above comment is a bit unclear because it seems you are saying
there is some problem even when copy_data is false. Are you missing
'not' after 'could' in the comment?

5.
 XXX Now, this can even lead to a deadlock if the prepare
  * transaction is waiting to get it logically replicated for
- * distributed 2PC. Currently, we don't have an in-core
- * implementation of prepares for distributed 2PC but some
- * out-of-core logical replication solution can have such an
- * implementation. They need to inform users to not have locks
- * on catalog tables in such transactions.
+ * distributed 2PC. This can be avoided by disallowing to
+ * prepare transactions that have locked [user] catalog tables
+ * exclusively.

Can we slightly modify this part of the comment as: "This can be
avoided by disallowing to prepare transactions that have locked [user]
catalog tables exclusively but as of now we ask users not to do such
operation"?

6.
+AllTablesyncsReady(void)
+{
+ bool found_busy = false;
+ bool started_tx = false;
+ bool has_subrels = false;
+
+ /* We need up-to-date sync state info for subscription tables here. */
+ has_subrels = FetchTableStates(_tx);
+
+ found_busy = list_length(table_states_not_ready) > 0;
+
+ if (started_tx)
+ {
+ CommitTransactionCommand();
+ pgstat_report_stat(false);
+ }
+
+ /*
+ * When there are no tables, then return false.
+ * When no tablesyncs are busy, then all are READY
+ */
+ return has_subrels && !found_busy;
+}

Do we really need found_busy variable in 

Re: [HACKERS] logical decoding of two-phase transactions

2021-05-28 Thread Amit Kapila
On Thu, May 27, 2021 at 8:05 AM Ajin Cherian  wrote:
>
> Thanks for confirmation. The problem seemed to be as you reported a
> table not closed when a transaction was committed.
> This seems to be because the function UpdateTwoPhaseState was
> committing a transaction inside the function when the caller of
> UpdateTwoPhaseState had
> a table open in CreateSubscription. This function was newly included
> in the CreateSubscription code, to handle the new use case of
> two_phase being enabled on
> create subscription if "copy_data = false". I don't think
> CreateSubscription required this to be inside a transaction and the
> committing of transaction
> was only meant for where this function was originally created to be
> used in the apply worker code (ApplyWorkerMain()).
> So, I removed the committing of the transaction from inside the
> function UpdateTwoPhaseState() and instead started and committed the
> transaction
>  prior to and after this function is invoked in the apply worker code.
>

You have made these changes in 0002 whereas they should be part of 0001.

One minor comment for 0001.
* Special case: if when tables were specified but copy_data is
+ * false then it is safe to enable two_phase up-front because
+ * those tables are already initially READY state. Note, if
+ * the subscription has no tables then enablement cannot be
+ * done here - we must leave the twophase state as PENDING, to
+ * allow ALTER SUBSCRIPTION ... REFRESH PUBLICATION to work.

Can we slightly modify this comment as: "Note that if tables were
specified but copy_data is false then it is safe to enable two_phase
up-front because those tables are already initially READY state. When
the subscription has no tables, we leave the twophase state as
PENDING, to allow ALTER SUBSCRIPTION ... REFRESH PUBLICATION to work."

Also, I don't see any test after you enable this special case. Is it
covered by existing tests, if not then let's try to add a test for
this?

-- 
With Regards,
Amit Kapila.




Re: [HACKERS] logical decoding of two-phase transactions

2021-05-28 Thread vignesh C
On Fri, May 28, 2021 at 9:14 AM Ajin Cherian  wrote:
>
> On Wed, May 26, 2021 at 6:53 PM vignesh C  wrote:
> >
> > On Tue, May 25, 2021 at 8:54 AM Ajin Cherian  wrote:
> > >
> > > On Fri, May 21, 2021 at 6:43 PM Peter Smith  wrote:
> > >
> > > > Fixed in v77-0001, v77-0002
> > >
> > > Attaching a new patch-set that rebases the patch, addresses review
> > > comments from Peter as well as a test failure reported by Tang. I've
> > > also added some new test case into patch-2 authored by Tang.
> >
> > Thanks for the updated patch, few comments:
> > 1)  Should "The end LSN of the prepare." be changed to "end LSN of the
> > prepare transaction."?
>
> No, this is the end LSN of the prepare. The prepare consists of multiple LSNs.
>
> > 2) Should the ";" be "," here?
> > +++ b/doc/src/sgml/catalogs.sgml
> > @@ -7639,6 +7639,18 @@ SCRAM-SHA-256$iteration
> > count:
> >
> >   
> >
> > +   subtwophasestate char
> > +  
> > +  
> > +   State code:
> > +   d = two_phase mode was not requested, so is 
> > disabled;
> > +   p = two_phase mode was requested, but is
> > pending enablement;
> > +   e = two_phase mode was requested, and is enabled.
> > +  
>
> no, I think the ";" is correct here, connecting multiple parts of the 
> sentence.
>
> >
> > 3) Should end_lsn be commit_end_lsn?
> > +   prepare_data->commit_end_lsn = pq_getmsgint64(in);
> > +   if (prepare_data->commit_end_lsn == InvalidXLogRecPtr)
> > elog(ERROR, "end_lsn is not set in commit prepared 
> > message");
> > +   prepare_data->prepare_time = pq_getmsgint64(in);
>
> Changed this.
>
> >
> > 4) This change is not required
> >
> > diff --git a/src/include/replication/pgoutput.h
> > b/src/include/replication/pgoutput.h
> > index 0dc460f..93c6731 100644
> > --- a/src/include/replication/pgoutput.h
> > +++ b/src/include/replication/pgoutput.h
> > @@ -29,5 +29,4 @@ typedef struct PGOutputData
> > boolmessages;
> > booltwo_phase;
> >  } PGOutputData;
> > -
>
> removed.
>
>
> >  #endif /* PGOUTPUT_H */
> >
> > 5) Will the worker receive commit prepared/rollback prepared as we
> > have skip logic to skip commit prepared / commit rollback in
> > pgoutput_rollback_prepared_txn and pgoutput_commit_prepared_txn:
> >
> > +* It is possible that we haven't received the prepare because
> > +* the transaction did not have any changes relevant to this
> > +* subscription and was essentially an empty prepare. In which case,
> > +* the walsender is optimized to drop the empty transaction and the
> > +* accompanying prepare. Silently ignore if we don't find the 
> > prepared
> > +* transaction.
> >  */
> > -   replorigin_session_origin_lsn = prepare_data.end_lsn;
> > -   replorigin_session_origin_timestamp = prepare_data.commit_time;
> > +   if (LookupGXact(gid, prepare_data.prepare_end_lsn,
> > +   prepare_data.prepare_time))
> > +   {
> >
> Commit prepared will be skipped if it happens in the same walsender's
> lifetime. But if the walsender restarts it no longer
> knows about the skipped prepare. In this case walsender will not skip
> the commit prepared. Hence, the logic for handling
> stray commit prepared in the apply worker.
>
>
> > 6) I'm not sure if we could add some tests for skip empty prepare
> > transactions, if possible add few tests.
>
> I've added a test case using pg_logical_slot_peek_binary_changes() for
> empty prepares
>  have a look.
>
> >
> > 7) We could add some debug level log messages for the transaction that
> > will be skipped.
>
> If this is for the test, I was able to add a test without debug messages.

The idea here is to include any debug logs which will help in
analyzing any bugs that we might get from an environment where debug
access might not be available.
Thanks for fixing the comments and posting an updated patch.

Regards,
Vignesh




Re: [HACKERS] logical decoding of two-phase transactions

2021-05-26 Thread Ajin Cherian
On Thu, May 27, 2021 at 11:20 AM tanghy.f...@fujitsu.com
 wrote:
>
> On Wed, May 26, 2021 10:13 PM Ajin Cherian  wrote:
> >
> > I've attached a patch that fixes this issue. Do test and confirm.
> >
>
> Thanks for your patch.
> I have tested and confirmed that the issue I reported has been fixed.

Thanks for confirmation. The problem seemed to be as you reported a
table not closed when a transaction was committed.
This seems to be because the function UpdateTwoPhaseState was
committing a transaction inside the function when the caller of
UpdateTwoPhaseState had
a table open in CreateSubscription. This function was newly included
in the CreateSubscription code, to handle the new use case of
two_phase being enabled on
create subscription if "copy_data = false". I don't think
CreateSubscription required this to be inside a transaction and the
committing of transaction
was only meant for where this function was originally created to be
used in the apply worker code (ApplyWorkerMain()).
So, I removed the committing of the transaction from inside the
function UpdateTwoPhaseState() and instead started and committed the
transaction
 prior to and after this function is invoked in the apply worker code.

regards,
Ajin Cherian
Fujitsu Australia




RE: [HACKERS] logical decoding of two-phase transactions

2021-05-26 Thread tanghy.f...@fujitsu.com
On Wed, May 26, 2021 10:13 PM Ajin Cherian  wrote:
> 
> I've attached a patch that fixes this issue. Do test and confirm.
> 

Thanks for your patch.
I have tested and confirmed that the issue I reported has been fixed.

Regards
Tang


Re: [HACKERS] logical decoding of two-phase transactions

2021-05-26 Thread vignesh C
On Tue, May 25, 2021 at 8:54 AM Ajin Cherian  wrote:
>
> On Fri, May 21, 2021 at 6:43 PM Peter Smith  wrote:
>
> > Fixed in v77-0001, v77-0002
>
> Attaching a new patch-set that rebases the patch, addresses review
> comments from Peter as well as a test failure reported by Tang. I've
> also added some new test case into patch-2 authored by Tang.

Thanks for the updated patch, few comments:
1)  Should "The end LSN of the prepare." be changed to "end LSN of the
prepare transaction."?

--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -7538,6 +7538,13 @@ are available since protocol version 3.
 
 Int64
 
+The end LSN of the prepare.
+
+
+
+
+Int64
+

2) Should the ";" be "," here?
+++ b/doc/src/sgml/catalogs.sgml
@@ -7639,6 +7639,18 @@ SCRAM-SHA-256$iteration
count:

  
   
+   subtwophasestate char
+  
+  
+   State code:
+   d = two_phase mode was not requested, so is disabled;
+   p = two_phase mode was requested, but is
pending enablement;
+   e = two_phase mode was requested, and is enabled.
+  

3) Should end_lsn be commit_end_lsn?
+   prepare_data->commit_end_lsn = pq_getmsgint64(in);
+   if (prepare_data->commit_end_lsn == InvalidXLogRecPtr)
elog(ERROR, "end_lsn is not set in commit prepared message");
+   prepare_data->prepare_time = pq_getmsgint64(in);

4) This change is not required

diff --git a/src/include/replication/pgoutput.h
b/src/include/replication/pgoutput.h
index 0dc460f..93c6731 100644
--- a/src/include/replication/pgoutput.h
+++ b/src/include/replication/pgoutput.h
@@ -29,5 +29,4 @@ typedef struct PGOutputData
boolmessages;
booltwo_phase;
 } PGOutputData;
-
 #endif /* PGOUTPUT_H */

5) Will the worker receive commit prepared/rollback prepared as we
have skip logic to skip commit prepared / commit rollback in
pgoutput_rollback_prepared_txn and pgoutput_commit_prepared_txn:

+* It is possible that we haven't received the prepare because
+* the transaction did not have any changes relevant to this
+* subscription and was essentially an empty prepare. In which case,
+* the walsender is optimized to drop the empty transaction and the
+* accompanying prepare. Silently ignore if we don't find the prepared
+* transaction.
 */
-   replorigin_session_origin_lsn = prepare_data.end_lsn;
-   replorigin_session_origin_timestamp = prepare_data.commit_time;
+   if (LookupGXact(gid, prepare_data.prepare_end_lsn,
+   prepare_data.prepare_time))
+   {

6) I'm not sure if we could add some tests for skip empty prepare
transactions, if possible add few tests.

7) We could add some debug level log messages for the transaction that
will be skipped.

Regards,
Vignesh




RE: [HACKERS] logical decoding of two-phase transactions

2021-05-25 Thread tanghy.f...@fujitsu.com
> > 13.
> > @@ -507,7 +558,16 @@ CreateSubscription(CreateSubscriptionStmt *stmt,
> > bool isTopLevel)
> >   {
> >   Assert(slotname);
> >
> > - walrcv_create_slot(wrconn, slotname, false,
> > + /*
> > + * Even if two_phase is set, don't create the slot with
> > + * two-phase enabled. Will enable it once all the tables are
> > + * synced and ready. This avoids race-conditions like prepared
> > + * transactions being skipped due to changes not being applied
> > + * due to checks in should_apply_changes_for_rel() when
> > + * tablesync for the corresponding tables are in progress. See
> > + * comments atop worker.c.
> > + */
> > + walrcv_create_slot(wrconn, slotname, false, false,
> >
> > Can't we enable two_phase if copy_data is false? Because in that case,
> > all relations will be in a READY state. If we do that then we should
> > also set two_phase state as 'enabled' during createsubscription. I
> > think we need to be careful to check that connect option is given and
> > copy_data is false before setting such a state. Now, I guess we may
> > not be able to optimize this to not set 'enabled' state when the
> > subscription has no rels.
> >
> 
> Fixed in v77-0001

I noticed this modification in v77-0001 and executed "CREATE SUBSCRIPTION ... 
WITH (two_phase = on, copy_data = false)", but it crashed.
-
postgres=# CREATE SUBSCRIPTION sub CONNECTION 'dbname=postgres' PUBLICATION pub 
WITH(two_phase = on, copy_data = false);
WARNING:  relcache reference leak: relation "pg_subscription" not closed
WARNING:  snapshot 0x34278d0 still active
NOTICE:  created replication slot "sub" on publisher
server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
The connection to the server was lost. Attempting reset: Failed.
!?>
-

There are two warnings and a segmentation fault in subscriber log:
-
2021-05-24 15:08:32.435 CST [2848572] WARNING:  relcache reference leak: 
relation "pg_subscription" not closed
2021-05-24 15:08:32.435 CST [2848572] WARNING:  snapshot 0x32ce8b0 still active
2021-05-24 15:08:33.012 CST [2848555] LOG:  server process (PID 2848572) was 
terminated by signal 11: Segmentation fault
2021-05-24 15:08:33.012 CST [2848555] DETAIL:  Failed process was running: 
CREATE SUBSCRIPTION sub CONNECTION 'dbname=postgres' PUBLICATION pub 
WITH(two_phase = on, copy_data = false);
-

The backtrace about segmentation fault is attached. It happened in table_close 
function, we got it because "CurrentResourceOwner" was NULL.

I think it was related with the first warning, which reported "relcache 
reference leak". The backtrace information is attached, too. When updating 
two-phase state in CreateSubscription function, it released resource owner and 
set "CurrentResourceOwner" as NULL in CommitTransaction function.

The second warning about "snapshot still active" was also happened in 
CommitTransaction function. It called AtEOXact_Snapshot function, checked 
leftover snapshots and reported the warning.
I debugged and found the snapshot was added in function PortalRunUtility by 
calling PushActiveSnapshot function, the address of "ActiveSnapshot" at this 
time was same as the address in warning.

In summary, when creating subscription with two_phase = on and copy_data = 
false, it calls UpdateTwoPhaseState function in CreateSubscription function to 
set two_phase state as 'enabled', and it checked and released relcache and 
snapshot too early so the NG happened. I think some change should be made to 
avoid it. Thought?

FYI
 I also tested the new released V78* at [1], the above NG still exists.
[1] 
https://www.postgresql.org/message-id/CAFPTHDab56twVmC%2B0a%3DRNcRw4KuyFdqzW0JAcvJdS63n_fRnOQ%40mail.gmail.com

Regards
Tang

#0  ResourceArrayRemove (resarr=resarr@entry=0x80, value=value@entry=46738512) 
at resowner.c:309
#1  0x00e46e96 in ResourceOwnerForgetRelationRef (owner=0x0, 
rel=rel@entry=0x2c92c50) at resowner.c:1127
#2  0x00dcdfb9 in RelationDecrementReferenceCount 
(rel=rel@entry=0x2c92c50) at relcache.c:2083
#3  0x00dcdff0 in RelationClose (relation=relation@entry=0x2c92c50) at 
relcache.c:2101
#4  0x004a7a50 in relation_close (relation=relation@entry=0x2c92c50, 
lockmode=lockmode@entry=3) at relation.c:213
#5  0x005bf6df in table_close (relation=relation@entry=0x2c92c50, 
lockmode=lockmode@entry=3) at table.c:169
#6  0x007c6687 in CreateSubscription (stmt=stmt@entry=0x2c30338, 
isTopLevel=isTopLevel@entry=true) at subscriptioncmds.c:590
#7  0x00bae4c8 in ProcessUtilitySlow (pstate=pstate@entry=0x2c52a40, 
pstmt=pstmt@entry=0x2c306a8,
queryString=queryString@entry=0x2c2f610 "create subscription sub connection 
'dbname=postgres' publication pub with(two_phase=on, copy_data = false);",
context=context@entry=PROCESS_UTILITY_TOPLEVEL, params=params@entry=0x0, 
queryEnv=queryEnv@entry=0x0, dest=0x2c30798, 

Re: [HACKERS] logical decoding of two-phase transactions

2021-05-21 Thread Peter Smith
On Tue, May 18, 2021 at 9:32 PM Amit Kapila  wrote:
>
> On Thu, May 13, 2021 at 3:20 PM Peter Smith  wrote:
> >
> > Please find attached the latest patch set v75*
> >
>
> Review comments for v75-0001-Add-support-for-prepared-transactions-to-built-i:
> ===
> 1.
> -   CREATE_REPLICATION_SLOT  class="parameter">slot_name [
> TEMPORARY ] { PHYSICAL [
> RESERVE_WAL ] | LOGICAL
> output_plugin [
> EXPORT_SNAPSHOT |
> NOEXPORT_SNAPSHOT | USE_SNAPSHOT
> ] }
> +   CREATE_REPLICATION_SLOT  class="parameter">slot_name [
> TEMPORARY ] [ TWO_PHASE ] {
> PHYSICAL [ RESERVE_WAL ] |
> LOGICAL  class="parameter">output_plugin [
> EXPORT_SNAPSHOT |
> NOEXPORT_SNAPSHOT | USE_SNAPSHOT
> ] }
>
>
> Can we do some testing of the code related to this in some way? One
> random idea could be to change the current subscriber-side code just
> for testing purposes to see if this works. Can we enhance and use
> pg_recvlogical to test this? It is possible that if you address
> comment number 13 below, this can be tested with Create Subscription
> command.
>

TODO

> 2.
> -   belong to the same transaction. It also sends changes of large in-progress
> -   transactions between a pair of Stream Start and Stream Stop messages. The
> -   last stream of such a transaction contains Stream Commit or Stream Abort
> -   message.
> +   belong to the same transaction. Similarly, all messages between a pair of
> +   Begin Prepare and Commit Prepared messages belong to the same transaction.
>
> I think here we need to write Prepare instead of Commit Prepared
> because Commit Prepared for a transaction can come at a later point of
> time and all the messages in-between won't belong to the same
> transaction.
>

Fixed in v77-0001

> 3.
> +
> +
> +
> +The following messages (Begin Prepare, Prepare, Commit Prepared,
> Rollback Prepared)
> +are available since protocol version 3.
> +
>
> I am not sure here marker like "TWO_PHASE Messages" is required. We
> don't have any such marker for streaming messages.
>

Fixed in v77-0001

> 4.
> +
> +Int64
> +
> +Timestamp of the prepare transaction.
>
> Isn't it better to write this description as "Prepare timestamp of the
> transaction" to match with the similar description of Commit
> timestamp. Also, there are similar occurances in the patch at other
> places, change those as well.
>

Fixed in v77-0001, v77-0002

> 5.
> +Begin Prepare
> +
> +
> ...
> +
> +Int32
> +
> +Xid of the subtransaction (will be same as xid of the
> transaction for top-level
> +transactions).
>
> The above description seems wrong to me. It should be Xid of the
> transaction as we won't receive Xid of subtransaction in Begin
> message. The same applies to the prepare/commit prepared/rollback
> prepared transaction messages as well, so change that as well
> accordingly.
>

Fixed in v77-0001, v77-0002

> 6.
> +Byte1('P')
> +
> +Identifies this message as a two-phase prepare
> transaction message.
> +
>
> In all the similar messages, we are using "Identifies the message as
> ...". I feel it is better to be consistent in this and similar
> messages in the patch.
>

Fixed in v77-0001, v77-0002

> 7.
> +
> +
> +Rollback Prepared
> +
> ..
> +
> +Int64
> +
> +The LSN of the prepare.
> +
>
> This should be end LSN of the prepared transaction.
>

Fixed in v77-0001

> 8.
> +bool
> +LookupGXact(const char *gid, XLogRecPtr prepare_end_lsn,
> + TimestampTz origin_prepare_timestamp)
> ..
> ..
> + /*
> + * We are neither expecting the collisions of GXACTs (same gid)
> + * between publisher and subscribers nor the apply worker restarts
> + * after prepared xacts,
>
> The second part of the comment ".. nor the apply worker restarts after
> prepared xacts .." is no longer true after commit 8bdb1332eb[1]. So,
> we can remove it.
>

Fixed in v77-0001

> 9.
> + /*
> + * Does the subscription have tables?
> + *
> + * If there were not-READY relations found then we know it does. But if
> + * table_state_no_ready was empty we still need to check again to see
> + * if there are 0 tables.
> + */
> + has_subrels = (list_length(table_states_not_ready) > 0) ||
>
> Typo in comments. /table_state_no_ready/table_state_not_ready
>

Fixed in v77-0001

> 10.
> + if (!twophase)
> + ereport(ERROR,
> + (errcode(ERRCODE_SYNTAX_ERROR),
> + errmsg("unrecognized subscription parameter: \"%s\"", defel->defname)));
>
> errmsg is not aligned properly. Can we make the error message clear,
> something like: "cannot change two_phase option"
>

Fixed in v77-0001.

I fixed the alignment, but did not modify the message text.This
message was already changed in v74 to make it more consistent with
similar errors. Please see Vignesh feedback  [1] comment #1.

> 11.
> @@ -69,7 +69,8 @@ parse_subscription_options(List *options,
>  char **synchronous_commit,
>  bool *refresh,
>  bool *binary_given, bool *binary,
> -bool 

Re: [HACKERS] logical decoding of two-phase transactions

2021-05-18 Thread Amit Kapila
On Thu, May 13, 2021 at 3:20 PM Peter Smith  wrote:
>
> Please find attached the latest patch set v75*
>

Review comments for v75-0001-Add-support-for-prepared-transactions-to-built-i:
===
1.
-   CREATE_REPLICATION_SLOT slot_name [
TEMPORARY ] { PHYSICAL [
RESERVE_WAL ] | LOGICAL
output_plugin [
EXPORT_SNAPSHOT |
NOEXPORT_SNAPSHOT | USE_SNAPSHOT
] }
+   CREATE_REPLICATION_SLOT slot_name [
TEMPORARY ] [ TWO_PHASE ] {
PHYSICAL [ RESERVE_WAL ] |
LOGICAL output_plugin [
EXPORT_SNAPSHOT |
NOEXPORT_SNAPSHOT | USE_SNAPSHOT
] }


Can we do some testing of the code related to this in some way? One
random idea could be to change the current subscriber-side code just
for testing purposes to see if this works. Can we enhance and use
pg_recvlogical to test this? It is possible that if you address
comment number 13 below, this can be tested with Create Subscription
command.

2.
-   belong to the same transaction. It also sends changes of large in-progress
-   transactions between a pair of Stream Start and Stream Stop messages. The
-   last stream of such a transaction contains Stream Commit or Stream Abort
-   message.
+   belong to the same transaction. Similarly, all messages between a pair of
+   Begin Prepare and Commit Prepared messages belong to the same transaction.

I think here we need to write Prepare instead of Commit Prepared
because Commit Prepared for a transaction can come at a later point of
time and all the messages in-between won't belong to the same
transaction.

3.
+
+
+
+The following messages (Begin Prepare, Prepare, Commit Prepared,
Rollback Prepared)
+are available since protocol version 3.
+

I am not sure here marker like "TWO_PHASE Messages" is required. We
don't have any such marker for streaming messages.

4.
+
+Int64
+
+Timestamp of the prepare transaction.

Isn't it better to write this description as "Prepare timestamp of the
transaction" to match with the similar description of Commit
timestamp. Also, there are similar occurances in the patch at other
places, change those as well.

5.
+Begin Prepare
+
+
...
+
+Int32
+
+Xid of the subtransaction (will be same as xid of the
transaction for top-level
+transactions).

The above description seems wrong to me. It should be Xid of the
transaction as we won't receive Xid of subtransaction in Begin
message. The same applies to the prepare/commit prepared/rollback
prepared transaction messages as well, so change that as well
accordingly.

6.
+Byte1('P')
+
+Identifies this message as a two-phase prepare
transaction message.
+

In all the similar messages, we are using "Identifies the message as
...". I feel it is better to be consistent in this and similar
messages in the patch.

7.
+
+
+Rollback Prepared
+
..
+
+Int64
+
+The LSN of the prepare.
+

This should be end LSN of the prepared transaction.

8.
+bool
+LookupGXact(const char *gid, XLogRecPtr prepare_end_lsn,
+ TimestampTz origin_prepare_timestamp)
..
..
+ /*
+ * We are neither expecting the collisions of GXACTs (same gid)
+ * between publisher and subscribers nor the apply worker restarts
+ * after prepared xacts,

The second part of the comment ".. nor the apply worker restarts after
prepared xacts .." is no longer true after commit 8bdb1332eb[1]. So,
we can remove it.

9.
+ /*
+ * Does the subscription have tables?
+ *
+ * If there were not-READY relations found then we know it does. But if
+ * table_state_no_ready was empty we still need to check again to see
+ * if there are 0 tables.
+ */
+ has_subrels = (list_length(table_states_not_ready) > 0) ||

Typo in comments. /table_state_no_ready/table_state_not_ready

10.
+ if (!twophase)
+ ereport(ERROR,
+ (errcode(ERRCODE_SYNTAX_ERROR),
+ errmsg("unrecognized subscription parameter: \"%s\"", defel->defname)));

errmsg is not aligned properly. Can we make the error message clear,
something like: "cannot change two_phase option"

11.
@@ -69,7 +69,8 @@ parse_subscription_options(List *options,
 char **synchronous_commit,
 bool *refresh,
 bool *binary_given, bool *binary,
-bool *streaming_given, bool *streaming)
+bool *streaming_given, bool *streaming,
+bool *twophase_given, bool *twophase)

This function already has 14 parameters and this patch adds 2 new
ones. Isn't it better to have a struct (ParseSubOptions) for these
parameters? I think that might lead to some code churn but we can have
that as a separate patch on top of which we can create two_pc patch.

12.
* The subscription two_phase commit implementation requires
+ * that replication has passed the initial table
+ * synchronization phase before the two_phase becomes properly
+ * enabled.

Can we slightly modify the starting of this sentence as:"The
subscription option 'two_phase' requires that ..."

13.
@@ -507,7 +558,16 @@ CreateSubscription(CreateSubscriptionStmt *stmt,
bool isTopLevel)
  {
  

RE: [HACKERS] logical decoding of two-phase transactions

2021-05-18 Thread tanghy.f...@fujitsu.com
Hi Ajin

>The above patch had some changes missing which resulted in some tap
>tests failing. Sending an updated patchset. Keeping the patchset
>version the same.

Thanks for your patch.  I see a problem about Segmentation fault when using it. 
Please take a look at this.
The steps to reproduce the problem are as follows.

--publisher--
create table test (a int primary key, b varchar);
create publication pub for table test;

--subscriber--
create table test (a int primary key, b varchar);
create subscription sub connection 'dbname=postgres' publication pub 
with(two_phase=on);

Then, I prepare, commit, rollback transactions and TRUNCATE table in a sql as 
follows:
-
BEGIN;
INSERT INTO test SELECT i, md5(i::text) FROM generate_series(1, 1) s(i);
PREPARE TRANSACTION 't1';
COMMIT PREPARED 't1';

BEGIN;
INSERT INTO test SELECT i, md5(i::text) FROM generate_series(10001, 2) s(i);
PREPARE TRANSACTION 't2';
ROLLBACK PREPARED 't2';

TRUNCATE test;
-

To make sure the problem produce easily, I looped above operations in my sql 
file about 10 times, then I can 100% reproduce it and got segmentation fault in 
publisher log as follows:
-
2021-05-18 16:30:56.952 CST [548189] postmaster LOG:  server process (PID 
548222) was terminated by signal 11: Segmentation fault
2021-05-18 16:30:56.952 CST [548189] postmaster DETAIL:  Failed process was 
running: START_REPLICATION SLOT "sub" LOGICAL 0/0 (proto_version '3', two_phase 
'on', publication_names '"pub"')
-

Here is the core dump information :
-
#0  0x0090afe4 in pq_sendstring (buf=buf@entry=0x251ca80, str=0x0) at 
pqformat.c:199
#1  0x00ab0a2b in logicalrep_write_begin_prepare (out=0x251ca80, 
txn=txn@entry=0x25346e8) at proto.c:124
#2  0x7f9528842dd6 in pgoutput_begin_prepare (ctx=ctx@entry=0x2514700, 
txn=txn@entry=0x25346e8) at pgoutput.c:495
#3  0x7f9528843f70 in pgoutput_truncate (ctx=0x2514700, txn=0x25346e8, 
nrelations=1, relations=0x262f678, change=0x25370b8) at pgoutput.c:905
#4  0x00aa57cb in truncate_cb_wrapper (cache=, 
txn=, nrelations=, relations=, 
change=)
at logical.c:1103
#5  0x00abf333 in ReorderBufferApplyTruncate (streaming=false, 
change=0x25370b8, relations=0x262f678, nrelations=1, txn=0x25346e8, 
rb=0x2516710)
at reorderbuffer.c:1918
#6  ReorderBufferProcessTXN (rb=rb@entry=0x2516710, txn=0x25346e8, 
commit_lsn=commit_lsn@entry=27517176, snapshot_now=, 
command_id=command_id@entry=0,
streaming=streaming@entry=false) at reorderbuffer.c:2278
#7  0x00ac0b14 in ReorderBufferReplay (txn=, 
rb=rb@entry=0x2516710, xid=xid@entry=738, commit_lsn=commit_lsn@entry=27517176,
end_lsn=end_lsn@entry=27517544, 
commit_time=commit_time@entry=674644388404356, origin_id=0, origin_lsn=0) at 
reorderbuffer.c:2591
#8  0x00ac1713 in ReorderBufferCommit (rb=0x2516710, xid=xid@entry=738, 
commit_lsn=27517176, end_lsn=27517544, 
commit_time=commit_time@entry=674644388404356,
origin_id=origin_id@entry=0, origin_lsn=0) at reorderbuffer.c:2615
#9  0x00a9f702 in DecodeCommit (ctx=ctx@entry=0x2514700, 
buf=buf@entry=0x7ffdd027c2b0, parsed=parsed@entry=0x7ffdd027c140, 
xid=xid@entry=738,
two_phase=) at decode.c:742
#10 0x00a9fc6c in DecodeXactOp (ctx=ctx@entry=0x2514700, 
buf=buf@entry=0x7ffdd027c2b0) at decode.c:278
#11 0x00aa1b75 in LogicalDecodingProcessRecord (ctx=0x2514700, 
record=0x2514ac0) at decode.c:142
#12 0x00af6db1 in XLogSendLogical () at walsender.c:2876
#13 0x00afb6aa in WalSndLoop (send_data=send_data@entry=0xaf6d49 
) at walsender.c:2306
#14 0x00afbdac in StartLogicalReplication (cmd=cmd@entry=0x24da288) at 
walsender.c:1206
#15 0x00afd646 in exec_replication_command (
cmd_string=cmd_string@entry=0x2452570 "START_REPLICATION SLOT \"sub\" 
LOGICAL 0/0 (proto_version '3', two_phase 'on', publication_names '\"pub\"')") 
at walsender.c:1646
#16 0x00ba3514 in PostgresMain (argc=argc@entry=1, 
argv=argv@entry=0x7ffdd027c560, dbname=, username=) at postgres.c:4482
#17 0x00a7284a in BackendRun (port=port@entry=0x2477b60) at 
postmaster.c:4491
#18 0x00a78bba in BackendStartup (port=port@entry=0x2477b60) at 
postmaster.c:4213
#19 0x00a78ff9 in ServerLoop () at postmaster.c:1745
#20 0x00a7bbdf in PostmasterMain (argc=argc@entry=3, 
argv=argv@entry=0x244bae0) at postmaster.c:1417
#21 0x0090dc80 in main (argc=3, argv=0x244bae0) at main.c:209
-

I noticed that it called pgoutput_truncate function and pgoutput_begin_prepare 
function. It seems odd because TRUNCATE is not in a prepared transaction in my 
case.

I tried to debug this to learn more and found that in pgoutput_truncate 
function, the value of in_prepared_txn was true. Later, it got a segmentation 
fault when it tried to get gid in logicalrep_write_begin_prepare function - it 
has no gid so we got the segmentation fault.

FYI:
I 

Re: [HACKERS] logical decoding of two-phase transactions

2021-05-18 Thread Peter Smith
On Sun, May 16, 2021 at 12:07 AM Ajin Cherian  wrote:
>
> On Thu, May 13, 2021 at 7:50 PM Peter Smith  wrote:
> >
> > Please find attached the latest patch set v75*
> >
> > Differences from v74* are:
> >
> > * Rebased to HEAD @ today.
> >
> > * v75 also addresses some of the feedback comments from Vignesh [1].
>
> Adding a patch to this patch-set that avoids empty transactions from
> being sent to the subscriber/replica. This patch is based on the
> logic that was proposed for empty transactions in the thread [1]. This
> patch uses that patch and handles empty prepared transactions
> as well. So, this will avoid empty prepared transactions from being
> sent to the subscriber/replica. This patch also avoids sending
> COMMIT PREPARED /ROLLBACK PREPARED if the prepared transaction was
> skipped provided the COMMIT /ROLLBACK happens
> prior to a restart of the walsender. If the COMMIT/ROLLBACK PREPARED
> happens after a restart, it will not be able know that the
> prepared transaction prior to the restart was not sent, in this case
> the apply worker of the subscription will check if a prepare of the
> same type exists
> and if it does not, it will silently ignore the COMMIT PREPARED
> (ROLLBACK PREPARED logic was already doing this).
> Do have a look and let me know if you have any comments.
>
> [1] - 
> https://www.postgresql.org/message-id/CAFPTHDYegcoS3xjGBj0XHfcdZr6Y35%2BYG1jq79TBD1VCkK7v3A%40mail.gmail.com
>

Hi Ajin.

I have applied the latest patch set v76*.

The patches applied cleanly.

All of the make, make check, and TAP subscriptions tests worked OK.

Below are my REVIEW COMMENTS for the v76-0003 part.

==

1.  File: doc/src/sgml/logicaldecoding.sgml

1.1

@@ -862,11 +862,19 @@ typedef void (*LogicalDecodePrepareCB) (struct
LogicalDecodingContext *ctx,
   The required commit_prepared_cb callback is called
   whenever a transaction COMMIT PREPARED has
been decoded.
   The gid field, which is part of the
-  txn parameter, can be used in this callback.
+  txn parameter, can be used in this callback. The
+  parameters prepare_end_lsn and
+  prepare_time can be used to check if the plugin
+  has received this PREPARE TRANSACTION in which case
+  it can apply the rollback, otherwise, it can skip the rollback
operation. The
+  gid alone is not sufficient because the downstream
+  node can have a prepared transaction with same identifier.

This is in the commit prepared section, but that new text is referring
to "it can apply to the rollback" etc.
Is this deliberate text, or maybe cut/paste error?

==

2. File: src/backend/replication/pgoutput/pgoutput.c

2.1

@@ -76,6 +78,7 @@ static void
pgoutput_stream_prepare_txn(LogicalDecodingContext *ctx,

 static bool publications_valid;
 static bool in_streaming;
+static bool in_prepared_txn;

Wondering why this is a module static flag. That makes it looks like
it somehow applies globally to all the functions in this scope, but
really I think this is just a txn property, right?
- e.g. why not use another member of the private TXN data instead? or
- e.g. why not use rbtxn_prepared(txn) macro?

--

2.2

@@ -404,10 +410,32 @@ pgoutput_startup(LogicalDecodingContext *ctx,
OutputPluginOptions *opt,
 static void
 pgoutput_begin_txn(LogicalDecodingContext *ctx, ReorderBufferTXN *txn)
 {
+ PGOutputTxnData*data = MemoryContextAllocZero(ctx->context,
+ sizeof(PGOutputTxnData));
+
+ (void)txn; /* keep compiler quiet */

I guess since now the arg "txn" is being used the added statement to
"keep compiler quiet" is now redundant, so should be removed.

--

2.3

+static void
+pgoutput_begin(LogicalDecodingContext *ctx, ReorderBufferTXN *txn)
+{
  bool send_replication_origin = txn->origin_id != InvalidRepOriginId;
+ PGOutputTxnData *data = (PGOutputTxnData *) txn->output_plugin_private;

  OutputPluginPrepareWrite(ctx, !send_replication_origin);
  logicalrep_write_begin(ctx->out, txn);
+ data->sent_begin_txn = true;


I wondered is it worth adding Assert(data); here?

--

2.4

@@ -422,8 +450,14 @@ static void
 pgoutput_commit_txn(LogicalDecodingContext *ctx, ReorderBufferTXN *txn,
  XLogRecPtr commit_lsn)
 {
+ PGOutputTxnData *data = (PGOutputTxnData *) txn->output_plugin_private;
+
  OutputPluginUpdateProgress(ctx);

I wondered is it worthwhile to add Assert(data); here also?

--

2.5
@@ -422,8 +450,14 @@ static void
 pgoutput_commit_txn(LogicalDecodingContext *ctx, ReorderBufferTXN *txn,
  XLogRecPtr commit_lsn)
 {
+ PGOutputTxnData *data = (PGOutputTxnData *) txn->output_plugin_private;
+
  OutputPluginUpdateProgress(ctx);

+ /* skip COMMIT message if nothing was sent */
+ if (!data->sent_begin_txn)
+ return;

Shouldn't this code also be freeing that allocated data? I think you
do free it in similar functions later in this patch.

--

2.6

@@ -435,10 +469,31 @@ pgoutput_commit_txn(LogicalDecodingContext *ctx,
ReorderBufferTXN *txn,
 static void
 

Re: [HACKERS] logical decoding of two-phase transactions

2021-05-17 Thread vignesh C
On Mon, May 17, 2021 at 6:10 PM Ajin Cherian  wrote:
>
> The above patch had some changes missing which resulted in some tap
> tests failing. Sending an updated patchset. Keeping the patchset
> version the same.

Thanks for the updated patch, the updated patch fixes the tap test failures.

Regards,
Vignesh




Re: [HACKERS] logical decoding of two-phase transactions

2021-05-13 Thread Peter Smith
On Mon, May 10, 2021 at 1:31 PM vignesh C  wrote:
>
> On Thu, Apr 29, 2021 at 2:23 PM Peter Smith  wrote:
> >
> > Please find attached the latest patch set v74*
> >
> > Differences from v73* are:
> >
> > * Rebased to HEAD @ 2 days ago.
> >
> > * v74 addresses most of the feedback comments from Vignesh posts [1][2][3].
> >
>
> Thanks for the updated patch.
> Few comments:
> 1) I felt skey[2] should be skey as we are just using one key here.
>
> +   ScanKeyData skey[2];
> +   SysScanDesc scan;
> +   boolhas_subrels;
> +
> +   rel = table_open(SubscriptionRelRelationId, AccessShareLock);
> +
> +   ScanKeyInit([nkeys++],
> +   Anum_pg_subscription_rel_srsubid,
> +   BTEqualStrategyNumber, F_OIDEQ,
> +   ObjectIdGetDatum(subid));
> +
> +   scan = systable_beginscan(rel, InvalidOid, false,
> + NULL, nkeys, skey);
> +
>

Fixed in v75.

> 2) I felt we can change lsn data type from Int64 to XLogRecPtr
> +
> +Int64
> +
> +The LSN of the prepare.
> +
> +
> +
> +
> +Int64
> +
> +The end LSN of the transaction.
> +
> +

Deferred.

>
> 3) I felt we can change lsn data type from Int32 to TransactionId
> +
> +Int32
> +
> +Xid of the subtransaction (will be same as xid of the
> transaction for top-level
> +transactions).
> +
> +

Deferred.

>
> 4) Should we change this to "The end LSN of the prepared transaction"
> just to avoid any confusion of it meaning commit/rollback.
> +
> +Int64
> +
> +The end LSN of the transaction.
> +
> +
>

Modified in v75 for message types 'b', 'P', 'K', 'r', 'p'.

> Similar problems related to comments 2 and 3 are being discussed at
> [1], we can change it accordingly based on the conclusion in the other
> thread.
> [1] - 
> https://www.postgresql.org/message-id/flat/CAHut%2BPs2JsSd_OpBR9kXt1Rt4bwyXAjh875gUpFw6T210ttO7Q%40mail.gmail.com#cf2a85d0623dcadfbb1204a196681313

Yes, I will defer addressing those feedback comments 2 and 3 pending
the outcome of your other patch of the above thread.

--
Kind Regards,
Peter Smith.
Fujitsu Australia




Re: [HACKERS] logical decoding of two-phase transactions

2021-05-10 Thread vignesh C
On Mon, May 10, 2021 at 10:51 AM Peter Smith  wrote:
>
> On Mon, May 10, 2021 at 1:31 PM vignesh C  wrote:
> >
>
> > 4) Should we change this to "The end LSN of the prepared transaction"
> > just to avoid any confusion of it meaning commit/rollback.
> > +
> > +Int64
> > +
> > +The end LSN of the transaction.
> > +
> > +
> >
>
> Can you please provide more details so I can be sure of the context of
> this feedback, e.g. there are multiple places that match that patch
> fragment provided. So was this suggestion to change all of them ( 'b',
> 'P', 'K' , 'r'  of patch 0001; and also 'p' of patch 0002) ?

My suggestion was for all of them.

Regards,
Vignesh




Re: [HACKERS] logical decoding of two-phase transactions

2021-05-09 Thread Peter Smith
On Mon, May 10, 2021 at 1:31 PM vignesh C  wrote:
>

> 4) Should we change this to "The end LSN of the prepared transaction"
> just to avoid any confusion of it meaning commit/rollback.
> +
> +Int64
> +
> +The end LSN of the transaction.
> +
> +
>

Can you please provide more details so I can be sure of the context of
this feedback, e.g. there are multiple places that match that patch
fragment provided. So was this suggestion to change all of them ( 'b',
'P', 'K' , 'r'  of patch 0001; and also 'p' of patch 0002) ?

--
Kind Regards,
Peter Smith.
Fujitsu Australia.




Re: [HACKERS] logical decoding of two-phase transactions

2021-05-09 Thread vignesh C
On Thu, Apr 29, 2021 at 2:23 PM Peter Smith  wrote:
>
> Please find attached the latest patch set v74*
>
> Differences from v73* are:
>
> * Rebased to HEAD @ 2 days ago.
>
> * v74 addresses most of the feedback comments from Vignesh posts [1][2][3].
>

Thanks for the updated patch.
Few comments:
1) I felt skey[2] should be skey as we are just using one key here.

+   ScanKeyData skey[2];
+   SysScanDesc scan;
+   boolhas_subrels;
+
+   rel = table_open(SubscriptionRelRelationId, AccessShareLock);
+
+   ScanKeyInit([nkeys++],
+   Anum_pg_subscription_rel_srsubid,
+   BTEqualStrategyNumber, F_OIDEQ,
+   ObjectIdGetDatum(subid));
+
+   scan = systable_beginscan(rel, InvalidOid, false,
+ NULL, nkeys, skey);
+

2) I felt we can change lsn data type from Int64 to XLogRecPtr
+
+Int64
+
+The LSN of the prepare.
+
+
+
+
+Int64
+
+The end LSN of the transaction.
+
+

3) I felt we can change lsn data type from Int32 to TransactionId
+
+Int32
+
+Xid of the subtransaction (will be same as xid of the
transaction for top-level
+transactions).
+
+

4) Should we change this to "The end LSN of the prepared transaction"
just to avoid any confusion of it meaning commit/rollback.
+
+Int64
+
+The end LSN of the transaction.
+
+

Similar problems related to comments 2 and 3 are being discussed at
[1], we can change it accordingly based on the conclusion in the other
thread.
[1] - 
https://www.postgresql.org/message-id/flat/CAHut%2BPs2JsSd_OpBR9kXt1Rt4bwyXAjh875gUpFw6T210ttO7Q%40mail.gmail.com#cf2a85d0623dcadfbb1204a196681313

Regards,
Vignesh




Re: [HACKERS] logical decoding of two-phase transactions

2021-04-29 Thread Peter Smith
On Tue, Apr 27, 2021 at 6:17 PM vignesh C  wrote:
>
> On Wed, Apr 21, 2021 at 12:13 PM Peter Smith  wrote:
> >
> > On Tue, Apr 20, 2021 at 3:45 PM Peter Smith  wrote:
> > >
> > > Please find attached the latest patch set v73`*
> > >
> > > Differences from v72* are:
> > >
> > > * Rebased to HEAD @ today (required because v72-0001 no longer applied 
> > > cleanly)
> > >
> > > * Minor documentation correction for protocol messages for Commit 
> > > Prepared ('K')
> > >
> > > * Non-functional code tidy (mostly proto.c) to reduce overloading
> > > different meanings to same member names for prepare/commit times.
> >
> >
> > Please find attached a re-posting of patch set v73*
> >
> > This is the same as yesterday's v73 but with a contrib module compile
> > error fixed.
>
> Few comments on
> v73-0002-Add-prepare-API-support-for-streaming-transactio.patch patch:

Thanks for your feedback comments. My replies are inline below.

> 1) There are slight differences in error message in case of Alter
> subscription ... drop publication, we can keep the error message
> similar:
> postgres=# ALTER SUBSCRIPTION mysub drop PUBLICATION mypub WITH
> (refresh = false, copy_data=true, two_phase=true);
> ERROR:  unrecognized subscription parameter: "copy_data"
> postgres=# ALTER SUBSCRIPTION mysub drop PUBLICATION mypub WITH
> (refresh = false, two_phase=true, streaming=true);
> ERROR:  cannot alter two_phase option

OK. Updated in v74.

>
> 2) We are sending txn->xid twice, I felt we should send only once in
> logicalrep_write_stream_prepare:
> +   /* transaction ID */
> +   Assert(TransactionIdIsValid(txn->xid));
> +   pq_sendint32(out, txn->xid);
> +
> +   /* send the flags field */
> +   pq_sendbyte(out, flags);
> +
> +   /* send fields */
> +   pq_sendint64(out, prepare_lsn);
> +   pq_sendint64(out, txn->end_lsn);
> +   pq_sendint64(out, txn->u_op_time.prepare_time);
> +   pq_sendint32(out, txn->xid);
> +
>

OK. Updated in v74.

> 3) We could remove xid and return prepare_data->xid
> +TransactionId
> +logicalrep_read_stream_prepare(StringInfo in,
> LogicalRepPreparedTxnData *prepare_data)
> +{
> +   TransactionId xid;
> +   uint8   flags;
> +
> +   xid = pq_getmsgint(in, 4);
>

OK. Updated in v74.

> 4) Here comments can be above apply_spooled_messages for better readability
> +   /*
> +* 1. Replay all the spooled operations - Similar code as for
> +* apply_handle_stream_commit (i.e. non two-phase stream commit)
> +*/
> +
> +   ensure_transaction();
> +
> +   nchanges = apply_spooled_messages(xid, prepare_data.prepare_lsn);
> +
>

Not done. It was deliberately commented this way because the part
below the comment is what is in apply_handle_stream_commit.

> 5) Similarly this below comment can be above PrepareTransactionBlock
> +   /*
> +* 2. Mark the transaction as prepared. - Similar code as for
> +* apply_handle_prepare (i.e. two-phase non-streamed prepare)
> +*/
> +
> +   /*
> +* BeginTransactionBlock is necessary to balance the 
> EndTransactionBlock
> +* called within the PrepareTransactionBlock below.
> +*/
> +   BeginTransactionBlock();
> +   CommitTransactionCommand();
> +
> +   /*
> +* Update origin state so we can restart streaming from correct 
> position
> +* in case of crash.
> +*/
> +   replorigin_session_origin_lsn = prepare_data.end_lsn;
> +   replorigin_session_origin_timestamp = prepare_data.prepare_time;
> +
> +   PrepareTransactionBlock(gid);
> +   CommitTransactionCommand();
> +
> +   pgstat_report_stat(false);
>

Not done. It is deliberately commented this way because the part below
the comment is what is in apply_handle_prepare.

> 6) There is a lot of common code between apply_handle_stream_prepare
> and apply_handle_prepare, if possible try to have a common function to
> avoid fixing at both places.
> +   /*
> +* 2. Mark the transaction as prepared. - Similar code as for
> +* apply_handle_prepare (i.e. two-phase non-streamed prepare)
> +*/
> +
> +   /*
> +* BeginTransactionBlock is necessary to balance the 
> EndTransactionBlock
> +* called within the PrepareTransactionBlock below.
> +*/
> +   BeginTransactionBlock();
> +   CommitTransactionCommand();
> +
> +   /*
> +* Update origin state so we can restart streaming from correct 
> position
> +* in case of crash.
> +*/
> +   replorigin_session_origin_lsn = prepare_data.end_lsn;
> +   replorigin_session_origin_timestamp = prepare_data.prepare_time;
> +
> +   PrepareTransactionBlock(gid);
> +   CommitTransactionCommand();
> +
> +   pgstat_report_stat(false);
> +
> +   store_flush_position(prepare_data.end_lsn);
>

Not done. If you diff those functions there are really only ~ 10
statements in common so I felt it is more readable to 

Re: [HACKERS] logical decoding of two-phase transactions

2021-04-29 Thread Peter Smith
On Tue, Apr 27, 2021 at 1:41 PM vignesh C  wrote:
>
> On Wed, Apr 21, 2021 at 12:13 PM Peter Smith  wrote:
> >
> > On Tue, Apr 20, 2021 at 3:45 PM Peter Smith  wrote:
> > >
> > > Please find attached the latest patch set v73`*
> > >
> > > Differences from v72* are:
> > >
> > > * Rebased to HEAD @ today (required because v72-0001 no longer applied 
> > > cleanly)
> > >
> > > * Minor documentation correction for protocol messages for Commit 
> > > Prepared ('K')
> > >
> > > * Non-functional code tidy (mostly proto.c) to reduce overloading
> > > different meanings to same member names for prepare/commit times.
> >
> >
> > Please find attached a re-posting of patch set v73*
>
> Few comments when I was having a look at the tests added:

Thanks for your feedback comments. My replies are inline below.

> 1) Can the below:
> +# check inserts are visible. 22 should be rolled back. 21 should be 
> committed.
> +$result = $node_subscriber->safe_psql('postgres', "SELECT count(*)
> FROM tab_full where a IN (21);");
> +is($result, qq(1), 'Rows committed are on the subscriber');
> +$result = $node_subscriber->safe_psql('postgres', "SELECT count(*)
> FROM tab_full where a IN (22);");
> +is($result, qq(0), 'Rows rolled back are not on the subscriber');
>
> be changed to:
> $result = $node_subscriber->safe_psql('postgres', "SELECT a FROM
> tab_full where a IN (21,22);");
> is($result, qq(21), 'Rows committed are on the subscriber');
>
> And Test count need to be reduced to "use Test::More tests => 19"
>
OK. Updated in v74.

> 2) we can change tx to transaction:
> +# check the tx state is prepared on subscriber(s)
> +$result = $node_B->safe_psql('postgres', "SELECT count(*) FROM
> pg_prepared_xacts;");
> +is($result, qq(1), 'transaction is prepared on subscriber B');
> +$result = $node_C->safe_psql('postgres', "SELECT count(*) FROM
> pg_prepared_xacts;");
> +is($result, qq(1), 'transaction is prepared on subscriber C');
>
OK. Updated in v74

> 3) There are few more instances present in the same file, those also
> can be changed.
OK. I found no others in the same file, but there were similar cases
in the 021 TAP test. Those were also updated in v74/

>
> 4) Can the below:
>  check inserts are visible at subscriber(s).
> # 22 should be rolled back.
> # 21 should be committed.
> $result = $node_B->safe_psql('postgres', "SELECT count(*) FROM
> tab_full where a IN (21);");
> is($result, qq(1), 'Rows committed are present on subscriber B');
> $result = $node_B->safe_psql('postgres', "SELECT count(*) FROM
> tab_full where a IN (22);");
> is($result, qq(0), 'Rows rolled back are not present on subscriber B');
> $result = $node_C->safe_psql('postgres', "SELECT count(*) FROM
> tab_full where a IN (21);");
> is($result, qq(1), 'Rows committed are present on subscriber C');
> $result = $node_C->safe_psql('postgres', "SELECT count(*) FROM
> tab_full where a IN (22);");
> is($result, qq(0), 'Rows rolled back are not present on subscriber C');
>
> be changed to:
> $result = $node_B->safe_psql('postgres', "SELECT a FROM tab_full where
> a IN (21,22);");
> is($result, qq(21), 'Rows committed are on the subscriber');
> $result = $node_C->safe_psql('postgres', "SELECT a FROM tab_full where
> a IN (21,22);");
> is($result, qq(21), 'Rows committed are on the subscriber');
>
> And Test count need to be reduced to "use Test::More tests => 27"
>
OK. Updated in v74.

> 5) should we change "Two phase commit" to "Two phase commit state" :
> +   /*
> +* Binary, streaming, and two_phase are only supported
> in v14 and
> +* higher
> +*/
> if (pset.sversion >= 14)
> appendPQExpBuffer(,
>   ", subbinary
> AS \"%s\"\n"
> - ", substream
> AS \"%s\"\n",
> + ", substream
> AS \"%s\"\n"
> + ",
> subtwophasestate AS \"%s\"\n",
>
> gettext_noop("Binary"),
> -
> gettext_noop("Streaming"));
> +
> gettext_noop("Streaming"),
> +
> gettext_noop("Two phase commit"));
>
Not updated. I think the column name is already the longest one and
this just makes it even longer - far too long IMO. I am not sure what
is better having the “state” suffix. After all, booleans are also
states. Anyway, I did not make this change now but if people feel
strongly about it then I can revisit it.

--
Kind Regards,
Peter Smith.
Fujitsu Australia




Re: [HACKERS] logical decoding of two-phase transactions

2021-04-29 Thread Peter Smith
On Mon, Apr 26, 2021 at 9:22 PM vignesh C  wrote:
>
> On Wed, Apr 21, 2021 at 12:13 PM Peter Smith  wrote:
> >
> > On Tue, Apr 20, 2021 at 3:45 PM Peter Smith  wrote:
> > >
> > > Please find attached the latest patch set v73`*
> > >
> > > Differences from v72* are:
> > >
> > > * Rebased to HEAD @ today (required because v72-0001 no longer applied 
> > > cleanly)
> > >
> > > * Minor documentation correction for protocol messages for Commit 
> > > Prepared ('K')
> > >
> > > * Non-functional code tidy (mostly proto.c) to reduce overloading
> > > different meanings to same member names for prepare/commit times.
> >
> >
> > Please find attached a re-posting of patch set v73*
> >
> > This is the same as yesterday's v73 but with a contrib module compile
> > error fixed.
>
> Thanks for the updated patch, few comments:

Thanks for your feedback comments, My replies are inline below.

> 1) Should "final_lsn not set in begin message" be "prepare_lsn not set
> in begin message"
> +logicalrep_read_begin_prepare(StringInfo in,
> LogicalRepPreparedTxnData *begin_data)
> +{
> +   /* read fields */
> +   begin_data->prepare_lsn = pq_getmsgint64(in);
> +   if (begin_data->prepare_lsn == InvalidXLogRecPtr)
> +   elog(ERROR, "final_lsn not set in begin message");
>

OK. Updated in v74.

> 2) Should "These commands" be "ALTER SUBSCRIPTION ... REFRESH
> PUBLICATION and ALTER SUBSCRIPTION ... SET/ADD PUBLICATION ..." as
> copy_data cannot be specified with alter subscription .. drop
> publication.
> +   These commands also cannot be executed with copy_data =
> true
> +   when the subscription has two_phase commit enabled. See
> +   column subtwophasestate of
> +to know the actual
> two-phase state.

OK. Updated in v74. While technically more correct, I think rewording
it as suggested makes the doc harder to understand. But I have
reworded it slightly to account for the fact that the copy_data
setting is not possible with the DROP.

>
> 3) Byte1('A') should be Byte1('r') as we
> have defined LOGICAL_REP_MSG_ROLLBACK_PREPARED as r.
> +Rollback Prepared
> +
> +
> +
> +
> +
> +
> +Byte1('A')
> +
> +Identifies this message as the rollback of a
> two-phase transaction message.
> +
> +

OK. Updated in v74.

>
> 4) Should "Check if the prepared transaction with the given GID and
> lsn is around." be
> "Check if the prepared transaction with the given GID, lsn & timestamp
> is around."
> +/*
> + * LookupGXact
> + * Check if the prepared transaction with the given GID
> and lsn is around.
> + *
> + * Note that we always compare with the LSN where prepare ends because that 
> is
> + * what is stored as origin_lsn in the 2PC file.
> + *
> + * This function is primarily used to check if the prepared transaction
> + * received from the upstream (remote node) already exists. Checking only GID
> + * is not sufficient because a different prepared xact with the same GID can
> + * exist on the same node. So, we are ensuring to match origin_lsn and
> + * origin_timestamp of prepared xact to avoid the possibility of a match of
> + * prepared xact from two different nodes.
> + */

OK. Updated in v74.

>
> 5) Should we change "The LSN of the prepare." to "The LSN of the begin 
> prepare."
> +Begin Prepare
> +
> +
> +
> +
> +
> +
> +Byte1('b')
> +
> +Identifies this message as the beginning of a
> two-phase transaction message.
> +
> +
> +
> +
> +Int64
> +
> +The LSN of the prepare.
> +
> +
>

Not updated. The PG Docs is correct as-is I think.

>
> 6) Similarly in cases of "Commit Prepared" and "Rollback Prepared"

Not updated. AFAIK these are correct – it really is LSN of the PREPARE
just like it says.

>
> 7) No need to initialize has_subrels as we will always assign the
> value returned by HeapTupleIsValid
> +HasSubscriptionRelations(Oid subid)
> +{
> +   Relationrel;
> +   int nkeys = 0;
> +   ScanKeyData skey[2];
> +   SysScanDesc scan;
> +   boolhas_subrels = false;
> +
> +   rel = table_open(SubscriptionRelRelationId, AccessShareLock);

OK. Updated in v74.

>
> 8) We could include errhint, like errhint("Option \"two_phase\"
> specified more than once") to specify a more informative error
> message.
> +   else if (strcmp(defel->defname, "two_phase") == 0)
> +   {
> +   if (two_phase_option_given)
> +   ereport(ERROR,
> +   
> (errcode(ERRCODE_SYNTAX_ERROR),
> +errmsg("conflicting
> or redundant options")));
> +   two_phase_option_given = true;
> +
> +   data->two_phase = defGetBoolean(defel);
> +   }
>

Not updated. Yes, maybe it would be better like you say, but the code
would then be inconsistent with every other option in this function.
Perhaps your idea can be raised as a separate 

Re: [HACKERS] logical decoding of two-phase transactions

2021-04-28 Thread Ajin Cherian
Modified pgbench's "tpcb-like" builtin query as below to do two-phase
commits and then ran a 4 cascade replication setup.

"BEGIN;\n"
"UPDATE pgbench_accounts SET abalance = abalance + :delta
WHERE aid = :aid;\n"
"SELECT abalance FROM pgbench_accounts WHERE aid = :aid;\n"
"UPDATE pgbench_tellers SET tbalance = tbalance + :delta WHERE
tid = :tid;\n"
"UPDATE pgbench_branches SET bbalance = bbalance + :delta
WHERE bid = :bid;\n"
"INSERT INTO pgbench_history (tid, bid, aid, delta, mtime)
VALUES (:tid, :bid, :aid, :delta, CURRENT_TIMESTAMP);\n"
"PREPARE TRANSACTION ':aid:';\n"
"COMMIT PREPARED ':aid:';\n"

The tests ran fine and all 4 cascaded servers replicated the changes
correctly. All the subscriptions were configured with two_phase
enabled.

regards,
Ajin Cherian
Fujitsu Australia




Re: [HACKERS] logical decoding of two-phase transactions

2021-04-27 Thread vignesh C
On Wed, Apr 21, 2021 at 12:13 PM Peter Smith  wrote:
>
> On Tue, Apr 20, 2021 at 3:45 PM Peter Smith  wrote:
> >
> > Please find attached the latest patch set v73`*
> >
> > Differences from v72* are:
> >
> > * Rebased to HEAD @ today (required because v72-0001 no longer applied 
> > cleanly)
> >
> > * Minor documentation correction for protocol messages for Commit Prepared 
> > ('K')
> >
> > * Non-functional code tidy (mostly proto.c) to reduce overloading
> > different meanings to same member names for prepare/commit times.
>
>
> Please find attached a re-posting of patch set v73*
>
> This is the same as yesterday's v73 but with a contrib module compile
> error fixed.

Few comments on
v73-0002-Add-prepare-API-support-for-streaming-transactio.patch patch:
1) There are slight differences in error message in case of Alter
subscription ... drop publication, we can keep the error message
similar:
postgres=# ALTER SUBSCRIPTION mysub drop PUBLICATION mypub WITH
(refresh = false, copy_data=true, two_phase=true);
ERROR:  unrecognized subscription parameter: "copy_data"
postgres=# ALTER SUBSCRIPTION mysub drop PUBLICATION mypub WITH
(refresh = false, two_phase=true, streaming=true);
ERROR:  cannot alter two_phase option

2) We are sending txn->xid twice, I felt we should send only once in
logicalrep_write_stream_prepare:
+   /* transaction ID */
+   Assert(TransactionIdIsValid(txn->xid));
+   pq_sendint32(out, txn->xid);
+
+   /* send the flags field */
+   pq_sendbyte(out, flags);
+
+   /* send fields */
+   pq_sendint64(out, prepare_lsn);
+   pq_sendint64(out, txn->end_lsn);
+   pq_sendint64(out, txn->u_op_time.prepare_time);
+   pq_sendint32(out, txn->xid);
+

3) We could remove xid and return prepare_data->xid
+TransactionId
+logicalrep_read_stream_prepare(StringInfo in,
LogicalRepPreparedTxnData *prepare_data)
+{
+   TransactionId xid;
+   uint8   flags;
+
+   xid = pq_getmsgint(in, 4);

4) Here comments can be above apply_spooled_messages for better readability
+   /*
+* 1. Replay all the spooled operations - Similar code as for
+* apply_handle_stream_commit (i.e. non two-phase stream commit)
+*/
+
+   ensure_transaction();
+
+   nchanges = apply_spooled_messages(xid, prepare_data.prepare_lsn);
+

5) Similarly this below comment can be above PrepareTransactionBlock
+   /*
+* 2. Mark the transaction as prepared. - Similar code as for
+* apply_handle_prepare (i.e. two-phase non-streamed prepare)
+*/
+
+   /*
+* BeginTransactionBlock is necessary to balance the EndTransactionBlock
+* called within the PrepareTransactionBlock below.
+*/
+   BeginTransactionBlock();
+   CommitTransactionCommand();
+
+   /*
+* Update origin state so we can restart streaming from correct position
+* in case of crash.
+*/
+   replorigin_session_origin_lsn = prepare_data.end_lsn;
+   replorigin_session_origin_timestamp = prepare_data.prepare_time;
+
+   PrepareTransactionBlock(gid);
+   CommitTransactionCommand();
+
+   pgstat_report_stat(false);

6) There is a lot of common code between apply_handle_stream_prepare
and apply_handle_prepare, if possible try to have a common function to
avoid fixing at both places.
+   /*
+* 2. Mark the transaction as prepared. - Similar code as for
+* apply_handle_prepare (i.e. two-phase non-streamed prepare)
+*/
+
+   /*
+* BeginTransactionBlock is necessary to balance the EndTransactionBlock
+* called within the PrepareTransactionBlock below.
+*/
+   BeginTransactionBlock();
+   CommitTransactionCommand();
+
+   /*
+* Update origin state so we can restart streaming from correct position
+* in case of crash.
+*/
+   replorigin_session_origin_lsn = prepare_data.end_lsn;
+   replorigin_session_origin_timestamp = prepare_data.prepare_time;
+
+   PrepareTransactionBlock(gid);
+   CommitTransactionCommand();
+
+   pgstat_report_stat(false);
+
+   store_flush_position(prepare_data.end_lsn);

7) two-phase commit is slightly misleading, we can just mention
streaming prepare.
+ * PREPARE callback (for streaming two-phase commit).
+ *
+ * Notify the downstream to prepare the transaction.
+ */
+static void
+pgoutput_stream_prepare_txn(LogicalDecodingContext *ctx,
+   ReorderBufferTXN *txn,
+   XLogRecPtr prepare_lsn)

8) should we include Assert of in_streaming similar to other
pgoutput_stream*** functions.
+static void
+pgoutput_stream_prepare_txn(LogicalDecodingContext *ctx,
+   ReorderBufferTXN *txn,
+   XLogRecPtr prepare_lsn)
+{
+   Assert(rbtxn_is_streamed(txn));
+
+   

Re: [HACKERS] logical decoding of two-phase transactions

2021-04-26 Thread vignesh C
On Wed, Apr 21, 2021 at 12:13 PM Peter Smith  wrote:
>
> On Tue, Apr 20, 2021 at 3:45 PM Peter Smith  wrote:
> >
> > Please find attached the latest patch set v73`*
> >
> > Differences from v72* are:
> >
> > * Rebased to HEAD @ today (required because v72-0001 no longer applied 
> > cleanly)
> >
> > * Minor documentation correction for protocol messages for Commit Prepared 
> > ('K')
> >
> > * Non-functional code tidy (mostly proto.c) to reduce overloading
> > different meanings to same member names for prepare/commit times.
>
>
> Please find attached a re-posting of patch set v73*

Few comments when I was having a look at the tests added:
1) Can the below:
+# check inserts are visible. 22 should be rolled back. 21 should be committed.
+$result = $node_subscriber->safe_psql('postgres', "SELECT count(*)
FROM tab_full where a IN (21);");
+is($result, qq(1), 'Rows committed are on the subscriber');
+$result = $node_subscriber->safe_psql('postgres', "SELECT count(*)
FROM tab_full where a IN (22);");
+is($result, qq(0), 'Rows rolled back are not on the subscriber');

be changed to:
$result = $node_subscriber->safe_psql('postgres', "SELECT a FROM
tab_full where a IN (21,22);");
is($result, qq(21), 'Rows committed are on the subscriber');

And Test count need to be reduced to "use Test::More tests => 19"

2) we can change tx to transaction:
+# check the tx state is prepared on subscriber(s)
+$result = $node_B->safe_psql('postgres', "SELECT count(*) FROM
pg_prepared_xacts;");
+is($result, qq(1), 'transaction is prepared on subscriber B');
+$result = $node_C->safe_psql('postgres', "SELECT count(*) FROM
pg_prepared_xacts;");
+is($result, qq(1), 'transaction is prepared on subscriber C');

3) There are few more instances present in the same file, those also
can be changed.

4) Can the below:
 check inserts are visible at subscriber(s).
# 22 should be rolled back.
# 21 should be committed.
$result = $node_B->safe_psql('postgres', "SELECT count(*) FROM
tab_full where a IN (21);");
is($result, qq(1), 'Rows committed are present on subscriber B');
$result = $node_B->safe_psql('postgres', "SELECT count(*) FROM
tab_full where a IN (22);");
is($result, qq(0), 'Rows rolled back are not present on subscriber B');
$result = $node_C->safe_psql('postgres', "SELECT count(*) FROM
tab_full where a IN (21);");
is($result, qq(1), 'Rows committed are present on subscriber C');
$result = $node_C->safe_psql('postgres', "SELECT count(*) FROM
tab_full where a IN (22);");
is($result, qq(0), 'Rows rolled back are not present on subscriber C');

be changed to:
$result = $node_B->safe_psql('postgres', "SELECT a FROM tab_full where
a IN (21,22);");
is($result, qq(21), 'Rows committed are on the subscriber');
$result = $node_C->safe_psql('postgres', "SELECT a FROM tab_full where
a IN (21,22);");
is($result, qq(21), 'Rows committed are on the subscriber');

And Test count need to be reduced to "use Test::More tests => 27"

5) should we change "Two phase commit" to "Two phase commit state" :
+   /*
+* Binary, streaming, and two_phase are only supported
in v14 and
+* higher
+*/
if (pset.sversion >= 14)
appendPQExpBuffer(,
  ", subbinary
AS \"%s\"\n"
- ", substream
AS \"%s\"\n",
+ ", substream
AS \"%s\"\n"
+ ",
subtwophasestate AS \"%s\"\n",

gettext_noop("Binary"),
-
gettext_noop("Streaming"));
+
gettext_noop("Streaming"),
+
gettext_noop("Two phase commit"));

Regards,
Vignesh




Re: [HACKERS] logical decoding of two-phase transactions

2021-04-26 Thread vignesh C
On Wed, Apr 21, 2021 at 12:13 PM Peter Smith  wrote:
>
> On Tue, Apr 20, 2021 at 3:45 PM Peter Smith  wrote:
> >
> > Please find attached the latest patch set v73`*
> >
> > Differences from v72* are:
> >
> > * Rebased to HEAD @ today (required because v72-0001 no longer applied 
> > cleanly)
> >
> > * Minor documentation correction for protocol messages for Commit Prepared 
> > ('K')
> >
> > * Non-functional code tidy (mostly proto.c) to reduce overloading
> > different meanings to same member names for prepare/commit times.
>
>
> Please find attached a re-posting of patch set v73*
>
> This is the same as yesterday's v73 but with a contrib module compile
> error fixed.

Thanks for the updated patch, few comments:
1) Should "final_lsn not set in begin message" be "prepare_lsn not set
in begin message"
+logicalrep_read_begin_prepare(StringInfo in,
LogicalRepPreparedTxnData *begin_data)
+{
+   /* read fields */
+   begin_data->prepare_lsn = pq_getmsgint64(in);
+   if (begin_data->prepare_lsn == InvalidXLogRecPtr)
+   elog(ERROR, "final_lsn not set in begin message");

2) Should "These commands" be "ALTER SUBSCRIPTION ... REFRESH
PUBLICATION and ALTER SUBSCRIPTION ... SET/ADD PUBLICATION ..." as
copy_data cannot be specified with alter subscription .. drop
publication.
+   These commands also cannot be executed with copy_data =
true
+   when the subscription has two_phase commit enabled. See
+   column subtwophasestate of
+to know the actual
two-phase state.

3) Byte1('A') should be Byte1('r') as we
have defined LOGICAL_REP_MSG_ROLLBACK_PREPARED as r.
+Rollback Prepared
+
+
+
+
+
+
+Byte1('A')
+
+Identifies this message as the rollback of a
two-phase transaction message.
+
+

4) Should "Check if the prepared transaction with the given GID and
lsn is around." be
"Check if the prepared transaction with the given GID, lsn & timestamp
is around."
+/*
+ * LookupGXact
+ * Check if the prepared transaction with the given GID
and lsn is around.
+ *
+ * Note that we always compare with the LSN where prepare ends because that is
+ * what is stored as origin_lsn in the 2PC file.
+ *
+ * This function is primarily used to check if the prepared transaction
+ * received from the upstream (remote node) already exists. Checking only GID
+ * is not sufficient because a different prepared xact with the same GID can
+ * exist on the same node. So, we are ensuring to match origin_lsn and
+ * origin_timestamp of prepared xact to avoid the possibility of a match of
+ * prepared xact from two different nodes.
+ */

5) Should we change "The LSN of the prepare." to "The LSN of the begin prepare."
+Begin Prepare
+
+
+
+
+
+
+Byte1('b')
+
+Identifies this message as the beginning of a
two-phase transaction message.
+
+
+
+
+Int64
+
+The LSN of the prepare.
+
+


6) Similarly in cases of "Commit Prepared" and "Rollback Prepared"

7) No need to initialize has_subrels as we will always assign the
value returned by HeapTupleIsValid
+HasSubscriptionRelations(Oid subid)
+{
+   Relationrel;
+   int nkeys = 0;
+   ScanKeyData skey[2];
+   SysScanDesc scan;
+   boolhas_subrels = false;
+
+   rel = table_open(SubscriptionRelRelationId, AccessShareLock);

8) We could include errhint, like errhint("Option \"two_phase\"
specified more than once") to specify a more informative error
message.
+   else if (strcmp(defel->defname, "two_phase") == 0)
+   {
+   if (two_phase_option_given)
+   ereport(ERROR,
+   (errcode(ERRCODE_SYNTAX_ERROR),
+errmsg("conflicting
or redundant options")));
+   two_phase_option_given = true;
+
+   data->two_phase = defGetBoolean(defel);
+   }

9) We have a lot of function parameters for
parse_subscription_options, should we change it to struct?
@@ -69,7 +69,8 @@ parse_subscription_options(List *options,
   char **synchronous_commit,
   bool *refresh,
   bool *binary_given,
bool *binary,
-  bool
*streaming_given, bool *streaming)
+  bool
*streaming_given, bool *streaming,
+  bool
*twophase_given, bool *twophase)

10) Should we change " errhint("Use ALTER SUBSCRIPTION ...SET
PUBLICATION with refresh = false, or with copy_data = false, or use
DROP/CREATE SUBSCRIPTION.")" to  "errhint("Use ALTER SUBSCRIPTION
...SET/ADD PUBLICATION with refresh = false, or with copy_data =
false.")" as we don't support copy_data in ALTER subscription ... DROP
publication.
+  

Re: [HACKERS] logical decoding of two-phase transactions

2021-04-09 Thread Peter Smith
On Fri, Apr 9, 2021 at 6:40 PM Amit Kapila  wrote:
>
> On Fri, Apr 9, 2021 at 12:33 PM Peter Smith  wrote:
> >
> > On Mon, Dec 14, 2020 at 8:27 PM Amit Kapila  wrote:
> > >
> > > 2.
> > > + /*
> > > + * Flags are determined from the state of the transaction. We know we
> > > + * always get PREPARE first and then [COMMIT|ROLLBACK] PREPARED, so if
> > > + * it's already marked as committed then it has to be COMMIT PREPARED 
> > > (and
> > > + * likewise for abort / ROLLBACK PREPARED).
> > > + */
> > > + if (rbtxn_commit_prepared(txn))
> > > + flags = LOGICALREP_IS_COMMIT_PREPARED;
> > > + else if (rbtxn_rollback_prepared(txn))
> > > + flags = LOGICALREP_IS_ROLLBACK_PREPARED;
> > > + else
> > > + flags = LOGICALREP_IS_PREPARE;
> > >
> > > I don't like clubbing three different operations under one message
> > > LOGICAL_REP_MSG_PREPARE. It looks awkward to use new flags
> > > RBTXN_COMMIT_PREPARED and RBTXN_ROLLBACK_PREPARED in ReordeBuffer so
> > > that we can recognize these operations in corresponding callbacks. I
> > > think setting any flag in ReorderBuffer should not dictate the
> > > behavior in callbacks. Then also there are few things that are not
> > > common to those APIs like the patch has an Assert to say that the txn
> > > is marked with prepare flag for all three operations which I think is
> > > not true for Rollback Prepared after the restart. We don't ensure to
> > > set the Prepare flag if the Rollback Prepare happens after the
> > > restart. Then, we have to introduce separate flags to distinguish
> > > prepare/commit prepared/rollback prepared to distinguish multiple
> > > operations sent as protocol messages. Also, all these operations are
> > > mutually exclusive so it will be better to send separate messages for
> > > each of these and I have changed it accordingly in the attached patch.
> > >
> >
> > While looking at the two-phase protocol messages (with a view to
> > documenting them) I noticed that the messages for
> > LOGICAL_REP_MSG_PREPARE, LOGICAL_REP_MSG_COMMIT_PREPARED,
> > LOGICAL_REP_MSG_ROLLBACK_PREPARED are all sending and receiving flag
> > bytes which *always* has a value 0.
> >
> > --
> > e.g.
> > uint8   flags = 0;
> > pq_sendbyte(out, flags);
> >
> > and
> > /* read flags */
> > uint8   flags = pq_getmsgbyte(in);
> > if (flags != 0)
> > elog(ERROR, "unrecognized flags %u in commit prepare message", 
> > flags);
> > --
> >
> > I think this patch version v31 is where the flags became redundant.
> >
>
> I think this has been kept for future use similar to how we have in
> logicalrep_write_commit. So, I think we can keep them unused for now.
> We can document it similar commit message ('C') [1].
>
> [1] - 
> https://www.postgresql.org/docs/devel/protocol-logicalrep-message-formats.html
>

Yeah, we can do that. And if nobody else gives feedback about this
then I will do exactly like you suggested.

But I don't understand why we are even trying to "future proof" the
protocol by keeping redundant flags lying around on the off-chance
that maybe one day they could be useful.

Isn't that what the protocol version number is for? e.g. If there did
become some future need for some flags then just add them at that time
and bump the protocol version.

And, even if we wanted to, I think we cannot use these existing flags
in future without bumping the protocol version, because the current
protocol docs say that flag value must be zero!

--
Kind Regards,
Peter Smith.
Fujitsu Australia




Re: [HACKERS] logical decoding of two-phase transactions

2021-04-09 Thread Amit Kapila
On Fri, Apr 9, 2021 at 12:33 PM Peter Smith  wrote:
>
> On Mon, Dec 14, 2020 at 8:27 PM Amit Kapila  wrote:
> >
> > 2.
> > + /*
> > + * Flags are determined from the state of the transaction. We know we
> > + * always get PREPARE first and then [COMMIT|ROLLBACK] PREPARED, so if
> > + * it's already marked as committed then it has to be COMMIT PREPARED (and
> > + * likewise for abort / ROLLBACK PREPARED).
> > + */
> > + if (rbtxn_commit_prepared(txn))
> > + flags = LOGICALREP_IS_COMMIT_PREPARED;
> > + else if (rbtxn_rollback_prepared(txn))
> > + flags = LOGICALREP_IS_ROLLBACK_PREPARED;
> > + else
> > + flags = LOGICALREP_IS_PREPARE;
> >
> > I don't like clubbing three different operations under one message
> > LOGICAL_REP_MSG_PREPARE. It looks awkward to use new flags
> > RBTXN_COMMIT_PREPARED and RBTXN_ROLLBACK_PREPARED in ReordeBuffer so
> > that we can recognize these operations in corresponding callbacks. I
> > think setting any flag in ReorderBuffer should not dictate the
> > behavior in callbacks. Then also there are few things that are not
> > common to those APIs like the patch has an Assert to say that the txn
> > is marked with prepare flag for all three operations which I think is
> > not true for Rollback Prepared after the restart. We don't ensure to
> > set the Prepare flag if the Rollback Prepare happens after the
> > restart. Then, we have to introduce separate flags to distinguish
> > prepare/commit prepared/rollback prepared to distinguish multiple
> > operations sent as protocol messages. Also, all these operations are
> > mutually exclusive so it will be better to send separate messages for
> > each of these and I have changed it accordingly in the attached patch.
> >
>
> While looking at the two-phase protocol messages (with a view to
> documenting them) I noticed that the messages for
> LOGICAL_REP_MSG_PREPARE, LOGICAL_REP_MSG_COMMIT_PREPARED,
> LOGICAL_REP_MSG_ROLLBACK_PREPARED are all sending and receiving flag
> bytes which *always* has a value 0.
>
> --
> e.g.
> uint8   flags = 0;
> pq_sendbyte(out, flags);
>
> and
> /* read flags */
> uint8   flags = pq_getmsgbyte(in);
> if (flags != 0)
> elog(ERROR, "unrecognized flags %u in commit prepare message", flags);
> --
>
> I think this patch version v31 is where the flags became redundant.
>

I think this has been kept for future use similar to how we have in
logicalrep_write_commit. So, I think we can keep them unused for now.
We can document it similar commit message ('C') [1].

[1] - 
https://www.postgresql.org/docs/devel/protocol-logicalrep-message-formats.html

-- 
With Regards,
Amit Kapila.




Re: [HACKERS] logical decoding of two-phase transactions

2021-04-09 Thread Peter Smith
On Mon, Dec 14, 2020 at 8:27 PM Amit Kapila  wrote:
>
> 2.
> + /*
> + * Flags are determined from the state of the transaction. We know we
> + * always get PREPARE first and then [COMMIT|ROLLBACK] PREPARED, so if
> + * it's already marked as committed then it has to be COMMIT PREPARED (and
> + * likewise for abort / ROLLBACK PREPARED).
> + */
> + if (rbtxn_commit_prepared(txn))
> + flags = LOGICALREP_IS_COMMIT_PREPARED;
> + else if (rbtxn_rollback_prepared(txn))
> + flags = LOGICALREP_IS_ROLLBACK_PREPARED;
> + else
> + flags = LOGICALREP_IS_PREPARE;
>
> I don't like clubbing three different operations under one message
> LOGICAL_REP_MSG_PREPARE. It looks awkward to use new flags
> RBTXN_COMMIT_PREPARED and RBTXN_ROLLBACK_PREPARED in ReordeBuffer so
> that we can recognize these operations in corresponding callbacks. I
> think setting any flag in ReorderBuffer should not dictate the
> behavior in callbacks. Then also there are few things that are not
> common to those APIs like the patch has an Assert to say that the txn
> is marked with prepare flag for all three operations which I think is
> not true for Rollback Prepared after the restart. We don't ensure to
> set the Prepare flag if the Rollback Prepare happens after the
> restart. Then, we have to introduce separate flags to distinguish
> prepare/commit prepared/rollback prepared to distinguish multiple
> operations sent as protocol messages. Also, all these operations are
> mutually exclusive so it will be better to send separate messages for
> each of these and I have changed it accordingly in the attached patch.
>

While looking at the two-phase protocol messages (with a view to
documenting them) I noticed that the messages for
LOGICAL_REP_MSG_PREPARE, LOGICAL_REP_MSG_COMMIT_PREPARED,
LOGICAL_REP_MSG_ROLLBACK_PREPARED are all sending and receiving flag
bytes which *always* has a value 0.

--
e.g.
uint8   flags = 0;
pq_sendbyte(out, flags);

and
/* read flags */
uint8   flags = pq_getmsgbyte(in);
if (flags != 0)
elog(ERROR, "unrecognized flags %u in commit prepare message", flags);
--

I think this patch version v31 is where the flags became redundant.

Is there some reason why these unused flags still remain in the protocol code?

Do you have any objection to me removing them?
Otherwise, it might seem strange to document a flag that has no function.

--
KInd Regards,
Peter Smith.
Fujitsu Australia




Re: [HACKERS] logical decoding of two-phase transactions

2021-04-01 Thread Peter Smith
On Thu, Apr 1, 2021 at 4:58 PM Ajin Cherian  wrote:
>
>
>
> On Thu, Apr 1, 2021 at 2:29 PM Amit Kapila  wrote:
>>
>> On Tue, Mar 30, 2021 at 5:34 AM Peter Smith  wrote:
>> >
>> > Please find attached the latest patch set v68*
>> >
>>
>> I think this patch is in much better shape than it was few versions
>> earlier but I feel still some more work and testing is required. We
>> can try to make it work with the streaming option and do something
>> about empty prepare transactions to reduce the need for users to set a
>> much higher value for max_prepared_xacts on subscribers. So, I propose
>> to move it to the next CF, what do you think?
>>
>
> I agree.

OK, done. Moved to next CF here: https://commitfest.postgresql.org/33/2914/

--
Kind Regards,
Peter Smith.
Fujitsu Australia.




Re: [HACKERS] logical decoding of two-phase transactions

2021-04-01 Thread vignesh C
On Thu, Apr 1, 2021 at 8:59 AM Amit Kapila  wrote:
>
> On Tue, Mar 30, 2021 at 5:34 AM Peter Smith  wrote:
> >
> > Please find attached the latest patch set v68*
> >
>
> I think this patch is in much better shape than it was few versions
> earlier but I feel still some more work and testing is required. We
> can try to make it work with the streaming option and do something
> about empty prepare transactions to reduce the need for users to set a
> much higher value for max_prepared_xacts on subscribers. So, I propose
> to move it to the next CF, what do you think?

+1 for moving it to the next PG version.

Regards,
Vignesh




Re: [HACKERS] logical decoding of two-phase transactions

2021-03-31 Thread Ajin Cherian
On Thu, Apr 1, 2021 at 2:29 PM Amit Kapila  wrote:

> On Tue, Mar 30, 2021 at 5:34 AM Peter Smith  wrote:
> >
> > Please find attached the latest patch set v68*
> >
>
> I think this patch is in much better shape than it was few versions
> earlier but I feel still some more work and testing is required. We
> can try to make it work with the streaming option and do something
> about empty prepare transactions to reduce the need for users to set a
> much higher value for max_prepared_xacts on subscribers. So, I propose
> to move it to the next CF, what do you think?
>
>
I agree.

regards,
Ajin Cherian
Fujitsu Australia


Re: [HACKERS] logical decoding of two-phase transactions

2021-03-31 Thread Amit Kapila
On Tue, Mar 30, 2021 at 5:34 AM Peter Smith  wrote:
>
> Please find attached the latest patch set v68*
>

I think this patch is in much better shape than it was few versions
earlier but I feel still some more work and testing is required. We
can try to make it work with the streaming option and do something
about empty prepare transactions to reduce the need for users to set a
much higher value for max_prepared_xacts on subscribers. So, I propose
to move it to the next CF, what do you think?

-- 
With Regards,
Amit Kapila.




Re: [HACKERS] logical decoding of two-phase transactions

2021-03-29 Thread vignesh C
On Tue, Mar 30, 2021 at 5:34 AM Peter Smith  wrote:
>
> Please find attached the latest patch set v68*
>
> Differences from v67* are:
>
> * Rebased to HEAD @ today.
>
> * v68 fixes an issue reported by Vignesh [1] where a scenario was
> found which still was able to cause a generated GID clash. Using
> Vignesh's test script I could reproduce the problem exactly as
> described. The fix makes the GID unique by including the subid. Now
> the same script runs to normal completion and produces good/expected
> output:
>
>  transaction |   gid|   prepared|
> owner   | database
> -+--+---+--+--
>  547 | pg_gid_16389_543 | 2021-03-30 10:32:36.87207+11  |
> postgres | postgres
>  555 | pg_gid_16390_543 | 2021-03-30 10:32:48.087771+11 |
> postgres | postgres
> (2 rows)
>

Thanks for the patch with the fix, the fix solves the issue reported.

Regards,
Vignesh




  1   2   3   4   5   >