1
0
mirror of https://github.com/postgres/postgres.git synced 2025-09-02 04:21:28 +03:00

Introduce logical decoding.

This feature, building on previous commits, allows the write-ahead log
stream to be decoded into a series of logical changes; that is,
inserts, updates, and deletes and the transactions which contain them.
It is capable of handling decoding even across changes to the schema
of the effected tables.  The output format is controlled by a
so-called "output plugin"; an example is included.  To make use of
this in a real replication system, the output plugin will need to be
modified to produce output in the format appropriate to that system,
and to perform filtering.

Currently, information can be extracted from the logical decoding
system only via SQL; future commits will add the ability to stream
changes via walsender.

Andres Freund, with review and other contributions from many other
people, including Álvaro Herrera, Abhijit Menon-Sen, Peter Gheogegan,
Kevin Grittner, Robert Haas, Heikki Linnakangas, Fujii Masao, Abhijit
Menon-Sen, Michael Paquier, Simon Riggs, Craig Ringer, and Steve
Singer.
This commit is contained in:
Robert Haas
2014-03-03 16:32:18 -05:00
parent de94b47c0a
commit b89e151054
89 changed files with 12998 additions and 194 deletions

View File

@@ -0,0 +1,19 @@
/*-------------------------------------------------------------------------
* decode.h
* PostgreSQL WAL to logical transformation
*
* Portions Copyright (c) 2012-2014, PostgreSQL Global Development Group
*
*-------------------------------------------------------------------------
*/
#ifndef DECODE_H
#define DECODE_H
#include "access/xlogreader.h"
#include "replication/reorderbuffer.h"
#include "replication/logical.h"
void LogicalDecodingProcessRecord(LogicalDecodingContext *ctx,
XLogRecord *record);
#endif

View File

@@ -0,0 +1,100 @@
/*-------------------------------------------------------------------------
* logical.h
* PostgreSQL logical decoding coordination
*
* Copyright (c) 2012-2014, PostgreSQL Global Development Group
*
*-------------------------------------------------------------------------
*/
#ifndef LOGICAL_H
#define LOGICAL_H
#include "replication/slot.h"
#include "access/xlog.h"
#include "access/xlogreader.h"
#include "replication/output_plugin.h"
struct LogicalDecodingContext;
typedef void (*LogicalOutputPluginWriterWrite) (
struct LogicalDecodingContext *lr,
XLogRecPtr Ptr,
TransactionId xid,
bool last_write
);
typedef LogicalOutputPluginWriterWrite LogicalOutputPluginWriterPrepareWrite;
typedef struct LogicalDecodingContext
{
/* memory context this is all allocated in */
MemoryContext context;
/* infrastructure pieces */
XLogReaderState *reader;
ReplicationSlot *slot;
struct ReorderBuffer *reorder;
struct SnapBuild *snapshot_builder;
OutputPluginCallbacks callbacks;
OutputPluginOptions options;
/*
* User specified options
*/
List *output_plugin_options;
/*
* User-Provided callback for writing/streaming out data.
*/
LogicalOutputPluginWriterPrepareWrite prepare_write;
LogicalOutputPluginWriterWrite write;
/*
* Output buffer.
*/
StringInfo out;
/*
* Private data pointer of the output plugin.
*/
void *output_plugin_private;
/*
* Private data pointer for the data writer.
*/
void *output_writer_private;
/*
* State for writing output.
*/
bool accept_writes;
bool prepared_write;
XLogRecPtr write_location;
TransactionId write_xid;
} LogicalDecodingContext;
extern void CheckLogicalDecodingRequirements(void);
extern LogicalDecodingContext *CreateInitDecodingContext(char *plugin,
List *output_plugin_options,
XLogPageReadCB read_page,
LogicalOutputPluginWriterPrepareWrite prepare_write,
LogicalOutputPluginWriterWrite do_write);
extern LogicalDecodingContext *CreateDecodingContext(
XLogRecPtr start_lsn,
List *output_plugin_options,
XLogPageReadCB read_page,
LogicalOutputPluginWriterPrepareWrite prepare_write,
LogicalOutputPluginWriterWrite do_write);
extern void DecodingContextFindStartpoint(LogicalDecodingContext *ctx);
extern bool DecodingContextReady(LogicalDecodingContext *ctx);
extern void FreeDecodingContext(LogicalDecodingContext *ctx);
extern void LogicalIncreaseXminForSlot(XLogRecPtr lsn, TransactionId xmin);
extern void LogicalIncreaseRestartDecodingForSlot(XLogRecPtr current_lsn,
XLogRecPtr restart_lsn);
extern void LogicalConfirmReceivedLocation(XLogRecPtr lsn);
#endif

