bdr: Use logical dump/reload and catchup mode during DB startup
authorCraig Ringer <[email protected]>
Mon, 21 Apr 2014 05:22:39 +0000 (13:22 +0800)
committerAndres Freund <[email protected]>
Thu, 3 Jul 2014 15:55:27 +0000 (17:55 +0200)
Install BDR extension in remote, track progress in bdr.bdr_nodes,
support pg_dump --snapshot and pg_restore for dump+reload apply,
use catchup mode.

contrib/bdr/bdr--0.5.sql
contrib/bdr/bdr.c
contrib/bdr/bdr.h
contrib/bdr/bdr_apply.c
contrib/bdr/bdr_init_replica.c [new file with mode: 0644]
contrib/bdr/scripts/bdr_initial_load [new file with mode: 0644]
contrib/bdr/worker.mk

index d76b53e686ba58a656d70ce7bf78c766453c0ac7..9e56d262ce2b385b694220bc6bf704eac3ff88e2 100644 (file)
@@ -217,6 +217,33 @@ END;
 $function$;
 
 
+
+-- The bdr_nodes table tracks members of a BDR group; it's only concerned with
+-- one database, so the local and foreign database names are implicit.  All we
+-- care about is the sysid.
+--
+-- The sysid must be a numeric (or string) because PostgreSQL has no uint64 SQL
+-- type.
+--
+-- In future we may support different local dbnames, so store the dbname too.
+-- It's even possible we might replicate from one local DB to another (though
+-- who knows why we'd want to) so the PK should be the (dbname, sysid) tuple.
+--
+CREATE TABLE bdr_nodes (
+    node_sysid numeric,
+    node_dbname name not null,
+    node_status "char" not null,
+    primary key(node_sysid, node_dbname),
+    check (node_status in ('i', 'c', 'r'))
+);
+
+COMMENT ON TABLE bdr_nodes IS 'All known nodes in this BDR group.';
+COMMENT ON COLUMN bdr_nodes.node_sysid IS 'system_identifier from the control file of the node';
+COMMENT ON COLUMN bdr_nodes.node_dbname IS 'local database name on the node';
+COMMENT ON COLUMN bdr_nodes.node_status IS 'Readiness of the node: [i]nitializing, [c]atchup, [r]eady. Doesn''t indicate connected/disconnected.';
+
+SELECT pg_catalog.pg_extension_config_dump('bdr_nodes', '');
+
 -- This type is tailored to use as input to get_object_address
 CREATE TYPE bdr.dropped_object AS
   (objtype text, objnames text[], objargs text[]);
index 6efc8e4bfc4a77ed61c73997dabd73a25efac470..36b109514e15daa1fa33dc943cee1c14059e6b78 100644 (file)
 #include "lib/stringinfo.h"
 #include "libpq/pqformat.h"
 #include "mb/pg_wchar.h"
+#include "postmaster/bgwriter.h"
 #include "replication/replication_identifier.h"
 #include "utils/builtins.h"
 #include "utils/guc.h"
 #include "utils/lsyscache.h"
 #include "utils/memutils.h"
 #include "utils/snapmgr.h"
+#include "utils/syscache.h"
 #include "utils/timestamp.h"
 
 /* sequencer */
 /* apply */
 #include "libpq-fe.h"
 
+
 #define MAXCONNINFO        1024
 
 static bool exit_worker = false;
 static int   n_configured_bdr_nodes = 0;
 ResourceOwner bdr_saved_resowner;
 static bool bdr_is_restart = false;
+Oid   BdrNodesRelid;
 
 /* GUC storage */
 static char *connections = NULL;
 static char *bdr_synchronous_commit = NULL;
-static int bdr_max_workers;
 int bdr_default_apply_delay;
+int bdr_max_workers;
 
 /* TODO: Remove when bdr_apply_main moved into bdr_apply.c */
 extern BdrApplyWorker *bdr_apply_worker;
 
-/*
- * Header for the shared memory segment ref'd by the BdrWorkerCtl ptr,
- * containing bdr_max_workers entries of BdrWorkerCon .
- */
-typedef struct BdrWorkerControl
-{
-   /* Must hold this lock when writing to BdrWorkerControl members */
-   LWLockId     lock;
-   /* Required only for bgworker restart issues: */
-   bool         launch_workers;
-   /* Set/unset by bdr_apply_pause()/_replay(). */
-   bool         pause_apply;
-   /* Array members, of size bdr_max_workers */
-   BdrWorker    slots[FLEXIBLE_ARRAY_MEMBER];
-} BdrWorkerControl;
-
 /* shmem init hook to chain to on startup, if any */
 static shmem_startup_hook_type prev_shmem_startup_hook = NULL;
+
 /* shortcut for finding the the worker shmem block */
-static BdrWorkerControl *BdrWorkerCtl = NULL;
+BdrWorkerControl *BdrWorkerCtl = NULL;
 
 /*
  * Used only in postmaster to pass data from _PG_init during
@@ -116,6 +105,8 @@ typedef struct BdrApplyWorkerConfigOptions
 {
    char *dsn;
    int   apply_delay;
+   bool  bdr_init_replica;
+   char *replica_local_dsn;
 } BdrApplyWorkerConfigOptions;
 
 PG_MODULE_MAGIC;
@@ -124,10 +115,6 @@ void       _PG_init(void);
 static void bdr_maintain_schema(void);
 static void bdr_worker_shmem_startup(void);
 static void bdr_worker_shmem_create_workers(void);
-static BdrWorker* bdr_worker_shmem_alloc(BdrWorkerType worker_type);
-static void bdr_worker_shmem_release(BdrWorker* worker,
-                                    BackgroundWorkerHandle *handle)
-   __attribute__((unused)); /* TODO: remove this attribute when function is used */
 
 Datum bdr_apply_pause(PG_FUNCTION_ARGS);
 Datum bdr_apply_resume(PG_FUNCTION_ARGS);
@@ -281,7 +268,7 @@ bdr_process_remote_action(StringInfo s)
  *   remote_sysid_i
  *   remote_tlid_i
  */
