# statements.
#
-my $main_in = '';
-my $main_out = '';
-my $main_timer = IPC::Run::timeout($TestLib::timeout_default);
-
-my $main_h =
- $node->background_psql('postgres', \$main_in, \$main_out,
- $main_timer, on_error_stop => 1);
-$main_in .= q(
+my $main_h = $node->background_psql('postgres');
+
+$main_h->query_safe(q(
BEGIN;
INSERT INTO tbl VALUES(0);
-\echo syncpoint1
-);
-pump $main_h until $main_out =~ /syncpoint1/ || $main_timer->is_expired;
-
-my $cic_in = '';
-my $cic_out = '';
-my $cic_timer = IPC::Run::timeout($TestLib::timeout_default);
-my $cic_h =
- $node->background_psql('postgres', \$cic_in, \$cic_out,
- $cic_timer, on_error_stop => 1);
-$cic_in .= q(
+));
+
+my $cic_h = $node->background_psql('postgres');
+
+$cic_h->query_until(qr/start/, q(
\echo start
CREATE INDEX CONCURRENTLY idx ON tbl(i);
-);
-pump $cic_h until $cic_out =~ /start/ || $cic_timer->is_expired;
+));
-$main_in .= q(
+$main_h->query_safe(q(
PREPARE TRANSACTION 'a';
-);
+));
-$main_in .= q(
+$main_h->query_safe(q(
BEGIN;
INSERT INTO tbl VALUES(0);
-\echo syncpoint2
-);
-pump $main_h until $main_out =~ /syncpoint2/ || $main_timer->is_expired;
+));
$node->safe_psql('postgres', q(COMMIT PREPARED 'a';));
-$main_in .= q(
+$main_h->query_safe(q(
PREPARE TRANSACTION 'b';
BEGIN;
INSERT INTO tbl VALUES(0);
-\echo syncpoint3
-);
-pump $main_h until $main_out =~ /syncpoint3/ || $main_timer->is_expired;
+));
$node->safe_psql('postgres', q(COMMIT PREPARED 'b';));
-$main_in .= q(
+$main_h->query_safe(q(
PREPARE TRANSACTION 'c';
COMMIT PREPARED 'c';
-);
-$main_h->pump_nb;
+));
-$main_h->finish;
-$cic_h->finish;
+$main_h->quit;
+$cic_h->quit;
$result = $node->psql('postgres', q(SELECT bt_index_check('idx',true)));
is($result, '0', 'bt_index_check after overlapping 2PC');
));
$node->restart;
-my $reindex_in = '';
-my $reindex_out = '';
-my $reindex_timer =
- IPC::Run::timeout($TestLib::timeout_default);
-my $reindex_h =
- $node->background_psql('postgres', \$reindex_in, \$reindex_out,
- $reindex_timer, on_error_stop => 1);
-$reindex_in .= q(
+my $reindex_h = $node->background_psql('postgres');
+$reindex_h->query_until(qr/start/, q(
\echo start
DROP INDEX CONCURRENTLY idx;
CREATE INDEX CONCURRENTLY idx ON tbl(i);
-);
-pump $reindex_h until $reindex_out =~ /start/ || $reindex_timer->is_expired;
+));
$node->safe_psql('postgres', "COMMIT PREPARED 'spans_restart'");
-$reindex_h->finish;
+$reindex_h->quit;
$result = $node->psql('postgres', q(SELECT bt_index_check('idx',true)));
is($result, '0', 'bt_index_check after 2PC and restart');
use PostgresNode;
use TestLib;
use Test::More;
-use IPC::Run qw(pump finish timer);
use Data::Dumper;
# Do nothing unless Makefile has told us that the build is --with-readline.
close $FH;
# fire up an interactive psql session
-my $in = '';
-my $out = '';
-
-my $timer = timer($TestLib::timeout_default);
-
-my $h = $node->interactive_psql('postgres', \$in, \$out, $timer);
-
-like($out, qr/psql/, "print startup banner");
+my $h = $node->interactive_psql('postgres');
# Simple test case: type something and see if psql responds as expected
sub check_completion
# report test failures from caller location
local $Test::Builder::Level = $Test::Builder::Level + 1;
- # reset output collector
- $out = "";
# restart per-command timer
- $timer->start($TestLib::timeout_default);
- # send the data to be sent
- $in .= $send;
- # wait ...
- pump $h until ($out =~ $pattern || $timer->is_expired);
- my $okay = ($out =~ $pattern && !$timer->is_expired);
+ $h->{timeout}->start($PostgreSQL::Test::Utils::timeout_default);
+
+ # send the data to be sent and wait for its result
+ my $out = $h->query_until($pattern, $send);
+ my $okay = ($out =~ $pattern && !$h->{timeout}->is_expired);
ok($okay, $annotation);
# for debugging, log actual output if it didn't match
local $Data::Dumper::Terse = 1;
clear_line();
# send psql an explicit \q to shut it down, else pty won't close properly
-$timer->start($TestLib::timeout_default);
-$in .= "\\q\n";
-finish $h or die "psql returned $?";
-$timer->reset;
+$h->quit or die "psql returned $?";
# done
$node->stop;
--- /dev/null
+
+# Copyright (c) 2021-2024, PostgreSQL Global Development Group
+
+=pod
+
+=head1 NAME
+
+PostgreSQL::Test::BackgroundPsql - class for controlling background psql processes
+
+=head1 SYNOPSIS
+
+ use PostgreSQL::Test::Cluster;
+
+ my $node = PostgreSQL::Test::Cluster->new('mynode');
+
+ # Create a data directory with initdb
+ $node->init();
+
+ # Start the PostgreSQL server
+ $node->start();
+
+ # Create and start an interactive psql session
+ my $isession = $node->interactive_psql('postgres');
+ # Apply timeout per query rather than per session
+ $isession->set_query_timer_restart();
+ # Run a query and get the output as seen by psql
+ my $ret = $isession->query("SELECT 1");
+ # Run a backslash command and wait until the prompt returns
+ $isession->query_until(qr/postgres #/, "\\d foo\n");
+ # Close the session and exit psql
+ $isession->quit;
+
+ # Create and start a background psql session
+ my $bsession = $node->background_psql('postgres');
+
+ # Run a query which is guaranteed to not return in case it fails
+ $bsession->query_safe("SELECT 1");
+ # Initiate a command which can be expected to terminate at a later stage
+ $bsession->query_until(qr/start/, q(
+ \echo start
+ CREATE INDEX CONCURRENTLY idx ON t(a);
+ ));
+ # Close the session and exit psql
+ $bsession->quit;
+
+=head1 DESCRIPTION
+
+PostgreSQL::Test::BackgroundPsql contains functionality for controlling
+a background or interactive psql session operating on a PostgreSQL node
+initiated by PostgreSQL::Test::Cluster.
+
+=cut
+
+package PostgreSQL::Test::BackgroundPsql;
+
+use strict;
+use warnings FATAL => 'all';
+
+use Carp;
+use Config;
+use IPC::Run;
+use PostgreSQL::Test::Utils qw(pump_until);
+use Test::More;
+
+=pod
+
+=head1 METHODS
+
+=over
+
+=item PostgreSQL::Test::BackgroundPsql->new(interactive, @psql_params, timeout)
+
+Builds a new object of class C<PostgreSQL::Test::BackgroundPsql> for either
+an interactive or background session and starts it. If C<interactive> is
+true then a PTY will be attached. C<psql_params> should contain the full
+command to run psql with all desired parameters and a complete connection
+string. For C<interactive> sessions, IO::Pty is required.
+
+=cut
+
+sub new
+{
+ my $class = shift;
+ my ($interactive, $psql_params, $timeout) = @_;
+ my $psql = {
+ 'stdin' => '',
+ 'stdout' => '',
+ 'stderr' => '',
+ 'query_timer_restart' => undef
+ };
+ my $run;
+
+ # This constructor should only be called from PostgreSQL::Test::Cluster
+ my ($package, $file, $line) = caller;
+ die
+ "Forbidden caller of constructor: package: $package, file: $file:$line"
+ unless $package->isa('PostgreSQL::Test::Cluster') || $package->isa('PostgresNode');
+
+ $psql->{timeout} = IPC::Run::timeout(
+ defined($timeout)
+ ? $timeout
+ : $PostgreSQL::Test::Utils::timeout_default);
+
+ if ($interactive)
+ {
+ $run = IPC::Run::start $psql_params,
+ '<pty<', \$psql->{stdin}, '>pty>', \$psql->{stdout}, '2>',
+ \$psql->{stderr},
+ $psql->{timeout};
+ }
+ else
+ {
+ $run = IPC::Run::start $psql_params,
+ '<', \$psql->{stdin}, '>', \$psql->{stdout}, '2>', \$psql->{stderr},
+ $psql->{timeout};
+ }
+
+ $psql->{run} = $run;
+
+ my $self = bless $psql, $class;
+
+ $self->_wait_connect();
+
+ return $self;
+}
+
+# Internal routine for awaiting psql starting up and being ready to consume
+# input.
+sub _wait_connect
+{
+ my ($self) = @_;
+
+ # Request some output, and pump until we see it. This means that psql
+ # connection failures are caught here, relieving callers of the need to
+ # handle those. (Right now, we have no particularly good handling for
+ # errors anyway, but that might be added later.)
+ my $banner = "background_psql: ready";
+ $self->{stdin} .= "\\echo $banner\n";
+ $self->{run}->pump()
+ until $self->{stdout} =~ /$banner/ || $self->{timeout}->is_expired;
+ $self->{stdout} = ''; # clear out banner
+
+ die "psql startup timed out" if $self->{timeout}->is_expired;
+}
+
+=pod
+
+=item $session->quit
+
+Close the session and clean up resources. Each test run must be closed with
+C<quit>.
+
+=cut
+
+sub quit
+{
+ my ($self) = @_;
+
+ $self->{stdin} .= "\\q\n";
+
+ return $self->{run}->finish;
+}
+
+=pod
+
+=item $session->reconnect_and_clear
+
+Terminate the current session and connect again.
+
+=cut
+
+sub reconnect_and_clear
+{
+ my ($self) = @_;
+
+ # If psql isn't dead already, tell it to quit as \q, when already dead,
+ # causes IPC::Run to unhelpfully error out with "ack Broken pipe:".
+ $self->{run}->pump_nb();
+ if ($self->{run}->pumpable())
+ {
+ $self->{stdin} .= "\\q\n";
+ }
+ $self->{run}->finish;
+
+ # restart
+ $self->{run}->run();
+ $self->{stdin} = '';
+ $self->{stdout} = '';
+
+ $self->_wait_connect();
+}
+
+=pod
+
+=item $session->query()
+
+Executes a query in the current session and returns the output in scalar
+context and (output, error) in list context where error is 1 in case there
+was output generated on stderr when executing the query.
+
+=cut
+
+sub query
+{
+ my ($self, $query) = @_;
+ my $ret;
+ my $output;
+ local $Test::Builder::Level = $Test::Builder::Level + 1;
+
+ note "issuing query via background psql: $query";
+
+ $self->{timeout}->start() if (defined($self->{query_timer_restart}));
+
+ # Feed the query to psql's stdin, followed by \n (so psql processes the
+ # line), by a ; (so that psql issues the query, if it doesn't include a ;
+ # itself), and a separator echoed with \echo, that we can wait on.
+ my $banner = "background_psql: QUERY_SEPARATOR";
+ $self->{stdin} .= "$query\n;\n\\echo $banner\n";
+
+ pump_until($self->{run}, $self->{timeout}, \$self->{stdout}, qr/$banner/);
+
+ die "psql query timed out" if $self->{timeout}->is_expired;
+ $output = $self->{stdout};
+
+ # remove banner again, our caller doesn't care
+ $output =~ s/\n$banner$//s;
+
+ # clear out output for the next query
+ $self->{stdout} = '';
+
+ $ret = $self->{stderr} eq "" ? 0 : 1;
+
+ return wantarray ? ($output, $ret) : $output;
+}
+
+=pod
+
+=item $session->query_safe()
+
+Wrapper around C<query> which errors out if the query failed to execute.
+Query failure is determined by it producing output on stderr.
+
+=cut
+
+sub query_safe
+{
+ my ($self, $query) = @_;
+
+ my $ret = $self->query($query);
+
+ if ($self->{stderr} ne "")
+ {
+ die "query failed: $self->{stderr}";
+ }
+
+ return $ret;
+}
+
+=pod
+
+=item $session->query_until(until, query)
+
+Issue C<query> and wait for C<until> appearing in the query output rather than
+waiting for query completion. C<query> needs to end with newline and semicolon
+(if applicable, interactive psql input may not require it) for psql to process
+the input.
+
+=cut
+
+sub query_until
+{
+ my ($self, $until, $query) = @_;
+ my $ret;
+ local $Test::Builder::Level = $Test::Builder::Level + 1;
+
+ $self->{timeout}->start() if (defined($self->{query_timer_restart}));
+ $self->{stdin} .= $query;
+
+ pump_until($self->{run}, $self->{timeout}, \$self->{stdout}, $until);
+
+ die "psql query timed out" if $self->{timeout}->is_expired;
+
+ $ret = $self->{stdout};
+
+ # clear out output for the next query
+ $self->{stdout} = '';
+
+ return $ret;
+}
+
+=pod
+
+=item $session->set_query_timer_restart()
+
+Configures the timer to be restarted before each query such that the defined
+timeout is valid per query rather than per test run.
+
+=back
+
+=cut
+
+sub set_query_timer_restart
+{
+ my $self = shift;
+
+ $self->{query_timer_restart} = 1;
+ return $self->{query_timer_restart};
+}
+
+1;
use Socket;
use Test::More;
use TestLib ();
+use PostgreSQL::Test::BackgroundPsql ();
use Time::HiRes qw(usleep);
use Scalar::Util qw(blessed);
=pod
-=item $node->background_psql($dbname, \$stdin, \$stdout, $timer, %params) => harness
+=item $node->background_psql($dbname, %params) => PostgreSQL::Test::BackgroundPsql instance
-Invoke B<psql> on B<$dbname> and return an IPC::Run harness object, which the
-caller may use to send input to B<psql>. The process's stdin is sourced from
-the $stdin scalar reference, and its stdout and stderr go to the $stdout
-scalar reference. This allows the caller to act on other parts of the system
-while idling this backend.
-
-The specified timer object is attached to the harness, as well. It's caller's
-responsibility to set the timeout length (usually
-$TestLib::timeout_default), and to restart the timer after
-each command if the timeout is per-command.
+Invoke B<psql> on B<$dbname> and return a BackgroundPsql object.
psql is invoked in tuples-only unaligned mode with reading of B<.psqlrc>
disabled. That may be overridden by passing extra psql parameters.
later are the caller's problem. psql runs with on_error_stop by default so
that it will stop running sql and return 3 if passed SQL results in an error.
-Be sure to "finish" the harness when done with it.
+Be sure to "quit" the returned object when done with it.
=over
set, so SQL execution is stopped at the first error and exit code 3 is
returned. Set B<on_error_stop> to 0 to ignore errors instead.
+=item timeout => 'interval'
+
+Set a timeout for a background psql session. By default, timeout of
+$PostgreSQL::Test::Utils::timeout_default is set up.
+
=item replication => B<value>
If set, add B<replication=value> to the conninfo string.
sub background_psql
{
- my ($self, $dbname, $stdin, $stdout, $timer, %params) = @_;
+ my ($self, $dbname, %params) = @_;
local %ENV = $self->_get_env();
my $replication = $params{replication};
+ my $timeout = undef;
my @psql_params = (
$self->installed_command('psql'),
'-');
$params{on_error_stop} = 1 unless defined $params{on_error_stop};
+ $timeout = $params{timeout} if defined $params{timeout};
push @psql_params, '-v', 'ON_ERROR_STOP=1' if $params{on_error_stop};
push @psql_params, @{ $params{extra_params} }
if defined $params{extra_params};
- # Ensure there is no data waiting to be sent:
- $$stdin = "" if ref($stdin);
- # IPC::Run would otherwise append to existing contents:
- $$stdout = "" if ref($stdout);
-
- my $harness = IPC::Run::start \@psql_params,
- '<', $stdin, '>', $stdout, $timer;
-
- # Request some output, and pump until we see it. This means that psql
- # connection failures are caught here, relieving callers of the need to
- # handle those. (Right now, we have no particularly good handling for
- # errors anyway, but that might be added later.)
- my $banner = "background_psql: ready";
- $$stdin = "\\echo $banner\n";
- pump $harness until $$stdout =~ /$banner/ || $timer->is_expired;
-
- die "psql startup timed out" if $timer->is_expired;
-
- return $harness;
+ return PostgreSQL::Test::BackgroundPsql->new(0, \@psql_params, $timeout);
}
=pod
-=item $node->interactive_psql($dbname, \$stdin, \$stdout, $timer, %params) => harness
+=item $node->interactive_psql($dbname, %params) => BackgroundPsql instance
-Invoke B<psql> on B<$dbname> and return an IPC::Run harness object,
-which the caller may use to send interactive input to B<psql>.
-The process's stdin is sourced from the $stdin scalar reference,
-and its stdout and stderr go to the $stdout scalar reference.
-ptys are used so that psql thinks it's being called interactively.
+Invoke B<psql> on B<$dbname> and return a BackgroundPsql object, which the
+caller may use to send interactive input to B<psql>.
-The specified timer object is attached to the harness, as well. It's caller's
-responsibility to set the timeout length (usually
-$TestLib::timeout_default), and to restart the timer after
-each command if the timeout is per-command.
+A timeout of $PostgreSQL::Test::Utils::timeout_default is set up.
psql is invoked in tuples-only unaligned mode with reading of B<.psqlrc>
disabled. That may be overridden by passing extra psql parameters.
Dies on failure to invoke psql, or if psql fails to connect.
Errors occurring later are the caller's problem.
-Be sure to "finish" the harness when done with it.
-
-The only extra parameter currently accepted is
+Be sure to "quit" the returned object when done with it.
=over
sub interactive_psql
{
- my ($self, $dbname, $stdin, $stdout, $timer, %params) = @_;
+ my ($self, $dbname, %params) = @_;
local %ENV = $self->_get_env();
push @psql_params, @{ $params{extra_params} }
if defined $params{extra_params};
- # Ensure there is no data waiting to be sent:
- $$stdin = "" if ref($stdin);
- # IPC::Run would otherwise append to existing contents:
- $$stdout = "" if ref($stdout);
-
- my $harness = IPC::Run::start \@psql_params,
- '<pty<', $stdin, '>pty>', $stdout, $timer;
-
- # Pump until we see psql's help banner. This ensures that callers
- # won't write anything to the pty before it's ready, avoiding an
- # implementation issue in IPC::Run. Also, it means that psql
- # connection failures are caught here, relieving callers of
- # the need to handle those. (Right now, we have no particularly
- # good handling for errors anyway, but that might be added later.)
- pump $harness
- until $$stdout =~ /Type "help" for help/ || $timer->is_expired;
-
- die "psql startup timed out" if $timer->is_expired;
-
- return $harness;
+ return PostgreSQL::Test::BackgroundPsql->new(1, \@psql_params);
}
# Common sub of pgbench-invoking interfaces. Makes any requested script files
use TestLib;
use Test::More tests => 13;
use File::Copy;
-use IPC::Run ();
use Scalar::Util qw(blessed);
my ($stdout, $stderr, $ret);
# a longrunning psql that we can use to trigger conflicts
-my $psql_timeout = IPC::Run::timer($PostgreSQL::Test::Utils::timeout_default);
-my %psql_standby = ('stdin' => '', 'stdout' => '');
-$psql_standby{run} =
- $node_standby->background_psql($test_db, \$psql_standby{stdin},
- \$psql_standby{stdout},
- $psql_timeout);
-$psql_standby{stdout} = '';
-
+my $psql_standby = $node_standby->background_psql($test_db,
+ on_error_stop => 0);
my $expected_conflicts = 0;
# DECLARE and use a cursor on standby, causing buffer with the only block of
# the relation to be pinned on the standby
-$psql_standby{stdin} .= qq[
- BEGIN;
- DECLARE $cursor1 CURSOR FOR SELECT b FROM $table1;
- FETCH FORWARD FROM $cursor1;
- ];
+my $res = $psql_standby->query_safe(qq[
+ BEGIN;
+ DECLARE $cursor1 CURSOR FOR SELECT b FROM $table1;
+ FETCH FORWARD FROM $cursor1;
+]);
# FETCH FORWARD should have returned a 0 since all values of b in the table
# are 0
-ok(pump_until_standby(qr/^0$/m),
- "$sect: cursor with conflicting pin established");
+like($res, qr/^0$/m, "$sect: cursor with conflicting pin established");
# to check the log starting now for recovery conflict messages
my $log_location = -s $node_standby->logfile;
$node_primary->wait_for_catchup($node_standby, 'replay', $primary_lsn);
check_conflict_log("User was holding shared buffer pin for too long");
-reconnect_and_clear();
+$psql_standby->reconnect_and_clear();
check_conflict_stat("bufferpin");
$node_primary->wait_for_catchup($node_standby, 'replay', $primary_lsn);
# DECLARE and FETCH from cursor on the standby
-$psql_standby{stdin} .= qq[
+$res = $psql_standby->query_safe(qq[
BEGIN;
DECLARE $cursor1 CURSOR FOR SELECT b FROM $table1;
FETCH FORWARD FROM $cursor1;
- ];
-ok( pump_until(
- $psql_standby{run}, $psql_timeout,
- \$psql_standby{stdout}, qr/^0$/m,),
- "$sect: cursor with conflicting snapshot established");
+ ]);
+like($res, qr/^0$/m, "$sect: cursor with conflicting snapshot established");
# Do some HOT updates
$node_primary->safe_psql($test_db,
check_conflict_log(
"User query might have needed to see row versions that must be removed");
-reconnect_and_clear();
+$psql_standby->reconnect_and_clear();
check_conflict_stat("snapshot");
$expected_conflicts++;
# acquire lock to conflict with
-$psql_standby{stdin} .= qq[
+$res = $psql_standby->query_safe(qq[
BEGIN;
LOCK TABLE $table1 IN ACCESS SHARE MODE;
SELECT 1;
- ];
-ok(pump_until_standby(qr/^1$/m), "$sect: conflicting lock acquired");
+ ]);
+like($res, qr/^1$/m, "$sect: conflicting lock acquired");
# DROP TABLE containing block which standby has in a pinned buffer
$node_primary->safe_psql($test_db, qq[DROP TABLE $table1;]);
$node_primary->wait_for_catchup($node_standby, 'replay', $primary_lsn);
check_conflict_log("User was holding a relation lock for too long");
-reconnect_and_clear();
+$psql_standby->reconnect_and_clear();
check_conflict_stat("lock");
## DECLARE a cursor for a query which, with sufficiently low work_mem, will
## spill tuples into temp files in the temporary tablespace created during
## setup.
-#$psql_standby{stdin} .= qq[
+#$res = $psql_standby->query_safe(qq[
# BEGIN;
# SET work_mem = '64kB';
# DECLARE $cursor1 CURSOR FOR
# SELECT count(*) FROM generate_series(1,6000);
# FETCH FORWARD FROM $cursor1;
-# ];
-#ok(pump_until_standby(qr/^6000$/m),
+# ]);
+#like($res, qr/^6000$/m,
# "$sect: cursor with conflicting temp file established");
#
## Drop the tablespace currently containing spill files for the query on the
#
#check_conflict_log(
# "User was or might have been using tablespace that must be dropped");
-#reconnect_and_clear();
+#$psql_standby->reconnect_and_clear();
#check_conflict_stat("tablespace");
$primary_lsn = $node_primary->lsn('flush');
$node_primary->wait_for_catchup($node_standby, 'replay', $primary_lsn);
-$psql_standby{stdin} .= qq[
+$res = $psql_standby->query_until(qr/^1$/m, qq[
BEGIN;
-- hold pin
DECLARE $cursor1 CURSOR FOR SELECT a FROM $table1;
FETCH FORWARD FROM $cursor1;
-- wait for lock held by prepared transaction
SELECT * FROM $table2;
- ];
-ok( pump_until(
- $psql_standby{run}, $psql_timeout,
- \$psql_standby{stdout}, qr/^1$/m,),
- "$sect: cursor holding conflicting pin, also waiting for lock, established"
-);
+ ]);
+ok( 1, "$sect: cursor holding conflicting pin, also waiting for lock, established");
# just to make sure we're waiting for lock already
ok( $node_standby->poll_query_until(
$node_primary->wait_for_catchup($node_standby, 'replay', $primary_lsn);
check_conflict_log("User transaction caused buffer deadlock with recovery.");
-reconnect_and_clear();
+$psql_standby->reconnect_and_clear();
check_conflict_stat("deadlock");
# clean up for next tests
# explicitly shut down psql instances gracefully - to avoid hangs or worse on
# windows
-$psql_standby{stdin} .= "\\q\n";
-$psql_standby{run}->finish;
+$psql_standby->quit;
$node_standby->stop();
$node_primary->stop();
done_testing();
-
-sub pump_until_standby
-{
- my $match = shift;
-
- return pump_until($psql_standby{run}, $psql_timeout,
- \$psql_standby{stdout}, $match);
-}
-
-sub reconnect_and_clear
-{
- # If psql isn't dead already, tell it to quit as \q, when already dead,
- # causes IPC::Run to unhelpfully error out with "ack Broken pipe:".
- $psql_standby{run}->pump_nb();
- if ($psql_standby{run}->pumpable())
- {
- $psql_standby{stdin} .= "\\q\n";
- }
- $psql_standby{run}->finish;
-
- # restart
- $psql_standby{run}->run();
- $psql_standby{stdin} = '';
- $psql_standby{stdout} = '';
-
- # Run query to ensure connection has finished re-establishing
- $psql_standby{stdin} .= qq[SELECT 1;\n];
- die unless pump_until_standby(qr/^1$/m);
- $psql_standby{stdout} = '';
-}
-
sub check_conflict_log
{
my $message = shift;
# interruption happens at the appropriate moment, we lock pg_tablespace. DROP
# DATABASE scans pg_tablespace once it has reached the "irreversible" part of
# dropping the database, making it a suitable point to wait.
-my $bgpsql_in = '';
-my $bgpsql_out = '';
-my $bgpsql_err = '';
-my $bgpsql_timer = IPC::Run::timer($PostgreSQL::Test::Utils::timeout_default);
-my $bgpsql = $node->background_psql('postgres', \$bgpsql_in, \$bgpsql_out,
- $bgpsql_timer, on_error_stop => 0);
-$bgpsql_out = '';
-$bgpsql_in .= "SELECT pg_backend_pid();\n";
-
-pump_until($bgpsql, $bgpsql_timer, \$bgpsql_out, qr/\d/);
-
-my $pid = $bgpsql_out;
-$bgpsql_out = '';
+my $bgpsql = $node->background_psql('postgres', on_error_stop => 0);
+my $pid = $bgpsql->query('SELECT pg_backend_pid()');
# create the database, prevent drop database via lock held by a 2PC transaction
-$bgpsql_in .= qq(
+ok( $bgpsql->query_safe(
+ qq(
CREATE DATABASE regression_invalid_interrupt;
BEGIN;
LOCK pg_tablespace;
- PREPARE TRANSACTION 'lock_tblspc';
- \\echo done
-);
-
-ok(pump_until($bgpsql, $bgpsql_timer, \$bgpsql_out, qr/done/),
+ PREPARE TRANSACTION 'lock_tblspc';)),
"blocked DROP DATABASE completion");
-$bgpsql_out = '';
# Try to drop. This will wait due to the still held lock.
-$bgpsql_in .= qq(
- DROP DATABASE regression_invalid_interrupt;
- \\echo DROP DATABASE completed
-);
-$bgpsql->pump_nb;
+$bgpsql->query_until(qr//, "DROP DATABASE regression_invalid_interrupt;\n");
# Ensure we're waiting for the lock
$node->poll_query_until('postgres',
# wait for cancellation to be processed
ok( pump_until(
- $bgpsql, $bgpsql_timer, \$bgpsql_out, qr/DROP DATABASE completed/),
+ $bgpsql->{run}, $bgpsql->{timeout},
+ \$bgpsql->{stderr}, qr/canceling statement due to user request/),
"cancel processed");
-$bgpsql_out = '';
+$bgpsql->{stderr} = '';
# verify that connection to the database aren't allowed
is($node->psql('regression_invalid_interrupt', ''),
# To properly drop the database, we need to release the lock previously preventing
# doing so.
-$bgpsql_in .= qq(
- ROLLBACK PREPARED 'lock_tblspc';
- \\echo ROLLBACK PREPARED
-);
-ok(pump_until($bgpsql, $bgpsql_timer, \$bgpsql_out, qr/ROLLBACK PREPARED/),
+ok($bgpsql->query_safe(qq(ROLLBACK PREPARED 'lock_tblspc')),
"unblock DROP DATABASE");
-$bgpsql_out = '';
-is($node->psql('postgres', "DROP DATABASE regression_invalid_interrupt"),
- 0, "DROP DATABASE invalid_interrupt");
+ok($bgpsql->query(qq(DROP DATABASE regression_invalid_interrupt)),
+ "DROP DATABASE invalid_interrupt");
-$bgpsql_in .= "\\q\n";
-$bgpsql->finish();
+$bgpsql->quit();
done_testing();
is($result, qq(2|2|2), 'check initial data was copied to subscriber');
# Interleave a pair of transactions, each exceeding the 64kB limit.
-my $in = '';
-my $out = '';
-
-my $timer = IPC::Run::timeout($TestLib::timeout_default);
-
-my $h = $node_publisher->background_psql('postgres', \$in, \$out, $timer,
+my $h = $node_publisher->background_psql('postgres',
on_error_stop => 0);
-$in .= q{
+$h->query_safe(q{
BEGIN;
INSERT INTO test_tab SELECT i, md5(i::text) FROM generate_series(3, 5000) s(i);
UPDATE test_tab SET b = md5(b) WHERE mod(a,2) = 0;
DELETE FROM test_tab WHERE mod(a,3) = 0;
-};
-$h->pump_nb;
+});
$node_publisher->safe_psql(
'postgres', q{
COMMIT;
});
-$in .= q{
-COMMIT;
-\q
-};
-$h->finish; # errors make the next test fail, so ignore them here
+$h->query_safe('COMMIT');
+# errors make the next test fail, so ignore them here
+$h->quit;
$node_publisher->wait_for_catchup($appname);