View File

@@ -0,0 +1,24 @@
/*-------------------------------------------------------------------------
* logicalfuncs.h
* PostgreSQL WAL to logical transformation support functions
*
* Copyright (c) 2012-2014, PostgreSQL Global Development Group
*
*-------------------------------------------------------------------------
*/
#ifndef LOGICALFUNCS_H
#define LOGICALFUNCS_H
#include "replication/logical.h"
extern int logical_read_local_xlog_page(XLogReaderState *state,
XLogRecPtr targetPagePtr,
int reqLen, XLogRecPtr targetRecPtr,
char *cur_page, TimeLineID *pageTLI);
extern Datum pg_logical_slot_get_changes(PG_FUNCTION_ARGS);
extern Datum pg_logical_slot_get_binary_changes(PG_FUNCTION_ARGS);
extern Datum pg_logical_slot_peek_changes(PG_FUNCTION_ARGS);
extern Datum pg_logical_slot_peek_binary_changes(PG_FUNCTION_ARGS);
#endif

View File

@@ -0,0 +1,98 @@
/*-------------------------------------------------------------------------
* output_plugin.h
* PostgreSQL Logical Decode Plugin Interface
*
* Copyright (c) 2012-2014, PostgreSQL Global Development Group
*
*-------------------------------------------------------------------------
*/
#ifndef OUTPUT_PLUGIN_H
#define OUTPUT_PLUGIN_H
#include "replication/reorderbuffer.h"
struct LogicalDecodingContext;
struct OutputPluginCallbacks;
typedef enum OutputPluginOutputType
{
OUTPUT_PLUGIN_BINARY_OUTPUT,
OUTPUT_PLUGIN_TEXTUAL_OUTPUT
} OutputPluginOutputType;
/*
* Options set by the output plugin, in the startup callback.
*/
typedef struct OutputPluginOptions
{
OutputPluginOutputType output_type;
} OutputPluginOptions;
/*
* Type of the shared library symbol _PG_output_plugin_init that is looked up
* when loading an output plugin shared library.
*/
typedef void (*LogicalOutputPluginInit)(struct OutputPluginCallbacks *cb);
/*
* Callback that gets called in a user-defined plugin. ctx->private_data can
* be set to some private data.
*
* "is_init" will be set to "true" if the decoding slot just got defined. When
* the same slot is used from there one, it will be "false".
*/
typedef void (*LogicalDecodeStartupCB) (
struct LogicalDecodingContext *ctx,
OutputPluginOptions *options,
bool is_init
);
/*
* Callback called for every (explicit or implicit) BEGIN of a successful
* transaction.
*/
typedef void (*LogicalDecodeBeginCB) (
struct LogicalDecodingContext *,
ReorderBufferTXN *txn);
/*
* Callback for every individual change in a successful transaction.
*/
typedef void (*LogicalDecodeChangeCB) (
struct LogicalDecodingContext *,
ReorderBufferTXN *txn,
Relation relation,
ReorderBufferChange *change
);
/*
* Called for every (explicit or implicit) COMMIT of a successful transaction.
*/
typedef void (*LogicalDecodeCommitCB) (
struct LogicalDecodingContext *,
ReorderBufferTXN *txn,
XLogRecPtr commit_lsn);
/*
* Called to shutdown an output plugin.
*/
typedef void (*LogicalDecodeShutdownCB) (
struct LogicalDecodingContext *
);
/*
* Output plugin callbacks
*/
typedef struct OutputPluginCallbacks
{
LogicalDecodeStartupCB startup_cb;
LogicalDecodeBeginCB begin_cb;
LogicalDecodeChangeCB change_cb;
LogicalDecodeCommitCB commit_cb;
LogicalDecodeShutdownCB shutdown_cb;
} OutputPluginCallbacks;
void OutputPluginPrepareWrite(struct LogicalDecodingContext *ctx, bool last_write);
void OutputPluginWrite(struct LogicalDecodingContext *ctx, bool last_write);
#endif /* OUTPUT_PLUGIN_H */

View File