-static PGconn*
+PGconn*
 bdr_connect(char *conninfo_repl,
            char* remote_ident, size_t remote_ident_length,
            NameData* slot_name,
@@ -378,8 +365,21 @@ bdr_connect(char *conninfo_repl,
 }
 
 /*
+ * ----------
  * Create a slot on a remote node, and the corresponding local replication
  * identifier.
+ *
+ * Arguments:
+ *   streamConn        Connection to use for slot creation
+ *   slot_name     Name of the slot to create
+ *   remote_ident  Identifier for the remote end
+ *
+ * Out parameters:
+ *   replication_identifier        Created local replication identifier
+ *   snapshot                  If !NULL, snapshot ID of slot snapshot
+ *
+ * If a snapshot is returned it must be pfree()'d by the caller.
+ * ----------
  */
 /*
  * TODO we should really handle the case where the slot already exists but
@@ -409,6 +409,8 @@ bdr_create_slot(PGconn *streamConn, Name slot_name,
 
    if (PQresultStatus(res) != PGRES_TUPLES_OK)
    {
+       /* TODO: Should test whether this error is 'already exists' and carry on */
+
        elog(FATAL, "could not send replication command \"%s\": status %s: %s\n",
             query.data,
             PQresStatus(PQresultStatus(res)), PQresultErrorMessage(res));
@@ -422,7 +424,8 @@ bdr_create_slot(PGconn *streamConn, Name slot_name,
    CurrentResourceOwner = bdr_saved_resowner;
    elog(LOG, "created replication identifier %u", *replication_identifier);
 
-   *snapshot = pstrdup(PQgetvalue(res, 0, 2));
+   if (snapshot)
+       *snapshot = pstrdup(PQgetvalue(res, 0, 2));
 
    PQclear(res);
 }
@@ -492,6 +495,79 @@ bdr_get_worker_option(const char * worker_name, const char * option_name,
    return optval;
 }
 
+/*
+ *----------------------
+ * Connect to the BDR remote end, IDENTIFY_SYSTEM, and CREATE_SLOT if necessary.
+ * Generates slot name, replication identifier.
+ *
+ * Raises an error on failure, will not return null.
+ *
+ * Arguments:
+ *       connection_name:  bdr conn name from bdr.connections to get dsn from
+ *
+ * Returns:
+ *    the libpq connection
+ *
+ * Out parameters:
+ *    out_slot_name: the generated name of the slot on the remote end
+ *    out_sysid:     the remote end's system identifier
+ *    out_timeline:  the remote end's current timeline
+ *    out_replication_identifier: The replication identifier for this connection
+ *
+ *----------------------
+ */
+PGconn*
+bdr_establish_connection_and_slot(Name connection_name, Name out_slot_name,
+   uint64 *out_sysid, TimeLineID* out_timeline, RepNodeId
+   *out_replication_identifier, char **out_snapshot)
+{
+   char        conninfo_repl[MAXCONNINFO + 75];
+   const char *dsn;
+   char        remote_ident[256];
+   PGconn     *streamConn;
+
+   dsn = bdr_get_worker_option(NameStr(*connection_name), "dsn", false);
+   snprintf(conninfo_repl, sizeof(conninfo_repl),
+            "%s replication=database fallback_application_name=bdr",
+            dsn);
+
+   /* Establish BDR conn and IDENTIFY_SYSTEM */
+   streamConn = bdr_connect(
+       conninfo_repl,
+       remote_ident, sizeof(remote_ident),
+       out_slot_name, out_sysid, out_timeline
+       );
+
+   StartTransactionCommand();
+   *out_replication_identifier = GetReplicationIdentifier(remote_ident, true);
+   CommitTransactionCommand();
+
+   if (OidIsValid(*out_replication_identifier))
+   {
+       elog(LOG, "found valid replication identifier %u",
+            *out_replication_identifier);
+       if (out_snapshot)
+           *out_snapshot = NULL;
+   }
+   else
+   {
+       /*
+        * Slot doesn't exist, create it.
+        *
+        * The per-db worker will create slots when we first init BDR, but new workers
+        * added afterwards are expected to create their own slots at connect time; that's
+        * when this runs.
+        */
+
+       /* create local replication identifier and a remote slot */
+       elog(LOG, "Creating new slot %s", NameStr(*out_slot_name));
+       bdr_create_slot(streamConn, out_slot_name, remote_ident,
+                       out_replication_identifier, out_snapshot);
+   }
+
+   return streamConn;
+}
+
 /*
  * Entry point and main loop for a BDR apply worker.
  *
@@ -500,21 +576,18 @@ bdr_get_worker_option(const char * worker_name, const char * option_name,
  *
  * TODO: move to bdr_apply.c
  */
-static void
+void
 bdr_apply_main(Datum main_arg)
 {
    PGconn     *streamConn;
    PGresult   *res;
    int         fd;
-   char        remote_ident[256];
    StringInfoData query;
-   char        conninfo_repl[MAXCONNINFO + 75];
    XLogRecPtr  last_received = InvalidXLogRecPtr;
    char       *sqlstate;
    RepNodeId   replication_identifier;
    XLogRecPtr  start_from;
    NameData    slot_name;
-   const char *dsn;
    BdrWorker  *bdr_worker_slot;
 
    Assert(IsBackgroundWorker);
@@ -527,48 +600,15 @@ bdr_apply_main(Datum main_arg)
 
    bdr_worker_init(NameStr(bdr_apply_worker->dbname));
 
-   CurrentResourceOwner =
-       ResourceOwnerCreate(NULL, "bdr apply top-level resource owner");
+   CurrentResourceOwner = ResourceOwnerCreate(NULL, "bdr apply top-level resource owner");
    bdr_saved_resowner = CurrentResourceOwner;
 
-   dsn = bdr_get_worker_option(NameStr(bdr_apply_worker->name),
-                               "dsn", false);
-   snprintf(conninfo_repl, sizeof(conninfo_repl),
-            "%s replication=database fallback_application_name=bdr",
-            dsn);
-
-   elog(LOG, "%s initialized on %s, remote %s",
-        MyBgworkerEntry->bgw_name, NameStr(bdr_apply_worker->dbname),
-        conninfo_repl);
-
-   /* Establish BDR conn and IDENTIFY_SYSTEM */
-   streamConn = bdr_connect(
-       conninfo_repl,
-       remote_ident, sizeof(remote_ident),
-       &slot_name,
-       &bdr_apply_worker->sysid,
-       &bdr_apply_worker->timeline
-       );
-
-   StartTransactionCommand();
-   replication_identifier = GetReplicationIdentifier(remote_ident, true);
-   CommitTransactionCommand();
-
-   if (OidIsValid(replication_identifier))
-       elog(LOG, "found valid replication identifier %u",
-            replication_identifier);
-   else
-   {
-       char *snapshot;
+   elog(LOG, "%s initialized on %s",
+        MyBgworkerEntry->bgw_name, NameStr(bdr_apply_worker->dbname));
 
-       elog(LOG, "Creating new slot %s", NameStr(slot_name));
-
-       /* create local replication identifier and a remote slot */
-       bdr_create_slot(streamConn, &slot_name, remote_ident,
-                       &replication_identifier, &snapshot);
-
-       /* TODO: Initialize database from remote */
-   }
+   streamConn = bdr_establish_connection_and_slot(
+       &bdr_apply_worker->name, &slot_name, &bdr_apply_worker->sysid,
+       &bdr_apply_worker->timeline, &replication_identifier, NULL);
 
    bdr_apply_worker->origin_id = replication_identifier;
 
@@ -792,6 +832,7 @@ static bool
 bdr_create_con_gucs(char  *name,
                    char **used_databases,
                    Size  *num_used_databases,
+                   char **database_initcons,
                    BdrApplyWorker *out_worker)
 {
    int         off;
@@ -803,6 +844,8 @@ bdr_create_con_gucs(char  *name,
    /* don't free, referenced by the guc machinery! */
    char       *optname_dsn = palloc(strlen(name) + 30);
    char       *optname_delay = palloc(strlen(name) + 30);
+   char       *optname_replica = palloc(strlen(name) + 30);
+   char       *optname_local_dsn = palloc(strlen(name) + 30);
    opts = palloc(sizeof(BdrApplyWorkerConfigOptions));
 
    Assert(process_shared_preload_libraries_in_progress);
@@ -829,6 +872,25 @@ bdr_create_con_gucs(char  *name,
                            GUC_UNIT_MS,
                            NULL, NULL, NULL);
 
+   sprintf(optname_replica, "bdr.%s_init_replica", name);
+   DefineCustomBoolVariable(optname_replica,
+                            optname_replica,
+                            NULL,
+                            &opts->bdr_init_replica,
+                            false,
+                            PGC_SIGHUP,
+                            0,
+                            NULL, NULL, NULL);
+
+   sprintf(optname_local_dsn, "bdr.%s_replica_local_dsn", name);
+   DefineCustomStringVariable(optname_local_dsn,
+                              optname_local_dsn,
+                              NULL,
+                              &opts->replica_local_dsn,
+                              NULL, PGC_POSTMASTER,
+                              GUC_NOT_IN_SAMPLE,
+                              NULL, NULL, NULL);
+
    if (!opts->dsn)
    {
        elog(WARNING, "no connection information for %s", name);
@@ -887,6 +949,19 @@ bdr_create_con_gucs(char  *name,
            pstrdup(NameStr(out_worker->dbname));
    }
 
+   /*
+    * Make sure that at most one of the worker configs for each DB can be
+    * configured to run initialization.
+    */
+   if (opts->bdr_init_replica)
+   {
+       if (database_initcons[off] != NULL)
+           elog(ERROR, "Connections %s and %s on database %s both have bdr_init_replica enabled, cannot continue",
+               name, database_initcons[off], used_databases[off]);
+       else
+           database_initcons[off] = name; /* no need to pstrdup, see _PG_init */
+   }
+
    /* optname vars and opts intentionally leaked, see above */
    return true;
 }
@@ -912,6 +987,7 @@ bdr_launch_apply_workers(char *dbname)
    apply_worker.bgw_flags = BGWORKER_SHMEM_ACCESS |
        BGWORKER_BACKEND_DATABASE_CONNECTION;
    apply_worker.bgw_start_time = BgWorkerStart_RecoveryFinished;
+   /* TODO: For EXEC_BACKEND we must use bgw_library_name & bgw_function_name */
    apply_worker.bgw_main = bdr_apply_main;
    apply_worker.bgw_restart_time = 5;
    apply_worker.bgw_notify_pid = 0;
@@ -1000,36 +1076,35 @@ bdr_node_count()
  * for each BDR connection.
  *
  * Since the worker is fork()ed from the postmaster, all globals initialised in
- * _PG_init remain valid (TODO: change this for EXEC_BACKEND support).
+ * _PG_init remain valid.
  *
  * This worker can use the SPI and shared memory.
  */
 static void
 bdr_perdb_worker_main(Datum main_arg)
 {
-   int             rc;
-   List           *apply_workers;
-   ListCell       *c;
-   BdrPerdbWorker *bdr_perdb_worker;
-   char           *dbname;
+   int               rc;
+   List             *apply_workers;
+   ListCell         *c;
+   BdrPerdbWorker   *bdr_perdb_worker;
 
    Assert(IsBackgroundWorker);
 
    /* FIXME: won't work with EXEC_BACKEND, change to index into shm array */
    bdr_perdb_worker = (BdrPerdbWorker *) DatumGetPointer(main_arg);
-   dbname = NameStr(bdr_perdb_worker->dbname);
 
-   bdr_worker_init(dbname);
+   bdr_worker_init(NameStr(bdr_perdb_worker->dbname));
 
-   CurrentResourceOwner =
-       ResourceOwnerCreate(NULL, "bdr seq top-level resource owner");
+   CurrentResourceOwner = ResourceOwnerCreate(NULL, "bdr seq top-level resource owner");
    bdr_saved_resowner = CurrentResourceOwner;
 
-   elog(LOG, "Starting bdr apply workers for db %s",
-        NameStr(bdr_perdb_worker->dbname));
+   /* Do we need to init the local DB from a remote node? */
+   bdr_init_replica(&bdr_perdb_worker->dbname);
+
+   elog(LOG, "Starting bdr apply workers for db %s", NameStr(bdr_perdb_worker->dbname));
 
    /* Launch the apply workers */
-   apply_workers = bdr_launch_apply_workers(dbname);
+   apply_workers = bdr_launch_apply_workers(NameStr(bdr_perdb_worker->dbname));
 
    /*
     * For now, just free the bgworker handles. Later we'll probably want them
@@ -1222,7 +1297,7 @@ bdr_worker_shmem_create_workers(void)
  *
  * To release a block, use bdr_worker_shmem_release(...)
  */
-static BdrWorker*
+BdrWorker*
 bdr_worker_shmem_alloc(BdrWorkerType worker_type)
 {
    int i;
@@ -1250,8 +1325,11 @@ bdr_worker_shmem_alloc(BdrWorkerType worker_type)
  * re-used.
  *
  * The bgworker *must* no longer be running.
+ *
+ * If passed, the bgworker handle is checked to ensure the worker
+ * is not still running before the slot is released.
  */
-static void
+void
 bdr_worker_shmem_release(BdrWorker* worker, BackgroundWorkerHandle *handle)
 {
    LWLockAcquire(BdrWorkerCtl->lock, LW_EXCLUSIVE);
@@ -1300,6 +1378,7 @@ _PG_init(void)
    char       *connections_tmp;
 
    char      **used_databases;
+   char      **database_initcons;
    Size        num_used_databases = 0;
 
    if (!process_shared_preload_libraries_in_progress)
@@ -1352,6 +1431,19 @@ _PG_init(void)
                            GUC_UNIT_MS,
                            NULL, NULL, NULL);
 
+   /*
+    * We can't use the temp_tablespace safely for our dumps, because Pg's
+    * crash recovery is very careful to delete only particularly formatted
+    * files. Instead for now just allow user to specify dump storage.
+    */
+   DefineCustomStringVariable("bdr.temp_dump_directory",
+                              "Directory to store dumps for local restore",
+                              NULL,
+                              &bdr_temp_dump_directory,
+                              "/tmp", PGC_SIGHUP,
+                              0,
+                              NULL, NULL, NULL);
+
    /* if nothing is configured, we're done */
    if (connections == NULL)
    {
@@ -1384,7 +1476,7 @@ _PG_init(void)
    {
        bdr_max_workers = list_length(connames) * 2;
        elog(LOG, "bdr: bdr_max_workers unset, configuring for %d workers",
-            bdr_max_workers);
+               bdr_max_workers);
    }
 
    /* Set up a ProcessUtility_hook to stop unsupported commands being run */
@@ -1405,6 +1497,11 @@ _PG_init(void)
 
    /* Names of all databases we're going to be doing BDR for */
    used_databases = palloc0(sizeof(char *) * list_length(connames));
+   /*
+    * For each db named in used_databases, the corresponding index is the name
+    * of the conn with bdr_init_replica=t if any.
+    */
+   database_initcons = palloc0(sizeof(char *) * list_length(connames));
 
    /*
     * Read all connections and create their BdrApplyWorker structs, validating
@@ -1422,12 +1519,10 @@ _PG_init(void)
        name = (char *) lfirst(c);
 
        if (!bdr_create_con_gucs(name, used_databases, &num_used_databases,
-                                apply_worker))
+                                database_initcons, apply_worker))
            continue;
-
        apply_worker->origin_id = InvalidRepNodeId;
-       bdr_startup_context->workers = lcons(apply_worker,
-                                            bdr_startup_context->workers);
+       bdr_startup_context->workers = lcons(apply_worker, bdr_startup_context->workers);
    }
 
    /*
@@ -1437,6 +1532,13 @@ _PG_init(void)
    list_free(connames);
    connames = NIL;
 
+   /*
+    * We've ensured there are no duplicate init connections, no need to
+    * remember which conn is the bdr_init_replica conn anymore. The contents
+    * are just pointers into connections_tmp so we don't want to free them.
+    */
+   pfree(database_initcons);
+
    /*
     * We now need to register one static bgworker per database.  When started,
     * this worker will continue setup - doing any required initialization of
@@ -1455,6 +1557,7 @@ _PG_init(void)
    perdb_worker.bgw_flags = BGWORKER_SHMEM_ACCESS |
        BGWORKER_BACKEND_DATABASE_CONNECTION;
    perdb_worker.bgw_start_time = BgWorkerStart_RecoveryFinished;
+   /* TODO: For EXEC_BACKEND we must use bgw_library_name & bgw_function_name */
    perdb_worker.bgw_main = bdr_perdb_worker_main;
    perdb_worker.bgw_restart_time = 5;
    perdb_worker.bgw_notify_pid = 0;
@@ -1577,6 +1680,8 @@ bdr_maintain_schema(void)
 
        BdrVotesRelid = bdr_lookup_relid("bdr_votes", schema_oid);
 
+       BdrNodesRelid = bdr_lookup_relid("bdr_nodes", schema_oid);
+
        QueuedDropsRelid = bdr_lookup_relid("bdr_queued_drops", schema_oid);
    }
    else
index 415af96a7d8cb4fa2267c09750d19899bdab2d9d..425e8019e07f3c18ea9bb823933ffce0d36d9a00 100644 (file)
 #define BDR_SLOT_NAME_FORMAT "bdr_%u_%s_%u_%u__%s"
 #define BDR_NODE_ID_FORMAT "bdr_"UINT64_FORMAT"_%u_%u_%u_%s"
 
+#define BDR_INIT_REPLICA_CMD "bdr_initial_load"
+
+/* forward delcs from other headers */
+typedef struct BackgroundWorkerHandle BackgroundWorkerHandle;
+typedef struct pg_conn PGconn;
+
 /*
  * Flags to indicate which fields are present in a commit record sent by the
  * output plugin.
@@ -41,13 +47,17 @@ typedef struct BdrApplyWorker
    /* connection name specified in configuration */
    NameData name;
 
+   /* TODO: Remove these from shm, into bdr worker global state */
    RepNodeId origin_id;
-
    uint64 sysid;
-
    TimeLineID timeline;
 
-   /* If not InvalidXLogRecPtr, stop replay at this point and exit */
+   /*
+    * If not InvalidXLogRecPtr, stop replay at this point and exit.
+    *
+    * To save shmem space in apply workers, this is reset to InvalidXLogRecPtr
+    * if replay is successfully completed instead of setting a separate flag.
+    */
    XLogRecPtr replay_stop_lsn;
 
    /* Request that the remote forward all changes from other nodes */
@@ -105,9 +115,32 @@ typedef struct BdrWorker
 
 /* GUCs */
 extern int bdr_default_apply_delay;
+extern int bdr_max_workers;
+extern char *bdr_temp_dump_directory;
+
+/*
+ * Header for the shared memory segment ref'd by the BdrWorkerCtl ptr,
+ * containing bdr_max_workers entries of BdrWorkerCon .
+ */
+typedef struct BdrWorkerControl
+{
+   /* Must hold this lock when writing to BdrWorkerControl members */
+   LWLockId     lock;
+   /* Required only for bgworker restart issues: */
+   bool         launch_workers;
+   /* Set/unset by bdr_apply_pause()/_replay(). */
+   bool         pause_apply;
+   /* Array members, of size bdr_max_workers */
+   BdrWorker    slots[FLEXIBLE_ARRAY_MEMBER];
+} BdrWorkerControl;
+
+extern BdrWorkerControl *BdrWorkerCtl;
 
 extern ResourceOwner bdr_saved_resowner;
 
+/* bdr_nodes table oid */
+extern Oid BdrNodesRelid;
+
 /* DDL replication support */
 extern Oid QueuedDDLCommandsRelid;
 extern Oid QueuedDropsRelid;
@@ -162,7 +195,29 @@ extern bool bdr_get_float8byval(void);
 extern bool bdr_get_integer_timestamps(void);
 extern bool bdr_get_bigendian(void);
 
+/* initialize a new bdr member */
+extern void bdr_init_replica(Name dbname);
+
+/* shared memory management */
+extern BdrWorker* bdr_worker_shmem_alloc(BdrWorkerType worker_type);
+extern void bdr_worker_shmem_release(BdrWorker* worker, BackgroundWorkerHandle *handle);
+
 /* forbid commands we do not support currently (or never will) */
 extern void init_bdr_commandfilter(void);
 
+/* background workers */
+extern void bdr_apply_main(Datum main_arg);
+
+/* helpers shared by multiple worker types */
+extern PGconn*
+bdr_connect(char *conninfo_repl,
+           char* remote_ident, size_t remote_ident_length,
+           NameData* slot_name,
+           uint64* remote_sysid_i, TimeLineID *remote_tlid_i);
+
+extern PGconn*
+bdr_establish_connection_and_slot(Name connection_name, Name out_slot_name,
+   uint64 *out_sysid, TimeLineID* out_timeline, RepNodeId
+   *out_replication_identifier, char **out_snapshot);
+
 #endif /* BDR_H */
index fe687e151516cffbb794eb5d3af8a833075cc819..b86b0f29ce52fb2b8bad76a74b2675821c77e508 100644 (file)
 
 #include "bdr.h"
 
+/* These are always necessary for a bgworker */
+#include "miscadmin.h"
+#include "postmaster/bgworker.h"
+#include "storage/ipc.h"
+#include "storage/latch.h"
+#include "storage/lwlock.h"
+#include "storage/proc.h"
+#include "storage/shmem.h"
+
 #include "pgstat.h"
 
 #include "access/committs.h"
@@ -36,6 +45,7 @@
 #include "funcapi.h"
 
 #include "libpq/pqformat.h"
+#include "libpq-fe.h"
 
 #include "miscadmin.h"
 
diff --git a/contrib/bdr/bdr_init_replica.c b/contrib/bdr/bdr_init_replica.c
new file mode 100644 (file)
index 0000000..7b7e4b6
--- /dev/null
@@ -0,0 +1,1095 @@
+/* -------------------------------------------------------------------------
+ *
+ * bdr_init_replica.c
+ *     Populate a new bdr node from the data in an existing node
+ *
+ * Use dump and restore, then bdr catchup mode, to bring up a new
+ * bdr node into a bdr group. Allows a new blank database to be
+ * introduced into an existing, already-working bdr group.
+ *
+ * Copyright (C) 2012-2014, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ *     contrib/bdr/bdr_init_replica.c
+ *
+ * -------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "bdr.h"
+
+/* These are always necessary for a bgworker */
+#include "miscadmin.h"
+#include "postmaster/bgworker.h"
+#include "storage/ipc.h"
+#include "storage/latch.h"
+#include "storage/lwlock.h"
+#include "storage/proc.h"
+#include "storage/shmem.h"
+
+#include "libpq/pqformat.h"
+#include "libpq-fe.h"
+
+#include "access/heapam.h"
+#include "access/xact.h"
+
+#include "replication/walreceiver.h"
+#include "replication/replication_identifier.h"
+
+#include "postmaster/bgwriter.h"
+
+#include "utils/builtins.h"
+#include "utils/syscache.h"
+
+/* bdr_init_replica */
+#include <unistd.h>
+#include <sys/types.h>
+#include <sys/wait.h>
+#include <sys/errno.h>
+#include <sys/stat.h>
+#include "postmaster/postmaster.h"
+
+char *bdr_temp_dump_directory = NULL;
+
+static void bdr_exec_init_replica(Name conn_name, char *snapshot);
+
+static void bdr_catchup_to_lsn(PGconn *conn, Name dbname, Name conn_name,
+                              XLogRecPtr target_lsn);
+
+/*
+ * Search BdrWorkerCtl for a worker in dbname with init_replica set and
+ * return it. The first worker found is returned (previous code should've
+ * ensured there can only be one). If no match is found, return null.
+ *
+ * Must be called with at least a share lock on BdrWorkerCtl->lock
+ */
+static BdrWorker*
+find_init_replica_worker(Name dbname)
+{
+   Size off;
+
+   Assert(LWLockHeldByMe(BdrWorkerCtl->lock));
+   /* Check whether one of our connections has init_replica set */
+   for (off = 0; off < bdr_max_workers; off++)
+   {
+       if (BdrWorkerCtl->slots[off].worker_type == BDR_WORKER_APPLY)
+       {
+           BdrApplyWorker *aw = &BdrWorkerCtl->slots[off].worker_data.apply_worker;
+           if (strcmp(NameStr(aw->dbname), NameStr(*dbname)) == 0)
+           {
+               const char *init_replica_str;
+               bool init_replica = false;
+               init_replica_str = bdr_get_worker_option(NameStr(aw->name),
+                                                        "init_replica", true);
+               if (init_replica_str
+                   && parse_bool(init_replica_str, &init_replica)
+                   && init_replica)
+                   return &BdrWorkerCtl->slots[off];
+           }
+       }
+   }
+   return NULL;
+}
+
+/*
+ * Get this node's status value from the remote's bdr.bdr_nodes table
+ * and return it.
+ *
+ * If no row is found, '\0' is returned.
+ */
+static char
+bdr_get_remote_status(PGconn *pgconn, Name dbname)
+{
+   PGresult *res;
+   char status;
+   StringInfoData query;
+   char escaped_dbname[NAMEDATALEN*2+1];
+   int escape_error;
+
+   initStringInfo(&query);
+
+   PQescapeStringConn(pgconn, &escaped_dbname[0], NameStr(*dbname), NAMEDATALEN, &escape_error);
+   if (escape_error)
+       elog(FATAL, "Failed to escape local dbname %s: %s",
+            NameStr(*dbname), PQerrorMessage(pgconn));
+
+   appendStringInfo(&query,
+                    "SELECT node_status FROM bdr.bdr_nodes WHERE node_sysid = " UINT64_FORMAT " AND node_dbname = '%s' FOR UPDATE",
+                    GetSystemIdentifier(), escaped_dbname);
+   res = PQexec(pgconn, query.data);
+   if (PQresultStatus(res) != PGRES_TUPLES_OK)
+   {
+       elog(FATAL, "bdr %s: query failed during bdr init - \"%s\": status %s: %s\n",
+            NameStr(*dbname), query.data, PQresStatus(PQresultStatus(res)),
+            PQresultErrorMessage(res));
+   }
+   if (PQntuples(res) == 0)
+       /* No row found on remote, we're starting from scratch */
+       status = '\0';
+   else
+   {
+       char *status_str = PQgetvalue(res, 0, 0);
+       Assert(strlen(status_str) == 1);
+       status = status_str[0];
+   }
+   PQclear(res);
+
+   return status;
+}
+
+/*
+ * Update/delete/insert in bdr.bdr_nodes to ensure that the bdr.bdr_nodes row
+ * for this worker's node ID matches the passed status before returning.
+ *
+ * The special case '\0' means "remove the row".
+ *
+ * No fancy upsert games are required here because we ensure that only one
+ * worker can be initing any one database, and that node IDs are unique across
+ * a group of BDR nodes.
+ */
+static char
+bdr_set_remote_status(PGconn *pgconn, Name dbname, const char status, const char prev_status)
+{
+   PGresult *res;
+   char    *status_str;
+   StringInfoData query;
+   char escaped_dbname[NAMEDATALEN*2+1];
+   int escape_error;
+   const uint64 sysid = GetSystemIdentifier();
+
+   initStringInfo(&query);
+
+   if (status == prev_status)
+       /* No action required (we could check the remote, but meh) */
+       return status;
+
+   PQescapeStringConn(pgconn, &escaped_dbname[0], NameStr(*dbname), NAMEDATALEN, &escape_error);
+   if (escape_error)
+       elog(FATAL, "Failed to escape local dbname %s: %s",
+            NameStr(*dbname), PQerrorMessage(pgconn));
+
+   if (status == '\0')
+   {
+       char    new_status;
+       appendStringInfo(&query,
+                        "DELETE FROM bdr.bdr_nodes WHERE node_sysid = "
+                        UINT64_FORMAT
+                        " AND node_dbname = '%s' RETURNING node_status",
+                        sysid, escaped_dbname);
+       res = PQexec(pgconn, query.data);
+
+       elog(DEBUG2, "bdr %s: deleting bdr_nodes row with id " UINT64_FORMAT
+            " and node_dbname %s ", NameStr(*dbname), sysid, escaped_dbname);
+
+       if (PQresultStatus(res) != PGRES_TUPLES_OK)
+       {
+           elog(FATAL, "bdr %s: query failed during bdr init - \"%s\": status %s: %s\n",
+                NameStr(*dbname), query.data,
+                PQresStatus(PQresultStatus(res)), PQresultErrorMessage(res));
+       }
+       if (PQntuples(res) == 0)
+       {
+           /* If prev_status was '\0' we wouldn't be here, so we should've got a returned value */
+           elog(FATAL, "bdr %s: bdr.bdr_nodes row for node_sysid="
+                UINT64_FORMAT
+                ", dbname='%s' missing, expected row with status=%c",
+                NameStr(*dbname), sysid, escaped_dbname, (int)prev_status);
+       }
+       status_str = PQgetvalue(res, 0, 0);
+       Assert(strlen(status_str) == 1);
+       new_status = status_str[0];
+
+       if (new_status != prev_status)
+       {
+           elog(FATAL, "bdr %s: bdr.bdr_nodes row for node_sysid="
+                UINT64_FORMAT
+                ", dbname='%s' had status=%c, expected status=%c",
+                NameStr(*dbname), sysid, escaped_dbname, (int)new_status, (int)prev_status);
+       }
+
+       PQclear(res);
+   }
+   else
+   {
+       char    new_status;
+       appendStringInfo(&query,
+                        "UPDATE bdr.bdr_nodes "
+                        "SET node_status = '%c' "
+                        "WHERE node_sysid = " UINT64_FORMAT
+                        " AND node_dbname = '%s' RETURNING ("
+                        "SELECT node_status FROM bdr.bdr_nodes "
+                        "WHERE node_sysid = " UINT64_FORMAT
+                        " AND node_dbname = '%s')",
+                        (int)status, sysid, escaped_dbname, sysid,
+                        escaped_dbname);
+
+       res = PQexec(pgconn, query.data);
+
+       elog(DEBUG2, "bdr %s: update row with id "
+            UINT64_FORMAT
+            " and node_dbname %s from %c to %c",
+            NameStr(*dbname), sysid, escaped_dbname, prev_status, status);
+
+       if (PQresultStatus(res) != PGRES_TUPLES_OK)
+       {
+           elog(FATAL,
+                "bdr %s: query failed during bdr init - \"%s\": status %s: %s\n",
+                NameStr(*dbname), query.data,
+                PQresStatus(PQresultStatus(res)), PQresultErrorMessage(res));
+       }
+       if (PQntuples(res) != 0)
+       {
+           /* Updated a row */
+           status_str = PQgetvalue(res, 0, 0);
+           Assert(strlen(status_str) == 1);
+           new_status = status_str[0];
+           if (new_status != prev_status)
+           {
+               elog(FATAL,
+                    "bdr %s: bdr.bdr_nodes row for node_sysid=" UINT64_FORMAT
+                    ", dbname='%s' had status=%c, expected status=%c",
+                    NameStr(*dbname), sysid, escaped_dbname, (int)new_status,
+                    (int)prev_status);
+           }
+
+           PQclear(res);
+       }
+       else
+       {
+           /* No rows affected, insert a new row instead */
+           PQclear(res);
+           resetStringInfo(&query);
+           appendStringInfo(&query,
+                            "INSERT INTO bdr.bdr_nodes (node_sysid, node_dbname, node_status) VALUES (" UINT64_FORMAT ", '%s', '%c');",
+                            sysid, escaped_dbname, (int)status);
+           res = PQexec(pgconn, query.data);
+
+           elog(DEBUG2, "bdr %s: insert row with id " UINT64_FORMAT
+                " and node_dbname %s from %c to %c",
+                NameStr(*dbname), sysid, escaped_dbname, prev_status, status);
+
+           if (PQresultStatus(res) != PGRES_COMMAND_OK)
+           {
+               elog(FATAL,
+                    "bdr %s: query failed during bdr init - \"%s\": status %s: %s\n",
+                    NameStr(*dbname), query.data,
+                    PQresStatus(PQresultStatus(res)),
+                    PQresultErrorMessage(res));
+           }
+           PQclear(res);
+       }
+   }
+
+   return status;
+}
+
+static XLogRecPtr
+bdr_get_remote_lsn(PGconn *conn)
+{
+   const char *query =
+       "SELECT pg_xlog_location_diff(pg_current_xlog_insert_location(), '0/0')";
+   char       *lsn_str;
+   char       *lsn_str_end;
+   XLogRecPtr  lsn;
+   PGresult   *res;
+
+   res = PQexec(conn, query);
+   if (PQresultStatus(res) != PGRES_TUPLES_OK)
+   {
+       elog(ERROR, "Unable to get remote LSN, query %s failed with status %s: %s\n",
+           query, PQresStatus(PQresultStatus(res)), PQresultErrorMessage(res));
+   }
+   Assert(PQntuples(res) == 1);
+   lsn_str = PQgetvalue(res, 0, 0);
+   /* lsn_str's remote type is numeric, but we know it has to fit in an XLogRecPtr */
+   /* TODO: Less ugly way to do this */
+   lsn = (XLogRecPtr) strtoul(lsn_str, &lsn_str_end, 10);
+   if (*lsn_str_end != '\0')
+       elog(ERROR, "Unable to parse remote LSN value %s as unsigned long int", lsn_str);
+   PQclear(res);
+   return lsn;
+}
+
+/*
+ * Make sure the bdr extension is installed on the other end. If it's
+ * a known extension but not present in the current DB, try to CREATE EXTENSION
+ * it.
+ */
+static void
+bdr_ensure_ext_installed(PGconn *pgconn, Name bdr_conn_name)
+{
+   PGresult *res;
+   const char *q_bdr_installed =
+       "SELECT default_version, installed_version "
+       "FROM pg_catalog.pg_available_extensions WHERE name = 'bdr';";
+
+   res = PQexec(pgconn, q_bdr_installed);
+   if (PQresultStatus(res) != PGRES_TUPLES_OK)
+   {
+       elog(ERROR, "Unable to get remote bdr extension version; query %s failed with %s: %s\n",
+           q_bdr_installed, PQresStatus(PQresultStatus(res)), PQresultErrorMessage(res));
+   }
+
+   if (PQntuples(res) == 1)
+   {
+       char *default_version;
+       /*
+        * bdr ext is known to Pg, check install state, install if missing.
+        *
+        * Right now we don't check the installed version or try to upgrade.
+        */
+       default_version = PQgetvalue(res, 0, 0);
+       Assert(default_version != NULL);
+       if (PQgetisnull(res, 0, 1))
+       {
+           /* bdr ext present but not installed; try to create */
+           PQclear(res);
+
+           res = PQexec(pgconn, "CREATE EXTENSION IF NOT EXISTS btree_gist;");
+           if (PQresultStatus(res) != PGRES_COMMAND_OK)
+           {
+               ereport(ERROR,
+                       (errmsg("Unable to 'CREATE EXTENSION btree_gist;' on bdr connection %s: state %s: %s",
+                        NameStr(*bdr_conn_name), PQresStatus(PQresultStatus(res)), PQresultErrorMessage(res)),
+                        errhint("CREATE EXTENSION btree_gist; as a superuser.")));
+           }
+           PQclear(res);
+
+           res = PQexec(pgconn, "CREATE EXTENSION bdr;");
+           if (PQresultStatus(res) != PGRES_COMMAND_OK)
+           {
+               ereport(ERROR,
+                       (errmsg("Unable to 'CREATE EXTENSION bdr;' on bdr connection %s: state %s: %s",
+                        NameStr(*bdr_conn_name), PQresStatus(PQresultStatus(res)), PQresultErrorMessage(res)),
+                        errhint("Make sure BDR is in shared_preload_libraries and CREATE EXTENSION bdr; as superuser.")));
+           }
+           PQclear(res);
+       }
+   }
+   else if (PQntuples(res) == 0)
+   {
+       /* bdr ext is not known to Pg at all */
+       ereport(ERROR,
+               (errmsg("Remote PostgreSQL install for bdr connection %s does not have bdr extension installed",
+                NameStr(*bdr_conn_name)),
+                errdetail("no entry with name 'bdr' in pg_available_extensions; did you install BDR?")));
+   }
+   else
+   {
+       Assert(false); /* Should not get >1 tuples */
+   }
+}
+
+/*
+ * Delete a replication identifier.
+ *
+ * This should really be in the replication identifier support code in changeset extraction,
+ * as DeleteReplicationIdentifier or DropReplicationIdentifier.
+ *
+ * If no matching identifier is found, takes no action.
+ */
+static void
+bdr_delete_replication_identifier(RepNodeId repid)
+{
+   HeapTuple       tuple = NULL;
+   Relation        rel;
+   ItemPointerData tid;
+
+   /*
+    * Exclusively lock pg_replication_identifier
+    */
+   rel = heap_open(ReplicationIdentifierRelationId, RowExclusiveLock);
+
+   /*
+    * Look it up from the syscache and get a copy we can safely
+    * modify.
+    */
+   tuple = GetReplicationInfoByIdentifier(repid, true);
+   if (HeapTupleIsValid(tuple))
+   {
+       tid = tuple->t_self;
+       ReleaseSysCache(tuple);
+       simple_heap_delete(rel, &tid);
+   }
+   heap_close(rel, RowExclusiveLock);
+
+   /*
+    * We should CHECKPOINT after this to make sure replication
+    * identifier state gets flushed.
+    */
+   RequestCheckpoint(CHECKPOINT_IMMEDIATE|CHECKPOINT_FORCE);
+}
+
+static void
+bdr_drop_slot_and_replication_identifier(Name connection_name, Name dbname)
+{
+
+   char        conninfo_repl[MAXCONNINFO + 75];
+   const char *dsn;
+   char        remote_ident[256];
+   PGconn     *streamConn;
+   RepNodeId   replication_identifier;
+   NameData    slot_name;
+   TimeLineID  timeline;
+   uint64      sysid;
+   PGresult   *res;
+   StringInfoData query;
+   char       *sqlstate;
+
+   elog(LOG, "bdr %s: Dropping slot and local ident from connection %s",
+        NameStr(*dbname), NameStr(*connection_name));
+
+   dsn = bdr_get_worker_option(NameStr(*connection_name), "dsn", false);
+   snprintf(conninfo_repl, sizeof(conninfo_repl),
+            "%s replication=database fallback_application_name=bdr",
+            dsn);
+
+   /* Establish BDR conn and IDENTIFY_SYSTEM */
+   streamConn = bdr_connect(
+       conninfo_repl,
+       remote_ident, sizeof(remote_ident),
+       &slot_name, &sysid, &timeline
+       );
+
+   StartTransactionCommand();
+   replication_identifier = GetReplicationIdentifier(remote_ident, true);
+
+   if (OidIsValid(replication_identifier))
+   {
+       /* Local replication identifier exists and must be dropped. */
+       elog(DEBUG2, "bdr %s: Deleting local replication identifier %hu",
+            NameStr(*dbname), replication_identifier);
+       bdr_delete_replication_identifier(replication_identifier);
+   }
+   else
+   {
+       elog(DEBUG2, "bdr %s: No local replication identifier to delete",
+            NameStr(*dbname));
+   }
+
+   /*
+    * Remove corresponding remote slot if it exists. We can't query
+    * whether it exists or not silently over the replication protocol,
+    * so we just try it and cope if it's missing.
+    */
+   initStringInfo(&query);
+   appendStringInfo(&query, "DROP_REPLICATION_SLOT %s", NameStr(slot_name));
+   res = PQexec(streamConn, query.data);
+   if (PQresultStatus(res) == PGRES_COMMAND_OK)
+   {
+       elog(DEBUG2, "bdr %s: remote replication slot %s deleted",
+            NameStr(*dbname), NameStr(slot_name));
+   }
+   else
+   {
+       /* SQLSTATE 42704 expected; others are error conditions */
+       sqlstate = PQresultErrorField(res, PG_DIAG_SQLSTATE);
+       if (strcmp(sqlstate, "42704") != 0)
+       {
+           ereport(ERROR,
+                   (errmsg("'DROP_REPLICATION_SLOT %s' on bdr connection %s failed with sqlstate %s: %s",
+                    NameStr(slot_name), NameStr(*connection_name), sqlstate, PQresultErrorMessage(res))));
+       }
+       else
+       {
+           elog(DEBUG2, "bdr %s: No slot to delete", NameStr(*dbname));
+       }
+   }
+   CommitTransactionCommand();
+   PQclear(res);
+   PQfinish(streamConn);
+}
+
+static void
+bdr_init_replica_cleanup_tmpdir(int errcode, Datum tmpdir)
+{
+   struct stat st;
+   const char* dir = DatumGetCString(tmpdir);
+
+   if (stat(dir, &st) == 0)
+       if (!rmtree(dir, true))
+           elog(WARNING, "Failed to clean up bdr dump temporary directory %s on exit/error", dir);
+}
+
+/*
+ * Use a script to copy the contents of a remote node using pg_dump and apply
+ * it to the local node. Runs during slot creation to bring up a new logical
+ * replica from an existing node.
+ */
+static void
+bdr_exec_init_replica(Name conn_name, char *snapshot)
+{
+   pid_t pid;
+   char *bindir;
+   char *tmpdir;
+   char *remote_dsn;
+   char *replica_local_dsn;
+   char  bdr_init_replica_script_path[MAXPGPATH];
+   const char *envvar;
+   StringInfoData path;
+   int   saved_errno;
+
+   initStringInfo(&path);
+
+   bindir = pstrdup(my_exec_path);
+   get_parent_directory(bindir);
+
+   replica_local_dsn = pstrdup(bdr_get_worker_option(NameStr(*conn_name), "replica_local_dsn", false));
+   remote_dsn  = pstrdup(bdr_get_worker_option(NameStr(*conn_name), "dsn", false));
+
+   if (!find_other_exec(my_exec_path, BDR_INIT_REPLICA_CMD,
+                        BDR_INIT_REPLICA_CMD " " PG_VERSION,
+                        &bdr_init_replica_script_path[0]))
+   {
+       elog(ERROR, "bdr: failed to find " BDR_INIT_REPLICA_CMD
+            " in Pg bin dir or wrong version (expected %s)",
+            PG_VERSION);
+   }
+
+   if (!replica_local_dsn)
+       elog(FATAL, "bdr init_replica: no replica_local_dsn specified");
+
+   tmpdir = palloc(strlen(bdr_temp_dump_directory)+32);
+   sprintf(tmpdir, "%s/postgres-bdr-%s.%d", bdr_temp_dump_directory,
+           snapshot, getpid());
+
+   if (mkdir(tmpdir, 0700))
+   {
+       saved_errno = errno;
+       if (saved_errno == EEXIST)
+       {
+           /*
+            * Target is an existing dir that somehow wasn't cleaned up or
+            * something more sinister. We'll just die here, and let the
+            * postmaster relaunch us and retry the whole operation.
+            */
+           elog(ERROR, "bdr init_replica: Temporary dump directory %s exists: %s",
+                tmpdir, strerror(saved_errno));
+       }
+       else
+       {
+           elog(ERROR, "bdr init_replica: Failed to create temp directory: %s",
+                strerror(saved_errno));
+       }
+   }
+
+   pid = fork();
+   if (pid < 0)
+       elog(FATAL, "can't fork to create initial replica");
+   else if (pid == 0)
+   {
+       int n = 0;
+
+       char * envp[] = {
+           NULL, /* to be replaced with PATH */
+           NULL
+       };
+       char *const argv[] = {
+           bdr_init_replica_script_path,
+           "--snapshot", snapshot,
+           "--source", remote_dsn,
+           "--target", replica_local_dsn,
+           "--tmp-directory", tmpdir,
+           NULL
+       };
+
+       envvar = getenv("PATH");
+       appendStringInfoString(&path, "PATH=");
+       appendStringInfoString(&path, bindir);
+       if (envvar != NULL)
+       {
+           appendStringInfoString(&path, ":");
+           appendStringInfoString(&path, envvar);
+       }
+       envp[0] = path.data;
+
+       elog(LOG, "Creating replica with: %s --snapshot %s --source \"%s\" --target \"%s\" --tmp-directory \"%s\"",
+            bdr_init_replica_script_path, snapshot, remote_dsn,
+            replica_local_dsn, tmpdir);
+
+       n = execve(bdr_init_replica_script_path, argv, envp);
+       if (n < 0)
+           _exit(n);
+   }
+   else
+   {
+       pid_t res;
+       int exitstatus = 0;
+
+       elog(DEBUG3, "Waiting for %s pid %d",
+            bdr_init_replica_script_path, pid);
+
+       PG_ENSURE_ERROR_CLEANUP(bdr_init_replica_cleanup_tmpdir,
+                               CStringGetDatum(tmpdir));
+       {
+           do
+           {
+               res = waitpid(pid, &exitstatus, WNOHANG);
+               if (res < 0)
+               {
+                   if (errno == EINTR || errno == EAGAIN)
+                       continue;
+                   elog(FATAL, "bdr_exec_init_replica: error calling waitpid");
+               }
+               else if (res == pid)
+                   break;
+
+               pg_usleep(10 * 1000);
+               CHECK_FOR_INTERRUPTS();
+           }
+           while (1);
+
+           elog(DEBUG3, "%s exited with waitpid return status %d",
+                bdr_init_replica_script_path, exitstatus);
+
+           if (exitstatus != 0)
+           {
+               if (WIFEXITED(exitstatus))
+                   elog(FATAL, "bdr: %s exited with exit code %d",
+                        bdr_init_replica_script_path, WEXITSTATUS(exitstatus));
+               if (WIFSIGNALED(exitstatus))
+                   elog(FATAL, "bdr: %s exited due to signal %d",
+                        bdr_init_replica_script_path, WTERMSIG(exitstatus));
+               elog(FATAL, "bdr: %s exited for an unknown reason with waitpid return %d",
+                    bdr_init_replica_script_path, exitstatus);
+           }
+       }
+       PG_END_ENSURE_ERROR_CLEANUP(bdr_init_replica_cleanup_tmpdir,
+                                   PointerGetDatum(tmpdir));
+       bdr_init_replica_cleanup_tmpdir(0, CStringGetDatum(tmpdir));
+   }
+
+   pfree(replica_local_dsn);
+   pfree(remote_dsn);
+   pfree(tmpdir);
+}
+
+/*
+ * Determine whether we need to initialize the database from a remote
+ * node and perform the required initialization if so.
+ */
+void
+bdr_init_replica(Name dbname)
+{
+   const char *connstr;
+   char status;
+   XLogRecPtr min_remote_lsn;
+   PGconn *nonrepl_init_conn;
+   StringInfoData query;
+   BdrWorker  *init_replica_worker;
+   Name init_conn_name;
+
+   initStringInfo(&query);
+
+   elog(DEBUG2, "bdr %s: bdr_init_replica",
+        NameStr(*dbname));
+
+   /*
+    * Before starting workers we must determine if we need to copy
+    * initial state from a remote node. This is only necessary if
+    * there is a connection with init_replica set and we do not yet
+    * have an entry in the local "bdr.bdr_nodes" table for our node
+    * ID showing initialisation to be complete.
+    */
+   LWLockAcquire(BdrWorkerCtl->lock, LW_SHARED);
+   init_replica_worker = find_init_replica_worker(dbname);
+   LWLockRelease(BdrWorkerCtl->lock);
+   /* No connections have init_replica=t, nothing to do */
+   if (!init_replica_worker)
+   {
+       elog(DEBUG2, "bdr %s: nothing to do in bdr_init_replica",
+            NameStr(*dbname));
+       return;
+   }
+
+   init_conn_name = &init_replica_worker->worker_data.apply_worker.name;
+   elog(DEBUG2, "bdr %s: bdr_init_replica init from connection %s",
+        NameStr(*dbname), NameStr(*init_conn_name));
+
+   /*
+    * Check the local bdr.bdr_nodes over SPI or direct scan to see if
+    * there's an entry for ourselves in ready mode already.
+    *
+    * This is an optimisation we don't need to do yet...
+    */
+   /*TODO
+   (status, min_remote_lsn) = get_node_status_from_local();
+   if (status == 'r')
+       return;
+   */
+
+   /*
+    * Test to see if there's an entry in the remote's bdr.bdr_nodes for our
+    * system identifier. If there is, that'll tell us what stage of startup
+    * we are up to and let us resume an incomplete start.
+    */
+   connstr = bdr_get_worker_option(NameStr(*init_conn_name), "dsn", false);
+   nonrepl_init_conn = PQconnectdb(connstr);
+   if (PQstatus(nonrepl_init_conn) != CONNECTION_OK)
+   {
+       ereport(FATAL,
+               (errmsg("bdr %s: could not connect to the upstream server in non-replication mode: %s",
+                       NameStr(*dbname), PQerrorMessage(nonrepl_init_conn))));
+   }
+
+   bdr_ensure_ext_installed(nonrepl_init_conn, dbname);
+   elog(DEBUG2, "bdr %s: bdr extension is installed in remote",
+        NameStr(*dbname));
+
+   /* Get the bdr.bdr_nodes status field for our node id from the remote */
+   status = bdr_get_remote_status(nonrepl_init_conn, dbname);
+   switch (status)
+   {
+       case '\0':
+           elog(DEBUG2, "bdr %s: initializing from clean state", NameStr(*dbname));
+           break;
+
+       case 'r':
+           /*
+            * Init has been completed, but we didn't check our local bdr.bdr_nodes,
+            * or the final update hasn't propagated yet.
+            *
+            * All we need to do is catch up, we already replayed enough to be
+            * consistent and start up in normal mode last time around
+            */
+           elog(DEBUG2, "bdr %s: init already completed, nothing to do",
+                NameStr(*dbname));
+           return;
+
+       case 'c':
+           /*
+            * We were in catchup mode when we died. We need to resume catchup mode
+            * up to the expected LSN before switching over.
+            *
+            * To do that all we need to do is fall through without doing any
+            * slot re-creation, dump/apply, etc, and pick up when we do catchup.
+            *
+            * We won't know what the original catchup target point is, but we can
+            * just catch up to whatever xlog position the server is currently at.
+            */
+           elog(DEBUG2, "bdr %s: dump applied, need to continue catchup",
+                NameStr(*dbname));
+           break;
+
+       case 'i':
+           /*
+            * A previous init attempt seems to have failed. Clean up, then fall through
+            * to start setup again.
+            *
+            * We can't just re-use the slot and replication identifier that were created
+            * last time (if they were), because we have no way of getting the slot's exported
+            * snapshot after CREATE_REPLICATION_SLOT.
+            */
+           elog(DEBUG2, "bdr %s: previous failed initalization detected, cleaning up",
+                NameStr(*dbname));
+           bdr_drop_slot_and_replication_identifier(init_conn_name, dbname);
+           status = bdr_set_remote_status(nonrepl_init_conn, dbname, '\0', status);
+           break;
+
+       default:
+           Assert(false); // Unhandled case
+           break;
+   }
+
+   if (status == '\0')
+   {
+       int         off;
+       int        *my_conn_idxs;
+       int         n_conns = 0;
+       char       *init_snapshot = NULL;
+       PGconn     *init_repl_conn = NULL;
+
+       elog(LOG, "bdr %s: initializing from remote db", NameStr(*dbname));
+
+       /* We're starting from scratch or have cleaned up a previous failed attempt */
+       status = bdr_set_remote_status(nonrepl_init_conn, dbname, 'i', status);
+
+       my_conn_idxs = (int*)palloc(sizeof(Size) * bdr_max_workers);
+
+       /*
+        * Collect a list of connections to make slots for.
+        */
+       LWLockAcquire(BdrWorkerCtl->lock, LW_SHARED);
+       for (off = 0; off < bdr_max_workers; off++)
+           if (BdrWorkerCtl->slots[off].worker_type == BDR_WORKER_APPLY
+               && (strcmp(NameStr(BdrWorkerCtl->slots[off].worker_data.apply_worker.dbname), NameStr(*dbname)) == 0))
+               my_conn_idxs[n_conns++] = off;
+       LWLockRelease(BdrWorkerCtl->lock);
+
+       elog(DEBUG2, "bdr %s: creating slots for %d nodes",
+            NameStr(*dbname), n_conns);
+
+       /*
+        * For each connection, ensure its slot exists.
+        *
+        * Do it one by one rather than fiddling with async libpq queries. If this
+        * needs to be parallelized later, it should probably be done by launching
+        * each apply worker and letting them create their own slots, then having
+        * them wait until signalled/unlatched before proceeding with actual
+        * replication. That'll save us another round of connections too.
+        *
+        * We don't attempt any cleanup if slot creation fails, we just bail out
+        * and leave any already-created slots in place.
+        */
+       for (off = 0; off < n_conns; off++)
+       {
+           BdrWorker *w = &BdrWorkerCtl->slots[my_conn_idxs[off]];
+           char *snapshot = NULL;
+           PGconn *conn = NULL;
+           RepNodeId replication_identifier;
+           NameData slot_name;
+           uint64 sysid;
+           TimeLineID timeline;
+
+           elog(DEBUG1, "bdr %s: checking/creating slot for %s",
+                NameStr(*dbname), NameStr(w->worker_data.apply_worker.name));
+           /*
+            * Create the slot on the remote. The returned remote sysid and
+            * timeline, the slot name, and the local replication identifier
+            * are all discarded; they're not needed here, and will be obtained
+            * again by the apply workers when they're launched after init.
+            */
+           conn = bdr_establish_connection_and_slot(&w->worker_data.apply_worker.name, &slot_name, &sysid, &timeline, &replication_identifier, &snapshot);
+           /* Always throws rather than returning failure */
+           Assert(conn);
+
+           if (&BdrWorkerCtl->slots[off] == init_replica_worker)
+           {
+               /*
+                * We need to keep the snapshot ID returned by CREATE SLOT so
+                * we can pass it to pg_dump to get a consistent dump from the
+                * remote slot's start point.
+                *
+                * The snapshot is only valid for the lifetime of the
+                * replication connection we created it with, so we must keep
+                * that connection around until the dump finishes.
+                */
+               if (!snapshot)
+                   elog(ERROR, "bdr %s: init_replica failed to create snapshot!",
+                        NameStr(*dbname));
+               init_snapshot = snapshot;
+               init_repl_conn = conn;
+           }
+           else
+           {
+               /*
+                * Just throw the returned info away; we only needed to create the slot
+                * so its replication identifier can be advanced during catchup.
+                */
+               if (snapshot)
+                   pfree(snapshot);
+               PQfinish(conn);
+           }
+       }
+
+       pfree(my_conn_idxs);
+
+       /* If we get here, we should have a valid snapshot to dump */
+       Assert(init_snapshot != NULL);
+       Assert(init_repl_conn != NULL);
+
+       /*
+        * Execute the dump and apply its self.
+        *
+        * Note that the bdr extension tables override pg_dump's default and
+        * ask to be included in dumps. In particular, bdr.bdr_nodes will get
+        * copied over.
+        */
+       elog(LOG, "bdr %s: creating and restoring dump for %s",
+            NameStr(*dbname), NameStr(*init_conn_name));
+       bdr_exec_init_replica(init_conn_name, init_snapshot);
+       PQfinish(init_repl_conn);
+
+       pfree(init_snapshot);
+       status = bdr_set_remote_status(nonrepl_init_conn, dbname, 'c', status);
+   }
+
+   Assert(status == 'c');
+
+   /* Launch the catchup worker and wait for it to finish */
+   elog(LOG, "bdr %s: launching catchup mode apply worker", NameStr(*dbname));
+   min_remote_lsn = bdr_get_remote_lsn(nonrepl_init_conn);
+   bdr_catchup_to_lsn(nonrepl_init_conn, dbname, init_conn_name, min_remote_lsn);
+   status = bdr_set_remote_status(nonrepl_init_conn, dbname, 'r', status);
+
+   elog(LOG, "bdr %s: catchup worker finished, ready for normal replication",
+        NameStr(*dbname));
+   PQfinish(nonrepl_init_conn);
+}
+
+/*
+ * Cleanup function after catchup; makes sure we free the bgworker
+ * slot for the catchup worker.
+ */
+static void
+bdr_catchup_to_lsn_cleanup(int code, Datum offset)
+{
+   int worker_shmem_idx = DatumGetInt32(offset);
+
+   /* Clear the worker's shared memory struct now we're done with it */
+   bdr_worker_shmem_release(&BdrWorkerCtl->slots[worker_shmem_idx], NULL);
+}
+
+/*
+ * Launch a temporary apply worker in catchup mode (forward_changesets=t),
+ * set to replay until the passed LSN.
+ *
+ * This worker will receive and apply all changes the remote server has
+ * received since the snapshot we got our dump from was taken, including
+ * those from other servers, and will advance the replication identifiers
+ * associated with each remote node appropriately.
+ *
+ * When we finish applying and the worker exits, we'll be caught up with the
+ * remote and in a consistent state where all our local replication identifiers
+ * are consistent with the actual state of the local DB.
+ */
+static void
+bdr_catchup_to_lsn(PGconn *conn, Name dbname, Name conn_name, XLogRecPtr target_lsn)
+{
+   int worker_shmem_idx;
+   pid_t bgw_pid;
+   BdrApplyWorker *catchup_worker;
+   BackgroundWorker bgw;
+   BackgroundWorkerHandle *bgw_handle;
+   BgwHandleStatus bgw_status;
+
+   elog(DEBUG1, "Registering bdr apply catchup worker %s for db %s to lsn %X/%X",
+        NameStr(*conn_name), NameStr(*dbname),
+        (uint32)(target_lsn>>32), (uint32)target_lsn);
+
+   /* Create the shm entry for the catchup worker */
+   LWLockAcquire(BdrWorkerCtl->lock, LW_SHARED);
+   for (worker_shmem_idx = 0; worker_shmem_idx < bdr_max_workers; worker_shmem_idx++)
+       if (BdrWorkerCtl->slots[worker_shmem_idx].worker_type == BDR_WORKER_EMPTY_SLOT)
+           break;
+   if (worker_shmem_idx == bdr_max_workers)
+   {
+       LWLockRelease(BdrWorkerCtl->lock);
+       elog(ERROR, "No free bdr worker slots, bdr_max_workers=%d too low",
+            bdr_max_workers);
+   }
+   BdrWorkerCtl->slots[worker_shmem_idx].worker_type = BDR_WORKER_APPLY;
+   catchup_worker = &BdrWorkerCtl->slots[worker_shmem_idx].worker_data.apply_worker;
+   LWLockRelease(BdrWorkerCtl->lock);
+
+   /*
+    * Make sure we free the shmem slot for the catchup worker even if we
+    * hit an error.
+    *
+    * There's a small race between claiming the worker and entering the ensure
+    * cleanup block.  Real consequences, pretty much nil, since this is really
+    * just startup code.
+    */
+   PG_ENSURE_ERROR_CLEANUP(bdr_catchup_to_lsn_cleanup,
+                           Int32GetDatum(worker_shmem_idx));
+   {
+       pid_t prev_bgw_pid = 0;
+
+       /* Set up the BdrApplyWorker struct in shmem */
+       strncpy(NameStr(catchup_worker->name), NameStr(*conn_name), NAMEDATALEN);
+       strncpy(NameStr(catchup_worker->dbname), NameStr(*dbname), NAMEDATALEN);
+       catchup_worker->origin_id = InvalidRepNodeId;
+       catchup_worker->sysid = 0;
+       catchup_worker->timeline = 0;
+       /* Special parameters for a catchup worker only */
+       catchup_worker->replay_stop_lsn = target_lsn;
+       catchup_worker->forward_changesets = true;
+
+       /* and the BackgroundWorker, which is a regular apply worker */
+       bgw.bgw_flags = BGWORKER_SHMEM_ACCESS |
+           BGWORKER_BACKEND_DATABASE_CONNECTION;
+       bgw.bgw_start_time = BgWorkerStart_RecoveryFinished;
+       /* TODO: For EXEC_BACKEND we must use bgw_library_name & bgw_function_name */
+       bgw.bgw_main = bdr_apply_main;
+       /*
+        * Would prefer this to be BGW_NEVER_RESTART but it's not honoured for
+        * exit 0 anyway. Set a small delay to give us time to unregister the
+        * worker after it exits, before the replacement is started. See below
+        * in the switch handling the exit cases for more detail.
+        */
+       bgw.bgw_restart_time = 5;
+       bgw.bgw_notify_pid = MyProc->pid;
+       bgw.bgw_main_arg = Int32GetDatum(worker_shmem_idx);
+
+       snprintf(bgw.bgw_name, BGW_MAXLEN,
+                "bdr %s: catchup apply to %X/%X on %s",
+                NameStr(*dbname),
+                (uint32)(target_lsn>>32), (uint32)target_lsn,
+                NameStr(*conn_name));
+       bgw.bgw_name[BGW_MAXLEN-1] = '\0';
+
+       /* Launch the catchup worker and wait for it to start */
+       RegisterDynamicBackgroundWorker(&bgw, &bgw_handle);
+       bgw_status = WaitForBackgroundWorkerStartup(bgw_handle, &bgw_pid);
+       prev_bgw_pid = bgw_pid;
+
+       /*
+        * Sleep on our latch until we're woken by SIGUSR1 on bgworker state
+        * change, or by timeout. (We need a timeout because there's a race between
+        * bgworker start and our setting the latch; if it starts and dies again
+        * quickly we'll miss it and sleep forever w/o a timeout).
+        */
+       while (bgw_status == BGWH_STARTED && bgw_pid == prev_bgw_pid)
+       {
+           int rc;
+           rc = WaitLatch(&MyProc->procLatch,
+                          WL_LATCH_SET | WL_TIMEOUT | WL_POSTMASTER_DEATH,
+                          1000L);
+
+           ResetLatch(&MyProc->procLatch);
+
+           /* emergency bailout if postmaster has died */
+           if (rc & WL_POSTMASTER_DEATH)
+               proc_exit(1);
+
+           /* Is our worker still replaying? */
+           bgw_status = GetBackgroundWorkerPid(bgw_handle, &bgw_pid);
+       }
+       switch(bgw_status)
+       {
+           case BGWH_POSTMASTER_DIED:
+               proc_exit(1);
+               break;
+           case BGWH_STOPPED:
+           case BGWH_NOT_YET_STARTED:
+           case BGWH_STARTED:
+               /*
+                * You'd think we'd only get here in the STOPPED case, but Pg
+                * restarts our bgworker even if we set BGW_NEVER_RESTART if we
+                * exit 0.
+                *
+                * If the pid changes, we know the worker exited, even if it's
+                * reported as running. So we make sure we always terminate it.
+                */
+               TerminateBackgroundWorker(bgw_handle);
+               break;
+       }
+       pfree(bgw_handle);
+
+       /*
+        * Stopped doesn't mean *successful*. The worker might've errored out. We
+        * have no way of getting its exit status, so we have to rely on it setting
+        * something in shmem on successful exit. In this case it will set
+        * replay_stop_lsn to InvalidXLogRecPtr to indicate that replay is done.
+        */
+       if (catchup_worker->replay_stop_lsn != InvalidXLogRecPtr)
+       {
+           /* Worker must've died before it finished */
+           elog(ERROR,
+                "bdr %s: catchup worker exited before catching up to target LSN %X/%X",
+                NameStr(*dbname),
+                (uint32)(target_lsn>>32), (uint32)target_lsn);
+       }
+       else
+       {
+           elog(DEBUG1, "bdr %s: catchup worker caught up to target LSN",
+                NameStr(*dbname));
+       }
+   }
+   PG_END_ENSURE_ERROR_CLEANUP(bdr_catchup_to_lsn_cleanup,
+                               Int32GetDatum(worker_shmem_idx));
+
+   bdr_catchup_to_lsn_cleanup(0, Int32GetDatum(worker_shmem_idx));
+
+   /* We're caught up! */
+}
diff --git a/contrib/bdr/scripts/bdr_initial_load b/contrib/bdr/scripts/bdr_initial_load
new file mode 100644 (file)
index 0000000..b2b05d2
--- /dev/null
@@ -0,0 +1,73 @@
+#!/bin/bash
+#
+# bdr_initial_load copies data from a running BDR instance's database to the empty
+# local database that's due to be joined into the BDR group.
+#
+# It must exit with an error if the dump OR the load fails. The load must be
+# performed in a single transaction or must use --clean to flush any local
+# contents first, i.e. it must cope with being re-run if the restore is
+# interrupted.
+#
+
+JOBS=1
+
+i=0
+argv=("$@")
+while (($i < ${#argv[*]})); do
+    case "${argv[$i]}" in
+        --snapshot)
+            ((i++)); SNAPSHOT="${argv[$i]}"
+        ;;
+        --source)
+            ((i++)); SOURCE="${argv[$i]}"
+        ;;
+        --target)
+            ((i++)); TARGET="${argv[$i]}"
+        ;;
+        --tmp-directory)
+            ((i++)); TMPDIR="${argv[$i]}"
+        ;;
+        --jobs)
+            ((i++)); JOBS="${argv[$i]}"
+        ;;
+        --help)
+            echo "Usage: bdr_replica --source <dsn> --target <dsn> [--snapshot <name>] --dir /path/to/dir [--jobs N]"
+            echo "<dsn> is a libpq conninfo string, e.g. \"host=/tmp post=5433 dbnae=xxx\""
+            exit 0
+        ;;
+        *)
+            echo Unknown command-line option: ${argv[$i]}
+            exit 1
+        ;;
+    esac
+
+    ((i++))
+done
+
+if [ -z "$SOURCE" ]; then
+    echo Please specify a source DSN with '--source "port=nnn dbname=xxx"'; exit 1
+fi
+
+if [ -z "$TARGET" ]; then
+    echo Please specify a target DSN with '--target "port=nnn dbname=xxx"'; exit 1
+fi
+
+if [ -z "$TMPDIR" ]; then
+    echo Please specify a directory with '--temp-directory /path/to/dir'; exit 1
+fi
+
+SNAP=${SNAPSHOT:+"--snapshot $SNAPSHOT"}
+
+echo "Dumping remote database \"$SOURCE\" with $JOBS concurrent workers to \"$TMPDIR\""
+if ! pg_dump -j $JOBS $SNAP -F d -f $TMPDIR "$SOURCE"; then
+    echo "pg_dump of "$SOURCE" failed, aborting"
+    exit 1
+fi
+
+echo "Restoring dump to local DB \"$TARGET\" with $JOBS concurrent workers from \"$TMPDIR\""
+if ! PGOPTIONS="-c bdr.permit_unsafe_ddl_commands=true" pg_restore --exit-on-error --single-transaction -j $JOBS -F d -d "$TARGET" $TMPDIR; then
+    echo "pg_restore to "$TARGET" failed, aborting"
+    exit 2
+fi
+
+exit 0
index 9285f5c7ea59a95f4b437dc5253d9c35562a783a..6fe8a84990a5770f49927960d2122c98ab29c98b 100644 (file)
@@ -2,11 +2,13 @@
 
 MODULE_big = bdr
 OBJS = bdr.o bdr_apply.o bdr_compat.o bdr_commandfilter.o bdr_count.o \
-   bdr_seq.o
+   bdr_seq.o bdr_init_replica.o
 
 EXTENSION = bdr
 DATA = bdr--0.5.sql
 
+SCRIPTS = scripts/bdr_initial_load
+
 PG_CPPFLAGS = -I$(libpq_srcdir)
 SHLIB_LINK = $(libpq)
 SHLIB_PREREQS = submake-libpq