@@ -0,0 +1,351 @@
/*
* reorderbuffer.h
* PostgreSQL logical replay/reorder buffer management.
*
* Copyright (c) 2012-2014, PostgreSQL Global Development Group
*
* src/include/replication/reorderbuffer.h
*/
#ifndef REORDERBUFFER_H
#define REORDERBUFFER_H
#include "access/htup_details.h"
#include "lib/ilist.h"
#include "storage/sinval.h"
#include "utils/hsearch.h"
#include "utils/rel.h"
#include "utils/snapshot.h"
#include "utils/timestamp.h"
/* an individual tuple, stored in one chunk of memory */
typedef struct ReorderBufferTupleBuf
{
/* position in preallocated list */
slist_node node;
/* tuple, stored sequentially */
HeapTupleData tuple;
HeapTupleHeaderData header;
char data[MaxHeapTupleSize];
} ReorderBufferTupleBuf;
/* types of the change passed to a 'change' callback */
enum ReorderBufferChangeType
{
REORDER_BUFFER_CHANGE_INSERT,
REORDER_BUFFER_CHANGE_UPDATE,
REORDER_BUFFER_CHANGE_DELETE
};
/*
* a single 'change', can be an insert (with one tuple), an update (old, new),
* or a delete (old).
*
* The same struct is also used internally for other purposes but that should
* never be visible outside reorderbuffer.c.
*/
typedef struct ReorderBufferChange
{
XLogRecPtr lsn;
/* type of change */
union
{
enum ReorderBufferChangeType action;
/* do not leak internal enum values to the outside */
int action_internal;
};
/*
* Context data for the change, which part of the union is valid depends
* on action/action_internal.
*/
union
{
/* old, new tuples when action == *_INSERT|UPDATE|DELETE */
struct
{
/* relation that has been changed */
RelFileNode relnode;
/* valid for DELETE || UPDATE */
ReorderBufferTupleBuf *oldtuple;
/* valid for INSERT || UPDATE */
ReorderBufferTupleBuf *newtuple;
} tp;
/* new snapshot */
Snapshot snapshot;
/* new command id for existing snapshot in a catalog changing tx */
CommandId command_id;
/* new cid mapping for catalog changing transaction */
struct
{
RelFileNode node;
ItemPointerData tid;
CommandId cmin;
CommandId cmax;
CommandId combocid;
} tuplecid;
};
/*
* While in use this is how a change is linked into a transactions,
* otherwise it's the preallocated list.
*/
dlist_node node;
} ReorderBufferChange;
typedef struct ReorderBufferTXN
{
/*
* The transactions transaction id, can be a toplevel or sub xid.
*/
TransactionId xid;
/* did the TX have catalog changes */
bool has_catalog_changes;
/*
* Do we know this is a subxact?
*/
bool is_known_as_subxact;
/*
* LSN of the first data carrying, WAL record with knowledge about this
* xid. This is allowed to *not* be first record adorned with this xid, if
* the previous records aren't relevant for logical decoding.
*/
XLogRecPtr first_lsn;
/* ----
* LSN of the record that lead to this xact to be committed or
* aborted. This can be a
* * plain commit record
* * plain commit record, of a parent transaction
* * prepared transaction commit
* * plain abort record
* * prepared transaction abort
* * error during decoding
* ----
*/
XLogRecPtr final_lsn;
/*
* LSN pointing to the end of the commit record + 1.
*/
XLogRecPtr end_lsn;
/*
* LSN of the last lsn at which snapshot information reside, so we can
* restart decoding from there and fully recover this transaction from
* WAL.
*/
XLogRecPtr restart_decoding_lsn;
/*
* Commit time, only known when we read the actual commit record.
*/
TimestampTz commit_time;
/*
* Base snapshot or NULL.
*/
Snapshot base_snapshot;
XLogRecPtr base_snapshot_lsn;
/*
* How many ReorderBufferChange's do we have in this txn.
*
* Changes in subtransactions are *not* included but tracked separately.
*/
uint64 nentries;
/*
* How many of the above entries are stored in memory in contrast to being
* spilled to disk.
*/
uint64 nentries_mem;
/*
* List of ReorderBufferChange structs, including new Snapshots and new
* CommandIds
*/
dlist_head changes;
/*
* List of (relation, ctid) => (cmin, cmax) mappings for catalog tuples.
* Those are always assigned to the toplevel transaction. (Keep track of
* #entries to create a hash of the right size)
*/
dlist_head tuplecids;
uint64 ntuplecids;
/*
* On-demand built hash for looking up the above values.
*/
HTAB *tuplecid_hash;
/*
* Hash containing (potentially partial) toast entries. NULL if no toast
* tuples have been found for the current change.
*/
HTAB *toast_hash;
/*
* non-hierarchical list of subtransactions that are *not* aborted. Only
* used in toplevel transactions.
*/
dlist_head subtxns;
uint32 nsubtxns;
/*
* Stored cache invalidations. This is not a linked list because we get
* all the invalidations at once.
*/
uint32 ninvalidations;
SharedInvalidationMessage *invalidations;
/* ---
* Position in one of three lists:
* * list of subtransactions if we are *known* to be subxact
* * list of toplevel xacts (can be a as-yet unknown subxact)
* * list of preallocated ReorderBufferTXNs
* ---
*/
dlist_node node;
} ReorderBufferTXN;
/* so we can define the callbacks used inside struct ReorderBuffer itself */
typedef struct ReorderBuffer ReorderBuffer;
/* change callback signature */
typedef void (*ReorderBufferApplyChangeCB) (
ReorderBuffer *rb,
ReorderBufferTXN *txn,
Relation relation,
ReorderBufferChange *change);
/* begin callback signature */
typedef void (*ReorderBufferBeginCB) (
ReorderBuffer *rb,
ReorderBufferTXN *txn);
/* commit callback signature */
typedef void (*ReorderBufferCommitCB) (
ReorderBuffer *rb,
ReorderBufferTXN *txn,
XLogRecPtr commit_lsn);
struct ReorderBuffer
{
/*
* xid => ReorderBufferTXN lookup table
*/
HTAB *by_txn;
/*
* Transactions that could be a toplevel xact, ordered by LSN of the first
* record bearing that xid..
*/
dlist_head toplevel_by_lsn;
/*
* one-entry sized cache for by_txn. Very frequently the same txn gets
* looked up over and over again.
*/
TransactionId by_txn_last_xid;
ReorderBufferTXN *by_txn_last_txn;
/*
* Callacks to be called when a transactions commits.
*/
ReorderBufferBeginCB begin;
ReorderBufferApplyChangeCB apply_change;
ReorderBufferCommitCB commit;
/*
* Pointer that will be passed untouched to the callbacks.
*/
void *private_data;
/*
* Private memory context.
*/
MemoryContext context;
/*
* Data structure slab cache.
*
* We allocate/deallocate some structures very frequently, to avoid bigger
* overhead we cache some unused ones here.
*
* The maximum number of cached entries is controlled by const variables
* ontop of reorderbuffer.c
*/
/* cached ReorderBufferTXNs */
dlist_head cached_transactions;
Size nr_cached_transactions;
/* cached ReorderBufferChanges */
dlist_head cached_changes;
Size nr_cached_changes;
/* cached ReorderBufferTupleBufs */
slist_head cached_tuplebufs;
Size nr_cached_tuplebufs;
XLogRecPtr current_restart_decoding_lsn;
/* buffer for disk<->memory conversions */
char *outbuf;
Size outbufsize;
};
ReorderBuffer *ReorderBufferAllocate(void);
void ReorderBufferFree(ReorderBuffer *);
ReorderBufferTupleBuf *ReorderBufferGetTupleBuf(ReorderBuffer *);
void ReorderBufferReturnTupleBuf(ReorderBuffer *, ReorderBufferTupleBuf *tuple);
ReorderBufferChange *ReorderBufferGetChange(ReorderBuffer *);
void ReorderBufferReturnChange(ReorderBuffer *, ReorderBufferChange *);
void ReorderBufferQueueChange(ReorderBuffer *, TransactionId, XLogRecPtr lsn, ReorderBufferChange *);
void ReorderBufferCommit(ReorderBuffer *, TransactionId,
XLogRecPtr commit_lsn, XLogRecPtr end_lsn,
TimestampTz commit_time);
void ReorderBufferAssignChild(ReorderBuffer *, TransactionId, TransactionId, XLogRecPtr commit_lsn);
void ReorderBufferCommitChild(ReorderBuffer *, TransactionId, TransactionId,
XLogRecPtr commit_lsn, XLogRecPtr end_lsn);
void ReorderBufferAbort(ReorderBuffer *, TransactionId, XLogRecPtr lsn);
void ReorderBufferAbortOld(ReorderBuffer *, TransactionId xid);
void ReorderBufferForget(ReorderBuffer *, TransactionId, XLogRecPtr lsn);
void ReorderBufferSetBaseSnapshot(ReorderBuffer *, TransactionId, XLogRecPtr lsn, struct SnapshotData *snap);
void ReorderBufferAddSnapshot(ReorderBuffer *, TransactionId, XLogRecPtr lsn, struct SnapshotData *snap);
void ReorderBufferAddNewCommandId(ReorderBuffer *, TransactionId, XLogRecPtr lsn,
CommandId cid);
void ReorderBufferAddNewTupleCids(ReorderBuffer *, TransactionId, XLogRecPtr lsn,
RelFileNode node, ItemPointerData pt,
CommandId cmin, CommandId cmax, CommandId combocid);
void ReorderBufferAddInvalidations(ReorderBuffer *, TransactionId, XLogRecPtr lsn,
Size nmsgs, SharedInvalidationMessage *msgs);
bool ReorderBufferIsXidKnown(ReorderBuffer *, TransactionId xid);
void ReorderBufferXidSetCatalogChanges(ReorderBuffer *, TransactionId xid, XLogRecPtr lsn);
bool ReorderBufferXidHasCatalogChanges(ReorderBuffer *, TransactionId xid);
bool ReorderBufferXidHasBaseSnapshot(ReorderBuffer *, TransactionId xid);
ReorderBufferTXN *ReorderBufferGetOldestTXN(ReorderBuffer *);
void ReorderBufferSetRestartPoint(ReorderBuffer *, XLogRecPtr ptr);
void StartupReorderBuffer(void);
#endif

View File

@@ -16,6 +16,24 @@
#include "storage/shmem.h"
#include "storage/spin.h"
/*
* Behaviour of replication slots, upon release or crash.
*
* Slots marked as PERSISTENT are crashsafe and will not be dropped when
* released. Slots marked as EPHEMERAL will be dropped when released or after
* restarts.
*
* EPHEMERAL slots can be made PERSISTENT by calling ReplicationSlotPersist().
*/
typedef enum ReplicationSlotPersistency
{
RS_PERSISTENT,
RS_EPHEMERAL
} ReplicationSlotPersistency;
/*
* On-Disk data of a replication slot, preserved across restarts.
*/
typedef struct ReplicationSlotPersistentData
{
/* The slot's identifier */
@@ -24,6 +42,11 @@ typedef struct ReplicationSlotPersistentData
/* database the slot is active on */
Oid database;
/*
* The slot's behaviour when being dropped (or restored after a crash).
*/
ReplicationSlotPersistency persistency;
/*
* xmin horizon for data
*
@@ -32,9 +55,22 @@ typedef struct ReplicationSlotPersistentData
*/
TransactionId xmin;
/*
* xmin horizon for catalog tuples
*
* NB: This may represent a value that hasn't been written to disk yet;
* see notes for effective_xmin, below.
*/
TransactionId catalog_xmin;
/* oldest LSN that might be required by this replication slot */
XLogRecPtr restart_lsn;
/* oldest LSN that the client has acked receipt for */
XLogRecPtr confirmed_flush;
/* plugin name */
NameData plugin;
} ReplicationSlotPersistentData;
/*
@@ -67,12 +103,26 @@ typedef struct ReplicationSlot
* same as the persistent value (data.xmin).
*/
TransactionId effective_xmin;
TransactionId effective_catalog_xmin;
/* data surviving shutdowns and crashes */
ReplicationSlotPersistentData data;
/* is somebody performing io on this slot? */
LWLock *io_in_progress_lock;
/* all the remaining data is only used for logical slots */
/* ----
* When the client has confirmed flushes >= candidate_xmin_lsn we can
* advance the catalog xmin, when restart_valid has been passed,
* restart_lsn can be increased.
* ----
*/
TransactionId candidate_catalog_xmin;
XLogRecPtr candidate_xmin_lsn;
XLogRecPtr candidate_restart_valid;
XLogRecPtr candidate_restart_lsn;
} ReplicationSlot;
/*
@@ -97,8 +147,11 @@ extern Size ReplicationSlotsShmemSize(void);
extern void ReplicationSlotsShmemInit(void);
/* management of individual slots */
extern void ReplicationSlotCreate(const char *name, bool db_specific);
extern void ReplicationSlotCreate(const char *name, bool db_specific,
ReplicationSlotPersistency p);
extern void ReplicationSlotPersist(void);
extern void ReplicationSlotDrop(const char *name);
extern void ReplicationSlotAcquire(const char *name);
extern void ReplicationSlotRelease(void);
extern void ReplicationSlotSave(void);
@@ -106,15 +159,20 @@ extern void ReplicationSlotMarkDirty(void);
/* misc stuff */
extern bool ReplicationSlotValidateName(const char *name, int elevel);
extern void ReplicationSlotsComputeRequiredXmin(void);
extern void ReplicationSlotsComputeRequiredXmin(bool already_locked);
extern void ReplicationSlotsComputeRequiredLSN(void);
extern XLogRecPtr ReplicationSlotsComputeLogicalRestartLSN(void);
extern bool ReplicationSlotsCountDBSlots(Oid dboid, int *nslots, int *nactive);
extern void StartupReplicationSlots(XLogRecPtr checkPointRedo);
extern void CheckPointReplicationSlots(void);
extern void CheckSlotRequirements(void);
extern void ReplicationSlotAtProcExit(void);
/* SQL callable functions */
extern Datum pg_create_physical_replication_slot(PG_FUNCTION_ARGS);
extern Datum pg_create_logical_replication_slot(PG_FUNCTION_ARGS);
extern Datum pg_drop_replication_slot(PG_FUNCTION_ARGS);
extern Datum pg_get_replication_slots(PG_FUNCTION_ARGS);
#endif /* SLOT_H */

View File

@@ -0,0 +1,83 @@
/*-------------------------------------------------------------------------
*
* snapbuild.h
* Exports from replication/logical/snapbuild.c.
*
* Copyright (c) 2012-2014, PostgreSQL Global Development Group
*
* src/include/replication/snapbuild.h
*
*-------------------------------------------------------------------------
*/
#ifndef SNAPBUILD_H
#define SNAPBUILD_H
#include "access/xlogdefs.h"
#include "utils/snapmgr.h"
typedef enum
{
/*
* Initial state, we can't do much yet.
*/
SNAPBUILD_START,
/*
* We have collected enough information to decode tuples in transactions
* that started after this.
*
* Once we reached this we start to collect changes. We cannot apply them
* yet because the might be based on transactions that were still running
* when we reached them yet.
*/
SNAPBUILD_FULL_SNAPSHOT,
/*
* Found a point after hitting built_full_snapshot where all transactions
* that were running at that point finished. Till we reach that we hold
* off calling any commit callbacks.
*/
SNAPBUILD_CONSISTENT
} SnapBuildState;
/* forward declare so we don't have to expose the struct to the public */
struct SnapBuild;
typedef struct SnapBuild SnapBuild;
/* forward declare so we don't have to include reorderbuffer.h */
struct ReorderBuffer;
/* forward declare so we don't have to include heapam_xlog.h */
struct xl_heap_new_cid;
struct xl_running_xacts;
extern void CheckPointSnapBuild(void);
extern SnapBuild *AllocateSnapshotBuilder(struct ReorderBuffer *cache,
TransactionId xmin_horizon, XLogRecPtr start_lsn);
extern void FreeSnapshotBuilder(SnapBuild *cache);
extern void SnapBuildSnapDecRefcount(Snapshot snap);
extern const char *SnapBuildExportSnapshot(SnapBuild *snapstate);
extern void SnapBuildClearExportedSnapshot(void);
extern SnapBuildState SnapBuildCurrentState(SnapBuild *snapstate);
extern bool SnapBuildXactNeedsSkip(SnapBuild *snapstate, XLogRecPtr ptr);
extern void SnapBuildCommitTxn(SnapBuild *builder, XLogRecPtr lsn,
TransactionId xid, int nsubxacts,
TransactionId *subxacts);
extern void SnapBuildAbortTxn(SnapBuild *builder, XLogRecPtr lsn,
TransactionId xid, int nsubxacts,
TransactionId *subxacts);
extern bool SnapBuildProcessChange(SnapBuild *builder, TransactionId xid,
XLogRecPtr lsn);
extern void SnapBuildProcessNewCid(SnapBuild *builder, TransactionId xid,
XLogRecPtr lsn, struct xl_heap_new_cid *cid);
extern void SnapBuildProcessRunningXacts(SnapBuild *builder, XLogRecPtr lsn,
struct xl_running_xacts *running);
extern void SnapBuildSerializationPoint(SnapBuild *builder, XLogRecPtr lsn);
#endif /* SNAPBUILD_H */